From 9a2855fae7eee1eb150116f19af56453cc59ffc5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 14 Apr 2020 04:26:34 +0300 Subject: [PATCH 001/196] in-memory parts: preparation --- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 8 +-- .../MergeTreeDataPartWriterCompact.h | 4 +- ....cpp => MergeTreeDataPartWriterOnDisk.cpp} | 49 +++++++-------- ...iter.h => MergeTreeDataPartWriterOnDisk.h} | 59 ++++--------------- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 4 +- .../MergeTree/MergedBlockOutputStream.cpp | 31 ++++++---- .../MergeTree/MergedBlockOutputStream.h | 5 ++ .../MergedColumnOnlyOutputStream.cpp | 14 +++-- 10 files changed, 74 insertions(+), 104 deletions(-) rename src/Storages/MergeTree/{IMergeTreeDataPartWriter.cpp => MergeTreeDataPartWriterOnDisk.cpp} (90%) rename src/Storages/MergeTree/{IMergeTreeDataPartWriter.h => MergeTreeDataPartWriterOnDisk.h} (68%) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 134b2fc1ef0..24d65622fe6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -54,7 +54,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( const NamesAndTypesList & columns_list, - const std::vector & indices_to_recalc, + const MergeTreeIndices & indices_to_recalc, const CompressionCodecPtr & default_codec, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index e33d4a97cac..22282754d99 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -15,10 +15,10 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) -: IMergeTreeDataPartWriter(disk_, part_path_, - storage_, columns_list_, - indices_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_, true) + : MergeTreeDataPartWriterOnDisk(disk_, + part_path_, storage_, columns_list_, + indices_to_recalc_, marks_file_extension_, + default_codec_, settings_, index_granularity_, true) { using DataPart = MergeTreeDataPartCompact; String data_file_name = DataPart::DATA_FILE_NAME; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 0aff55588aa..c0aced61b95 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -1,10 +1,10 @@ -#include +#include namespace DB { /// Writes data part in compact format. -class MergeTreeDataPartWriterCompact : public IMergeTreeDataPartWriter +class MergeTreeDataPartWriterCompact : public MergeTreeDataPartWriterOnDisk { public: MergeTreeDataPartWriterCompact( diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp similarity index 90% rename from src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp rename to src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 8187799f4be..cdd7e592513 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -1,4 +1,4 @@ -#include +#include #include @@ -14,20 +14,20 @@ namespace constexpr auto INDEX_FILE_EXTENSION = ".idx"; } -void IMergeTreeDataPartWriter::Stream::finalize() +void MergeTreeDataPartWriterOnDisk::Stream::finalize() { compressed.next(); plain_file->next(); marks.next(); } -void IMergeTreeDataPartWriter::Stream::sync() +void MergeTreeDataPartWriterOnDisk::Stream::sync() { plain_file->sync(); marks_file->sync(); } -IMergeTreeDataPartWriter::Stream::Stream( +MergeTreeDataPartWriterOnDisk::Stream::Stream( const String & escaped_column_name_, DiskPtr disk_, const String & data_path_, @@ -47,7 +47,7 @@ IMergeTreeDataPartWriter::Stream::Stream( { } -void IMergeTreeDataPartWriter::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) +void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) { String name = escaped_column_name; @@ -62,7 +62,7 @@ void IMergeTreeDataPartWriter::Stream::addToChecksums(MergeTreeData::DataPart::C } -IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( +MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( DiskPtr disk_, const String & part_path_, const MergeTreeData & storage_, @@ -73,15 +73,13 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_, bool need_finish_last_granule_) - : disk(std::move(disk_)) + : IMergeTreeDataPartWriter(storage_, + columns_list_, indices_to_recalc_, + index_granularity_, settings_) + , disk(std::move(disk_)) , part_path(part_path_) - , storage(storage_) - , columns_list(columns_list_) , marks_file_extension(marks_file_extension_) - , index_granularity(index_granularity_) , default_codec(default_codec_) - , skip_indices(indices_to_recalc_) - , settings(settings_) , compute_granularity(index_granularity.empty()) , with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity) , need_finish_last_granule(need_finish_last_granule_) @@ -93,8 +91,6 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( disk->createDirectories(part_path); } -IMergeTreeDataPartWriter::~IMergeTreeDataPartWriter() = default; - static void fillIndexGranularityImpl( const Block & block, size_t index_granularity_bytes, @@ -155,7 +151,7 @@ static void fillIndexGranularityImpl( } } -void IMergeTreeDataPartWriter::fillIndexGranularity(const Block & block) +void MergeTreeDataPartWriterOnDisk::fillIndexGranularity(const Block & block) { const auto storage_settings = storage.getSettings(); fillIndexGranularityImpl( @@ -169,7 +165,7 @@ void IMergeTreeDataPartWriter::fillIndexGranularity(const Block & block) need_finish_last_granule); } -void IMergeTreeDataPartWriter::initPrimaryIndex() +void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() { if (storage.hasPrimaryKey()) { @@ -180,13 +176,13 @@ void IMergeTreeDataPartWriter::initPrimaryIndex() primary_index_initialized = true; } -void IMergeTreeDataPartWriter::initSkipIndices() +void MergeTreeDataPartWriterOnDisk::initSkipIndices() { for (const auto & index : skip_indices) { String stream_name = index->getFileName(); skip_indices_streams.emplace_back( - std::make_unique( + std::make_unique( stream_name, disk, part_path + stream_name, INDEX_FILE_EXTENSION, @@ -200,8 +196,9 @@ void IMergeTreeDataPartWriter::initSkipIndices() skip_indices_initialized = true; } -void IMergeTreeDataPartWriter::calculateAndSerializePrimaryIndex(const Block & primary_index_block, size_t rows) +void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Block & primary_index_block) { + size_t rows = primary_index_block.rows(); if (!primary_index_initialized) throw Exception("Primary index is not initialized", ErrorCodes::LOGICAL_ERROR); @@ -250,9 +247,9 @@ void IMergeTreeDataPartWriter::calculateAndSerializePrimaryIndex(const Block & p } } -void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices( - const Block & skip_indexes_block, size_t rows) +void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block & skip_indexes_block) { + size_t rows = skip_indexes_block.rows(); if (!skip_indices_initialized) throw Exception("Skip indices are not initialized", ErrorCodes::LOGICAL_ERROR); @@ -314,7 +311,7 @@ void IMergeTreeDataPartWriter::calculateAndSerializeSkipIndices( skip_index_data_mark = skip_index_current_data_mark; } -void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) +void MergeTreeDataPartWriterOnDisk::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) { bool write_final_mark = (with_final_mark && data_written); if (write_final_mark && compute_granularity) @@ -340,7 +337,7 @@ void IMergeTreeDataPartWriter::finishPrimaryIndexSerialization(MergeTreeData::Da } } -void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( +void MergeTreeDataPartWriterOnDisk::finishSkipIndicesSerialization( MergeTreeData::DataPart::Checksums & checksums) { for (size_t i = 0; i < skip_indices.size(); ++i) @@ -361,10 +358,4 @@ void IMergeTreeDataPartWriter::finishSkipIndicesSerialization( skip_index_filling.clear(); } -void IMergeTreeDataPartWriter::next() -{ - current_mark = next_mark; - index_offset = next_index_offset; -} - } diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h similarity index 68% rename from src/Storages/MergeTree/IMergeTreeDataPartWriter.h rename to src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 3e3496c88da..a5df9d4a389 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -16,7 +17,7 @@ namespace DB /// Writes data part to disk in different formats. /// Calculates and serializes primary and skip indices if needed. -class IMergeTreeDataPartWriter : private boost::noncopyable +class MergeTreeDataPartWriterOnDisk : public IMergeTreeDataPartWriter { public: using WrittenOffsetColumns = std::set; @@ -60,7 +61,7 @@ public: using StreamPtr = std::unique_ptr; - IMergeTreeDataPartWriter( + MergeTreeDataPartWriterOnDisk( DiskPtr disk, const String & part_path, const MergeTreeData & storage, @@ -72,75 +73,36 @@ public: const MergeTreeIndexGranularity & index_granularity, bool need_finish_last_granule); - virtual ~IMergeTreeDataPartWriter(); - - virtual void write( - const Block & block, const IColumn::Permutation * permutation = nullptr, - /* Blocks with already sorted index columns */ - const Block & primary_key_block = {}, const Block & skip_indexes_block = {}) = 0; - - void calculateAndSerializePrimaryIndex(const Block & primary_index_block, size_t rows); - void calculateAndSerializeSkipIndices(const Block & skip_indexes_block, size_t rows); - - /// Shift mark and offset to prepare read next mark. - /// You must call it after calling write method and optionally - /// calling calculations of primary and skip indices. - void next(); + void calculateAndSerializePrimaryIndex(const Block & primary_index_block) final; + void calculateAndSerializeSkipIndices(const Block & skip_indexes_block) final; /// Count index_granularity for block and store in `index_granularity` - void fillIndexGranularity(const Block & block); + void fillIndexGranularity(const Block & block) final; - const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } + void initSkipIndices() final; + void initPrimaryIndex() final; - Columns releaseIndexColumns() - { - return Columns(std::make_move_iterator(index_columns.begin()), std::make_move_iterator(index_columns.end())); - } + virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) final; + virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums) final; void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_) { written_offset_columns = written_offset_columns_; } - const MergeTreeIndices & getSkipIndices() { return skip_indices; } - - void initSkipIndices(); - void initPrimaryIndex(); - - virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0; - void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums); - void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums); - protected: using SerializationState = IDataType::SerializeBinaryBulkStatePtr; using SerializationStates = std::unordered_map; DiskPtr disk; String part_path; - const MergeTreeData & storage; - NamesAndTypesList columns_list; const String marks_file_extension; - - MergeTreeIndexGranularity index_granularity; - CompressionCodecPtr default_codec; - MergeTreeIndices skip_indices; - - MergeTreeWriterSettings settings; - bool compute_granularity; bool with_final_mark; bool need_finish_last_granule; - size_t current_mark = 0; - - /// The offset to the first row of the block for which you want to write the index. - size_t index_offset = 0; - - size_t next_mark = 0; - size_t next_index_offset = 0; - /// Number of marsk in data from which skip indices have to start /// aggregation. I.e. it's data mark number, not skip indices mark. size_t skip_index_data_mark = 0; @@ -151,7 +113,6 @@ protected: std::unique_ptr index_file_stream; std::unique_ptr index_stream; - MutableColumns index_columns; DataTypes index_types; /// Index columns values from the last row from the last block /// It's written to index file in the `writeSuffixAndFinalizePart` method diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 1e5640b4e23..b953bc9d31e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -22,7 +22,7 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) - : IMergeTreeDataPartWriter(disk_, part_path_, + : MergeTreeDataPartWriterOnDisk(disk_, part_path_, storage_, columns_list_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_, false) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 4e4f4806d53..a9cdf4ce8e9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -1,10 +1,10 @@ -#include +#include namespace DB { /// Writes data part in wide format. -class MergeTreeDataPartWriterWide : public IMergeTreeDataPartWriter +class MergeTreeDataPartWriterWide : public MergeTreeDataPartWriterOnDisk { public: diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 2b482ac7c29..f10c1b8d533 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -97,6 +97,24 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( else part_columns = *total_columns_list; + if (new_part->isStoredOnDisk()) + finalizePartOnDisk(new_part, part_columns, checksums); + + new_part->setColumns(part_columns); + new_part->rows_count = rows_count; + new_part->modification_time = time(nullptr); + new_part->index = writer->releaseIndexColumns(); + new_part->checksums = checksums; + new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); + new_part->index_granularity = writer->getIndexGranularity(); + new_part->calculateColumnsSizesOnDisk(); +} + +void MergedBlockOutputStream::finalizePartOnDisk( + const MergeTreeData::MutableDataPartPtr & new_part, + NamesAndTypesList & part_columns, + MergeTreeData::DataPart::Checksums & checksums) +{ if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) { new_part->partition.store(storage, disk, part_path, checksums); @@ -137,15 +155,6 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( auto out = disk->writeFile(part_path + "checksums.txt", 4096); checksums.write(*out); } - - new_part->setColumns(part_columns); - new_part->rows_count = rows_count; - new_part->modification_time = time(nullptr); - new_part->index = writer->releaseIndexColumns(); - new_part->checksums = checksums; - new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); - new_part->index_granularity = writer->getIndexGranularity(); - new_part->calculateColumnsSizesOnDisk(); } void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Permutation * permutation) @@ -165,8 +174,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm Block skip_indexes_block = getBlockAndPermute(block, skip_indexes_column_names, permutation); writer->write(block, permutation, primary_key_block, skip_indexes_block); - writer->calculateAndSerializeSkipIndices(skip_indexes_block, rows); - writer->calculateAndSerializePrimaryIndex(primary_key_block, rows); + writer->calculateAndSerializeSkipIndices(skip_indexes_block); + writer->calculateAndSerializePrimaryIndex(primary_key_block); writer->next(); rows_count += rows; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 5a92977640e..8abc02e55e0 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -53,6 +53,11 @@ private: */ void writeImpl(const Block & block, const IColumn::Permutation * permutation); + void finalizePartOnDisk( + const MergeTreeData::MutableDataPartPtr & new_part, + NamesAndTypesList & part_columns, + MergeTreeData::DataPart::Checksums & checksums); + private: NamesAndTypesList columns_list; diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 892b4eccfbc..f37bde33083 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -30,8 +31,12 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( std::move(writer_settings), index_granularity); - writer->setWrittenOffsetColumns(offset_columns_); - writer->initSkipIndices(); + auto * writer_on_disk = dynamic_cast(writer.get()); + if (!writer_on_disk) + throw Exception("MergedColumnOnlyOutputStream supports only parts stored on disk", ErrorCodes::NOT_IMPLEMENTED); + + writer_on_disk->setWrittenOffsetColumns(offset_columns_); + writer_on_disk->initSkipIndices(); } void MergedColumnOnlyOutputStream::write(const Block & block) @@ -44,12 +49,11 @@ void MergedColumnOnlyOutputStream::write(const Block & block) Block skip_indexes_block = getBlockAndPermute(block, skip_indexes_column_names, nullptr); - size_t rows = block.rows(); - if (!rows) + if (!block.rows()) return; writer->write(block); - writer->calculateAndSerializeSkipIndices(skip_indexes_block, rows); + writer->calculateAndSerializeSkipIndices(skip_indexes_block); writer->next(); } From ee4eb97c3fc3425e060853425d5008b937985a23 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 14 Apr 2020 04:27:27 +0300 Subject: [PATCH 002/196] in-memory parts: preparation --- .../MergeTree/IMergeTreeDataPartWriter.cpp | 38 +++++++++ .../MergeTree/IMergeTreeDataPartWriter.h | 78 +++++++++++++++++++ 2 files changed, 116 insertions(+) create mode 100644 src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp create mode 100644 src/Storages/MergeTree/IMergeTreeDataPartWriter.h diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp new file mode 100644 index 00000000000..c5e20a8a95a --- /dev/null +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -0,0 +1,38 @@ +#include + +namespace DB +{ + +IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( + const MergeTreeData & storage_, + const MergeTreeWriterSettings & settings_) + : storage(storage_), settings(settings_) {} + +IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( + const MergeTreeData & storage_, + const NamesAndTypesList & columns_list_, + const MergeTreeIndices & skip_indices_, + const MergeTreeIndexGranularity & index_granularity_, + const MergeTreeWriterSettings & settings_) + : storage(storage_) + , columns_list(columns_list_) + , skip_indices(skip_indices_) + , index_granularity(index_granularity_) + , settings(settings_) {} + +Columns IMergeTreeDataPartWriter::releaseIndexColumns() +{ + return Columns( + std::make_move_iterator(index_columns.begin()), + std::make_move_iterator(index_columns.end())); +} + +void IMergeTreeDataPartWriter::next() +{ + current_mark = next_mark; + index_offset = next_index_offset; +} + +IMergeTreeDataPartWriter::~IMergeTreeDataPartWriter() = default; + +} diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h new file mode 100644 index 00000000000..1cab6ae4b60 --- /dev/null +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -0,0 +1,78 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +/// Writes data part to disk in different formats. +/// Calculates and serializes primary and skip indices if needed. +class IMergeTreeDataPartWriter : private boost::noncopyable +{ +public: + IMergeTreeDataPartWriter(const MergeTreeData & storage_, + const MergeTreeWriterSettings & settings_); + + IMergeTreeDataPartWriter(const MergeTreeData & storage_, + const NamesAndTypesList & columns_list_, + const MergeTreeIndices & skip_indices_, + const MergeTreeIndexGranularity & index_granularity_, + const MergeTreeWriterSettings & settings_); + + virtual ~IMergeTreeDataPartWriter(); + + virtual void write( + const Block & block, const IColumn::Permutation * permutation = nullptr, + /* Blocks with already sorted index columns */ + const Block & primary_key_block = {}, const Block & skip_indexes_block = {}) = 0; + + virtual void calculateAndSerializePrimaryIndex(const Block & /* primary_index_block */) {} + virtual void calculateAndSerializeSkipIndices(const Block & /* skip_indexes_block */) {} + + /// Shift mark and offset to prepare read next mark. + /// You must call it after calling write method and optionally + /// calling calculations of primary and skip indices. + void next(); + + /// Count index_granularity for block and store in `index_granularity` + virtual void fillIndexGranularity(const Block & /* block */) {} + + virtual void initSkipIndices() {} + virtual void initPrimaryIndex() {} + + virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync = false) = 0; + virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */) {} + virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & /* checksums */) {} + + Columns releaseIndexColumns(); + const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } + const MergeTreeIndices & getSkipIndices() { return skip_indices; } + +protected: + const MergeTreeData & storage; + NamesAndTypesList columns_list; + MergeTreeIndices skip_indices; + MergeTreeIndexGranularity index_granularity; + MergeTreeWriterSettings settings; + + size_t current_mark = 0; + + /// The offset to the first row of the block for which you want to write the index. + size_t index_offset = 0; + + size_t next_mark = 0; + size_t next_index_offset = 0; + + MutableColumns index_columns; +}; + +} From 391f7c34be97d56aab30ca7c7b6ecf882c93c015 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 14 Apr 2020 22:47:19 +0300 Subject: [PATCH 003/196] in memory parts: basic read/write --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- .../MergeTree/IMergeTreeDataPartWriter.cpp | 5 +- .../MergeTree/IMergeTreeDataPartWriter.h | 6 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 8 ++ src/Storages/MergeTree/IMergeTreeReader.h | 2 + .../MergeTree/IMergedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 73 ++++++++++-- src/Storages/MergeTree/MergeTreeData.h | 9 ++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 14 +-- .../MergeTree/MergeTreeDataPartCompact.cpp | 1 - .../MergeTree/MergeTreeDataPartInMemory.cpp | 72 ++++++++++++ .../MergeTree/MergeTreeDataPartInMemory.h | 62 ++++++++++ .../MergeTreeDataPartWriterInMemory.cpp | 108 ++++++++++++++++++ .../MergeTreeDataPartWriterInMemory.h | 29 +++++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 + .../MergeTree/MergeTreeDataWriter.cpp | 19 +-- src/Storages/MergeTree/MergeTreeIOSettings.h | 4 + .../MergeTreeIndexGranularityInfo.cpp | 4 + .../MergeTree/MergeTreeReaderCompact.cpp | 1 + .../MergeTree/MergeTreeReaderInMemory.cpp | 74 ++++++++++++ .../MergeTree/MergeTreeReaderInMemory.h | 34 ++++++ .../MergeTree/MergeTreeReaderWide.cpp | 7 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 3 +- .../MergeTree/MergeTreeSelectProcessor.h | 1 - src/Storages/MergeTree/MergeTreeSettings.h | 3 + ...rgeTreeThreadSelectBlockInputProcessor.cpp | 6 +- ...MergeTreeThreadSelectBlockInputProcessor.h | 2 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 104 +++++++++++++++++ .../MergeTree/MergeTreeWriteAheadLog.h | 56 +++++++++ .../MergeTree/MergedBlockOutputStream.cpp | 14 ++- .../MergeTree/MergedBlockOutputStream.h | 2 +- 32 files changed, 683 insertions(+), 50 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp create mode 100644 src/Storages/MergeTree/MergeTreeDataPartInMemory.h create mode 100644 src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp create mode 100644 src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h create mode 100644 src/Storages/MergeTree/MergeTreeReaderInMemory.cpp create mode 100644 src/Storages/MergeTree/MergeTreeReaderInMemory.h create mode 100644 src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp create mode 100644 src/Storages/MergeTree/MergeTreeWriteAheadLog.h diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5d799d257bc..01dd6d5da47 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -380,7 +380,7 @@ String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize() const String IMergeTreeDataPart::getFullPath() const { - assertOnDisk(); + // assertOnDisk(); //TODO if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); @@ -390,7 +390,7 @@ String IMergeTreeDataPart::getFullPath() const String IMergeTreeDataPart::getFullRelativePath() const { - assertOnDisk(); + // assertOnDisk(); //TODO if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 784a3ff047b..32c5e6737b0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -284,7 +284,7 @@ public: size_t getFileSizeOrZero(const String & file_name) const; String getFullRelativePath() const; String getFullPath() const; - void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = false) const; + virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = false) const; void renameToDetached(const String & prefix) const; void makeCloneInDetached(const String & prefix) const; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index c5e20a8a95a..2d35b9ff723 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -5,8 +5,11 @@ namespace DB IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeData & storage_, + const NamesAndTypesList & columns_list_, const MergeTreeWriterSettings & settings_) - : storage(storage_), settings(settings_) {} + : storage(storage_) + , columns_list(columns_list_) + , settings(settings_) {} IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeData & storage_, diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 1cab6ae4b60..f745c47d5b4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -19,7 +19,9 @@ namespace DB class IMergeTreeDataPartWriter : private boost::noncopyable { public: - IMergeTreeDataPartWriter(const MergeTreeData & storage_, + IMergeTreeDataPartWriter( + const MergeTreeData & storage_, + const NamesAndTypesList & columns_list_, const MergeTreeWriterSettings & settings_); IMergeTreeDataPartWriter(const MergeTreeData & storage_, @@ -49,7 +51,7 @@ public: virtual void initSkipIndices() {} virtual void initPrimaryIndex() {} - virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync = false) = 0; + virtual void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) = 0; virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & /* checksums */) {} virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & /* checksums */) {} diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 8243983d837..a2984421c2a 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -248,4 +248,12 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) } } +void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) +{ + if (num_columns_to_read != columns.size()) + throw Exception("invalid number of columns passed to MergeTreeReader::readRows. " + "Expected " + toString(columns.size()) + ", " + "got " + toString(num_columns_to_read), ErrorCodes::LOGICAL_ERROR); +} + } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 02d8f67f9d0..79f7860d1cc 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -61,6 +61,8 @@ protected: /// Returns actual column type in part, which can differ from table metadata. NameAndTypePair getColumnFromPart(const NameAndTypePair & required_column) const; + void checkNumberOfColumns(size_t columns_num_to_read); + /// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size. ValueSizeMap avg_value_size_hints; /// Stores states for IDataType::deserializeBinaryBulk diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index c016ec325da..95db9010d3f 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -10,7 +10,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( const MergeTreeDataPartPtr & data_part) : storage(data_part->storage) , disk(data_part->disk) - , part_path(data_part->getFullRelativePath()) + , part_path(data_part->isStoredOnDisk() ? data_part->getFullRelativePath() : "") { } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index aaacea6f1e7..fa1097c125f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -248,6 +249,12 @@ MergeTreeData::MergeTreeData( String reason; if (!canUsePolymorphicParts(*settings, &reason) && !reason.empty()) LOG_WARNING(log, reason + " Settings 'min_bytes_for_wide_part' and 'min_bytes_for_wide_part' will be ignored."); + + if (settings->in_memory_parts_enable_wal) + { + auto disk = reserveSpace(0)->getDisk(); + write_ahead_log = std::make_shared(*this, disk); + } } @@ -894,17 +901,21 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) continue; part_names_with_disks.emplace_back(it->name(), disk_ptr); + + if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) + loadDataPartsFromWAL(disk_ptr, it->name()); } } auto part_lock = lockParts(); - data_parts_indexes.clear(); + // TODO: fix. + // data_parts_indexes.clear(); - if (part_names_with_disks.empty()) - { - LOG_DEBUG(log, "There is no data parts"); - return; - } + // if (part_names_with_disks.empty()) + // { + // LOG_DEBUG(log, "There is no data parts"); + // return; + // } /// Parallel loading of data parts. size_t num_threads = std::min(size_t(settings->max_part_loading_threads), part_names_with_disks.size()); @@ -1106,6 +1117,21 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) LOG_DEBUG(log, "Loaded data parts (" << data_parts_indexes.size() << " items)"); } +void MergeTreeData::loadDataPartsFromWAL(const DiskPtr & disk, const String & file_name) +{ + MergeTreeWriteAheadLog wal(*this, disk, file_name); + auto parts = wal.restore(); + for (auto & part : parts) + { + part->modification_time = time(nullptr); + /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later + part->state = DataPartState::Committed; + + if (!data_parts_indexes.insert(part).second) + throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); + } +} + /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. @@ -1544,6 +1570,21 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S } MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const +{ + if (!canUseAdaptiveGranularity()) + return MergeTreeDataPartType::WIDE; + + const auto settings = getSettings(); + if (bytes_uncompressed < settings->min_bytes_for_compact_part || rows_count < settings->min_rows_for_compact_part) + return MergeTreeDataPartType::IN_MEMORY; + + if (bytes_uncompressed < settings->min_bytes_for_wide_part || rows_count < settings->min_rows_for_wide_part) + return MergeTreeDataPartType::COMPACT; + + return MergeTreeDataPartType::WIDE; +} + +MergeTreeDataPartType MergeTreeData::choosePartTypeOnDisk(size_t bytes_uncompressed, size_t rows_count) const { if (!canUseAdaptiveGranularity()) return MergeTreeDataPartType::WIDE; @@ -1564,8 +1605,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart(const String & name, return std::make_shared(*this, name, part_info, disk, relative_path); else if (type == MergeTreeDataPartType::WIDE) return std::make_shared(*this, name, part_info, disk, relative_path); + else if (type == MergeTreeDataPartType::IN_MEMORY) + return std::make_shared(*this, name, part_info, disk, relative_path); else - throw Exception("Unknown type in part " + relative_path, ErrorCodes::UNKNOWN_PART_TYPE); + throw Exception("Unknown type of part " + relative_path, ErrorCodes::UNKNOWN_PART_TYPE); } static MergeTreeDataPartType getPartTypeFromMarkExtension(const String & mrk_ext) @@ -1876,6 +1919,13 @@ void MergeTreeData::renameTempPartAndReplace( addPartContributionToColumnSizes(part); } + auto * part_in_memory = dynamic_cast(part.get()); + if (part_in_memory && getSettings()->in_memory_parts_enable_wal) + { + auto wal = getWriteAheadLog(); + wal->write(part_in_memory->block, part_in_memory->name); + } + if (out_covered_parts) { for (DataPartPtr & covered_part : covered_parts) @@ -2699,6 +2749,8 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartS } } + LOG_DEBUG(log, "MergeTreeData::getDataPartsVector: " << res.size()); + return res; } @@ -3616,4 +3668,11 @@ MergeTreeData::AlterConversions MergeTreeData::getAlterConversionsForPart(const return result; } + +MergeTreeData::WriteAheadLogPtr MergeTreeData::getWriteAheadLog() const +{ + // std::lock_guard lock(wal_mutex); + return write_ahead_log; +} + } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d299d39726e..243156dab94 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -190,6 +191,7 @@ public: DataPartsLock lockParts() const { return DataPartsLock(data_parts_mutex); } MergeTreeDataPartType choosePartType(size_t bytes_uncompressed, size_t rows_count) const; + MergeTreeDataPartType choosePartTypeOnDisk(size_t bytes_uncompressed, size_t rows_count) const; /// After this method setColumns must be called MutableDataPartPtr createPart(const String & name, @@ -389,6 +391,7 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); + void loadDataPartsFromWAL(const DiskPtr & disk, const String & file_name); String getLogName() const { return log_name; } @@ -659,6 +662,9 @@ public: /// Return alter conversions for part which must be applied on fly. AlterConversions getAlterConversionsForPart(const MergeTreeDataPartPtr part) const; + using WriteAheadLogPtr = std::shared_ptr; + WriteAheadLogPtr getWriteAheadLog() const; + MergeTreeDataFormatVersion format_version; Context & global_context; @@ -957,6 +963,9 @@ private: CurrentlyMovingPartsTagger checkPartsForMove(const DataPartsVector & parts, SpacePtr space); bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason); + + WriteAheadLogPtr write_ahead_log; + // mutable std::mutex wal_mutex; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8bc871476ed..533378d4af6 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -84,7 +84,7 @@ void FutureMergedMutatedPart::assign(MergeTreeData::DataPartsVector parts_) sum_bytes_uncompressed += part->getTotalColumnsSize().data_uncompressed; } - auto future_part_type = parts_.front()->storage.choosePartType(sum_bytes_uncompressed, sum_rows); + auto future_part_type = parts_.front()->storage.choosePartTypeOnDisk(sum_bytes_uncompressed, sum_rows); assign(std::move(parts_), future_part_type); } @@ -1039,7 +1039,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor need_remove_expired_values = true; /// All columns from part are changed and may be some more that were missing before in part - if (isCompactPart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList())) + if (!isWidePart(source_part) || source_part->getColumns().isSubsetOf(updated_header.getNamesAndTypesList())) { auto part_indices = getIndicesForNewDataPart(data.skip_indices, for_file_renames); mutateAllPartColumns( @@ -1231,7 +1231,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands( { NameSet removed_columns_from_compact_part; NameSet already_changed_columns; - bool is_compact_part = isCompactPart(part); + bool is_wide_part = isWidePart(part); for (const auto & command : commands) { if (command.type == MutationCommand::Type::DELETE @@ -1257,14 +1257,14 @@ void MergeTreeDataMergerMutator::splitMutationCommands( for_file_renames.push_back(command); } - else if (is_compact_part && command.type == MutationCommand::Type::DROP_COLUMN) + else if (!is_wide_part && command.type == MutationCommand::Type::DROP_COLUMN) { removed_columns_from_compact_part.emplace(command.column_name); for_file_renames.push_back(command); } else if (command.type == MutationCommand::Type::RENAME_COLUMN) { - if (is_compact_part) + if (!is_wide_part) { for_interpreter.push_back( { @@ -1282,7 +1282,7 @@ void MergeTreeDataMergerMutator::splitMutationCommands( } } - if (is_compact_part) + if (!is_wide_part) { /// If it's compact part than we don't need to actually remove files from disk /// we just don't read dropped columns @@ -1558,9 +1558,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( merge_entry->bytes_written_uncompressed += block.bytes(); } - new_data_part->minmax_idx = std::move(minmax_idx); - mutating_stream->readSuffix(); out.writeSuffixAndFinalizePart(new_data_part); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 24d65622fe6..6789b22f01a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -1,7 +1,6 @@ #include "MergeTreeDataPartCompact.h" #include #include -#include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp new file mode 100644 index 00000000000..c154ccef4fe --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -0,0 +1,72 @@ +#include "MergeTreeDataPartInMemory.h" +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + + +MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( + MergeTreeData & storage_, + const String & name_, + const DiskPtr & disk_, + const std::optional & relative_path_) + : IMergeTreeDataPart(storage_, name_, disk_, relative_path_, Type::IN_MEMORY) +{ +} + +MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( + const MergeTreeData & storage_, + const String & name_, + const MergeTreePartInfo & info_, + const DiskPtr & disk_, + const std::optional & relative_path_) + : IMergeTreeDataPart(storage_, name_, info_, disk_, relative_path_, Type::IN_MEMORY) +{ +} + +IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( + const NamesAndTypesList & columns_to_read, + const MarkRanges & mark_ranges, + UncompressedCache * /* uncompressed_cache */, + MarkCache * /* mark_cache */, + const MergeTreeReaderSettings & reader_settings, + const ValueSizeMap & /* avg_value_size_hints */, + const ReadBufferFromFileBase::ProfileCallback & /* profile_callback */) const +{ + auto ptr = std::static_pointer_cast(shared_from_this()); + return std::make_unique( + ptr, columns_to_read, mark_ranges, reader_settings); +} + +IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( + const NamesAndTypesList & columns_list, + const std::vector & /* indices_to_recalc */, + const CompressionCodecPtr & /* default_codec */, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & /* computed_index_granularity */) const +{ + auto ptr = std::static_pointer_cast(shared_from_this()); + return std::make_unique(ptr, columns_list, writer_settings); +} + + +void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName & /*each_columns_size*/, ColumnSize & /*total_size*/) const +{ + // throw Exception("calculateEachColumnSizesOnDisk of in memory part", ErrorCodes::NOT_IMPLEMENTED); +} + +void MergeTreeDataPartInMemory::loadIndexGranularity() +{ + throw Exception("loadIndexGranularity of in memory part", ErrorCodes::NOT_IMPLEMENTED); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h new file mode 100644 index 00000000000..9fe0c139626 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -0,0 +1,62 @@ +#pragma once + +#include + +namespace DB +{ + +class MergeTreeDataPartInMemory : public IMergeTreeDataPart +{ +public: + MergeTreeDataPartInMemory( + const MergeTreeData & storage_, + const String & name_, + const MergeTreePartInfo & info_, + const DiskPtr & disk_, + const std::optional & relative_path_ = {}); + + MergeTreeDataPartInMemory( + MergeTreeData & storage_, + const String & name_, + const DiskPtr & disk_, + const std::optional & relative_path_ = {}); + + MergeTreeReaderPtr getReader( + const NamesAndTypesList & columns, + const MarkRanges & mark_ranges, + UncompressedCache * uncompressed_cache, + MarkCache * mark_cache, + const MergeTreeReaderSettings & reader_settings_, + const ValueSizeMap & avg_value_size_hints, + const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; + + MergeTreeWriterPtr getWriter( + const NamesAndTypesList & columns_list, + const std::vector & indices_to_recalc, + const CompressionCodecPtr & default_codec_, + const MergeTreeWriterSettings & writer_settings, + const MergeTreeIndexGranularity & computed_index_granularity) const override; + + bool isStoredOnDisk() const override { return false; } + + bool hasColumnFiles(const String & /* column_name */, const IDataType & /* type */) const override { return true; } + + String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } + + void renameTo(const String & /*new_relative_path*/, bool /*remove_new_dir_if_exists*/) const override {} + + mutable Block block; + +private: + void checkConsistency(bool /* require_part_metadata */) const override {} + + /// Loads marks index granularity into memory + void loadIndexGranularity() override; + + /// Compact parts doesn't support per column size, only total size + void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; +}; + +using DataPartInMemoryPtr = std::shared_ptr; + +} diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp new file mode 100644 index 00000000000..3d0d67e1ed6 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -0,0 +1,108 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +MergeTreeDataPartWriterInMemory::MergeTreeDataPartWriterInMemory( + const DataPartInMemoryPtr & part_, + const NamesAndTypesList & columns_list_, + const MergeTreeWriterSettings & settings_) + : IMergeTreeDataPartWriter(part_->storage, columns_list_, settings_) + , part(part_) {} + +void MergeTreeDataPartWriterInMemory::write( + const Block & block, const IColumn::Permutation * permutation, + const Block & primary_key_block, const Block & /* skip_indexes_block */) +{ + if (block_written) + throw Exception("DataPartWriterInMemory supports only one write", ErrorCodes::LOGICAL_ERROR); + + Block result_block; + if (permutation) + { + for (const auto & it : columns_list) + { + if (primary_key_block.has(it.name)) + result_block.insert(primary_key_block.getByName(it.name)); + else + { + auto column = block.getByName(it.name); + column.column = column.column->permute(*permutation, 0); + result_block.insert(column); + } + } + } + else + { + result_block = block; + } + + part->block = std::move(result_block); + block_written = true; +} + +void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Block & primary_index_block) +{ + size_t rows = primary_index_block.rows(); + if (!rows) + return; + + index_granularity.appendMark(rows); + index_granularity.appendMark(0); + + size_t primary_columns_num = primary_index_block.columns(); + index_columns.resize(primary_columns_num); + for (size_t i = 0; i < primary_columns_num; ++i) + { + const auto & primary_column = *primary_index_block.getByPosition(i).column; + index_columns[i] = primary_column.cloneEmpty(); + index_columns[i]->insertFrom(primary_column, 0); + index_columns[i]->insertFrom(primary_column, rows - 1); + } +} + +static MergeTreeDataPartChecksum createUncompressedChecksum(size_t size, SipHash & hash) +{ + MergeTreeDataPartChecksum checksum; + checksum.uncompressed_size = size; + hash.get128(checksum.uncompressed_hash.first, checksum.uncompressed_hash.second); + return checksum; +} + +void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) +{ + UNUSED(checksums); + SipHash hash; + part->block.updateHash(hash); + checksums.files["data.bin"] = createUncompressedChecksum(part->block.bytes(), hash); +} + +void MergeTreeDataPartWriterInMemory::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) +{ + UNUSED(checksums); + if (index_columns.empty()) + return; + + SipHash hash; + size_t index_size = 0; + size_t rows = index_columns[0]->size(); + for (size_t i = 0; i < rows; ++i) + { + for (const auto & col : index_columns) + { + col->updateHashWithValue(i, hash); + index_size += col->byteSize(); + } + } + + checksums.files["primary.idx"] = createUncompressedChecksum(index_size, hash); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h new file mode 100644 index 00000000000..c9b57e5e4b6 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h @@ -0,0 +1,29 @@ +#include +#include + +namespace DB +{ + +/// Writes data part in memory. +class MergeTreeDataPartWriterInMemory : public IMergeTreeDataPartWriter +{ +public: + MergeTreeDataPartWriterInMemory( + const DataPartInMemoryPtr & part_, + const NamesAndTypesList & columns_list_, + const MergeTreeWriterSettings & settings_); + + void write(const Block & block, const IColumn::Permutation * permutation, + const Block & primary_key_block, const Block & skip_indexes_block) override; + + void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; + + void calculateAndSerializePrimaryIndex(const Block & primary_index_block) override; + void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) override; + +private: + DataPartInMemoryPtr part; + bool block_written = false; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 816af8db3e9..c468dc07583 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -170,6 +170,8 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( const unsigned num_streams, const PartitionIdToMaxBlock * max_block_numbers_to_read) const { + LOG_DEBUG(log, "readFromParts size: " << parts.size()); + size_t part_index = 0; /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 23a60ddab78..c508110a4f0 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -243,16 +243,19 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa new_data_part->minmax_idx = std::move(minmax_idx); new_data_part->is_temp = true; - /// The name could be non-unique in case of stale files from previous runs. - String full_path = new_data_part->getFullRelativePath(); - - if (new_data_part->disk->exists(full_path)) + if (new_data_part->isStoredOnDisk()) { - LOG_WARNING(log, "Removing old temporary directory " + fullPath(new_data_part->disk, full_path)); - new_data_part->disk->removeRecursive(full_path); - } + /// The name could be non-unique in case of stale files from previous runs. + String full_path = new_data_part->getFullRelativePath(); - new_data_part->disk->createDirectories(full_path); + if (new_data_part->disk->exists(full_path)) + { + LOG_WARNING(log, "Removing old temporary directory " + fullPath(new_data_part->disk, full_path)); + new_data_part->disk->removeRecursive(full_path); + } + + new_data_part->disk->createDirectories(full_path); + } /// If we need to calculate some columns to sort. if (data.hasSortingKey() || data.hasSkipIndices()) diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index f5c57659052..756ffc9f38c 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -17,6 +17,8 @@ struct MergeTreeReaderSettings struct MergeTreeWriterSettings { + MergeTreeWriterSettings() = default; + MergeTreeWriterSettings(const Settings & global_settings, bool can_use_adaptive_granularity_, size_t aio_threshold_, bool blocks_are_granules_size_ = false) : min_compress_block_size(global_settings.min_compress_block_size) @@ -31,6 +33,8 @@ struct MergeTreeWriterSettings bool can_use_adaptive_granularity; bool blocks_are_granules_size; + /// true if we write temporary files during alter. size_t estimated_size = 0; }; + } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index c481140cb84..ed0e7e55fc8 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -72,6 +72,8 @@ size_t MergeTreeIndexGranularityInfo::getMarkSizeInBytes(size_t columns_num) con return is_adaptive ? getAdaptiveMrkSizeWide() : getNonAdaptiveMrkSizeWide(); else if (type == MergeTreeDataPartType::COMPACT) return getAdaptiveMrkSizeCompact(columns_num); + else if (type == MergeTreeDataPartType::IN_MEMORY) + return 0; else throw Exception("Unknown part type", ErrorCodes::UNKNOWN_PART_TYPE); } @@ -88,6 +90,8 @@ std::string getAdaptiveMrkExtension(MergeTreeDataPartType part_type) return ".mrk2"; else if (part_type == MergeTreeDataPartType::COMPACT) return ".mrk3"; + else if (part_type == MergeTreeDataPartType::IN_MEMORY) + return ""; else throw Exception("Unknown part type", ErrorCodes::UNKNOWN_PART_TYPE); } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index a895149e12e..a63397b9b9c 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -102,6 +102,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, size_t read_rows = 0; size_t num_columns = columns.size(); + checkNumberOfColumns(num_columns); MutableColumns mutable_columns(num_columns); auto column_it = columns.begin(); diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp new file mode 100644 index 00000000000..8c61a879270 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -0,0 +1,74 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_READ_ALL_DATA; + extern const int ARGUMENT_OUT_OF_BOUND; +} + + +MergeTreeReaderInMemory::MergeTreeReaderInMemory( + DataPartInMemoryPtr data_part_, + NamesAndTypesList columns_, + MarkRanges mark_ranges_, + MergeTreeReaderSettings settings_) + : IMergeTreeReader(data_part_, std::move(columns_), + nullptr, nullptr, std::move(mark_ranges_), + std::move(settings_), {}) + , part_in_memory(std::move(data_part_)) +{ +} + +size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool /* continue_reading */, size_t max_rows_to_read, Columns & res_columns) +{ + size_t total_marks = data_part->index_granularity.getMarksCount(); + if (from_mark >= total_marks) + throw Exception("Mark " + toString(from_mark) + " is out of bound. Max mark: " + + toString(total_marks), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + size_t num_columns = res_columns.size(); + checkNumberOfColumns(num_columns); + + size_t part_rows = part_in_memory->block.rows(); + if (total_rows_read >= part_rows) + throw Exception("Cannot read data in MergeTreeReaderInMemory. Rows already read: " + + toString(total_rows_read) + ". Rows in part: " + toString(part_rows), ErrorCodes::CANNOT_READ_ALL_DATA); + + auto column_it = columns.begin(); + size_t rows_read = 0; + for (size_t i = 0; i < num_columns; ++i, ++column_it) + { + auto [name, type] = getColumnFromPart(*column_it); + if (!part_in_memory->block.has(name)) + continue; + + const auto block_column = part_in_memory->block.getByPosition(i).column; + if (total_rows_read == 0 && part_rows <= max_rows_to_read) + { + res_columns[i] = block_column; + rows_read = part_rows; + } + else + { + if (res_columns[i] == nullptr) + res_columns[i] = type->createColumn(); + + auto mutable_column = res_columns[i]->assumeMutable(); + rows_read = std::min(max_rows_to_read, part_rows - total_rows_read); + mutable_column->insertRangeFrom(*block_column, total_rows_read, rows_read); + res_columns[i] = std::move(mutable_column); + } + } + + total_rows_read += rows_read; + return rows_read; +} + +} diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h new file mode 100644 index 00000000000..6d64801682e --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class MergeTreeDataPartInMemory; +using DataPartInMemoryPtr = std::shared_ptr; + +/// Reader for InMemory parts +class MergeTreeReaderInMemory : public IMergeTreeReader +{ +public: + MergeTreeReaderInMemory( + DataPartInMemoryPtr data_part_, + NamesAndTypesList columns_, + MarkRanges mark_ranges_, + MergeTreeReaderSettings settings_); + + /// Return the number of rows has been read or zero if there is no columns to read. + /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark + size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override; + + bool canReadIncompleteGranules() const override { return true; } + +private: + size_t total_rows_read = 0; + DataPartInMemoryPtr part_in_memory; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 1a03acb5758..c15ed240b82 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -21,7 +21,6 @@ namespace namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int MEMORY_LIMIT_EXCEEDED; } @@ -61,11 +60,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si try { size_t num_columns = columns.size(); - - if (res_columns.size() != num_columns) - throw Exception("invalid number of columns passed to MergeTreeReader::readRows. " - "Expected " + toString(num_columns) + ", " - "got " + toString(res_columns.size()), ErrorCodes::LOGICAL_ERROR); + checkNumberOfColumns(num_columns); /// Pointers to offset columns that are common to the nested data structure columns. /// If append is true, then the value will be equal to nullptr and will be used only to diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index def01b192d5..4228ca2b472 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -36,8 +36,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( data_part{owned_data_part_}, all_mark_ranges(std::move(mark_ranges_)), part_index_in_query(part_index_in_query_), - check_columns(check_columns_), - path(data_part->getFullRelativePath()) + check_columns(check_columns_) { /// Let's estimate total number of rows for progress bar. for (const auto & range : all_mark_ranges) diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 4c64bfb6a18..c1c0cd6f782 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -64,7 +64,6 @@ private: size_t part_index_in_query = 0; bool check_columns; - String path; bool is_first_task = true; Logger * log = &Logger::get("MergeTreeSelectProcessor"); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 02c852b4f4b..f3f0e5a05d3 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -31,6 +31,9 @@ struct MergeTreeSettings : public SettingsCollection /** Data storing format settigns. */ \ M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ + M(SettingUInt64, min_bytes_for_compact_part, 0, "Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ + M(SettingUInt64, min_rows_for_compact_part, 0, "Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ + M(SettingBool, in_memory_parts_enable_wal, 0, "", 0) \ \ /** Merge settings. */ \ M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp index aa8c550839d..d8784843e56 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp @@ -57,7 +57,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() return false; } - const std::string path = task->data_part->getFullRelativePath(); + const std::string part_name = task->data_part->name; /// Allows pool to reduce number of threads in case of too slow reads. auto profile_callback = [this](ReadBufferFromFileBase::ProfileInfo info_) { pool->profileFeedback(info_); }; @@ -82,7 +82,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() else { /// in other case we can reuse readers, anyway they will be "seeked" to required mark - if (path != last_readed_part_path) + if (part_name != last_readed_part_name) { auto rest_mark_ranges = pool->getRestMarks(*task->data_part, task->mark_ranges[0]); /// retain avg_value_size_hints @@ -97,7 +97,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() } } - last_readed_part_path = path; + last_readed_part_name = part_name; return true; } diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h index e214696b705..01b227de19c 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h @@ -43,7 +43,7 @@ private: size_t min_marks_to_read; /// Last part readed in this thread - std::string last_readed_part_path; + std::string last_readed_part_name; /// Names from header. Used in order to order columns in read blocks. Names ordered_names; }; diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp new file mode 100644 index 00000000000..d874a10d3f2 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -0,0 +1,104 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_FORMAT_VERSION; +} + +// WALBlockOutputStream::WALBlockOutputStream(WriteBuffer & out_, const Block & header_) +// : NativeBlockOutputStream(out_, 0, header_), out(out_) {} + +// void WALBlockOutputStream::write(const Block & block, const String & part_name) +// { +// writeIntBinary(0, out); +// writeString(part_name, out); +// NativeBlockOutputStream::write(block); +// } + +MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( + const MergeTreeData & storage_, + const DiskPtr & disk_, + const String & name) + : storage(storage_) + , disk(disk_) + , path(storage.getFullPathOnDisk(disk) + name) + , out(disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) + , block_out(std::make_unique(*out, 0, storage.getSampleBlock())) {} + + +void MergeTreeWriteAheadLog::write(const Block & block, const String & part_name) +{ + std::lock_guard lock(write_mutex); + + auto part_info = MergeTreePartInfo::fromPartName(part_name, storage.format_version); + min_block_number = std::min(min_block_number, part_info.min_block); + max_block_number = std::max(max_block_number, part_info.max_block); + + writeIntBinary(static_cast(0), *out); /// version + writeStringBinary(part_name, *out); + block_out->write(block); + block_out->flush(); + + if (out->count() > MAX_WAL_BYTES) + rotate(lock); +} + +void MergeTreeWriteAheadLog::rotate(const std::lock_guard & /*write_lock*/) +{ + String new_name = String(WAL_FILE_NAME) + "_" + + toString(min_block_number) + "_" + + toString(max_block_number) + WAL_FILE_EXTENSION; + + Poco::File(path).renameTo(storage.getFullPathOnDisk(disk) + new_name); + out = disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); + block_out = std::make_unique(*out, 0, storage.getSampleBlock()); + min_block_number = std::numeric_limits::max(); + max_block_number = 0; +} + +MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() +{ + std::lock_guard lock(write_mutex); + + MergeTreeData::MutableDataPartsVector result; + auto in = disk->readFile(path, DBMS_DEFAULT_BUFFER_SIZE); + NativeBlockInputStream block_in(*in, 0); + + while (!in->eof()) + { + UInt8 version; + String part_name; + readIntBinary(version, *in); + if (version != 0) + throw Exception("Unknown WAL format version: " + toString(version), ErrorCodes::UNKNOWN_FORMAT_VERSION); + + readStringBinary(part_name, *in); + auto part = storage.createPart( + part_name, + MergeTreeDataPartType::IN_MEMORY, + MergeTreePartInfo::fromPartName(part_name, storage.format_version), + storage.reserveSpace(0)->getDisk(), + part_name); + + auto block = block_in.read(); + + part->minmax_idx.update(block, storage.minmax_idx_columns); + MergedBlockOutputStream part_out(part, block.getNamesAndTypesList(), {}, nullptr); + part_out.writePrefix(); + part_out.write(block); + part_out.writeSuffixAndFinalizePart(part); + + result.push_back(std::move(part)); + } + + return result; +} + +} diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h new file mode 100644 index 00000000000..2014fba18de --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -0,0 +1,56 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +// class WALBlockOutputStream : public NativeBlockOutputStream +// { +// public: +// WALBlockOutputStream(WriteBuffer & out_, const Block & header_); +// void write(const Block & block, const String & part_name); + +// private: +// WriteBuffer & out; +// }; + +// class WALBlockInputStream : public NativeBlockInputStream +// { +// }; + +class MergeTreeData; + +class MergeTreeWriteAheadLog +{ +public: + constexpr static auto WAL_FILE_NAME = "wal"; + constexpr static auto WAL_FILE_EXTENSION = ".bin"; + constexpr static size_t MAX_WAL_BYTES = 1024; + + MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, + const String & name = String(WAL_FILE_NAME) + WAL_FILE_EXTENSION); + + void write(const Block & block, const String & part_name); + std::vector> restore(); + +private: + void rotate(const std::lock_guard & write_lock); + + const MergeTreeData & storage; + DiskPtr disk; + String path; + + std::unique_ptr out; + std::unique_ptr block_out; + + Int64 min_block_number = std::numeric_limits::max(); + Int64 max_block_number = 0; + + mutable std::mutex write_mutex; +}; + +} diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index f10c1b8d533..0a39a66d7ba 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -36,8 +36,11 @@ MergedBlockOutputStream::MergedBlockOutputStream( : IMergedBlockOutputStream(data_part) , columns_list(columns_list_) { - MergeTreeWriterSettings writer_settings(data_part->storage.global_context.getSettings(), - data_part->storage.canUseAdaptiveGranularity(), aio_threshold, blocks_are_granules_size); + MergeTreeWriterSettings writer_settings( + storage.global_context.getSettings(), + storage.canUseAdaptiveGranularity(), + aio_threshold, + blocks_are_granules_size); if (aio_threshold > 0 && !merged_column_to_size.empty()) { @@ -49,7 +52,8 @@ MergedBlockOutputStream::MergedBlockOutputStream( } } - disk->createDirectories(part_path); + if (!part_path.empty()) + disk->createDirectories(part_path); writer = data_part->getWriter(columns_list, skip_indices, default_codec, writer_settings); writer->initPrimaryIndex(); @@ -107,7 +111,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( new_part->checksums = checksums; new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->index_granularity = writer->getIndexGranularity(); - new_part->calculateColumnsSizesOnDisk(); + // new_part->calculateColumnsSizesOnDisk(); // TODO: Fix } void MergedBlockOutputStream::finalizePartOnDisk( @@ -165,7 +169,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm return; std::unordered_set skip_indexes_column_names_set; - for (const auto & index : storage.skip_indices) + for (const auto & index : writer->getSkipIndices()) std::copy(index->columns.cbegin(), index->columns.cend(), std::inserter(skip_indexes_column_names_set, skip_indexes_column_names_set.end())); Names skip_indexes_column_names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end()); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 8abc02e55e0..9097f78b749 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -60,7 +60,7 @@ private: private: NamesAndTypesList columns_list; - + IMergeTreeDataPart::MinMaxIndex minmax_idx; size_t rows_count = 0; }; From 4069dbcc58f86e25c0e5ea4b255dc5463a12d5c4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 20 Apr 2020 04:38:38 +0300 Subject: [PATCH 004/196] in-memory parts: add waiting for insert --- src/Core/Settings.h | 2 ++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 4 ++++ .../MergeTree/MergeTreeBlockOutputStream.cpp | 12 ++++++++++++ src/Storages/MergeTree/MergeTreeBlockOutputStream.h | 5 +++-- src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp | 11 +++++++++++ src/Storages/MergeTree/MergeTreeDataPartInMemory.h | 5 +++++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + src/Storages/StorageMergeTree.cpp | 9 ++++++++- 10 files changed, 52 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 325abc16f3f..b38a05e73ea 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -89,6 +89,8 @@ struct Settings : public SettingsCollection \ M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \ \ + M(SettingMilliseconds, insert_in_memory_parts_timeout, 600000, "", 0) \ + \ M(SettingUInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) \ M(SettingUInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \ \ diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 01dd6d5da47..57bc040ab45 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -914,4 +914,9 @@ bool isWidePart(const MergeTreeDataPartPtr & data_part) return (data_part && data_part->getType() == MergeTreeDataPartType::WIDE); } +bool isInMemoryPart(const MergeTreeDataPartPtr & data_part) +{ + return (data_part && data_part->getType() == MergeTreeDataPartType::IN_MEMORY); +} + } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 32c5e6737b0..b1fb2554c76 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -91,6 +91,9 @@ public: virtual bool supportsVerticalMerge() const { return false; } + virtual bool waitUntilMerged(size_t /* timeout */) const { return true; } + virtual void notifyMerged() const {} + /// NOTE: Returns zeros if column files are not found in checksums. /// Otherwise return information about column size on disk. ColumnSize getColumnSize(const String & column_name, const IDataType & /* type */) const; @@ -354,5 +357,6 @@ using MergeTreeDataPartPtr = std::shared_ptr; bool isCompactPart(const MergeTreeDataPartPtr & data_part); bool isWidePart(const MergeTreeDataPartPtr & data_part); +bool isInMemoryPart(const MergeTreeDataPartPtr & data_part); } diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index be3caf98ad4..4f9500f973e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int TIMEOUT_EXCEEDED; +} + Block MergeTreeBlockOutputStream::getHeader() const { return storage.getSampleBlock(); @@ -26,6 +31,13 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); + if (isInMemoryPart(part) && storage.getSettings()->in_memory_parts_insert_sync) + { + if (!part->waitUntilMerged(in_memory_parts_timeout)) + throw Exception("Timeout exceeded while waiting to write part " + + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); + } + /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. if (storage.merging_mutating_task_handle) storage.merging_mutating_task_handle->wake(); diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h index 8f957d631d3..d91794bc50e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -13,8 +13,8 @@ class StorageMergeTree; class MergeTreeBlockOutputStream : public IBlockOutputStream { public: - MergeTreeBlockOutputStream(StorageMergeTree & storage_, size_t max_parts_per_block_) - : storage(storage_), max_parts_per_block(max_parts_per_block_) {} + MergeTreeBlockOutputStream(StorageMergeTree & storage_, size_t max_parts_per_block_, size_t in_memory_parts_timeout_) + : storage(storage_), max_parts_per_block(max_parts_per_block_), in_memory_parts_timeout(in_memory_parts_timeout_) {} Block getHeader() const override; void write(const Block & block) override; @@ -22,6 +22,7 @@ public: private: StorageMergeTree & storage; size_t max_parts_per_block; + size_t in_memory_parts_timeout; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 243156dab94..387647c5db3 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index c154ccef4fe..41b35757ed8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -58,6 +58,17 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( return std::make_unique(ptr, columns_list, writer_settings); } +bool MergeTreeDataPartInMemory::waitUntilMerged(size_t timeout) const +{ + auto lock = storage.lockParts(); + return is_merged.wait_for(lock, std::chrono::milliseconds(timeout), + [this]() { return state == State::Outdated; }); +} + +void MergeTreeDataPartInMemory::notifyMerged() const +{ + is_merged.notify_one(); +} void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName & /*each_columns_size*/, ColumnSize & /*total_size*/) const { diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 9fe0c139626..29c01805529 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -45,9 +45,14 @@ public: void renameTo(const String & /*new_relative_path*/, bool /*remove_new_dir_if_exists*/) const override {} + bool waitUntilMerged(size_t timeout) const override; + void notifyMerged() const override; + mutable Block block; private: + mutable std::condition_variable is_merged; + void checkConsistency(bool /* require_part_metadata */) const override {} /// Loads marks index granularity into memory diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index f3f0e5a05d3..5544d267f89 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -34,6 +34,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, min_bytes_for_compact_part, 0, "Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ M(SettingUInt64, min_rows_for_compact_part, 0, "Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(SettingBool, in_memory_parts_enable_wal, 0, "", 0) \ + M(SettingBool, in_memory_parts_insert_sync, 0, "", 0) \ \ /** Merge settings. */ \ M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 1aac6717728..3a2bdf83903 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -158,7 +159,10 @@ std::optional StorageMergeTree::totalBytes() const BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Context & context) { - return std::make_shared(*this, context.getSettingsRef().max_partitions_per_insert_block); + const auto & settings = context.getSettingsRef(); + return std::make_shared( + *this, settings.max_partitions_per_insert_block, + settings.insert_in_memory_parts_timeout.totalMilliseconds()); } void StorageMergeTree::checkTableCanBeDropped() const @@ -630,6 +634,9 @@ bool StorageMergeTree::merge( throw; } + for (const auto & part : future_part.parts) + part->notifyMerged(); + return true; } From 1789d6fa8213761d8309c925ea67ea291c8f0230 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Apr 2020 02:07:11 +0300 Subject: [PATCH 005/196] add a test just in case --- .../test_polymorphic_parts/test.py | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index f7256de9d9a..2382dba863f 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -52,6 +52,7 @@ node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) settings_default = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0} +settings_compact_only = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 1000000, 'min_bytes_for_wide_part' : 0} settings_not_adaptive = {'index_granularity' : 64, 'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0} node3 = cluster.add_instance('node3', config_dir="configs", with_zookeeper=True) @@ -69,6 +70,7 @@ def start_cluster(): cluster.start() create_tables('polymorphic_table', [node1, node2], [settings_default, settings_default], "shard1") + create_tables('compact_parts_only', [node1, node2], [settings_compact_only, settings_compact_only], "shard1") create_tables('non_adaptive_table', [node1, node2], [settings_not_adaptive, settings_default], "shard1") create_tables('polymorphic_table_compact', [node3, node4], [settings_compact, settings_wide], "shard2") create_tables('polymorphic_table_wide', [node3, node4], [settings_wide, settings_compact], "shard2") @@ -138,6 +140,31 @@ def test_polymorphic_parts_basics(start_cluster, first_node, second_node): second_node.query("SELECT count(ss) FROM polymorphic_table") == "2000\n" second_node.query("SELECT uniqExact(ss) FROM polymorphic_table") == "600\n" +# Checks mostly that merge from compact part to compact part works. +def test_compact_parts_only(start_cluster): + for i in range(20): + insert_random_data('compact_parts_only', node1, 100) + insert_random_data('compact_parts_only', node2, 100) + + node1.query("SYSTEM SYNC REPLICA compact_parts_only", timeout=20) + node2.query("SYSTEM SYNC REPLICA compact_parts_only", timeout=20) + + assert node1.query("SELECT count() FROM compact_parts_only") == "4000\n" + assert node2.query("SELECT count() FROM compact_parts_only") == "4000\n" + + assert node1.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'compact_parts_only' AND active") == "Compact\n" + assert node2.query("SELECT DISTINCT part_type FROM system.parts WHERE table = 'compact_parts_only' AND active") == "Compact\n" + + node1.query("OPTIMIZE TABLE compact_parts_only FINAL") + node2.query("SYSTEM SYNC REPLICA compact_parts_only", timeout=20) + assert node2.query("SELECT count() FROM compact_parts_only") == "4000\n" + + expected = "Compact\t1\n" + assert TSV(node1.query("SELECT part_type, count() FROM system.parts " \ + "WHERE table = 'compact_parts_only' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) + assert TSV(node2.query("SELECT part_type, count() FROM system.parts " \ + "WHERE table = 'compact_parts_only' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) + # Check that follower replicas create parts of the same type, which leader has chosen at merge. @pytest.mark.parametrize( From 42997bce868ee1d75cc1d63d24d43a603a967064 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Apr 2020 20:14:49 +0300 Subject: [PATCH 006/196] im-memory parts: replication --- src/Storages/MergeTree/DataPartsExchange.cpp | 154 +++++++++++++----- src/Storages/MergeTree/DataPartsExchange.h | 10 +- .../MergeTreeDataPartWriterOnDisk.cpp | 1 + .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../MergeTree/MergeTreeWriteAheadLog.h | 2 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 12 +- src/Storages/StorageReplicatedMergeTree.h | 2 +- src/Storages/System/StorageSystemParts.cpp | 14 +- .../test_polymorphic_parts/test.py | 50 ++++-- 10 files changed, 178 insertions(+), 70 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index c656fbf0c58..1ecffdef5e1 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -1,6 +1,9 @@ #include +#include +#include #include #include +#include #include #include #include @@ -54,6 +57,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo int client_protocol_version = parse(params.get("client_protocol_version", "0")); String part_name = params.get("part"); + String part_type = params.get("part_type", "Wide"); // TODO: correct type with old versions const auto data_settings = data.getSettings(); @@ -84,24 +88,16 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo LOG_TRACE(log, "Sending part " << part_name); try - { + { auto storage_lock = data.lockStructureForShare( - false, RWLockImpl::NO_QUERY, data.getSettings()->lock_acquire_timeout_for_background_operations); + false, RWLockImpl::NO_QUERY, data.getSettings()->lock_acquire_timeout_for_background_operations); - MergeTreeData::DataPartPtr part = findPart(part_name); + auto part = findPart(part_name); CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedSend}; - /// We'll take a list of files from the list of checksums. - MergeTreeData::DataPart::Checksums checksums = part->checksums; - /// Add files that are not in the checksum list. - checksums.files["checksums.txt"]; - checksums.files["columns.txt"]; - - MergeTreeData::DataPart::Checksums data_checksums; - if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) - writeBinary(checksums.getTotalSizeOnDisk(), out); + writeBinary(part->checksums.getTotalSizeOnDisk(), out); if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS) { @@ -110,37 +106,10 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo writeBinary(ttl_infos_buffer.str(), out); } - writeBinary(checksums.files.size(), out); - for (const auto & it : checksums.files) - { - String file_name = it.first; - - auto disk = part->disk; - String path = part->getFullRelativePath() + file_name; - - UInt64 size = disk->getFileSize(path); - - writeStringBinary(it.first, out); - writeBinary(size, out); - - auto file_in = disk->readFile(path); - HashingWriteBuffer hashing_out(out); - copyData(*file_in, hashing_out, blocker.getCounter()); - - if (blocker.isCancelled()) - throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); - - if (hashing_out.count() != size) - throw Exception("Unexpected size of file " + path, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); - - writePODBinary(hashing_out.getHash(), out); - - if (file_name != "checksums.txt" && - file_name != "columns.txt") - data_checksums.addFile(file_name, hashing_out.count(), hashing_out.getHash()); - } - - part->checksums.checkEqual(data_checksums, false); + if (part_type == "InMemory") + sendPartFromMemory(part, out, storage_lock); + else + sendPartFromDisk(part, out, storage_lock); } catch (const NetException &) { @@ -160,6 +129,61 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo } } +void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, TableStructureReadLockHolder &) +{ + auto part_in_memory = dynamic_cast(part.get()); + if (!part_in_memory) + throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::NO_SUCH_DATA_PART); // TODO error code + + NativeBlockOutputStream block_out(out, 0, data.getSampleBlock()); + block_out.write(part_in_memory->block); + + // TODO send checksums +} + +void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, TableStructureReadLockHolder &) +{ + /// We'll take a list of files from the list of checksums. + MergeTreeData::DataPart::Checksums checksums = part->checksums; + /// Add files that are not in the checksum list. + checksums.files["checksums.txt"]; + checksums.files["columns.txt"]; + + MergeTreeData::DataPart::Checksums data_checksums; + + writeBinary(checksums.files.size(), out); + for (const auto & it : checksums.files) + { + String file_name = it.first; + + auto disk = part->disk; + String path = part->getFullRelativePath() + file_name; + + UInt64 size = disk->getFileSize(path); + + writeStringBinary(it.first, out); + writeBinary(size, out); + + auto file_in = disk->readFile(path); + HashingWriteBuffer hashing_out(out); + copyData(*file_in, hashing_out, blocker.getCounter()); + + if (blocker.isCancelled()) + throw Exception("Transferring part to replica was cancelled", ErrorCodes::ABORTED); + + if (hashing_out.count() != size) + throw Exception("Unexpected size of file " + path, ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART); + + writePODBinary(hashing_out.getHash(), out); + + if (file_name != "checksums.txt" && + file_name != "columns.txt") + data_checksums.addFile(file_name, hashing_out.count(), hashing_out.getHash()); + } + + part->checksums.checkEqual(data_checksums, false); +} + MergeTreeData::DataPartPtr Service::findPart(const String & name) { /// It is important to include PreCommitted and Outdated parts here because remote replicas cannot reliably @@ -174,6 +198,7 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( const String & part_name, + const String & part_type, const String & replica_path, const String & host, int port, @@ -196,6 +221,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( { {"endpoint", getEndpointId(replica_path)}, {"part", part_name}, + {"part_type", part_type}, {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS)}, {"compress", "false"} }); @@ -244,10 +270,48 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } - return downloadPart(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); + return part_type == "InMemory" ? downloadPartToMemory(part_name, replica_path, in) + : downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); } -MergeTreeData::MutableDataPartPtr Fetcher::downloadPart( +MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( + const String & part_name, + const String & /* replica_path */, + PooledReadWriteBufferFromHTTP & in) +{ + NativeBlockInputStream block_in(in, 0); + auto block = block_in.read(); + MergeTreeData::MutableDataPartPtr new_data_part = + std::make_shared(data, part_name, nullptr); + + new_data_part->is_temp = true; + new_data_part->setColumns(block.getNamesAndTypesList()); + new_data_part->minmax_idx.update(block, data.minmax_idx_columns); + + auto partition_block = block; + data.partition_key_expr->execute(partition_block); + auto & partition = new_data_part->partition.value; + size_t partition_columns_num = data.partition_key_sample.columns(); + partition.resize(partition_columns_num); + + for (size_t i = 0; i < partition_columns_num; ++i) + { + const auto & column_name = data.partition_key_sample.getByPosition(i).name; + const auto & partition_column = partition_block.getByName(column_name).column; + partition[i] = (*partition_column)[0]; + } + + MergedBlockOutputStream part_out(new_data_part, block.getNamesAndTypesList(), {}, nullptr); + part_out.writePrefix(); + part_out.write(block); + part_out.writeSuffixAndFinalizePart(new_data_part); + + // TODO validate checksums + + return new_data_part; +} + +MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( const String & part_name, const String & replica_path, bool to_detached, diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index c0e8c0d2331..f17836cf9f3 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -31,6 +31,8 @@ public: private: MergeTreeData::DataPartPtr findPart(const String & name); + void sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, TableStructureReadLockHolder & storage_lock); + void sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, TableStructureReadLockHolder & storage_lock); private: /// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish, @@ -52,6 +54,7 @@ public: /// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory. MergeTreeData::MutableDataPartPtr fetchPart( const String & part_name, + const String & part_type, const String & replica_path, const String & host, int port, @@ -66,7 +69,7 @@ public: ActionBlocker blocker; private: - MergeTreeData::MutableDataPartPtr downloadPart( + MergeTreeData::MutableDataPartPtr downloadPartToDisk( const String & part_name, const String & replica_path, bool to_detached, @@ -74,6 +77,11 @@ private: const ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in); + MergeTreeData::MutableDataPartPtr downloadPartToMemory( + const String & part_name, + const String & replica_path, + PooledReadWriteBufferFromHTTP & in); + MergeTreeData & data; Logger * log; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index cdd7e592513..2e8c068aaf8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -222,6 +222,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc /// Write index. The index contains Primary Key value for each `index_granularity` row. + while (current_mark < ) for (size_t i = index_offset; i < rows;) { if (storage.hasPrimaryKey()) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index d874a10d3f2..5d2d9270a89 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -60,7 +60,7 @@ void MergeTreeWriteAheadLog::rotate(const std::lock_guard & /*write_ out = disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); block_out = std::make_unique(*out, 0, storage.getSampleBlock()); min_block_number = std::numeric_limits::max(); - max_block_number = 0; + max_block_number = std::numeric_limits::min(); } MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 2014fba18de..4868012f6c0 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -48,7 +48,7 @@ private: std::unique_ptr block_out; Int64 min_block_number = std::numeric_limits::max(); - Int64 max_block_number = 0; + Int64 max_block_number = std::numeric_limits::min(); mutable std::mutex write_mutex; }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 72255081e6b..37ef8e61ef2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -239,6 +239,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo log_entry.new_part_name = part_name; log_entry.quorum = quorum; log_entry.block_id = block_id; + log_entry.new_part_type = part->getType(); /// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3f907541a3c..424c054be54 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1400,7 +1400,8 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) try { String part_name = entry.actual_new_part_name.empty() ? entry.new_part_name : entry.actual_new_part_name; - if (!fetchPart(part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) + String part_type = entry.new_part_type.toString(); + if (!fetchPart(part_name, part_type, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) return false; } catch (Exception & e) @@ -1744,7 +1745,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (interserver_scheme != address.scheme) throw Exception("Interserver schemas are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR); - part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, source_replica_path, + part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, "Wide", source_replica_path, // TODO: fix part type address.host, address.replication_port, timeouts, user, password, interserver_scheme, false, TMP_PREFIX + "fetch_"); /// TODO: check columns_version of fetched part @@ -2693,7 +2694,8 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } -bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & source_replica_path, bool to_detached, size_t quorum) +bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & part_type, + const String & source_replica_path, bool to_detached, size_t quorum) { const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -2798,7 +2800,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin ErrorCodes::LOGICAL_ERROR); return fetcher.fetchPart( - part_name, source_replica_path, + part_name, part_type, source_replica_path, address.host, address.replication_port, timeouts, user_password.first, user_password.second, interserver_scheme, to_detached); }; @@ -4305,7 +4307,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const { try { - fetchPart(part, best_replica_path, true, 0); + fetchPart(part, "Wide", best_replica_path, true, 0); // TODO: fix part type } catch (const DB::Exception & e) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 01dd32614f9..589826fc2c6 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -454,7 +454,7 @@ private: * If quorum != 0, then the node for tracking the quorum is updated. * Returns false if part is already fetching right now. */ - bool fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum); + bool fetchPart(const String & part_name, const String & part_type, const String & replica_path, bool to_detached, size_t quorum); /// Required only to avoid races between executeLogEntry and fetchPartition std::unordered_set currently_fetching_parts; diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 2dfbf415100..e5a367239b4 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -19,7 +19,7 @@ StorageSystemParts::StorageSystemParts(const std::string & name_) { {"partition", std::make_shared()}, {"name", std::make_shared()}, - {"part_type", std::make_shared()}, + {"part_type", std::make_shared()}, {"active", std::make_shared()}, {"marks", std::make_shared()}, {"rows", std::make_shared()}, @@ -111,8 +111,16 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto columns_[i++]->insert(info.database); columns_[i++]->insert(info.table); columns_[i++]->insert(info.engine); - columns_[i++]->insert(part->disk->getName()); - columns_[i++]->insert(part->getFullPath()); + if (part->isStoredOnDisk()) + { + columns_[i++]->insert(part->disk->getName()); + columns_[i++]->insert(part->getFullPath()); + } + else + { + columns_[i++]->insertDefault(); + columns_[i++]->insertDefault(); + } if (has_state_column) columns_[i++]->insert(part->stateString()); diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 2382dba863f..cfcfc633c02 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -36,8 +36,8 @@ def create_tables(name, nodes, node_settings, shard): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/{name}', '{repl}') PARTITION BY toYYYYMM(date) ORDER BY id - SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes}, - min_rows_for_wide_part = {min_rows_for_wide_part}, min_bytes_for_wide_part = {min_bytes_for_wide_part} + SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes}, + min_rows_for_wide_part = {min_rows_for_wide_part}, min_rows_for_compact_part = {min_rows_for_compact_part} '''.format(name=name, shard=shard, repl=i, **settings)) def create_tables_old_format(name, nodes, shard): @@ -51,19 +51,24 @@ def create_tables_old_format(name, nodes, shard): node1 = cluster.add_instance('node1', config_dir="configs", with_zookeeper=True) node2 = cluster.add_instance('node2', config_dir="configs", with_zookeeper=True) -settings_default = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0} -settings_compact_only = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 1000000, 'min_bytes_for_wide_part' : 0} -settings_not_adaptive = {'index_granularity' : 64, 'index_granularity_bytes' : 0, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0} +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) -settings_compact = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0} -settings_wide = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 0, 'min_bytes_for_wide_part' : 0} +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) +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) +node10 = cluster.add_instance('node10', config_dir="configs", with_zookeeper=True) + @pytest.fixture(scope="module") def start_cluster(): try: @@ -75,6 +80,7 @@ def start_cluster(): create_tables('polymorphic_table_compact', [node3, node4], [settings_compact, settings_wide], "shard2") create_tables('polymorphic_table_wide', [node3, node4], [settings_wide, settings_compact], "shard2") create_tables_old_format('polymorphic_table', [node5, node6], "shard3") + create_tables('in_memory_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard4") yield cluster @@ -84,8 +90,8 @@ def start_cluster(): @pytest.mark.parametrize( ('first_node', 'second_node'), [ - (node1, node2), - (node5, node6) + (node1, node2), # compact parts + (node5, node6), # compact parts, old-format ] ) def test_polymorphic_parts_basics(start_cluster, first_node, second_node): @@ -198,8 +204,8 @@ def test_different_part_types_on_replicas(start_cluster, table, part_type): node7 = cluster.add_instance('node7', config_dir="configs", 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) -settings7 = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760} -settings8 = {'index_granularity' : 64, 'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_bytes_for_wide_part' : 0} +settings7 = {'index_granularity_bytes' : 10485760} +settings8 = {'index_granularity_bytes' : 10485760, 'min_rows_for_wide_part' : 512, 'min_rows_for_compact_part' : 0} @pytest.fixture(scope="module") def start_cluster_diff_versions(): @@ -212,7 +218,7 @@ def start_cluster_diff_versions(): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard5/{name}', '1') PARTITION BY toYYYYMM(date) ORDER BY id - SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes} + SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes} '''.format(name=name, **settings7) ) @@ -222,7 +228,7 @@ def start_cluster_diff_versions(): ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/shard5/{name}', '2') PARTITION BY toYYYYMM(date) ORDER BY id - SETTINGS index_granularity = {index_granularity}, index_granularity_bytes = {index_granularity_bytes}, + SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes}, min_rows_for_wide_part = {min_rows_for_wide_part}, min_bytes_for_wide_part = {min_bytes_for_wide_part} '''.format(name=name, **settings8) ) @@ -287,3 +293,21 @@ def test_polymorphic_parts_non_adaptive(start_cluster): "WHERE table = 'non_adaptive_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV("Wide\t2\n") assert node1.contains_in_log(" default.non_adaptive_table: Table can't create parts with adaptive granularity") + +def test_in_memory(start_cluster): + node9.query("SYSTEM STOP MERGES") + node10.query("SYSTEM STOP MERGES") + + for size in [200, 200, 300, 600]: + insert_random_data('in_memory_table', node9, size) + node10.query("SYSTEM SYNC REPLICA in_memory_table", timeout=20) + + assert node9.query("SELECT count() FROM in_memory_table") == "1300\n" + assert node10.query("SELECT count() FROM in_memory_table") == "1300\n" + + expected = "Compact\t1\nInMemory\t2\nWide\t1\n" + + assert TSV(node9.query("SELECT part_type, count() FROM system.parts " \ + "WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) + assert TSV(node10.query("SELECT part_type, count() FROM system.parts " \ + "WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) From b3cfce523c24e560a76f652a423cbad252e5cf5d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 5 May 2020 04:24:51 +0300 Subject: [PATCH 007/196] in-memory parts: test for basic functionality --- .../01130_in_memory_parts.reference | 27 +++++++++++++++ .../0_stateless/01130_in_memory_parts.sql | 33 +++++++++++++++++++ 2 files changed, 60 insertions(+) create mode 100644 tests/queries/0_stateless/01130_in_memory_parts.reference create mode 100644 tests/queries/0_stateless/01130_in_memory_parts.sql diff --git a/tests/queries/0_stateless/01130_in_memory_parts.reference b/tests/queries/0_stateless/01130_in_memory_parts.reference new file mode 100644 index 00000000000..dbf39a0b48f --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts.reference @@ -0,0 +1,27 @@ +Simple selects +0 0 +1 1 +2 2 +3 0 +4 1 +50 2 +51 0 +52 1 +53 2 +54 0 +34 +0 +Mutations and Alters +66 +1 1 +2 2 +4 1 +5 2 +7 1 +[1,1] +[] +[4,16] +[] +[7,49] +1 1 +2 1 diff --git a/tests/queries/0_stateless/01130_in_memory_parts.sql b/tests/queries/0_stateless/01130_in_memory_parts.sql new file mode 100644 index 00000000000..4c09eb19937 --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts.sql @@ -0,0 +1,33 @@ +DROP TABLE IF EXISTS in_memory; +CREATE TABLE in_memory (a UInt32, b UInt32) + ENGINE = MergeTree ORDER BY a + SETTINGS min_rows_for_compact_part = 0; + +INSERT INTO in_memory SELECT number, number % 3 FROM numbers(100); + +SELECT 'Simple selects'; + +SELECT * FROM in_memory ORDER BY a LIMIT 5; +SELECT * FROM in_memory ORDER BY a LIMIT 5 OFFSET 50; +SELECT count() FROM in_memory WHERE b = 0 SETTINGS max_block_size = 10; +-- Check index +SELECT count() FROM in_memory WHERE a > 100 SETTINGS max_rows_to_read = 0, force_primary_key = 1; + +SELECT 'Mutations and Alters'; +SET mutations_sync = 1; + +ALTER TABLE in_memory DELETE WHERE b = 0; + +SELECT count() FROM in_memory; +SELECT * FROM in_memory ORDER BY a LIMIT 5; + +ALTER TABLE in_memory ADD COLUMN arr Array(UInt64); +ALTER TABLE in_memory UPDATE arr = [a, a * a] WHERE b = 1; + +SELECT arr FROM in_memory ORDER BY a LIMIT 5; + +ALTER TABLE in_memory MODIFY COLUMN b String; +ALTER TABLE in_memory RENAME COLUMN b to str; +SELECT DISTINCT str, length(str) FROM in_memory ORDER BY str; + +DROP TABLE in_memory; From 14e8592e47716a2c998091cd5d83ddc366b76d3c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 5 May 2020 04:27:31 +0300 Subject: [PATCH 008/196] in-memory parts: send checksums --- src/Storages/MergeTree/DataPartsExchange.cpp | 18 +++++++++++------- src/Storages/MergeTree/MergeTreeData.cpp | 2 -- .../MergeTree/MergeTreeReaderInMemory.cpp | 2 +- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7d1c0fb43b5..bbee34e4a0b 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -29,6 +29,8 @@ namespace ErrorCodes extern const int CANNOT_WRITE_TO_OSTREAM; extern const int CHECKSUM_DOESNT_MATCH; extern const int INSECURE_PATH; + extern const int CORRUPTED_DATA; + extern const int LOGICAL_ERROR; } namespace DataPartsExchange @@ -133,12 +135,11 @@ void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteB { auto part_in_memory = dynamic_cast(part.get()); if (!part_in_memory) - throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::NO_SUCH_DATA_PART); // TODO error code - - NativeBlockOutputStream block_out(out, 0, data.getSampleBlock()); - block_out.write(part_in_memory->block); + throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR); - // TODO send checksums + NativeBlockOutputStream block_out(out, 0, data.getSampleBlock()); + part->checksums.write(out); + block_out.write(part_in_memory->block); } void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuffer & out, TableStructureReadLockHolder &) @@ -279,6 +280,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( const String & /* replica_path */, PooledReadWriteBufferFromHTTP & in) { + MergeTreeData::DataPart::Checksums checksums; + if (!checksums.read(in)) + throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA); + NativeBlockInputStream block_in(in, 0); auto block = block_in.read(); MergeTreeData::MutableDataPartPtr new_data_part = @@ -305,8 +310,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_data_part); - - // TODO validate checksums + new_data_part->checksums.checkEqual(checksums, /* have_uncompressed = */ true); return new_data_part; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index fa7bd9f77e5..9fd80765f0e 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2740,8 +2740,6 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartS } } - LOG_DEBUG(log, "MergeTreeData::getDataPartsVector: " << res.size()); - return res; } diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 8c61a879270..671b36dfe86 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -49,7 +49,7 @@ size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool /* continue_read if (!part_in_memory->block.has(name)) continue; - const auto block_column = part_in_memory->block.getByPosition(i).column; + const auto block_column = part_in_memory->block.getByName(name).column; if (total_rows_read == 0 && part_rows <= max_rows_to_read) { res_columns[i] = block_column; From 4878c91d07768374c9e81b5d483bbbf8d67f8928 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 5 May 2020 18:06:16 +0300 Subject: [PATCH 009/196] in-memory parts: better restore from wal --- src/Storages/MergeTree/DataPartsExchange.cpp | 16 +----- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/MergeTreeData.cpp | 50 +++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 1 - src/Storages/MergeTree/MergeTreePartition.cpp | 14 ++++++ src/Storages/MergeTree/MergeTreePartition.h | 2 + src/Storages/MergeTree/MergeTreeSettings.h | 2 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 14 ++---- .../MergeTree/MergeTreeWriteAheadLog.h | 18 +------ .../test_polymorphic_parts/test.py | 41 +++++++++++++-- 10 files changed, 88 insertions(+), 71 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index bbee34e4a0b..673c774ce5a 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -90,7 +90,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo LOG_TRACE(log, "Sending part " << part_name); try - { + { auto storage_lock = data.lockStructureForShare( false, RWLockImpl::NO_QUERY, data.getSettings()->lock_acquire_timeout_for_background_operations); @@ -292,19 +292,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( new_data_part->is_temp = true; new_data_part->setColumns(block.getNamesAndTypesList()); new_data_part->minmax_idx.update(block, data.minmax_idx_columns); - - auto partition_block = block; - data.partition_key_expr->execute(partition_block); - auto & partition = new_data_part->partition.value; - size_t partition_columns_num = data.partition_key_sample.columns(); - partition.resize(partition_columns_num); - - for (size_t i = 0; i < partition_columns_num; ++i) - { - const auto & column_name = data.partition_key_sample.getByPosition(i).name; - const auto & partition_column = partition_block.getByName(column_name).column; - partition[i] = (*partition_column)[0]; - } + new_data_part->partition.create(data, block, 0); MergedBlockOutputStream part_out(new_data_part, block.getNamesAndTypesList(), {}, nullptr); part_out.writePrefix(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index b1fb2554c76..8943a9fcb1f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -354,6 +354,7 @@ private: using MergeTreeDataPartState = IMergeTreeDataPart::State; using MergeTreeDataPartPtr = std::shared_ptr; +using MergeTreeMutableDataPartPtr = std::shared_ptr; bool isCompactPart(const MergeTreeDataPartPtr & data_part); bool isWidePart(const MergeTreeDataPartPtr & data_part); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9fd80765f0e..67a30934e2c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -248,8 +248,8 @@ MergeTreeData::MergeTreeData( if (settings->in_memory_parts_enable_wal) { - auto disk = reserveSpace(0)->getDisk(); - write_ahead_log = std::make_shared(*this, disk); + auto disk = makeEmptyReservationOnLargestDisk()->getDisk(); + write_ahead_log = std::make_shared(*this, std::move(disk)); } } @@ -859,6 +859,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) const auto settings = getSettings(); std::vector> part_names_with_disks; + MutableDataPartsVector parts_from_wal; Strings part_file_names; auto disks = getStoragePolicy()->getDisks(); @@ -899,19 +900,23 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) part_names_with_disks.emplace_back(it->name(), disk_ptr); if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) - loadDataPartsFromWAL(disk_ptr, it->name()); + { + MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); + auto current_parts = wal.restore(); + for (auto & part : current_parts) + parts_from_wal.push_back(std::move(part)); + } } } auto part_lock = lockParts(); - // TODO: fix. - // data_parts_indexes.clear(); + data_parts_indexes.clear(); - // if (part_names_with_disks.empty()) - // { - // LOG_DEBUG(log, "There is no data parts"); - // return; - // } + if (part_names_with_disks.empty() && parts_from_wal.empty()) + { + LOG_DEBUG(log, "There is no data parts"); + return; + } /// Parallel loading of data parts. size_t num_threads = std::min(size_t(settings->max_part_loading_threads), part_names_with_disks.size()); @@ -1043,6 +1048,16 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) pool.wait(); + for (auto & part : parts_from_wal) + { + part->modification_time = time(nullptr); + /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later + part->state = DataPartState::Committed; + + if (!data_parts_indexes.insert(part).second) + throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); + } + if (has_non_adaptive_parts && has_adaptive_parts && !settings->enable_mixed_granularity_parts) throw Exception("Table contains parts with adaptive and non adaptive marks, but `setting enable_mixed_granularity_parts` is disabled", ErrorCodes::LOGICAL_ERROR); @@ -1110,21 +1125,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) LOG_DEBUG(log, "Loaded data parts (" << data_parts_indexes.size() << " items)"); } -void MergeTreeData::loadDataPartsFromWAL(const DiskPtr & disk, const String & file_name) -{ - MergeTreeWriteAheadLog wal(*this, disk, file_name); - auto parts = wal.restore(); - for (auto & part : parts) - { - part->modification_time = time(nullptr); - /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->state = DataPartState::Committed; - - if (!data_parts_indexes.insert(part).second) - throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); - } -} - /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 618668b0d87..4ae1a4bb0cb 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -370,7 +370,6 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks); - void loadDataPartsFromWAL(const DiskPtr & disk, const String & file_name); String getLogName() const { return log_name; } diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 000d0abad43..3124b16a138 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -152,4 +152,18 @@ void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr checksums.files["partition.dat"].file_hash = out_hashing.getHash(); } +void MergeTreePartition::create(const MergeTreeData & storage, Block block, size_t row) +{ + storage.partition_key_expr->execute(block); + size_t partition_columns_num = storage.partition_key_sample.columns(); + value.resize(partition_columns_num); + + for (size_t i = 0; i < partition_columns_num; ++i) + { + const auto & column_name = storage.partition_key_sample.getByPosition(i).name; + const auto & partition_column = block.getByName(column_name).column; + partition_column->get(row, value[i]); + } +} + } diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 2a589339ba8..d91022f655f 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -36,6 +36,8 @@ public: void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void assign(const MergeTreePartition & other) { value.assign(other.value); } + + void create(const MergeTreeData & storage, Block block, size_t row); }; } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 20010eb8f4c..68e240f9d7e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,7 +33,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ M(SettingUInt64, min_bytes_for_compact_part, 0, "Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ M(SettingUInt64, min_rows_for_compact_part, 0, "Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ - M(SettingBool, in_memory_parts_enable_wal, 0, "", 0) \ + M(SettingBool, in_memory_parts_enable_wal, 1, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(SettingBool, in_memory_parts_insert_sync, 0, "", 0) \ \ /** Merge settings. */ \ diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 84091f904e6..e5c0c370ae2 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -12,15 +12,6 @@ namespace ErrorCodes extern const int UNKNOWN_FORMAT_VERSION; } -// WALBlockOutputStream::WALBlockOutputStream(WriteBuffer & out_, const Block & header_) -// : NativeBlockOutputStream(out_, 0, header_), out(out_) {} - -// void WALBlockOutputStream::write(const Block & block, const String & part_name) -// { -// writeIntBinary(0, out); -// writeString(part_name, out); -// NativeBlockOutputStream::write(block); -// } MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( const MergeTreeData & storage_, @@ -28,7 +19,7 @@ MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( const String & name) : storage(storage_) , disk(disk_) - , path(storage.getFullPathOnDisk(disk) + name) + , path(storage.getRelativeDataPath() + name) { init(); } @@ -93,8 +84,9 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() part_name); auto block = block_in.read(); - part->minmax_idx.update(block, storage.minmax_idx_columns); + part->partition.create(storage, block, 0); + MergedBlockOutputStream part_out(part, block.getNamesAndTypesList(), {}, nullptr); part_out.writePrefix(); part_out.write(block); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 50bb9aa5e13..7a0e5759624 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -8,20 +8,6 @@ namespace DB { -// class WALBlockOutputStream : public NativeBlockOutputStream -// { -// public: -// WALBlockOutputStream(WriteBuffer & out_, const Block & header_); -// void write(const Block & block, const String & part_name); - -// private: -// WriteBuffer & out; -// }; - -// class WALBlockInputStream : public NativeBlockInputStream -// { -// }; - class MergeTreeData; class MergeTreeWriteAheadLog @@ -29,13 +15,13 @@ class MergeTreeWriteAheadLog public: constexpr static auto WAL_FILE_NAME = "wal"; constexpr static auto WAL_FILE_EXTENSION = ".bin"; - constexpr static size_t MAX_WAL_BYTES = 1024; + constexpr static size_t MAX_WAL_BYTES = 1024 * 1024 * 1024; MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, const String & name = String(WAL_FILE_NAME) + WAL_FILE_EXTENSION); void write(const Block & block, const String & part_name); - std::vector> restore(); + std::vector restore(); private: void init(); diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 1cd917a12bb..8cca8aa1072 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -39,7 +39,8 @@ def create_tables(name, nodes, node_settings, shard): PARTITION BY toYYYYMM(date) ORDER BY id SETTINGS index_granularity = 64, index_granularity_bytes = {index_granularity_bytes}, - min_rows_for_wide_part = {min_rows_for_wide_part}, min_rows_for_compact_part = {min_rows_for_compact_part} + min_rows_for_wide_part = {min_rows_for_wide_part}, min_rows_for_compact_part = {min_rows_for_compact_part}, + in_memory_parts_enable_wal = 1 '''.format(name=name, shard=shard, repl=i, **settings)) def create_tables_old_format(name, nodes, shard): @@ -68,8 +69,8 @@ node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['confi 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) -node10 = cluster.add_instance('node10', config_dir="configs", with_zookeeper=True) +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, stay_alive=True) @pytest.fixture(scope="module") def start_cluster(): @@ -83,6 +84,7 @@ def start_cluster(): create_tables('polymorphic_table_wide', [node3, node4], [settings_wide, settings_compact], "shard2") create_tables_old_format('polymorphic_table', [node5, node6], "shard3") create_tables('in_memory_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard4") + create_tables('wal_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard4") yield cluster @@ -314,6 +316,39 @@ def test_in_memory(start_cluster): assert TSV(node10.query("SELECT part_type, count() FROM system.parts " \ "WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) +def test_in_memory_wal(start_cluster): + node9.query("SYSTEM STOP MERGES") + node10.query("SYSTEM STOP MERGES") + + for i in range(5): + insert_random_data('wal_table', node9, 50) + node10.query("SYSTEM SYNC REPLICA wal_table", timeout=20) + + assert node9.query("SELECT count() FROM wal_table") == "250\n" + assert node10.query("SELECT count() FROM wal_table") == "250\n" + + assert node9.query("SELECT count() FROM system.parts WHERE table = 'wal_table' AND part_type = 'InMemory'") == '5\n' + assert node10.query("SELECT count() FROM system.parts WHERE table = 'wal_table' AND part_type = 'InMemory'") == '5\n' + + # WAL works at inserts + node9.restart_clickhouse(kill=True) + time.sleep(5) + assert node9.query("SELECT count() FROM wal_table") == "250\n" + + # WAL works at fetches + node10.restart_clickhouse(kill=True) + time.sleep(5) + assert node10.query("SELECT count() FROM wal_table") == "250\n" + + node9.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0") + insert_random_data('wal_table', node9, 50) + assert node9.query("SELECT count() FROM wal_table") == "300\n" + + # Data is lost without WAL + node9.restart_clickhouse(kill=True) + time.sleep(5) + assert node9.query("SELECT count() FROM wal_table") == "250\n" + def test_polymorphic_parts_index(start_cluster): node1.query(''' CREATE TABLE index_compact(a UInt32, s String) From 4a396090f8cfb2a19df73ddb6e85682792bd8479 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 7 May 2020 01:24:00 +0300 Subject: [PATCH 010/196] fix build --- src/Storages/StorageReplicatedMergeTree.cpp | 1 - src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a5cd4b6629a..9b138af30bf 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2714,7 +2714,6 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) } - void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) { auto zookeeper = getZooKeeper(); diff --git a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp index 7488b6ea44a..d9ddb8e9722 100644 --- a/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp +++ b/src/Storages/tests/gtest_aux_funcs_for_adaptive_granularity.cpp @@ -3,7 +3,7 @@ #include // I know that inclusion of .cpp is not good at all -#include +#include using namespace DB; static Block getBlockWithSize(size_t required_size_in_bytes, size_t size_of_row_in_bytes) From 4c03f4870366feb230f231bf7ad54a0c670ca59a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 May 2020 23:08:15 +0300 Subject: [PATCH 011/196] in-memory parts: better restoring --- src/Storages/MergeTree/DataPartsExchange.cpp | 17 +++-- src/Storages/MergeTree/DataPartsExchange.h | 1 - .../MergeTree/IMergeTreeDataPartWriter.cpp | 6 +- .../MergeTree/IMergeTreeDataPartWriter.h | 1 + src/Storages/MergeTree/MergeSelector.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 12 ++-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 + .../MergeTreeDataPartWriterInMemory.cpp | 43 ++++-------- .../MergeTreeDataPartWriterInMemory.h | 1 - .../MergeTreeDataPartWriterOnDisk.cpp | 1 - .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 5 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 - .../MergeTree/MergeTreeWriteAheadLog.cpp | 68 ++++++++++++++----- .../MergeTree/MergeTreeWriteAheadLog.h | 6 +- src/Storages/StorageReplicatedMergeTree.cpp | 11 ++- src/Storages/StorageReplicatedMergeTree.h | 2 +- tests/integration/helpers/network.py | 2 +- .../configs/do_not_merge.xml | 6 ++ .../test_polymorphic_parts/test.py | 63 ++++++++++++----- .../01130_in_memory_parts.reference | 8 +++ .../0_stateless/01130_in_memory_parts.sql | 8 ++- 21 files changed, 168 insertions(+), 99 deletions(-) create mode 100644 tests/integration/test_polymorphic_parts/configs/do_not_merge.xml diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 673c774ce5a..f61d80e63ac 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -40,6 +40,7 @@ namespace { constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE = 1; constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS = 2; +constexpr auto REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE = 3; std::string getEndpointId(const std::string & node_id) @@ -59,7 +60,6 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo int client_protocol_version = parse(params.get("client_protocol_version", "0")); String part_name = params.get("part"); - String part_type = params.get("part_type", "Wide"); // TODO: correct type with old versions const auto data_settings = data.getSettings(); @@ -79,7 +79,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo } /// We pretend to work as older server version, to be sure that client will correctly process our version - response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS))}); + response.addCookie({"server_protocol_version", toString(std::min(client_protocol_version, REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE))}); ++total_sends; SCOPE_EXIT({--total_sends;}); @@ -108,7 +108,10 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo writeBinary(ttl_infos_buffer.str(), out); } - if (part_type == "InMemory") + if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE) + writeStringBinary(part->getType().toString(), out); + + if (isInMemoryPart(part)) sendPartFromMemory(part, out, storage_lock); else sendPartFromDisk(part, out, storage_lock); @@ -199,7 +202,6 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( const String & part_name, - const String & part_type, const String & replica_path, const String & host, int port, @@ -222,8 +224,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( { {"endpoint", getEndpointId(replica_path)}, {"part", part_name}, - {"part_type", part_type}, - {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE_AND_TTL_INFOS)}, + {"client_protocol_version", toString(REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE)}, {"compress", "false"} }); @@ -271,6 +272,10 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( reservation = data.makeEmptyReservationOnLargestDisk(); } + String part_type = "Wide"; + if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) + readStringBinary(part_type, in); + return part_type == "InMemory" ? downloadPartToMemory(part_name, replica_path, in) : downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); } diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index f17836cf9f3..217434d7e41 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -54,7 +54,6 @@ public: /// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory. MergeTreeData::MutableDataPartPtr fetchPart( const String & part_name, - const String & part_type, const String & replica_path, const String & host, int port, diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 2d35b9ff723..523774da233 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -9,7 +9,8 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeWriterSettings & settings_) : storage(storage_) , columns_list(columns_list_) - , settings(settings_) {} + , settings(settings_) + , with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity){} IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeData & storage_, @@ -21,7 +22,8 @@ IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( , columns_list(columns_list_) , skip_indices(skip_indices_) , index_granularity(index_granularity_) - , settings(settings_) {} + , settings(settings_) + , with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity) {} Columns IMergeTreeDataPartWriter::releaseIndexColumns() { diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index 6ecdcf581c2..f03a442d990 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -65,6 +65,7 @@ protected: MergeTreeIndices skip_indices; MergeTreeIndexGranularity index_granularity; MergeTreeWriterSettings settings; + bool with_final_mark; size_t next_mark = 0; size_t next_index_offset = 0; diff --git a/src/Storages/MergeTree/MergeSelector.h b/src/Storages/MergeTree/MergeSelector.h index 3c3cd8190ac..24612b367d5 100644 --- a/src/Storages/MergeTree/MergeSelector.h +++ b/src/Storages/MergeTree/MergeSelector.h @@ -60,7 +60,7 @@ public: const Partitions & partitions, const size_t max_total_size_to_merge) = 0; - virtual ~IMergeSelector() {} + virtual ~IMergeSelector() = default; }; } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 049cc212a7b..bbe051a2476 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -245,12 +245,6 @@ MergeTreeData::MergeTreeData( String reason; if (!canUsePolymorphicParts(*settings, &reason) && !reason.empty()) LOG_WARNING(log, reason + " Settings 'min_bytes_for_wide_part' and 'min_bytes_for_wide_part' will be ignored."); - - if (settings->in_memory_parts_enable_wal) - { - auto disk = makeEmptyReservationOnLargestDisk()->getDisk(); - write_ahead_log = std::make_shared(*this, std::move(disk)); - } } @@ -1121,6 +1115,12 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } + if (settings->in_memory_parts_enable_wal) + { + auto disk = makeEmptyReservationOnLargestDisk()->getDisk(); + write_ahead_log = std::make_shared(*this, std::move(disk)); + } + calculateColumnSizesImpl(); LOG_DEBUG(log, "Loaded data parts (" << data_parts_indexes.size() << " items)"); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 1e5f80d44b0..ff56b407998 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -231,12 +231,14 @@ bool MergeTreeDataMergerMutator::selectPartsToMerge( { /// Check predicate only for first part in each partition. if (!prev_part) + { /* Parts can be merged with themselves for TTL needs for example. * So we have to check if this part is currently being inserted with quorum and so on and so forth. * Obviously we have to check it manually only for the first part * of each partition because it will be automatically checked for a pair of parts. */ if (!can_merge_callback(nullptr, part, nullptr)) continue; + } const String & partition_id = part->info.partition_id; if (!prev_partition_id || partition_id != *prev_partition_id || (prev_part && !can_merge_callback(*prev_part, part, nullptr))) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index 3d0d67e1ed6..39e9757d81c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -27,21 +27,22 @@ void MergeTreeDataPartWriterInMemory::write( Block result_block; if (permutation) { - for (const auto & it : columns_list) + for (const auto & col : columns_list) { - if (primary_key_block.has(it.name)) - result_block.insert(primary_key_block.getByName(it.name)); + if (primary_key_block.has(col.name)) + result_block.insert(primary_key_block.getByName(col.name)); else { - auto column = block.getByName(it.name); - column.column = column.column->permute(*permutation, 0); - result_block.insert(column); + auto permuted = block.getByName(col.name); + permuted.column = permuted.column->permute(*permutation, 0); + result_block.insert(permuted); } } } else { - result_block = block; + for (const auto & col : columns_list) + result_block.insert(block.getByName(col.name)); } part->block = std::move(result_block); @@ -55,7 +56,8 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl return; index_granularity.appendMark(rows); - index_granularity.appendMark(0); + if (with_final_mark) + index_granularity.appendMark(0); size_t primary_columns_num = primary_index_block.columns(); index_columns.resize(primary_columns_num); @@ -64,7 +66,8 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl const auto & primary_column = *primary_index_block.getByPosition(i).column; index_columns[i] = primary_column.cloneEmpty(); index_columns[i]->insertFrom(primary_column, 0); - index_columns[i]->insertFrom(primary_column, rows - 1); + if (with_final_mark) + index_columns[i]->insertFrom(primary_column, rows - 1); } } @@ -78,31 +81,9 @@ static MergeTreeDataPartChecksum createUncompressedChecksum(size_t size, SipHash void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) { - UNUSED(checksums); SipHash hash; part->block.updateHash(hash); checksums.files["data.bin"] = createUncompressedChecksum(part->block.bytes(), hash); } -void MergeTreeDataPartWriterInMemory::finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) -{ - UNUSED(checksums); - if (index_columns.empty()) - return; - - SipHash hash; - size_t index_size = 0; - size_t rows = index_columns[0]->size(); - for (size_t i = 0; i < rows; ++i) - { - for (const auto & col : index_columns) - { - col->updateHashWithValue(i, hash); - index_size += col->byteSize(); - } - } - - checksums.files["primary.idx"] = createUncompressedChecksum(index_size, hash); -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h index c9b57e5e4b6..425066a802e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h @@ -19,7 +19,6 @@ public: void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) override; void calculateAndSerializePrimaryIndex(const Block & primary_index_block) override; - void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) override; private: DataPartInMemoryPtr part; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 3ada9973477..0543e6420ee 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -80,7 +80,6 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( , marks_file_extension(marks_file_extension_) , default_codec(default_codec_) , compute_granularity(index_granularity.empty()) - , with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity) { if (settings.blocks_are_granules_size && !index_granularity.empty()) throw Exception("Can't take information about index granularity from blocks, when non empty index_granularity array specified", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 149de2d9f30..bb54b964793 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -82,8 +82,8 @@ public: void initSkipIndices() final; void initPrimaryIndex() final; - virtual void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) final; - virtual void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums) final; + void finishPrimaryIndexSerialization(MergeTreeData::DataPart::Checksums & checksums) final; + void finishSkipIndicesSerialization(MergeTreeData::DataPart::Checksums & checksums) final; void setWrittenOffsetColumns(WrittenOffsetColumns * written_offset_columns_) { @@ -100,7 +100,6 @@ protected: CompressionCodecPtr default_codec; bool compute_granularity; - bool with_final_mark; bool need_finish_last_granule; /// Number of marsk in data from which skip indices have to start diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c8aa15e62f3..580c95b34dd 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -165,8 +165,6 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( const unsigned num_streams, const PartitionIdToMaxBlock * max_block_numbers_to_read) const { - LOG_DEBUG(log, "readFromParts size: " << parts.size()); - size_t part_index = 0; /// If query contains restrictions on the virtual column `_part` or `_part_index`, select only parts suitable for it. diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index e5c0c370ae2..02c45dcfb64 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -10,16 +10,18 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_FORMAT_VERSION; + extern const int CANNOT_READ_ALL_DATA; } MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( const MergeTreeData & storage_, const DiskPtr & disk_, - const String & name) + const String & name_) : storage(storage_) , disk(disk_) - , path(storage.getRelativeDataPath() + name) + , name(name_) + , path(storage.getRelativeDataPath() + name_) { init(); } @@ -29,7 +31,7 @@ void MergeTreeWriteAheadLog::init() out = disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); block_out = std::make_unique(*out, 0, storage.getSampleBlock()); min_block_number = std::numeric_limits::max(); - max_block_number = std::numeric_limits::min(); + max_block_number = -1; } void MergeTreeWriteAheadLog::write(const Block & block, const String & part_name) @@ -55,7 +57,7 @@ void MergeTreeWriteAheadLog::rotate() + toString(min_block_number) + "_" + toString(max_block_number) + WAL_FILE_EXTENSION; - Poco::File(path).renameTo(storage.getFullPathOnDisk(disk) + new_name); + disk->replaceFile(path, storage.getRelativeDataPath() + new_name); init(); } @@ -69,29 +71,61 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() while (!in->eof()) { + MergeTreeData::MutableDataPartPtr part; UInt8 version; String part_name; - readIntBinary(version, *in); - if (version != 0) - throw Exception("Unknown WAL format version: " + toString(version), ErrorCodes::UNKNOWN_FORMAT_VERSION); + Block block; - readStringBinary(part_name, *in); - auto part = storage.createPart( - part_name, - MergeTreeDataPartType::IN_MEMORY, - MergeTreePartInfo::fromPartName(part_name, storage.format_version), - storage.reserveSpace(0)->getDisk(), - part_name); + try + { + readIntBinary(version, *in); + if (version != 0) + throw Exception("Unknown WAL format version: " + toString(version), ErrorCodes::UNKNOWN_FORMAT_VERSION); - auto block = block_in.read(); - part->minmax_idx.update(block, storage.minmax_idx_columns); - part->partition.create(storage, block, 0); + readStringBinary(part_name, *in); + + part = storage.createPart( + part_name, + MergeTreeDataPartType::IN_MEMORY, + MergeTreePartInfo::fromPartName(part_name, storage.format_version), + storage.reserveSpace(0)->getDisk(), + part_name); + + block = block_in.read(); + } + catch (const Exception & e) + { + if (e.code() == ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == ErrorCodes::UNKNOWN_FORMAT_VERSION) + { + LOG_WARNING(&Logger::get(storage.getLogName() + " (WriteAheadLog)"), + "WAL file '" << path << "' is broken. " << e.displayText()); + + /// If file is broken, do not write new parts to it. + /// But if it contains any part rotate and save them. + if (max_block_number == -1) + Poco::File(path).remove(); + else if (name == DEFAULT_WAL_FILE) + rotate(); + + break; + } + throw; + } MergedBlockOutputStream part_out(part, block.getNamesAndTypesList(), {}, nullptr); + + part->minmax_idx.update(block, storage.minmax_idx_columns); + if (storage.partition_key_expr) + part->partition.create(storage, block, 0); + if (storage.hasSortingKey()) + storage.sorting_key_expr->execute(block); + part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(part); + min_block_number = std::min(min_block_number, part->info.min_block); + max_block_number = std::max(max_block_number, part->info.max_block); result.push_back(std::move(part)); } diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 7a0e5759624..22665048f56 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -15,10 +15,11 @@ class MergeTreeWriteAheadLog public: constexpr static auto WAL_FILE_NAME = "wal"; constexpr static auto WAL_FILE_EXTENSION = ".bin"; + constexpr static auto DEFAULT_WAL_FILE = "wal.bin"; constexpr static size_t MAX_WAL_BYTES = 1024 * 1024 * 1024; MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, - const String & name = String(WAL_FILE_NAME) + WAL_FILE_EXTENSION); + const String & name = DEFAULT_WAL_FILE); void write(const Block & block, const String & part_name); std::vector restore(); @@ -29,13 +30,14 @@ private: const MergeTreeData & storage; DiskPtr disk; + String name; String path; std::unique_ptr out; std::unique_ptr block_out; Int64 min_block_number = std::numeric_limits::max(); - Int64 max_block_number = std::numeric_limits::min(); + Int64 max_block_number = -1; mutable std::mutex write_mutex; }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9b138af30bf..73034ef2e7e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1409,8 +1409,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) try { String part_name = entry.actual_new_part_name.empty() ? entry.new_part_name : entry.actual_new_part_name; - String part_type = entry.new_part_type.toString(); - if (!fetchPart(part_name, part_type, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) + if (!fetchPart(part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) return false; } catch (Exception & e) @@ -1754,7 +1753,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (interserver_scheme != address.scheme) throw Exception("Interserver schemas are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR); - part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, "Wide", source_replica_path, // TODO: fix part type + part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, source_replica_path, address.host, address.replication_port, timeouts, user, password, interserver_scheme, false, TMP_PREFIX + "fetch_"); /// TODO: check columns_version of fetched part @@ -2766,7 +2765,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) } -bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & part_type, +bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const String & source_replica_path, bool to_detached, size_t quorum) { const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -2872,7 +2871,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Strin ErrorCodes::LOGICAL_ERROR); return fetcher.fetchPart( - part_name, part_type, source_replica_path, + part_name, source_replica_path, address.host, address.replication_port, timeouts, user_password.first, user_password.second, interserver_scheme, to_detached); }; @@ -4384,7 +4383,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const { try { - fetchPart(part, "Wide", best_replica_path, true, 0); // TODO: fix part type + fetchPart(part, best_replica_path, true, 0); } catch (const DB::Exception & e) { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 728ac792dab..70fb48e9b35 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -456,7 +456,7 @@ private: * If quorum != 0, then the node for tracking the quorum is updated. * Returns false if part is already fetching right now. */ - bool fetchPart(const String & part_name, const String & part_type, const String & replica_path, bool to_detached, size_t quorum); + bool fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum); /// Required only to avoid races between executeLogEntry and fetchPartition std::unordered_set currently_fetching_parts; diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index 3ba8ae3f9fd..5d738126f07 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -13,7 +13,7 @@ class PartitionManager: Can act as a context manager: - with pm as PartitionManager(): + with PartitionManager() as pm: pm.partition_instances(instance1, instance2) ... # At exit all partitions are removed automatically. diff --git a/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml b/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml new file mode 100644 index 00000000000..bc2dae31ad6 --- /dev/null +++ b/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml @@ -0,0 +1,6 @@ + + + 1 + 2 + + diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 8cca8aa1072..ba8b4b6b725 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -8,6 +8,7 @@ import struct from helpers.test_tools import TSV from helpers.test_tools import assert_eq_with_retry from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) @@ -69,8 +70,8 @@ node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['confi 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, stay_alive=True) +node9 = cluster.add_instance('node9', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) +node10 = cluster.add_instance('node10', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) @pytest.fixture(scope="module") def start_cluster(): @@ -317,37 +318,65 @@ def test_in_memory(start_cluster): "WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) def test_in_memory_wal(start_cluster): - node9.query("SYSTEM STOP MERGES") - node10.query("SYSTEM STOP MERGES") + # Merges are disabled in config for i in range(5): insert_random_data('wal_table', node9, 50) node10.query("SYSTEM SYNC REPLICA wal_table", timeout=20) - assert node9.query("SELECT count() FROM wal_table") == "250\n" - assert node10.query("SELECT count() FROM wal_table") == "250\n" + def check(node, rows, parts): + node.query("SELECT count() FROM wal_table") == "{}\n".format(rows) + node.query("SELECT count() FROM system.parts WHERE table = 'wal_table' AND part_type = 'InMemory'") == "{}\n".format(parts) - assert node9.query("SELECT count() FROM system.parts WHERE table = 'wal_table' AND part_type = 'InMemory'") == '5\n' - assert node10.query("SELECT count() FROM system.parts WHERE table = 'wal_table' AND part_type = 'InMemory'") == '5\n' + check(node9, 250, 5) + check(node10, 250, 5) # WAL works at inserts node9.restart_clickhouse(kill=True) - time.sleep(5) - assert node9.query("SELECT count() FROM wal_table") == "250\n" + check(node9, 250, 5) # WAL works at fetches node10.restart_clickhouse(kill=True) - time.sleep(5) - assert node10.query("SELECT count() FROM wal_table") == "250\n" + check(node10, 250, 5) - node9.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0") insert_random_data('wal_table', node9, 50) - assert node9.query("SELECT count() FROM wal_table") == "300\n" + node10.query("SYSTEM SYNC REPLICA wal_table", timeout=20) + + # Disable replication + with PartitionManager() as pm: + pm.partition_instances(node9, node10) + check(node9, 300, 6) + + wal_file = os.path.join(node9.path, "database/data/default/wal_table/wal.bin") + # Corrupt wal file + open(wal_file, 'rw+').truncate(os.path.getsize(wal_file) - 10) + node9.restart_clickhouse(kill=True) + + # Broken part is lost, but other restored successfully + check(node9, 250, 5) + # WAL with blocks from 0 to 4 + broken_wal_file = os.path.join(node9.path, "database/data/default/wal_table/wal_0_4.bin") + assert os.path.exists(broken_wal_file) + + # Fetch lost part from replica + node9.query("SYSTEM SYNC REPLICA wal_table", timeout=20) + check(node9, 300, 6) + + #Check that new data is written to new wal, but old is still exists for restoring + assert os.path.getsize(wal_file) > 0 + assert os.path.getsize(broken_wal_file) # Data is lost without WAL - node9.restart_clickhouse(kill=True) - time.sleep(5) - assert node9.query("SELECT count() FROM wal_table") == "250\n" + node9.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0") + with PartitionManager() as pm: + pm.partition_instances(node9, node10) + + insert_random_data('wal_table', node9, 50) + check(node9, 350, 7) + + node9.restart_clickhouse(kill=True) + check(node9, 300, 6) + def test_polymorphic_parts_index(start_cluster): node1.query(''' diff --git a/tests/queries/0_stateless/01130_in_memory_parts.reference b/tests/queries/0_stateless/01130_in_memory_parts.reference index dbf39a0b48f..ae32d3ea7a3 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts.reference +++ b/tests/queries/0_stateless/01130_in_memory_parts.reference @@ -1,3 +1,4 @@ +InMemory 2 Simple selects 0 0 1 1 @@ -11,6 +12,8 @@ Simple selects 54 0 34 0 +20 +10 Mutations and Alters 66 1 1 @@ -25,3 +28,8 @@ Mutations and Alters [7,49] 1 1 2 1 +1 [1,1] +2 [] +4 [4,16] +5 [] +7 [7,49] diff --git a/tests/queries/0_stateless/01130_in_memory_parts.sql b/tests/queries/0_stateless/01130_in_memory_parts.sql index 4c09eb19937..b704fbdf081 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts.sql @@ -1,9 +1,10 @@ DROP TABLE IF EXISTS in_memory; CREATE TABLE in_memory (a UInt32, b UInt32) ENGINE = MergeTree ORDER BY a - SETTINGS min_rows_for_compact_part = 0; + SETTINGS min_rows_for_compact_part = 1000; INSERT INTO in_memory SELECT number, number % 3 FROM numbers(100); +SELECT DISTINCT part_type, marks FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory' AND active; SELECT 'Simple selects'; @@ -12,6 +13,8 @@ SELECT * FROM in_memory ORDER BY a LIMIT 5 OFFSET 50; SELECT count() FROM in_memory WHERE b = 0 SETTINGS max_block_size = 10; -- Check index SELECT count() FROM in_memory WHERE a > 100 SETTINGS max_rows_to_read = 0, force_primary_key = 1; +SELECT count() FROM in_memory WHERE a >= 10 AND a < 30 SETTINGS force_primary_key = 1; +SELECT DISTINCT blockSize() FROM in_memory SETTINGS max_block_size = 10; SELECT 'Mutations and Alters'; SET mutations_sync = 1; @@ -29,5 +32,8 @@ SELECT arr FROM in_memory ORDER BY a LIMIT 5; ALTER TABLE in_memory MODIFY COLUMN b String; ALTER TABLE in_memory RENAME COLUMN b to str; SELECT DISTINCT str, length(str) FROM in_memory ORDER BY str; +ALTER TABLE in_memory DROP COLUMN str; + +SELECT * FROM in_memory ORDER BY a LIMIT 5; DROP TABLE in_memory; From e8262ccaf67ed975cb8b07b198adeaa7dc47ab2c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 15 May 2020 03:53:12 +0300 Subject: [PATCH 012/196] in-memory parts: add perf test --- tests/performance/polymorphic_parts.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/performance/polymorphic_parts.xml b/tests/performance/polymorphic_parts.xml index a8e305953d0..46e308848da 100644 --- a/tests/performance/polymorphic_parts.xml +++ b/tests/performance/polymorphic_parts.xml @@ -13,6 +13,13 @@ SAMPLE BY intHash32(UserID) SETTINGS min_bytes_for_wide_part = '10M' + + CREATE TABLE hits_memory AS hits_10m_single ENGINE = MergeTree() + PARTITION BY toYYYYMM(EventDate) + ORDER BY (CounterID, EventDate, intHash32(UserID)) + SAMPLE BY intHash32(UserID) + SETTINGS min_bytes_for_compact_part = '1M', min_bytes_for_wide_part = '10M', in_memory_parts_enable_wal = 1 + CREATE TABLE hits_buffer AS hits_10m_single ENGINE = Buffer(merge, hits, 16, 10, 100, 10000, 1000000, 10000000, 100000000) @@ -28,6 +35,11 @@ INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(1000) INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(10000) + INSERT INTO hits_memory(UserID) VALUES (rand()) + INSERT INTO hits_memory(UserID) SELECT rand() FROM numbers(100) + INSERT INTO hits_memory(UserID) SELECT rand() FROM numbers(1000) + INSERT INTO hits_memory(UserID) SELECT rand() FROM numbers(10000) + INSERT INTO hits_buffer(UserID) VALUES (rand()) INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(100) INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(1000) @@ -35,5 +47,6 @@ DROP TABLE IF EXISTS hits_wide DROP TABLE IF EXISTS hits_compact + DROP TABLE IF EXISTS hits_memory DROP TABLE IF EXISTS hits_buffer From 4fb6492b08538d42089eca6b2b3322c843869b92 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 20 May 2020 15:02:02 +0300 Subject: [PATCH 013/196] in-memory parts: delay merges --- .../MergeTree/MergeTreeBlockOutputStream.cpp | 22 +++++++++++++++---- src/Storages/MergeTree/MergeTreeData.cpp | 21 ++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 19 ++++++++++++++++ .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTreeDataPartWriterInMemory.cpp | 7 +++--- .../MergeTree/MergeTreeWriteAheadLog.cpp | 5 ++++- src/Storages/StorageMergeTree.cpp | 7 ++++-- 7 files changed, 71 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 4f9500f973e..67ba2ba8d3e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -31,11 +32,24 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); - if (isInMemoryPart(part) && storage.getSettings()->in_memory_parts_insert_sync) + if (auto * part_in_memory = dynamic_cast(part.get())) { - if (!part->waitUntilMerged(in_memory_parts_timeout)) - throw Exception("Timeout exceeded while waiting to write part " - + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); + storage.in_memory_merges_throttler.add(part_in_memory->block.bytes(), part_in_memory->rows_count); + + auto settings = storage.getSettings(); + if (settings->in_memory_parts_insert_sync) + { + if (!part->waitUntilMerged(in_memory_parts_timeout)) + throw Exception("Timeout exceeded while waiting to write part " + + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); + } + else if (storage.merging_mutating_task_handle && !storage.in_memory_merges_throttler.needDelayMerge()) + { + storage.in_memory_merges_throttler.reset(); + storage.merging_mutating_task_handle->wake(); + } + + return; } /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bbe051a2476..2de2bb8656b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -142,6 +142,7 @@ MergeTreeData::MergeTreeData( , data_parts_by_info(data_parts_indexes.get()) , data_parts_by_state_and_info(data_parts_indexes.get()) , parts_mover(this) + , in_memory_merges_throttler(storage_settings.get()->min_bytes_for_compact_part, storage_settings.get()->min_rows_for_compact_part) { if (relative_data_path.empty()) throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -3673,4 +3674,24 @@ NamesAndTypesList MergeTreeData::getVirtuals() const NameAndTypePair("_sample_factor", std::make_shared()), }; } + +bool MergeTreeData::MergesThrottler::needDelayMerge() const +{ + std::lock_guard lock(mutex); + return (!max_bytes || have_bytes < max_bytes) && (!max_rows || have_rows < max_rows); +} + +void MergeTreeData::MergesThrottler::add(size_t bytes, size_t rows) +{ + std::lock_guard lock(mutex); + have_bytes += bytes; + have_rows += rows; +} + +void MergeTreeData::MergesThrottler::reset() +{ + have_bytes = 0; + have_rows = 0; +} + } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index ad83c2bf0d7..56e9ab1da7a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -921,6 +921,25 @@ protected: bool areBackgroundMovesNeeded() const; + struct MergesThrottler + { + mutable std::mutex mutex; + size_t have_bytes = 0; + size_t have_rows = 0; + + size_t max_bytes; + size_t max_rows; + + MergesThrottler(size_t max_bytes_, size_t max_rows_) + : max_bytes(max_bytes_), max_rows(max_rows_) {} + + bool needDelayMerge() const; + void add(size_t bytes, size_t rows); + void reset(); + }; + + MergesThrottler in_memory_merges_throttler; + private: /// RAII Wrapper for atomic work with currently moving parts /// Acuire them in constructor and remove them in destructor diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index ff56b407998..1420e7d1bff 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -998,7 +998,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor commands_for_part.emplace_back(command); } - if (!isStorageTouchedByMutations(storage_from_source_part, commands_for_part, context_for_reading)) + if (source_part->isStoredOnDisk() && !isStorageTouchedByMutations(storage_from_source_part, commands_for_part, context_for_reading)) { LOG_TRACE(log, "Part " << source_part->name << " doesn't change up to mutation version " << future_part.part_info.mutation); return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index 39e9757d81c..88c53107acf 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -45,6 +45,9 @@ void MergeTreeDataPartWriterInMemory::write( result_block.insert(block.getByName(col.name)); } + index_granularity.appendMark(result_block.rows()); + if (with_final_mark) + index_granularity.appendMark(0); part->block = std::move(result_block); block_written = true; } @@ -55,10 +58,6 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl if (!rows) return; - index_granularity.appendMark(rows); - if (with_final_mark) - index_granularity.appendMark(0); - size_t primary_columns_num = primary_index_block.columns(); index_columns.resize(primary_columns_num); for (size_t i = 0; i < primary_columns_num; ++i) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 02c45dcfb64..5ceedbe1f55 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -11,6 +11,7 @@ namespace ErrorCodes { extern const int UNKNOWN_FORMAT_VERSION; extern const int CANNOT_READ_ALL_DATA; + extern const int BAD_DATA_PART_NAME; } @@ -95,7 +96,9 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() } catch (const Exception & e) { - if (e.code() == ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == ErrorCodes::UNKNOWN_FORMAT_VERSION) + if (e.code() == ErrorCodes::CANNOT_READ_ALL_DATA + || e.code() == ErrorCodes::UNKNOWN_FORMAT_VERSION + || e.code() == ErrorCodes::BAD_DATA_PART_NAME) { LOG_WARNING(&Logger::get(storage.getLogName() + " (WriteAheadLog)"), "WAL file '" << path << "' is broken. " << e.displayText()); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 37044a9780b..ad2e0317882 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -822,8 +822,11 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() ///TODO: read deduplicate option from table config if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/)) - return BackgroundProcessingPoolTaskResult::SUCCESS; - + { + return in_memory_merges_throttler.needDelayMerge() + ? BackgroundProcessingPoolTaskResult::NOTHING_TO_DO + : BackgroundProcessingPoolTaskResult::SUCCESS; + } if (tryMutatePart()) return BackgroundProcessingPoolTaskResult::SUCCESS; From 6d8749b100eaf9144d24291ece74c8c644a9e991 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 May 2020 01:16:08 +0300 Subject: [PATCH 014/196] in-memory parts: faster computation of checksums --- src/Columns/ColumnAggregateFunction.cpp | 7 +++++++ src/Columns/ColumnAggregateFunction.h | 2 ++ src/Columns/ColumnArray.cpp | 6 ++++++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnConst.h | 5 +++++ src/Columns/ColumnDecimal.cpp | 6 ++++++ src/Columns/ColumnDecimal.h | 1 + src/Columns/ColumnFixedString.cpp | 6 ++++++ src/Columns/ColumnFixedString.h | 2 ++ src/Columns/ColumnFunction.h | 5 +++++ src/Columns/ColumnLowCardinality.cpp | 6 ++++++ src/Columns/ColumnLowCardinality.h | 2 ++ src/Columns/ColumnNullable.cpp | 6 ++++++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnString.h | 6 ++++++ src/Columns/ColumnTuple.cpp | 6 ++++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVector.cpp | 6 ++++++ src/Columns/ColumnVector.h | 2 ++ src/Columns/IColumn.h | 2 ++ src/Columns/IColumnDummy.h | 4 ++++ src/Columns/IColumnUnique.h | 5 +++++ src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h | 2 +- 25 files changed, 92 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 2f3a766b8f5..86cc3047230 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -309,6 +309,13 @@ void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const } } +void ColumnAggregateFunction::updateHashFast(SipHash & hash) const +{ + /// Fallback to per-element hashing, as there is no faster way + for (size_t i = 0; i < size(); ++i) + updateHashWithValue(i, hash); +} + /// The returned size is less than real size. The reason is that some parts of /// aggregate function data may be allocated on shared arenas. These arenas are /// used for several blocks, and also may be updated concurrently from other diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index f257351a4d0..dfae7f6f774 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -162,6 +162,8 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; + size_t byteSize() const override; size_t allocatedBytes() const override; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 7dba8e857cc..a67e37c6258 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -250,6 +250,12 @@ void ColumnArray::updateWeakHash32(WeakHash32 & hash) const } } +void ColumnArray::updateHashFast(SipHash & hash) const +{ + offsets->updateHashFast(hash); + data->updateHashFast(hash); +} + void ColumnArray::insert(const Field & x) { const Array & array = DB::get(x); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 15a1d1bd91a..fcf9ea3fc12 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -62,6 +62,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insert(const Field & x) override; void insertFrom(const IColumn & src_, size_t n) override; diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 560d4d63a10..799bdc8b657 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -165,6 +165,11 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override + { + data->updateHashFast(hash); + } + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 5396389294a..1b32f1457c8 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -89,6 +89,12 @@ void ColumnDecimal::updateWeakHash32(WeakHash32 & hash) const } } +template +void ColumnDecimal::updateHashFast(SipHash & hash) const +{ + hash.update(reinterpret_cast(data.data()), size() * sizeof(data[0])); +} + template void ColumnDecimal::getPermutation(bool reverse, size_t limit, int , IColumn::Permutation & res) const { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 62e414a676b..d5aa9c9a2a6 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -106,6 +106,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 57ae4cbdedf..d8bb31d8d11 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -124,6 +124,12 @@ void ColumnFixedString::updateWeakHash32(WeakHash32 & hash) const } } +void ColumnFixedString::updateHashFast(SipHash & hash) const +{ + hash.update(n); + hash.update(reinterpret_cast(chars.data()), size() * n); +} + template struct ColumnFixedString::less { diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 74c4f3c74f2..b04660ee0dd 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -110,6 +110,8 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; + int compareAt(size_t p1, size_t p2, const IColumn & rhs_, int /*nan_direction_hint*/) const override { const ColumnFixedString & rhs = assert_cast(rhs_); diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index 1bde48559fe..bb4bccadae3 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -106,6 +106,11 @@ public: throw Exception("updateWeakHash32 is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + void updateHashFast(SipHash &) const override + { + throw Exception("updateHashFast is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + void popBack(size_t) override { throw Exception("popBack is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index e87b3b4cbf6..c804ae9e1bf 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -257,6 +257,12 @@ void ColumnLowCardinality::updateWeakHash32(WeakHash32 & hash) const idx.updateWeakHash(hash, dict_hash); } +void ColumnLowCardinality::updateHashFast(SipHash & hash) const +{ + idx.getPositions()->updateHashFast(hash); + getDictionary().getNestedColumn()->updateHashFast(hash); +} + void ColumnLowCardinality::gather(ColumnGathererStream & gatherer) { gatherer.gather(*this); diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index e641cc177f3..996940aaa2d 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -92,6 +92,8 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash &) const override; + ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override { return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint)); diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 55ce1401073..fe9e81605dc 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -64,6 +64,12 @@ void ColumnNullable::updateWeakHash32(WeakHash32 & hash) const hash_data[row] = old_hash_data[row]; } +void ColumnNullable::updateHashFast(SipHash & hash) const +{ + null_map->updateHashFast(hash); + nested_column->updateHashFast(hash); +} + MutableColumnPtr ColumnNullable::cloneResized(size_t new_size) const { MutableColumnPtr new_nested_col = getNestedColumn().cloneResized(new_size); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 5443d8b0187..4dfc0007e94 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -85,6 +85,7 @@ public: ColumnPtr replicate(const Offsets & replicate_offsets) const override; void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; void getExtremes(Field & min, Field & max) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 32116880014..53686b7f744 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -190,6 +190,12 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override + { + hash.update(reinterpret_cast(offsets.data()), size() * sizeof(offsets[0])); + hash.update(reinterpret_cast(chars.data()), size() * sizeof(chars[0])); + } + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 59552c67f14..b20e01caf76 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -194,6 +194,12 @@ void ColumnTuple::updateWeakHash32(WeakHash32 & hash) const column->updateWeakHash32(hash); } +void ColumnTuple::updateHashFast(SipHash & hash) const +{ + for (const auto & column : columns) + column->updateHashFast(hash); +} + void ColumnTuple::insertRangeFrom(const IColumn & src, size_t start, size_t length) { const size_t tuple_size = columns.size(); diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 3533b602a1b..fde539e60da 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -62,6 +62,7 @@ public: const char * deserializeAndInsertFromArena(const char * pos) override; void updateHashWithValue(size_t n, SipHash & hash) const override; void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 9b128fcffec..6a0693d646d 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -84,6 +84,12 @@ void ColumnVector::updateWeakHash32(WeakHash32 & hash) const } } +template +void ColumnVector::updateHashFast(SipHash & hash) const +{ + hash.update(reinterpret_cast(data.data()), size() * sizeof(data[0])); +} + template struct ColumnVector::less { diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 3551efe890c..b86ee5d8093 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -162,6 +162,8 @@ public: void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; + size_t byteSize() const override { return data.size() * sizeof(data[0]); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 4af593bb658..7cba4b9918d 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -206,6 +206,8 @@ public: /// WeakHash32 must have the same size as column. virtual void updateWeakHash32(WeakHash32 & hash) const = 0; + virtual void updateHashFast(SipHash & hash) const = 0; + /** Removes elements that don't match the filter. * Is used in WHERE and HAVING operations. * If result_size_hint > 0, then makes advance reserve(result_size_hint) for the result column; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 00604fb87d0..0b9682a0fba 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -69,6 +69,10 @@ public: { } + void updateHashFast(SipHash & /*hash*/) const override + { + } + void insertFrom(const IColumn &, size_t) override { ++s; diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index af5d9878a3b..ee9342c1fdf 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -141,6 +141,11 @@ public: { throw Exception("Method updateWeakHash32 is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } + + void updateHashFast(SipHash &) const override + { + throw Exception("Method updateHashFast is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } }; using ColumnUniquePtr = IColumnUnique::ColumnUniquePtr; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index 88c53107acf..b31a0cbe51a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -81,7 +81,8 @@ static MergeTreeDataPartChecksum createUncompressedChecksum(size_t size, SipHash void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) { SipHash hash; - part->block.updateHash(hash); + for (const auto & column : part->block) + column.column->updateHashFast(hash); checksums.files["data.bin"] = createUncompressedChecksum(part->block.bytes(), hash); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 0543e6420ee..db3d2cecf37 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -21,7 +21,7 @@ void MergeTreeDataPartWriterOnDisk::Stream::finalize() marks.next(); } -void MergeTreeDataPartWriterOnDisk::Stream::sync() +void MergeTreeDataPartWriterOnDisk::Stream::sync() const { plain_file->sync(); marks_file->sync(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index bb54b964793..145db10c62c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -54,7 +54,7 @@ public: void finalize(); - void sync(); + void sync() const; void addToChecksums(IMergeTreeDataPart::Checksums & checksums); }; From d8342e5b120a716f861fe9fd9f262a7bc34e5efc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 27 May 2020 23:05:55 +0300 Subject: [PATCH 015/196] in-memory parts: better restore and clear stale wal files --- src/Storages/MergeTree/DataPartsExchange.cpp | 6 +- src/Storages/MergeTree/DataPartsExchange.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 - .../MergeTree/MergeTreeBlockOutputStream.cpp | 9 +- src/Storages/MergeTree/MergeTreeData.cpp | 61 +++++++++++ src/Storages/MergeTree/MergeTreeData.h | 3 + src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/MergeTreeWriteAheadLog.cpp | 24 ++++- .../MergeTree/MergeTreeWriteAheadLog.h | 4 +- .../ReplicatedMergeTreeCleanupThread.cpp | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 7 +- src/Storages/StorageMergeTree.cpp | 19 +++- src/Storages/StorageReplicatedMergeTree.cpp | 18 +++- .../configs/do_not_merge.xml | 2 + .../test_polymorphic_parts/test.py | 100 +++++++++++++----- 15 files changed, 213 insertions(+), 48 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index f61d80e63ac..5e0ee324e3a 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -276,13 +276,13 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) readStringBinary(part_type, in); - return part_type == "InMemory" ? downloadPartToMemory(part_name, replica_path, in) + return part_type == "InMemory" ? downloadPartToMemory(part_name, std::move(reservation), in) : downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( const String & part_name, - const String & /* replica_path */, + ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in) { MergeTreeData::DataPart::Checksums checksums; @@ -292,7 +292,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( NativeBlockInputStream block_in(in, 0); auto block = block_in.read(); MergeTreeData::MutableDataPartPtr new_data_part = - std::make_shared(data, part_name, nullptr); + std::make_shared(data, part_name, reservation->getDisk()); new_data_part->is_temp = true; new_data_part->setColumns(block.getNamesAndTypesList()); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 217434d7e41..1aa87d9f307 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -78,7 +78,7 @@ private: MergeTreeData::MutableDataPartPtr downloadPartToMemory( const String & part_name, - const String & replica_path, + ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in); MergeTreeData & data; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index a76251c2512..bb51a4e8e67 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -380,8 +380,6 @@ String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize() const String IMergeTreeDataPart::getFullPath() const { - // assertOnDisk(); //TODO - if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); @@ -390,8 +388,6 @@ String IMergeTreeDataPart::getFullPath() const String IMergeTreeDataPart::getFullRelativePath() const { - // assertOnDisk(); //TODO - if (relative_path.empty()) throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 67ba2ba8d3e..1b3f80b4e09 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -49,12 +49,13 @@ void MergeTreeBlockOutputStream::write(const Block & block) storage.merging_mutating_task_handle->wake(); } - return; + continue; } - - /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. - if (storage.merging_mutating_task_handle) + else if (storage.merging_mutating_task_handle) + { + /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. storage.merging_mutating_task_handle->wake(); + } } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 2de2bb8656b..c6bc54eabe8 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -58,6 +58,7 @@ #include #include #include +#include namespace ProfileEvents @@ -1046,6 +1047,9 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) for (auto & part : parts_from_wal) { + if (getActiveContainingPart(part->info, DataPartState::Committed, part_lock)) + continue; + part->modification_time = time(nullptr); /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later part->state = DataPartState::Committed; @@ -1322,6 +1326,61 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re } } +void MergeTreeData::clearOldWriteAheadLogs() +{ + DataPartsVector parts = getDataPartsVector(); + std::vector> all_block_numbers_on_disk; + std::vector> block_numbers_on_disk; + + for (const auto & part : parts) + if (part->isStoredOnDisk()) + all_block_numbers_on_disk.emplace_back(part->info.min_block, part->info.max_block); + + if (all_block_numbers_on_disk.empty()) + return; + + std::sort(all_block_numbers_on_disk.begin(), all_block_numbers_on_disk.end()); + block_numbers_on_disk.push_back(all_block_numbers_on_disk[0]); + for (size_t i = 1; i < all_block_numbers_on_disk.size(); ++i) + { + if (all_block_numbers_on_disk[i].first == all_block_numbers_on_disk[i - 1].second + 1) + block_numbers_on_disk.back().second = all_block_numbers_on_disk[i].second; + else + block_numbers_on_disk.push_back(all_block_numbers_on_disk[i]); + } + + auto is_range_on_disk = [&block_numbers_on_disk](Int64 min_block, Int64 max_block) + { + auto lower = std::upper_bound(block_numbers_on_disk.begin(), block_numbers_on_disk.end(), std::make_pair(min_block, -1L)); + if (lower != block_numbers_on_disk.end() && min_block >= lower->first && max_block <= lower->second) + return true; + + if (lower != block_numbers_on_disk.begin()) + { + --lower; + if (min_block >= lower->first && max_block <= lower->second) + return true; + } + + return false; + }; + + auto disks = getStoragePolicy()->getDisks(); + for (auto disk_it = disks.rbegin(); disk_it != disks.rend(); ++disk_it) + { + auto disk_ptr = *disk_it; + for (auto it = disk_ptr->iterateDirectory(relative_data_path); it->isValid(); it->next()) + { + auto min_max_block_number = MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(it->name()); + if (min_max_block_number && is_range_on_disk(min_max_block_number->first, min_max_block_number->second)) + { + LOG_DEBUG(log, "Removing from filesystem outdated WAL file " + it->name()); + disk_ptr->remove(relative_data_path + it->name()); + } + } + } +} + void MergeTreeData::rename(const String & new_table_path, const StorageID & new_table_id) { auto disks = getStoragePolicy()->getDisks(); @@ -1875,6 +1934,7 @@ void MergeTreeData::renameTempPartAndReplace( DataPartPtr covering_part; DataPartsVector covered_parts = getActivePartsToReplace(part_info, part_name, covering_part, lock); + DataPartsVector covered_parts_in_memory; if (covering_part) { @@ -3690,6 +3750,7 @@ void MergeTreeData::MergesThrottler::add(size_t bytes, size_t rows) void MergeTreeData::MergesThrottler::reset() { + std::lock_guard lock(mutex); have_bytes = 0; have_rows = 0; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 56e9ab1da7a..d56f6d57283 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -492,6 +492,9 @@ public: void clearOldPartsFromFilesystem(bool force = false); void clearPartsFromFilesystem(const DataPartsVector & parts); + /// Delete WAL files containing parts, that all already stored on disk. + void clearOldWriteAheadLogs(); + /// Delete all directories which names begin with "tmp" /// Set non-negative parameter value to override MergeTreeSettings temporary_directories_lifetime /// Must be called with locked lockStructureForShare(). diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 68e240f9d7e..0012ed89b77 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -35,6 +35,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, min_rows_for_compact_part, 0, "Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(SettingBool, in_memory_parts_enable_wal, 1, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(SettingBool, in_memory_parts_insert_sync, 0, "", 0) \ + M(SettingUInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ \ /** Merge settings. */ \ M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 5ceedbe1f55..e2dcea290de 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include namespace DB @@ -48,7 +49,8 @@ void MergeTreeWriteAheadLog::write(const Block & block, const String & part_name block_out->write(block); block_out->flush(); - if (out->count() > MAX_WAL_BYTES) + auto max_wal_bytes = storage.getSettings()->write_ahead_log_max_bytes; + if (out->count() > max_wal_bytes) rotate(); } @@ -106,7 +108,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() /// If file is broken, do not write new parts to it. /// But if it contains any part rotate and save them. if (max_block_number == -1) - Poco::File(path).remove(); + disk->remove(path); else if (name == DEFAULT_WAL_FILE) rotate(); @@ -135,4 +137,22 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() return result; } +std::optional +MergeTreeWriteAheadLog::tryParseMinMaxBlockNumber(const String & filename) +{ + Int64 min_block; + Int64 max_block; + ReadBufferFromString in(filename); + if (!checkString(WAL_FILE_NAME, in) + || !checkChar('_', in) + || !tryReadIntText(min_block, in) + || !checkChar('_', in) + || !tryReadIntText(max_block, in)) + { + return {}; + } + + return std::make_pair(min_block, max_block); +} + } diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 22665048f56..3081d51ecac 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -16,7 +16,6 @@ public: constexpr static auto WAL_FILE_NAME = "wal"; constexpr static auto WAL_FILE_EXTENSION = ".bin"; constexpr static auto DEFAULT_WAL_FILE = "wal.bin"; - constexpr static size_t MAX_WAL_BYTES = 1024 * 1024 * 1024; MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, const String & name = DEFAULT_WAL_FILE); @@ -24,6 +23,9 @@ public: void write(const Block & block, const String & part_name); std::vector restore(); + using MinMaxBlockNumber = std::pair; + static std::optional tryParseMinMaxBlockNumber(const String & filename); + private: void init(); void rotate(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index b1164f6621c..cbb0b3f5c0e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -54,6 +54,7 @@ void ReplicatedMergeTreeCleanupThread::run() void ReplicatedMergeTreeCleanupThread::iterate() { storage.clearOldPartsAndRemoveFromZK(); + storage.clearOldWriteAheadLogs(); { /// TODO: Implement tryLockStructureForShare. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 80e7e033525..39b7a6968a9 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1014,7 +1014,12 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( auto part = data.getPartIfExists(name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); if (part) - sum_parts_size_in_bytes += part->getBytesOnDisk(); + { + if (auto * part_in_memory = dynamic_cast(part.get())) + sum_parts_size_in_bytes += part_in_memory->block.bytes(); + else + sum_parts_size_in_bytes += part->getBytesOnDisk(); + } } if (merger_mutator.merges_blocker.isCancelled()) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ad2e0317882..73ae4cdd33c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -87,6 +87,7 @@ StorageMergeTree::StorageMergeTree( void StorageMergeTree::startup() { clearOldPartsFromFilesystem(); + clearOldWriteAheadLogs(); /// Temporary directories contain incomplete results of merges (after forced restart) /// and don't allow to reinitialize them, so delete each of them immediately @@ -632,8 +633,22 @@ bool StorageMergeTree::merge( new_part = merger_mutator.mergePartsToTemporaryPart( future_part, *merge_entry, table_lock_holder, time(nullptr), merging_tagger->reserved_space, deduplicate, force_ttl); + merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); + DataPartsVector parts_to_remove_immediately; + for (const auto & part : future_part.parts) + { + part->notifyMerged(); + if (isInMemoryPart(part)) + { + modifyPartState(part, DataPartState::Deleting); + parts_to_remove_immediately.push_back(part); + } + } + + removePartsFinally(parts_to_remove_immediately); + merging_tagger->is_successful = true; write_part_log({}); } @@ -644,9 +659,6 @@ bool StorageMergeTree::merge( throw; } - for (const auto & part : future_part.parts) - part->notifyMerged(); - return true; } @@ -818,6 +830,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() clearOldTemporaryDirectories(); } clearOldMutations(); + clearOldWriteAheadLogs(); } ///TODO: read deduplicate option from table config diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ab88b93fe01..2c6131caf63 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -298,6 +298,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( /// Temporary directories contain untinalized results of Merges or Fetches (after forced restart) /// and don't allow to reinitialize them, so delete each of them immediately clearOldTemporaryDirectories(0); + clearOldWriteAheadLogs(); } createNewZooKeeperNodes(); @@ -1050,7 +1051,8 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) for (auto & part_ptr : parts) { ttl_infos.update(part_ptr->ttl_infos); - max_volume_index = std::max(max_volume_index, getStoragePolicy()->getVolumeIndexByDisk(part_ptr->disk)); + if (part_ptr->isStoredOnDisk()) + max_volume_index = std::max(max_volume_index, getStoragePolicy()->getVolumeIndexByDisk(part_ptr->disk)); } ReservationPtr reserved_space = reserveSpacePreferringTTLRules(estimated_space_for_merge, ttl_infos, time(nullptr), max_volume_index); @@ -1091,6 +1093,20 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) try { checkPartChecksumsAndCommit(transaction, part); + + DataPartsVector parts_to_remove_immediatly; + for (const auto & part_ptr : parts) + { + part_ptr->notifyMerged(); + if (isInMemoryPart(part_ptr)) + { + modifyPartState(part_ptr, DataPartState::Deleting); + parts_to_remove_immediatly.push_back(part_ptr); + } + } + + tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_immediatly); + removePartsFinally(parts_to_remove_immediatly); } catch (const Exception & e) { diff --git a/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml b/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml index bc2dae31ad6..8b57af4f48e 100644 --- a/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml +++ b/tests/integration/test_polymorphic_parts/configs/do_not_merge.xml @@ -2,5 +2,7 @@ 1 2 + 100 + 0 diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index ba8b4b6b725..362204c307d 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -70,8 +70,11 @@ node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['confi 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", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) -node10 = cluster.add_instance('node10', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) +node9 = cluster.add_instance('node9', config_dir="configs", with_zookeeper=True) +node10 = cluster.add_instance('node10', config_dir="configs", 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) @pytest.fixture(scope="module") def start_cluster(): @@ -85,7 +88,8 @@ def start_cluster(): create_tables('polymorphic_table_wide', [node3, node4], [settings_wide, settings_compact], "shard2") create_tables_old_format('polymorphic_table', [node5, node6], "shard3") create_tables('in_memory_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard4") - create_tables('wal_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard4") + create_tables('wal_table', [node11, node12], [settings_in_memory, settings_in_memory], "shard4") + create_tables('restore_table', [node11, node12], [settings_in_memory, settings_in_memory], "shard5") yield cluster @@ -317,66 +321,106 @@ def test_in_memory(start_cluster): assert TSV(node10.query("SELECT part_type, count() FROM system.parts " \ "WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV(expected) + node9.query("SYSTEM START MERGES") + node10.query("SYSTEM START MERGES") + + assert_eq_with_retry(node9, "OPTIMIZE TABLE in_memory_table FINAL SETTINGS optimize_throw_if_noop = 1", "") + node10.query("SYSTEM SYNC REPLICA in_memory_table", timeout=20) + + assert node9.query("SELECT count() FROM in_memory_table") == "1300\n" + assert node10.query("SELECT count() FROM in_memory_table") == "1300\n" + + assert TSV(node9.query("SELECT part_type, count() FROM system.parts " \ + "WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV("Wide\t1\n") + assert TSV(node10.query("SELECT part_type, count() FROM system.parts " \ + "WHERE table = 'in_memory_table' AND active GROUP BY part_type ORDER BY part_type")) == TSV("Wide\t1\n") + def test_in_memory_wal(start_cluster): # Merges are disabled in config for i in range(5): - insert_random_data('wal_table', node9, 50) - node10.query("SYSTEM SYNC REPLICA wal_table", timeout=20) + insert_random_data('wal_table', node11, 50) + node12.query("SYSTEM SYNC REPLICA wal_table", timeout=20) def check(node, rows, parts): node.query("SELECT count() FROM wal_table") == "{}\n".format(rows) node.query("SELECT count() FROM system.parts WHERE table = 'wal_table' AND part_type = 'InMemory'") == "{}\n".format(parts) - check(node9, 250, 5) - check(node10, 250, 5) + check(node11, 250, 5) + check(node12, 250, 5) # WAL works at inserts - node9.restart_clickhouse(kill=True) - check(node9, 250, 5) + node11.restart_clickhouse(kill=True) + check(node11, 250, 5) # WAL works at fetches - node10.restart_clickhouse(kill=True) - check(node10, 250, 5) + node12.restart_clickhouse(kill=True) + check(node12, 250, 5) - insert_random_data('wal_table', node9, 50) - node10.query("SYSTEM SYNC REPLICA wal_table", timeout=20) + insert_random_data('wal_table', node11, 50) + node12.query("SYSTEM SYNC REPLICA wal_table", timeout=20) # Disable replication with PartitionManager() as pm: - pm.partition_instances(node9, node10) - check(node9, 300, 6) + pm.partition_instances(node11, node12) + check(node11, 300, 6) - wal_file = os.path.join(node9.path, "database/data/default/wal_table/wal.bin") + wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal.bin") # Corrupt wal file open(wal_file, 'rw+').truncate(os.path.getsize(wal_file) - 10) - node9.restart_clickhouse(kill=True) + node11.restart_clickhouse(kill=True) # Broken part is lost, but other restored successfully - check(node9, 250, 5) + check(node11, 250, 5) # WAL with blocks from 0 to 4 - broken_wal_file = os.path.join(node9.path, "database/data/default/wal_table/wal_0_4.bin") + broken_wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal_0_4.bin") assert os.path.exists(broken_wal_file) # Fetch lost part from replica - node9.query("SYSTEM SYNC REPLICA wal_table", timeout=20) - check(node9, 300, 6) + node11.query("SYSTEM SYNC REPLICA wal_table", timeout=20) + check(node11, 300, 6) #Check that new data is written to new wal, but old is still exists for restoring assert os.path.getsize(wal_file) > 0 - assert os.path.getsize(broken_wal_file) + assert os.path.exists(broken_wal_file) # Data is lost without WAL - node9.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0") + node11.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0") with PartitionManager() as pm: - pm.partition_instances(node9, node10) + pm.partition_instances(node11, node12) - insert_random_data('wal_table', node9, 50) - check(node9, 350, 7) + insert_random_data('wal_table', node11, 50) + check(node11, 350, 7) - node9.restart_clickhouse(kill=True) - check(node9, 300, 6) + node11.restart_clickhouse(kill=True) + check(node11, 300, 6) +def test_in_memory_wal_rotate(start_cluster): + # Write every part to single wal + node11.query("ALTER TABLE restore_table MODIFY SETTING write_ahead_log_max_bytes = 10") + for i in range(5): + insert_random_data('restore_table', node11, 50) + + for i in range(5): + wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i)) + assert os.path.exists(wal_file) + + for node in [node11, node12]: + node.query("ALTER TABLE restore_table MODIFY SETTING number_of_free_entries_in_pool_to_lower_max_size_of_merge = 0") + node.query("ALTER TABLE restore_table MODIFY SETTING max_bytes_to_merge_at_max_space_in_pool = 10000000") + + assert_eq_with_retry(node11, "OPTIMIZE TABLE restore_table FINAL SETTINGS optimize_throw_if_noop = 1", "") + # Restart to be sure, that clearing stale logs task was ran + node11.restart_clickhouse(kill=True) + + for i in range(5): + wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i)) + assert not os.path.exists(wal_file) + + # New wal file was created and ready to write part to it + wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal.bin") + assert os.path.exists(wal_file) + assert os.path.getsize(wal_file) == 0 def test_polymorphic_parts_index(start_cluster): node1.query(''' From 4769ce727176007239387909a30a1e5a7d11453a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 29 May 2020 02:59:13 +0300 Subject: [PATCH 016/196] in-memory parts: several fixes --- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/MergeTreeReaderInMemory.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 13 +++++++----- .../01130_in_memory_parts_default.reference | 1 + .../01130_in_memory_parts_default.sql | 20 +++++++++++++++++++ 6 files changed, 32 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_default.reference create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_default.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c6bc54eabe8..d9b6b5e8780 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1351,7 +1351,7 @@ void MergeTreeData::clearOldWriteAheadLogs() auto is_range_on_disk = [&block_numbers_on_disk](Int64 min_block, Int64 max_block) { - auto lower = std::upper_bound(block_numbers_on_disk.begin(), block_numbers_on_disk.end(), std::make_pair(min_block, -1L)); + auto lower = std::lower_bound(block_numbers_on_disk.begin(), block_numbers_on_disk.end(), std::make_pair(min_block, -1L)); if (lower != block_numbers_on_disk.end() && min_block >= lower->first && max_block <= lower->second) return true; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 29c01805529..b264ff73436 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -39,7 +39,7 @@ public: bool isStoredOnDisk() const override { return false; } - bool hasColumnFiles(const String & /* column_name */, const IDataType & /* type */) const override { return true; } + bool hasColumnFiles(const String & column_name, const IDataType & /* type */) const override { return !!getColumnPosition(column_name); } String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 671b36dfe86..5e4c3e49e3b 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -49,7 +49,7 @@ size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool /* continue_read if (!part_in_memory->block.has(name)) continue; - const auto block_column = part_in_memory->block.getByName(name).column; + const auto & block_column = part_in_memory->block.getByName(name).column; if (total_rows_read == 0 && part_rows <= max_rows_to_read) { res_columns[i] = block_column; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 73ae4cdd33c..c97319456ce 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -637,13 +637,16 @@ bool StorageMergeTree::merge( merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); DataPartsVector parts_to_remove_immediately; - for (const auto & part : future_part.parts) { - part->notifyMerged(); - if (isInMemoryPart(part)) + auto lock = lockParts(); + for (const auto & part : future_part.parts) { - modifyPartState(part, DataPartState::Deleting); - parts_to_remove_immediately.push_back(part); + part->notifyMerged(); + if (isInMemoryPart(part)) + { + modifyPartState(part, DataPartState::Deleting); + parts_to_remove_immediately.push_back(part); + } } } diff --git a/tests/queries/0_stateless/01130_in_memory_parts_default.reference b/tests/queries/0_stateless/01130_in_memory_parts_default.reference new file mode 100644 index 00000000000..c51afdb6658 --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_default.reference @@ -0,0 +1 @@ +0 bbb_aaa diff --git a/tests/queries/0_stateless/01130_in_memory_parts_default.sql b/tests/queries/0_stateless/01130_in_memory_parts_default.sql new file mode 100644 index 00000000000..61e20c84f3d --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_default.sql @@ -0,0 +1,20 @@ +-- Test 01266_default_prewhere_reqq, but with in-memory parts +DROP TABLE IF EXISTS t1; + +CREATE TABLE t1 +( + date Date, + s1 String, + s2 String +) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(date) ORDER BY (date, s1) +SETTINGS index_granularity = 8192, min_rows_for_compact_part = 1000, min_rows_for_wide_part = 1000; + + +set max_threads=1; + +insert into t1 (date, s1,s2) values(today()-1,'aaa','bbb'); +alter table t1 add column s3 String DEFAULT concat(s2,'_',s1); +-- insert into t1 (date, s1,s2) values(today(),'aaa2','bbb2'); +select ignore(date), s3 from t1 where s2='bbb'; + +DROP TABLE t1; From c919840722add2d34f006a1c3242e5767642b775 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 29 May 2020 18:02:12 +0300 Subject: [PATCH 017/196] in-memory parts: partition commands --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 - src/Storages/MergeTree/IMergeTreeDataPart.h | 9 ++- src/Storages/MergeTree/MergeTreeData.cpp | 16 ++-- .../MergeTree/MergeTreeDataPartInMemory.cpp | 34 +++++++- .../MergeTree/MergeTreeDataPartInMemory.h | 4 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 77 +++++++++++++------ .../MergeTree/MergeTreeWriteAheadLog.h | 10 ++- .../0_stateless/01130_in_memory_parts.sql | 2 +- .../01130_in_memory_parts_partitons.reference | 22 ++++++ .../01130_in_memory_parts_partitons.sql | 25 ++++++ 10 files changed, 162 insertions(+), 41 deletions(-) create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_partitons.reference create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_partitons.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index bb51a4e8e67..a0370dfad16 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -794,15 +794,11 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) void IMergeTreeDataPart::renameToDetached(const String & prefix) const { - assertOnDisk(); renameTo(getRelativePathForDetachedPart(prefix)); } void IMergeTreeDataPart::makeCloneInDetached(const String & prefix) const { - assertOnDisk(); - LOG_INFO(storage.log, "Detaching " << relative_path); - String destination_path = storage.relative_data_path + getRelativePathForDetachedPart(prefix); /// Backup is not recursive (max_level is 0), so do not copy inner directories diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 8943a9fcb1f..5ce8c854156 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -287,9 +287,10 @@ public: size_t getFileSizeOrZero(const String & file_name) const; String getFullRelativePath() const; String getFullPath() const; - virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = false) const; + void renameToDetached(const String & prefix) const; - void makeCloneInDetached(const String & prefix) const; + virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = false) const; + virtual void makeCloneInDetached(const String & prefix) const; /// Makes full clone of part in detached/ on another disk void makeCloneOnDiskDetached(const ReservationPtr & reservation) const; @@ -324,6 +325,8 @@ protected: /// disk using columns and checksums. virtual void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0; + String getRelativePathForDetachedPart(const String & prefix) const; + private: /// In compact parts order of columns is necessary NameToPosition column_name_to_position; @@ -348,8 +351,6 @@ private: void loadTTLInfos(); void loadPartitionAndMinMaxIndex(); - - String getRelativePathForDetachedPart(const String & prefix) const; }; using MergeTreeDataPartState = IMergeTreeDataPart::State; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d9b6b5e8780..e5c4744296d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -896,11 +896,17 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) part_names_with_disks.emplace_back(it->name(), disk_ptr); - if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) + /// Create and correctly initialize global WAL object, if it's needed + if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE && settings->in_memory_parts_enable_wal) + { + write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); + for (auto && part : write_ahead_log->restore()) + parts_from_wal.push_back(std::move(part)); + } + else if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) { MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); - auto current_parts = wal.restore(); - for (auto & part : current_parts) + for (auto && part : wal.restore()) parts_from_wal.push_back(std::move(part)); } } @@ -1120,7 +1126,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } - if (settings->in_memory_parts_enable_wal) + if (settings->in_memory_parts_enable_wal && !write_ahead_log) { auto disk = makeEmptyReservationOnLargestDisk()->getDisk(); write_ahead_log = std::make_shared(*this, std::move(disk)); @@ -1976,7 +1982,7 @@ void MergeTreeData::renameTempPartAndReplace( if (part_in_memory && getSettings()->in_memory_parts_enable_wal) { auto wal = getWriteAheadLog(); - wal->write(part_in_memory->block, part_in_memory->name); + wal->addPart(part_in_memory->block, part_in_memory->name); } if (out_covered_parts) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 41b35757ed8..5d376b88b74 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -1,9 +1,11 @@ #include "MergeTreeDataPartInMemory.h" #include +#include #include #include #include - +#include +#include namespace DB { @@ -58,6 +60,36 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( return std::make_unique(ptr, columns_list, writer_settings); } +void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix) const +{ + String detached_path = getRelativePathForDetachedPart(prefix); + String destination_path = storage.getRelativeDataPath() + getRelativePathForDetachedPart(prefix); + + auto new_type = storage.choosePartTypeOnDisk(block.bytes(), rows_count); + auto new_data_part = storage.createPart(name, new_type, info, disk, detached_path); + + new_data_part->setColumns(columns); + new_data_part->partition.value.assign(partition.value); + new_data_part->minmax_idx = minmax_idx; + + if (disk->exists(destination_path)) + { + LOG_WARNING(&Logger::get(storage.getLogName()), "Removing old temporary directory " + disk->getPath() + destination_path); + disk->removeRecursive(destination_path); + } + + disk->createDirectories(destination_path); + + auto compression_codec = storage.global_context.chooseCompressionCodec(0, 0); + MergedBlockOutputStream out(new_data_part, columns, storage.skip_indices, compression_codec); + out.writePrefix(); + out.write(block); + out.writeSuffixAndFinalizePart(new_data_part); + + if (storage.getSettings()->in_memory_parts_enable_wal) + storage.getWriteAheadLog()->dropPart(name); +} + bool MergeTreeDataPartInMemory::waitUntilMerged(size_t timeout) const { auto lock = storage.lockParts(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index b264ff73436..437bbb8e308 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -38,12 +38,10 @@ public: const MergeTreeIndexGranularity & computed_index_granularity) const override; bool isStoredOnDisk() const override { return false; } - bool hasColumnFiles(const String & column_name, const IDataType & /* type */) const override { return !!getColumnPosition(column_name); } - String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } - void renameTo(const String & /*new_relative_path*/, bool /*remove_new_dir_if_exists*/) const override {} + void makeCloneInDetached(const String & prefix) const override; bool waitUntilMerged(size_t timeout) const override; void notifyMerged() const override; diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index e2dcea290de..d3bec08073c 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -13,6 +13,7 @@ namespace ErrorCodes extern const int UNKNOWN_FORMAT_VERSION; extern const int CANNOT_READ_ALL_DATA; extern const int BAD_DATA_PART_NAME; + extern const int CORRUPTED_DATA; } @@ -36,7 +37,7 @@ void MergeTreeWriteAheadLog::init() max_block_number = -1; } -void MergeTreeWriteAheadLog::write(const Block & block, const String & part_name) +void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_name) { std::lock_guard lock(write_mutex); @@ -45,6 +46,7 @@ void MergeTreeWriteAheadLog::write(const Block & block, const String & part_name max_block_number = std::max(max_block_number, part_info.max_block); writeIntBinary(static_cast(0), *out); /// version + writeIntBinary(static_cast(ActionType::ADD_PART), *out); writeStringBinary(part_name, *out); block_out->write(block); block_out->flush(); @@ -54,6 +56,15 @@ void MergeTreeWriteAheadLog::write(const Block & block, const String & part_name rotate(); } +void MergeTreeWriteAheadLog::dropPart(const String & part_name) +{ + std::lock_guard lock(write_mutex); + + writeIntBinary(static_cast(0), *out); + writeIntBinary(static_cast(ActionType::DROP_PART), *out); + writeStringBinary(part_name, *out); +} + void MergeTreeWriteAheadLog::rotate() { String new_name = String(WAL_FILE_NAME) + "_" @@ -68,9 +79,10 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() { std::lock_guard lock(write_mutex); - MergeTreeData::MutableDataPartsVector result; + MergeTreeData::MutableDataPartsVector parts; auto in = disk->readFile(path, DBMS_DEFAULT_BUFFER_SIZE); NativeBlockInputStream block_in(*in, 0); + NameSet dropped_parts; while (!in->eof()) { @@ -78,6 +90,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() UInt8 version; String part_name; Block block; + ActionType action_type; try { @@ -85,22 +98,35 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() if (version != 0) throw Exception("Unknown WAL format version: " + toString(version), ErrorCodes::UNKNOWN_FORMAT_VERSION); + readIntBinary(action_type, *in); readStringBinary(part_name, *in); - part = storage.createPart( - part_name, - MergeTreeDataPartType::IN_MEMORY, - MergeTreePartInfo::fromPartName(part_name, storage.format_version), - storage.reserveSpace(0)->getDisk(), - part_name); + if (action_type == ActionType::DROP_PART) + { + dropped_parts.insert(part_name); + } + else if (action_type == ActionType::ADD_PART) + { + part = storage.createPart( + part_name, + MergeTreeDataPartType::IN_MEMORY, + MergeTreePartInfo::fromPartName(part_name, storage.format_version), + storage.reserveSpace(0)->getDisk(), + part_name); - block = block_in.read(); + block = block_in.read(); + } + else + { + throw Exception("Unknown action type: " + toString(static_cast(action_type)), ErrorCodes::CORRUPTED_DATA); + } } catch (const Exception & e) { if (e.code() == ErrorCodes::CANNOT_READ_ALL_DATA || e.code() == ErrorCodes::UNKNOWN_FORMAT_VERSION - || e.code() == ErrorCodes::BAD_DATA_PART_NAME) + || e.code() == ErrorCodes::BAD_DATA_PART_NAME + || e.code() == ErrorCodes::CORRUPTED_DATA) { LOG_WARNING(&Logger::get(storage.getLogName() + " (WriteAheadLog)"), "WAL file '" << path << "' is broken. " << e.displayText()); @@ -117,23 +143,30 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() throw; } - MergedBlockOutputStream part_out(part, block.getNamesAndTypesList(), {}, nullptr); + if (action_type == ActionType::ADD_PART) + { + MergedBlockOutputStream part_out(part, block.getNamesAndTypesList(), {}, nullptr); - part->minmax_idx.update(block, storage.minmax_idx_columns); - if (storage.partition_key_expr) - part->partition.create(storage, block, 0); - if (storage.hasSortingKey()) - storage.sorting_key_expr->execute(block); + part->minmax_idx.update(block, storage.minmax_idx_columns); + if (storage.partition_key_expr) + part->partition.create(storage, block, 0); + if (storage.hasSortingKey()) + storage.sorting_key_expr->execute(block); - part_out.writePrefix(); - part_out.write(block); - part_out.writeSuffixAndFinalizePart(part); + part_out.writePrefix(); + part_out.write(block); + part_out.writeSuffixAndFinalizePart(part); - min_block_number = std::min(min_block_number, part->info.min_block); - max_block_number = std::max(max_block_number, part->info.max_block); - result.push_back(std::move(part)); + min_block_number = std::min(min_block_number, part->info.min_block); + max_block_number = std::max(max_block_number, part->info.max_block); + parts.push_back(std::move(part)); + } } + MergeTreeData::MutableDataPartsVector result; + std::copy_if(parts.begin(), parts.end(), std::back_inserter(result), + [&dropped_parts](const auto & part) { return dropped_parts.count(part->name) == 0; }); + return result; } diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 3081d51ecac..e33cc8d534b 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -13,6 +13,13 @@ class MergeTreeData; class MergeTreeWriteAheadLog { public: + /// Append-only enum. It is serialized to WAL + enum class ActionType : UInt8 + { + ADD_PART = 0, + DROP_PART = 1, + }; + constexpr static auto WAL_FILE_NAME = "wal"; constexpr static auto WAL_FILE_EXTENSION = ".bin"; constexpr static auto DEFAULT_WAL_FILE = "wal.bin"; @@ -20,7 +27,8 @@ public: MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, const String & name = DEFAULT_WAL_FILE); - void write(const Block & block, const String & part_name); + void addPart(const Block & block, const String & part_name); + void dropPart(const String & part_name); std::vector restore(); using MinMaxBlockNumber = std::pair; diff --git a/tests/queries/0_stateless/01130_in_memory_parts.sql b/tests/queries/0_stateless/01130_in_memory_parts.sql index b704fbdf081..d6471cfb35f 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts.sql @@ -1,7 +1,7 @@ DROP TABLE IF EXISTS in_memory; CREATE TABLE in_memory (a UInt32, b UInt32) ENGINE = MergeTree ORDER BY a - SETTINGS min_rows_for_compact_part = 1000; + SETTINGS min_rows_for_compact_part = 1000, min_rows_for_compact_part = 1000; INSERT INTO in_memory SELECT number, number % 3 FROM numbers(100); SELECT DISTINCT part_type, marks FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory' AND active; diff --git a/tests/queries/0_stateless/01130_in_memory_parts_partitons.reference b/tests/queries/0_stateless/01130_in_memory_parts_partitons.reference new file mode 100644 index 00000000000..4f9c1b40e27 --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_partitons.reference @@ -0,0 +1,22 @@ +1 2 foo +1 3 bar +2 4 aa +2 5 bb +3 6 qq +3 7 ww +================== +2 4 aa +2 5 bb +3 6 qq +3 7 ww +================== +3 6 qq +3 7 ww +================== +2 4 aa +2 5 bb +3 6 qq +3 7 ww +2_4_4_0 Compact +3_3_3_0 InMemory +================== diff --git a/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql new file mode 100644 index 00000000000..88b0e3322ed --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS t2; + +CREATE TABLE t2(id UInt32, a UInt64, s String) + ENGINE = MergeTree ORDER BY a PARTITION BY id + SETTINGS min_rows_for_compact_part = 1000, min_rows_for_wide_part = 2000; + +INSERT INTO t2 VALUES (1, 2, 'foo'), (1, 3, 'bar'); +INSERT INTO t2 VALUES (2, 4, 'aa'), (2, 5, 'bb'); +INSERT INTO t2 VALUES (3, 6, 'qq'), (3, 7, 'ww'); + +SELECT * FROM t2 ORDER BY a; +SELECT '=================='; + +ALTER TABLE t2 DROP PARTITION 1; +SELECT * FROM t2 ORDER BY a; +SELECT '=================='; + +ALTER TABLE t2 DETACH PARTITION 2; +SELECT * FROM t2 ORDER BY a; +SELECT '=================='; + +ALTER TABLE t2 ATTACH PARTITION 2; +SELECT * FROM t2 ORDER BY a; +SELECT name, part_type FROM system.parts WHERE table = 't2' AND active AND database = currentDatabase() ORDER BY name; +SELECT '=================='; From fa05641b063408804d92cd9eed54572a20a0ee21 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 29 May 2020 19:58:08 +0300 Subject: [PATCH 018/196] in-memory parts: partition commands --- src/Storages/MergeTree/MergeTreeData.cpp | 17 +++++++++++- .../MergeTree/MergeTreeDataPartInMemory.cpp | 19 +++++++------ .../MergeTree/MergeTreeDataPartInMemory.h | 2 ++ .../01130_in_memory_parts_partitons.reference | 14 ++++++++++ .../01130_in_memory_parts_partitons.sql | 27 +++++++++++++++++++ 5 files changed, 70 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e5c4744296d..6b6f3a2f45d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2021,6 +2021,9 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect if (part->state != IMergeTreeDataPart::State::Outdated) modifyPartState(part,IMergeTreeDataPart::State::Outdated); + + if (isInMemoryPart(part) && write_ahead_log) + write_ahead_log->dropPart(part->name); } } @@ -3317,6 +3320,14 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( if (disk->exists(dst_part_path)) throw Exception("Part in " + fullPath(disk, dst_part_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); + /// If source part is in memory, flush it to disk and clone it already in on-disk format + if (auto * src_part_in_memory = dynamic_cast(src_part.get())) + { + auto flushed_part_path = tmp_part_prefix + src_part_in_memory->name; + src_part_in_memory->flushToDisk(relative_data_path, flushed_part_path); + src_part_path = src_part_in_memory->storage.relative_data_path + flushed_part_path + "/"; + } + LOG_DEBUG(log, "Cloning part " << fullPath(disk, src_part_path) << " to " << fullPath(disk, dst_part_path)); localBackup(disk, src_part_path, dst_part_path); disk->removeIfExists(dst_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH); @@ -3404,7 +3415,11 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & LOG_DEBUG(log, "Freezing part " << part->name << " snapshot will be placed at " + backup_path); String backup_part_path = backup_path + relative_data_path + part->relative_path; - localBackup(part->disk, part->getFullRelativePath(), backup_part_path); + if (auto part_in_memory = dynamic_cast(part.get())) + part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path); + else + localBackup(part->disk, part->getFullRelativePath(), backup_part_path); + part->disk->removeIfExists(backup_part_path + "/" + DELETE_ON_DESTROY_MARKER_PATH); part->is_frozen.store(true, std::memory_order_relaxed); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 5d376b88b74..bdcc00758d1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -13,6 +13,7 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; + extern const int DIRECTORY_ALREADY_EXISTS; } @@ -60,13 +61,12 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( return std::make_unique(ptr, columns_list, writer_settings); } -void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix) const +void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const String & new_relative_path) const { - String detached_path = getRelativePathForDetachedPart(prefix); - String destination_path = storage.getRelativeDataPath() + getRelativePathForDetachedPart(prefix); + String destination_path = base_path + new_relative_path; auto new_type = storage.choosePartTypeOnDisk(block.bytes(), rows_count); - auto new_data_part = storage.createPart(name, new_type, info, disk, detached_path); + auto new_data_part = storage.createPart(name, new_type, info, disk, new_relative_path); new_data_part->setColumns(columns); new_data_part->partition.value.assign(partition.value); @@ -74,8 +74,8 @@ void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix) const if (disk->exists(destination_path)) { - LOG_WARNING(&Logger::get(storage.getLogName()), "Removing old temporary directory " + disk->getPath() + destination_path); - disk->removeRecursive(destination_path); + throw Exception("Could not flush part " + quoteString(getFullPath()) + + ". Part in " + fullPath(disk, destination_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); } disk->createDirectories(destination_path); @@ -85,9 +85,12 @@ void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix) const out.writePrefix(); out.write(block); out.writeSuffixAndFinalizePart(new_data_part); +} - if (storage.getSettings()->in_memory_parts_enable_wal) - storage.getWriteAheadLog()->dropPart(name); +void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix) const +{ + String detached_path = getRelativePathForDetachedPart(prefix); + flushToDisk(storage.getRelativeDataPath(), detached_path); } bool MergeTreeDataPartInMemory::waitUntilMerged(size_t timeout) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 437bbb8e308..08e63ac182e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -43,6 +43,8 @@ public: void renameTo(const String & /*new_relative_path*/, bool /*remove_new_dir_if_exists*/) const override {} void makeCloneInDetached(const String & prefix) const override; + void flushToDisk(const String & base_path, const String & new_relative_path) const; + bool waitUntilMerged(size_t timeout) const override; void notifyMerged() const override; diff --git a/tests/queries/0_stateless/01130_in_memory_parts_partitons.reference b/tests/queries/0_stateless/01130_in_memory_parts_partitons.reference index 4f9c1b40e27..b9daa88b4ca 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts_partitons.reference +++ b/tests/queries/0_stateless/01130_in_memory_parts_partitons.reference @@ -20,3 +20,17 @@ 2_4_4_0 Compact 3_3_3_0 InMemory ================== +2 4 aa +2 5 bb +3 6 qq +3 7 ww +================== +2 4 aa +2 5 bb +3 6 cc +3 7 dd +t2 2_4_4_0 Compact +t2 3_6_6_0 Compact +t3 3_1_1_0 InMemory +================== +3_1_1_0 InMemory 1 diff --git a/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql index 88b0e3322ed..c07d65f114c 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts_partitons.sql @@ -4,6 +4,8 @@ CREATE TABLE t2(id UInt32, a UInt64, s String) ENGINE = MergeTree ORDER BY a PARTITION BY id SETTINGS min_rows_for_compact_part = 1000, min_rows_for_wide_part = 2000; +SYSTEM STOP MERGES t2; + INSERT INTO t2 VALUES (1, 2, 'foo'), (1, 3, 'bar'); INSERT INTO t2 VALUES (2, 4, 'aa'), (2, 5, 'bb'); INSERT INTO t2 VALUES (3, 6, 'qq'), (3, 7, 'ww'); @@ -23,3 +25,28 @@ ALTER TABLE t2 ATTACH PARTITION 2; SELECT * FROM t2 ORDER BY a; SELECT name, part_type FROM system.parts WHERE table = 't2' AND active AND database = currentDatabase() ORDER BY name; SELECT '=================='; + +DETACH TABLE t2; +ATTACH TABLE t2; + +SELECT * FROM t2 ORDER BY a; +SELECT '=================='; + +DROP TABLE IF EXISTS t3; + +CREATE TABLE t3(id UInt32, a UInt64, s String) + ENGINE = MergeTree ORDER BY a PARTITION BY id + SETTINGS min_rows_for_compact_part = 1000, min_rows_for_wide_part = 2000; + +INSERT INTO t3 VALUES (3, 6, 'cc'), (3, 7, 'dd'); +ALTER TABLE t2 REPLACE PARTITION 3 FROM t3; +SELECT * FROM t2 ORDER BY a; +SELECT table, name, part_type FROM system.parts WHERE table = 't2' AND active AND database = currentDatabase() ORDER BY name; +SELECT table, name, part_type FROM system.parts WHERE table = 't3' AND active AND database = currentDatabase() ORDER BY name; +SELECT '=================='; + +ALTER TABLE t3 FREEZE PARTITION 3; +SELECT name, part_type, is_frozen FROM system.parts WHERE table = 't3' AND active AND database = currentDatabase() ORDER BY name; + +DROP TABLE t2; +DROP TABLE t3; From e1970f6d28540e7baa6a5694a0d866ace3887d1b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 1 Jun 2020 20:52:09 +0300 Subject: [PATCH 019/196] in-memory parts: fix reading of nested --- src/Storages/MergeTree/IMergeTreeReader.cpp | 18 +++++- src/Storages/MergeTree/IMergeTreeReader.h | 5 +- .../MergeTree/MergeTreeReaderCompact.cpp | 20 +------ .../MergeTree/MergeTreeReaderCompact.h | 3 - .../MergeTree/MergeTreeReaderInMemory.cpp | 56 ++++++++++++++----- .../MergeTree/MergeTreeReaderInMemory.h | 2 + .../01130_in_memory_parts_nested.reference | 15 +++++ .../01130_in_memory_parts_nested.sql | 16 ++++++ 8 files changed, 96 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_nested.reference create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_nested.sql diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index a2984421c2a..624de2886a8 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -248,7 +248,23 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) } } -void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) +IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const String & column_name) const +{ + String table_name = Nested::extractTableName(column_name); + for (const auto & part_column : data_part->getColumns()) + { + if (typeid_cast(part_column.type.get())) + { + auto position = data_part->getColumnPosition(part_column.name); + if (position && Nested::extractTableName(part_column.name) == table_name) + return position; + } + } + + return {}; +} + +void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const { if (num_columns_to_read != columns.size()) throw Exception("invalid number of columns passed to MergeTreeReader::readRows. " diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 79f7860d1cc..90a43a61536 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -61,7 +61,7 @@ protected: /// Returns actual column type in part, which can differ from table metadata. NameAndTypePair getColumnFromPart(const NameAndTypePair & required_column) const; - void checkNumberOfColumns(size_t columns_num_to_read); + void checkNumberOfColumns(size_t columns_num_to_read) const; /// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size. ValueSizeMap avg_value_size_hints; @@ -79,6 +79,9 @@ protected: const MergeTreeData & storage; MarkRanges all_mark_ranges; + using ColumnPosition = std::optional; + ColumnPosition findColumnForOffsets(const String & column_name) const; + friend class MergeTreeRangeReader::DelayedStream; private: diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index a63397b9b9c..5b84069cc2c 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -84,12 +84,11 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( if (!position && typeid_cast(type.get())) { /// If array of Nested column is missing in part, - /// we have to read it's offsets if they exists. + /// we have to read its offsets if they exist. position = findColumnForOffsets(name); read_only_offsets[i] = (position != std::nullopt); } - column_positions[i] = std::move(position); } @@ -168,23 +167,6 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, return read_rows; } -MergeTreeReaderCompact::ColumnPosition MergeTreeReaderCompact::findColumnForOffsets(const String & column_name) -{ - String table_name = Nested::extractTableName(column_name); - for (const auto & part_column : data_part->getColumns()) - { - if (typeid_cast(part_column.type.get())) - { - auto position = data_part->getColumnPosition(part_column.name); - if (position && Nested::extractTableName(part_column.name) == table_name) - return position; - } - } - - return {}; -} - - 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) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 827306cd983..75d1da342fb 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -40,7 +40,6 @@ private: MergeTreeMarksLoader marks_loader; - using ColumnPosition = std::optional; /// Positions of columns in part structure. std::vector column_positions; /// Should we read full column or only it's offsets @@ -53,8 +52,6 @@ private: 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); - - ColumnPosition findColumnForOffsets(const String & column_name); }; } diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 5e4c3e49e3b..ce6eb44a50b 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include namespace DB @@ -24,10 +25,20 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( std::move(settings_), {}) , part_in_memory(std::move(data_part_)) { + for (const auto & name_and_type : columns) + { + auto [name, type] = getColumnFromPart(name_and_type); + if (!part_in_memory->block.has(name) && typeid_cast(type.get())) + if (auto offset_position = findColumnForOffsets(name)) + positions_for_offsets[name] = *offset_position; + } } -size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool /* continue_reading */, size_t max_rows_to_read, Columns & res_columns) +size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { + if (!continue_reading) + total_rows_read = 0; + size_t total_marks = data_part->index_granularity.getMarksCount(); if (from_mark >= total_marks) throw Exception("Mark " + toString(from_mark) + " is out of bound. Max mark: " @@ -41,34 +52,49 @@ size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool /* continue_read throw Exception("Cannot read data in MergeTreeReaderInMemory. Rows already read: " + toString(total_rows_read) + ". Rows in part: " + toString(part_rows), ErrorCodes::CANNOT_READ_ALL_DATA); + size_t rows_to_read = std::min(max_rows_to_read, part_rows - total_rows_read); auto column_it = columns.begin(); - size_t rows_read = 0; for (size_t i = 0; i < num_columns; ++i, ++column_it) { auto [name, type] = getColumnFromPart(*column_it); - if (!part_in_memory->block.has(name)) - continue; - const auto & block_column = part_in_memory->block.getByName(name).column; - if (total_rows_read == 0 && part_rows <= max_rows_to_read) - { - res_columns[i] = block_column; - rows_read = part_rows; - } - else + auto offsets_it = positions_for_offsets.find(name); + if (offsets_it != positions_for_offsets.end()) { + const auto & source_offsets = assert_cast( + *part_in_memory->block.getByPosition(offsets_it->second).column).getOffsets(); + if (res_columns[i] == nullptr) res_columns[i] = type->createColumn(); auto mutable_column = res_columns[i]->assumeMutable(); - rows_read = std::min(max_rows_to_read, part_rows - total_rows_read); - mutable_column->insertRangeFrom(*block_column, total_rows_read, rows_read); + auto & res_offstes = assert_cast(*mutable_column).getOffsets(); + for (size_t row = 0; row < rows_to_read; ++row) + res_offstes.push_back(source_offsets[total_rows_read + row]); + res_columns[i] = std::move(mutable_column); } + else if (part_in_memory->block.has(name)) + { + const auto & block_column = part_in_memory->block.getByName(name).column; + if (rows_to_read == part_rows) + { + res_columns[i] = block_column; + } + else + { + if (res_columns[i] == nullptr) + res_columns[i] = type->createColumn(); + + auto mutable_column = res_columns[i]->assumeMutable(); + mutable_column->insertRangeFrom(*block_column, total_rows_read, rows_to_read); + res_columns[i] = std::move(mutable_column); + } + } } - total_rows_read += rows_read; - return rows_read; + total_rows_read += rows_to_read; + return rows_to_read; } } diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h index 6d64801682e..196fc53725a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.h +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -29,6 +29,8 @@ public: private: size_t total_rows_read = 0; DataPartInMemoryPtr part_in_memory; + + std::unordered_map positions_for_offsets; }; } diff --git a/tests/queries/0_stateless/01130_in_memory_parts_nested.reference b/tests/queries/0_stateless/01130_in_memory_parts_nested.reference new file mode 100644 index 00000000000..abc233c46f4 --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_nested.reference @@ -0,0 +1,15 @@ +[0] +[0,0,0] +[0,0,0,0,0] +[0,0,0,0,0,0,0] +[0,0,0,0,0,0,0,0,0] +[0] +[0,2,4] +[0,2,4,6,8] +[0,2,4,6,8,10,12] +[0,2,4,6,8,10,12,14,16] +[0] [0] +[0,1,2] [0,2,4] +[0,1,2,3,4] [0,2,4,6,8] +[0,1,2,3,4,5,6] [0,2,4,6,8,10,12] +[0,1,2,3,4,5,6,7,8] [0,2,4,6,8,10,12,14,16] diff --git a/tests/queries/0_stateless/01130_in_memory_parts_nested.sql b/tests/queries/0_stateless/01130_in_memory_parts_nested.sql new file mode 100644 index 00000000000..c09593d01bc --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_nested.sql @@ -0,0 +1,16 @@ +-- Test 00576_nested_and_prewhere, but with in-memory parts. +DROP TABLE IF EXISTS nested; + +CREATE TABLE nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeTree ORDER BY x + SETTINGS min_rows_for_compact_part = 200000, min_rows_for_wide_part = 300000; + +INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000; + +ALTER TABLE nested ADD COLUMN n.b Array(UInt64); +SELECT DISTINCT n.b FROM nested PREWHERE filter; + +ALTER TABLE nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a); +SELECT DISTINCT n.c FROM nested PREWHERE filter; +SELECT DISTINCT n.a, n.c FROM nested PREWHERE filter; + +DROP TABLE nested; From caf1e4e8cc2261c8fef3b3b9d2e809029e00e92d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jun 2020 12:51:23 +0300 Subject: [PATCH 020/196] in-memory-parts: fixes --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 9 +++++++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++---- src/Storages/MergeTree/MergeTreeData.h | 3 ++- src/Storages/MergeTree/MergeTreeReaderInMemory.cpp | 4 ++++ .../0_stateless/01130_in_memory_parts_nested.reference | 5 +++++ .../queries/0_stateless/01130_in_memory_parts_nested.sql | 1 + 7 files changed, 26 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index a0370dfad16..3cb9a4c5924 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -766,7 +766,7 @@ void IMergeTreeDataPart::remove() const } } -String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const +String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const { /// Do not allow underscores in the prefix because they are used as separators. @@ -780,7 +780,7 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) */ for (int try_no = 0; try_no < 10; try_no++) { - res = "detached/" + (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : ""); + res = (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : ""); if (!disk->exists(getFullRelativePath() + res)) return res; @@ -792,6 +792,11 @@ String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) return res; } +String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const +{ + return "detached/" + getRelativePathForPrefix(prefix); +} + void IMergeTreeDataPart::renameToDetached(const String & prefix) const { renameTo(getRelativePathForDetachedPart(prefix)); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 5ce8c854156..3158419a16f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -301,6 +301,8 @@ public: static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from); void calculateColumnsSizesOnDisk(); + String getRelativePathForPrefix(const String & prefix) const; + protected: /// Total size of all columns, calculated once in calcuateColumnSizesOnDisk ColumnSize total_columns_size; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6b6f3a2f45d..df7d484f751 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1700,7 +1700,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::createPart( else { /// Didn't find any mark file, suppose that part is empty. - type = choosePartType(0, 0); + type = choosePartTypeOnDisk(0, 0); } return createPart(name, type, part_info, disk, relative_path); @@ -3323,9 +3323,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( /// If source part is in memory, flush it to disk and clone it already in on-disk format if (auto * src_part_in_memory = dynamic_cast(src_part.get())) { - auto flushed_part_path = tmp_part_prefix + src_part_in_memory->name; - src_part_in_memory->flushToDisk(relative_data_path, flushed_part_path); - src_part_path = src_part_in_memory->storage.relative_data_path + flushed_part_path + "/"; + const auto & src_relative_data_path = src_part_in_memory->storage.relative_data_path; + auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); + src_part_in_memory->flushToDisk(src_relative_data_path, flushed_part_path); + src_part_path = src_relative_data_path + flushed_part_path + "/"; } LOG_DEBUG(log, "Cloning part " << fullPath(disk, src_part_path) << " to " << fullPath(disk, dst_part_path)); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index d56f6d57283..31a05bf46b1 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -199,7 +199,8 @@ public: MergeTreeDataPartType type, const MergeTreePartInfo & part_info, const DiskPtr & disk, const String & relative_path) const; - /// After this methods 'loadColumnsChecksumsIndexes' must be called + /// Create part, that already exists on filesystem. + /// After this methods 'loadColumnsChecksumsIndexes' must be called. MutableDataPartPtr createPart(const String & name, const DiskPtr & disk, const String & relative_path) const; diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index ce6eb44a50b..924198b49f1 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -28,6 +28,9 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( for (const auto & name_and_type : columns) { auto [name, type] = getColumnFromPart(name_and_type); + + /// If array of Nested column is missing in part, + /// we have to read its offsets if they exist. if (!part_in_memory->block.has(name) && typeid_cast(type.get())) if (auto offset_position = findColumnForOffsets(name)) positions_for_offsets[name] = *offset_position; @@ -58,6 +61,7 @@ size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading { auto [name, type] = getColumnFromPart(*column_it); + /// Copy offsets, if array of Nested column is missing in part. auto offsets_it = positions_for_offsets.find(name); if (offsets_it != positions_for_offsets.end()) { diff --git a/tests/queries/0_stateless/01130_in_memory_parts_nested.reference b/tests/queries/0_stateless/01130_in_memory_parts_nested.reference index abc233c46f4..23e93d4be90 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts_nested.reference +++ b/tests/queries/0_stateless/01130_in_memory_parts_nested.reference @@ -4,6 +4,11 @@ [0,0,0,0,0,0,0] [0,0,0,0,0,0,0,0,0] [0] +[0,0,0] +[0,0,0,0,0] +[0,0,0,0,0,0,0] +[0,0,0,0,0,0,0,0,0] +[0] [0,2,4] [0,2,4,6,8] [0,2,4,6,8,10,12] diff --git a/tests/queries/0_stateless/01130_in_memory_parts_nested.sql b/tests/queries/0_stateless/01130_in_memory_parts_nested.sql index c09593d01bc..f643a65e041 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts_nested.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts_nested.sql @@ -8,6 +8,7 @@ INSERT INTO nested SELECT number, number % 2, range(number % 10) FROM system.num ALTER TABLE nested ADD COLUMN n.b Array(UInt64); SELECT DISTINCT n.b FROM nested PREWHERE filter; +SELECT DISTINCT n.b FROM nested PREWHERE filter SETTINGS max_block_size = 10; ALTER TABLE nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a); SELECT DISTINCT n.c FROM nested PREWHERE filter; From 11c4e9dde3c248e66692860337eb3ac92c618bb3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Jun 2020 21:59:18 +0300 Subject: [PATCH 021/196] in-memory parts: fix 'check' query --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 +++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeDataPartInMemory.h | 7 +------ src/Storages/MergeTree/checkDataPart.cpp | 3 +++ src/Storages/StorageMergeTree.cpp | 2 +- .../0_stateless/01130_in_memory_parts_check.reference | 1 + .../0_stateless/01130_in_memory_parts_check.sql | 10 ++++++++++ 9 files changed, 28 insertions(+), 14 deletions(-) create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_check.reference create mode 100644 tests/queries/0_stateless/01130_in_memory_parts_check.sql diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index ca5ad352564..6ef4cc15032 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -289,7 +289,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA); NativeBlockInputStream block_in(in, 0); - auto block = block_in.read(); + auto block = block_in.read(); auto volume = std::make_shared("volume_" + part_name, reservation->getDisk()); MergeTreeData::MutableDataPartPtr new_data_part = diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index e06ab8eed44..22accdbe190 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -889,6 +889,11 @@ void IMergeTreeDataPart::checkConsistencyBase() const } } +void IMergeTreeDataPart::checkConsistency(bool /* require_part_metadata */) const +{ + throw Exception("Method 'checkConsistency' is not implemented for part with type " + getType().toString(), ErrorCodes::NOT_IMPLEMENTED); +} + void IMergeTreeDataPart::calculateColumnsSizesOnDisk() { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 0921b24673c..f9f1965aa57 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -332,7 +332,7 @@ protected: void removeIfNeeded(); - virtual void checkConsistency(bool require_part_metadata) const = 0; + virtual void checkConsistency(bool require_part_metadata) const; void checkConsistencyBase() const; /// Fill each_columns_size and total_size with sizes from columns files on diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 9abfd44e42f..fac7e4982bb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -108,14 +108,14 @@ void MergeTreeDataPartInMemory::notifyMerged() const is_merged.notify_one(); } +void MergeTreeDataPartInMemory::renameTo(const String & new_relative_path, bool /* remove_new_dir_if_exists */) const +{ + relative_path = new_relative_path; +} + void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName & /*each_columns_size*/, ColumnSize & /*total_size*/) const { // throw Exception("calculateEachColumnSizesOnDisk of in memory part", ErrorCodes::NOT_IMPLEMENTED); } -void MergeTreeDataPartInMemory::loadIndexGranularity() -{ - throw Exception("loadIndexGranularity of in memory part", ErrorCodes::NOT_IMPLEMENTED); -} - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index ad9a583f5c8..3e2ec82b038 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -40,7 +40,7 @@ public: bool isStoredOnDisk() const override { return false; } bool hasColumnFiles(const String & column_name, const IDataType & /* type */) const override { return !!getColumnPosition(column_name); } String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } - void renameTo(const String & /*new_relative_path*/, bool /*remove_new_dir_if_exists*/) const override {} + void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const override; void makeCloneInDetached(const String & prefix) const override; void flushToDisk(const String & base_path, const String & new_relative_path) const; @@ -53,11 +53,6 @@ public: private: mutable std::condition_variable is_merged; - void checkConsistency(bool /* require_part_metadata */) const override {} - - /// Loads marks index granularity into memory - void loadIndexGranularity() override; - /// Compact parts doesn't support per column size, only total size void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; }; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 58ff2af9466..8395a7461a7 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -166,6 +166,9 @@ IMergeTreeDataPart::Checksums checkDataPart( bool require_checksums, std::function is_cancelled) { + if (!data_part->isStoredOnDisk()) + return data_part->checksums; + return checkDataPart( data_part->volume->getDisk(), data_part->getFullRelativePath(), diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index b32fcc1d188..44a942551fb 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1315,7 +1315,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, const Context & c /// If the checksums file is not present, calculate the checksums and write them to disk. String checksums_path = part_path + "checksums.txt"; String tmp_checksums_path = part_path + "checksums.txt.tmp"; - if (!disk->exists(checksums_path)) + if (part->isStoredOnDisk() && !disk->exists(checksums_path)) { try { diff --git a/tests/queries/0_stateless/01130_in_memory_parts_check.reference b/tests/queries/0_stateless/01130_in_memory_parts_check.reference new file mode 100644 index 00000000000..15f72836ff1 --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_check.reference @@ -0,0 +1 @@ +201901_1_1_0 1 diff --git a/tests/queries/0_stateless/01130_in_memory_parts_check.sql b/tests/queries/0_stateless/01130_in_memory_parts_check.sql new file mode 100644 index 00000000000..57cd1c83528 --- /dev/null +++ b/tests/queries/0_stateless/01130_in_memory_parts_check.sql @@ -0,0 +1,10 @@ +-- Part of 00961_check_table test, but with in-memory parts +SET check_query_single_value_result = 0; +DROP TABLE IF EXISTS mt_table; +CREATE TABLE mt_table (d Date, key UInt64, data String) ENGINE = MergeTree() PARTITION BY toYYYYMM(d) ORDER BY key + SETTINGS min_rows_for_compact_part = 1000, min_rows_for_compact_part = 1000; + +CHECK TABLE mt_table; +INSERT INTO mt_table VALUES (toDate('2019-01-02'), 1, 'Hello'), (toDate('2019-01-02'), 2, 'World'); +CHECK TABLE mt_table; +DROP TABLE mt_table; From df3dfd5b81311ab5a3555a91de9f6d46127358d0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jun 2020 01:00:02 +0300 Subject: [PATCH 022/196] fix clang-tidy build --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 +++---- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/IMergeTreeReader.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 8 ++++---- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 7 files changed, 12 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 6ef4cc15032..61561a8e3cf 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -135,7 +135,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out) { - auto part_in_memory = dynamic_cast(part.get()); + const auto * part_in_memory = dynamic_cast(part.get()); if (!part_in_memory) throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 22accdbe190..adefa52fc13 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -773,9 +773,6 @@ void IMergeTreeDataPart::remove() const String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const { - /// Do not allow underscores in the prefix because they are used as separators. - - assert(prefix.find_first_of('_') == String::npos); String res; /** If you need to detach a part, and directory into which we want to rename it already exists, @@ -798,12 +795,14 @@ String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix) const String IMergeTreeDataPart::getRelativePathForDetachedPart(const String & prefix) const { + /// Do not allow underscores in the prefix because they are used as separators. + assert(prefix.find_first_of('_') == String::npos); return "detached/" + getRelativePathForPrefix(prefix); } void IMergeTreeDataPart::renameToDetached(const String & prefix) const { - renameTo(getRelativePathForDetachedPart(prefix)); + renameTo(getRelativePathForDetachedPart(prefix), true); } void IMergeTreeDataPart::makeCloneInDetached(const String & prefix) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f9f1965aa57..f091d8ec519 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -297,7 +297,7 @@ public: String getFullPath() const; void renameToDetached(const String & prefix) const; - virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists = false) const; + virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const; virtual void makeCloneInDetached(const String & prefix) const; /// Makes full clone of part in detached/ on another disk diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 90a43a61536..98dbe9a6834 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -61,7 +61,7 @@ protected: /// Returns actual column type in part, which can differ from table metadata. NameAndTypePair getColumnFromPart(const NameAndTypePair & required_column) const; - void checkNumberOfColumns(size_t columns_num_to_read) const; + void checkNumberOfColumns(size_t num_columns_to_read) const; /// avg_value_size_hints are used to reduce the number of reallocations when creating columns of variable size. ValueSizeMap avg_value_size_hints; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d0659a2bcdd..ef526552e12 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1318,7 +1318,7 @@ void MergeTreeData::clearOldWriteAheadLogs() auto is_range_on_disk = [&block_numbers_on_disk](Int64 min_block, Int64 max_block) { - auto lower = std::lower_bound(block_numbers_on_disk.begin(), block_numbers_on_disk.end(), std::make_pair(min_block, -1L)); + auto lower = std::lower_bound(block_numbers_on_disk.begin(), block_numbers_on_disk.end(), std::make_pair(min_block, Int64(-1L))); if (lower != block_numbers_on_disk.end() && min_block >= lower->first && max_block <= lower->second) return true; @@ -1919,7 +1919,7 @@ void MergeTreeData::renameTempPartAndReplace( part->info = part_info; part->is_temp = false; part->state = DataPartState::PreCommitted; - part->renameTo(part_name); + part->renameTo(part_name, true); auto part_it = data_parts_indexes.insert(part).first; @@ -3271,7 +3271,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( throw Exception("Part in " + fullPath(disk, dst_part_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); /// If source part is in memory, flush it to disk and clone it already in on-disk format - if (auto * src_part_in_memory = dynamic_cast(src_part.get())) + if (const auto * src_part_in_memory = dynamic_cast(src_part.get())) { const auto & src_relative_data_path = src_part_in_memory->storage.relative_data_path; auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); @@ -3367,7 +3367,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & LOG_DEBUG(log, "Freezing part {} snapshot will be placed at {}", part->name, backup_path); String backup_part_path = backup_path + relative_data_path + part->relative_path; - if (auto part_in_memory = dynamic_cast(part.get())) + if (const auto * part_in_memory = dynamic_cast(part.get())) part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path); else localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 2a11f465805..11c12d47823 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1007,7 +1007,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( auto part = data.getPartIfExists(name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); if (part) { - if (auto * part_in_memory = dynamic_cast(part.get())) + if (const auto * part_in_memory = dynamic_cast(part.get())) sum_parts_size_in_bytes += part_in_memory->block.bytes(); else sum_parts_size_in_bytes += part->getBytesOnDisk(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a1e203a9769..9babf9476d1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -2896,7 +2896,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, } else { - part->renameTo("detached/" + part_name); + part->renameTo("detached/" + part_name, true); } } catch (...) From b312ac9786be81f59779372590d26078b699a8bf Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 4 Jun 2020 01:52:21 +0300 Subject: [PATCH 023/196] in-memory parts: fix columns sizes --- src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp | 10 ++++++++-- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++-- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- .../0_stateless/01130_in_memory_parts.reference | 3 +++ tests/queries/0_stateless/01130_in_memory_parts.sql | 3 +++ 5 files changed, 17 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index fac7e4982bb..0d930eba4e8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -113,9 +113,15 @@ void MergeTreeDataPartInMemory::renameTo(const String & new_relative_path, bool relative_path = new_relative_path; } -void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName & /*each_columns_size*/, ColumnSize & /*total_size*/) const +/// Calculates uncompressed sizes in memory. +void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const { - // throw Exception("calculateEachColumnSizesOnDisk of in memory part", ErrorCodes::NOT_IMPLEMENTED); + auto it = checksums.files.find("data.bin"); + if (it != checksums.files.end()) + total_size.data_uncompressed += it->second.uncompressed_size; + + for (const auto & column : columns) + each_columns_size[column.name].data_uncompressed += block.getByName(column.name).column->byteSize(); } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6dd48dc2bba..ad70a541611 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,8 +33,8 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ M(SettingUInt64, min_bytes_for_compact_part, 0, "Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ M(SettingUInt64, min_rows_for_compact_part, 0, "Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ - M(SettingBool, in_memory_parts_enable_wal, 1, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ - M(SettingBool, in_memory_parts_insert_sync, 0, "", 0) \ + M(SettingBool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ + M(SettingBool, in_memory_parts_insert_sync, false, "If true and in-memory parts are enabled, insert will wait while part will persist on disk in result of merge", 0) \ M(SettingUInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ \ /** Merge settings. */ \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index de5d8599fb7..af083901985 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -112,7 +112,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( new_part->checksums = checksums; new_part->setBytesOnDisk(checksums.getTotalSizeOnDisk()); new_part->index_granularity = writer->getIndexGranularity(); - // new_part->calculateColumnsSizesOnDisk(); // TODO: Fix + new_part->calculateColumnsSizesOnDisk(); } void MergedBlockOutputStream::finalizePartOnDisk( diff --git a/tests/queries/0_stateless/01130_in_memory_parts.reference b/tests/queries/0_stateless/01130_in_memory_parts.reference index ae32d3ea7a3..4a22f17c644 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts.reference +++ b/tests/queries/0_stateless/01130_in_memory_parts.reference @@ -1,4 +1,7 @@ +system.parts InMemory 2 +1 +1 Simple selects 0 0 1 1 diff --git a/tests/queries/0_stateless/01130_in_memory_parts.sql b/tests/queries/0_stateless/01130_in_memory_parts.sql index d6471cfb35f..21665faefd6 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts.sql @@ -4,7 +4,10 @@ CREATE TABLE in_memory (a UInt32, b UInt32) SETTINGS min_rows_for_compact_part = 1000, min_rows_for_compact_part = 1000; INSERT INTO in_memory SELECT number, number % 3 FROM numbers(100); +SELECT 'system.parts'; SELECT DISTINCT part_type, marks FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory' AND active; +SELECT DISTINCT data_uncompressed_bytes > 0 FROM system.parts WHERE database = currentDatabase() AND table = 'in_memory' AND active; +SELECT DISTINCT column_data_uncompressed_bytes > 0 FROM system.parts_columns WHERE database = currentDatabase() AND table = 'in_memory' AND active; SELECT 'Simple selects'; From 66e31d4311507350f8a08c30f34980e016cf7d2d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 5 Jun 2020 23:47:46 +0300 Subject: [PATCH 024/196] in-memory parts: several fixes --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 --- src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp | 7 ++++++- src/Storages/MergeTree/MergeTreeDataPartInMemory.h | 5 +++-- .../MergeTree/MergeTreeDataPartWriterInMemory.cpp | 5 +++++ src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 8 ++++---- src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++---- tests/queries/0_stateless/01130_in_memory_parts.reference | 1 + tests/queries/0_stateless/01130_in_memory_parts.sql | 4 ++++ 13 files changed, 35 insertions(+), 22 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 61561a8e3cf..7039951b256 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -135,7 +135,7 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out) { - const auto * part_in_memory = dynamic_cast(part.get()); + auto part_in_memory = asInMemoryPart(part); if (!part_in_memory) throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f091d8ec519..8b65ec29b97 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -95,9 +95,6 @@ public: virtual bool supportsVerticalMerge() const { return false; } - virtual bool waitUntilMerged(size_t /* timeout */) const { return true; } - virtual void notifyMerged() const {} - /// NOTE: Returns zeros if column files are not found in checksums. /// Otherwise return information about column size on disk. ColumnSize getColumnSize(const String & column_name, const IDataType & /* type */) const; diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 1b3f80b4e09..e5ee8b2be5e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -32,14 +32,14 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); - if (auto * part_in_memory = dynamic_cast(part.get())) + if (auto part_in_memory = asInMemoryPart(part)) { storage.in_memory_merges_throttler.add(part_in_memory->block.bytes(), part_in_memory->rows_count); auto settings = storage.getSettings(); if (settings->in_memory_parts_insert_sync) { - if (!part->waitUntilMerged(in_memory_parts_timeout)) + if (!part_in_memory->waitUntilMerged(in_memory_parts_timeout)) throw Exception("Timeout exceeded while waiting to write part " + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ef526552e12..098416e87ed 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1941,7 +1941,7 @@ void MergeTreeData::renameTempPartAndReplace( addPartContributionToColumnSizes(part); } - auto * part_in_memory = dynamic_cast(part.get()); + auto part_in_memory = asInMemoryPart(part); if (part_in_memory && getSettings()->in_memory_parts_enable_wal) { auto wal = getWriteAheadLog(); @@ -3271,7 +3271,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( throw Exception("Part in " + fullPath(disk, dst_part_path) + " already exists", ErrorCodes::DIRECTORY_ALREADY_EXISTS); /// If source part is in memory, flush it to disk and clone it already in on-disk format - if (const auto * src_part_in_memory = dynamic_cast(src_part.get())) + if (auto src_part_in_memory = asInMemoryPart(src_part)) { const auto & src_relative_data_path = src_part_in_memory->storage.relative_data_path; auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); @@ -3367,7 +3367,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & LOG_DEBUG(log, "Freezing part {} snapshot will be placed at {}", part->name, backup_path); String backup_part_path = backup_path + relative_data_path + part->relative_path; - if (const auto * part_in_memory = dynamic_cast(part.get())) + if (auto part_in_memory = asInMemoryPart(part)) part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path); else localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 4f6a5e38384..7c6204a5a32 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1455,7 +1455,7 @@ NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart( { /// In compact parts we read all columns, because they all stored in a /// single file - if (isCompactPart(source_part)) + if (!isWidePart(source_part)) return updated_header.getNamesAndTypesList(); NameSet removed_columns; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 0d930eba4e8..bec9d16209d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int DIRECTORY_ALREADY_EXISTS; } @@ -124,4 +123,10 @@ void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName each_columns_size[column.name].data_uncompressed += block.getByName(column.name).column->byteSize(); } +DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part) +{ + return std::dynamic_pointer_cast(part); +} + + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 3e2ec82b038..e48d9b8e201 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -45,8 +45,8 @@ public: void flushToDisk(const String & base_path, const String & new_relative_path) const; - bool waitUntilMerged(size_t timeout) const override; - void notifyMerged() const override; + bool waitUntilMerged(size_t timeout) const; + void notifyMerged() const; mutable Block block; @@ -58,5 +58,6 @@ private: }; using DataPartInMemoryPtr = std::shared_ptr; +DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index 917f2b862a9..2c50d5baee0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -79,6 +79,11 @@ static MergeTreeDataPartChecksum createUncompressedChecksum(size_t size, SipHash void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) { + /// If part is empty we still need to initialize block by empty columns. + if (!part_in_memory->block) + for (const auto & column : columns_list) + part_in_memory->block.insert(ColumnWithTypeAndName{column.type, column.name}); + SipHash hash; for (const auto & column : part_in_memory->block) column.column->updateHashFast(hash); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 11c12d47823..b367cf73a08 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1007,7 +1007,7 @@ bool ReplicatedMergeTreeQueue::shouldExecuteLogEntry( auto part = data.getPartIfExists(name, {MergeTreeDataPartState::PreCommitted, MergeTreeDataPartState::Committed, MergeTreeDataPartState::Outdated}); if (part) { - if (const auto * part_in_memory = dynamic_cast(part.get())) + if (auto part_in_memory = asInMemoryPart(part)) sum_parts_size_in_bytes += part_in_memory->block.bytes(); else sum_parts_size_in_bytes += part->getBytesOnDisk(); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 44a942551fb..b5cf716b079 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -681,11 +681,11 @@ bool StorageMergeTree::merge( auto lock = lockParts(); for (const auto & part : future_part.parts) { - part->notifyMerged(); - if (isInMemoryPart(part)) + if (auto part_in_memory = asInMemoryPart(part)) { - modifyPartState(part, DataPartState::Deleting); - parts_to_remove_immediately.push_back(part); + part_in_memory->notifyMerged(); + modifyPartState(part_in_memory, DataPartState::Deleting); + parts_to_remove_immediately.push_back(part_in_memory); } } } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 9babf9476d1..882b5593c76 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1089,11 +1089,11 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) DataPartsVector parts_to_remove_immediatly; for (const auto & part_ptr : parts) { - part_ptr->notifyMerged(); - if (isInMemoryPart(part_ptr)) + if (auto part_in_memory = asInMemoryPart(part_ptr)) { - modifyPartState(part_ptr, DataPartState::Deleting); - parts_to_remove_immediatly.push_back(part_ptr); + part_in_memory->notifyMerged(); + modifyPartState(part_in_memory, DataPartState::Deleting); + parts_to_remove_immediatly.push_back(part_in_memory); } } diff --git a/tests/queries/0_stateless/01130_in_memory_parts.reference b/tests/queries/0_stateless/01130_in_memory_parts.reference index 4a22f17c644..ad5435abb59 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts.reference +++ b/tests/queries/0_stateless/01130_in_memory_parts.reference @@ -36,3 +36,4 @@ Mutations and Alters 4 [4,16] 5 [] 7 [7,49] +0 diff --git a/tests/queries/0_stateless/01130_in_memory_parts.sql b/tests/queries/0_stateless/01130_in_memory_parts.sql index 21665faefd6..dca12a85841 100644 --- a/tests/queries/0_stateless/01130_in_memory_parts.sql +++ b/tests/queries/0_stateless/01130_in_memory_parts.sql @@ -39,4 +39,8 @@ ALTER TABLE in_memory DROP COLUMN str; SELECT * FROM in_memory ORDER BY a LIMIT 5; +-- in-memory parts works if they're empty. +ALTER TABLE in_memory DELETE WHERE 1; +SELECT count() FROM in_memory; + DROP TABLE in_memory; From a3ac224ae4e6427c30b3fd6d6af55c91d10079e8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 15 Jun 2020 20:41:44 +0300 Subject: [PATCH 025/196] in-memory parts: sync insert to replicated --- .../MergeTree/MergeTreeBlockOutputStream.cpp | 24 ++++++----- src/Storages/MergeTree/MergeTreeData.cpp | 21 ++++++---- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 11 +++-- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 3 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 19 +++++++-- .../ReplicatedMergeTreeBlockOutputStream.h | 3 +- src/Storages/StorageReplicatedMergeTree.cpp | 36 +++++++++------- .../test_polymorphic_parts/test.py | 42 +++++++++++++++++++ 10 files changed, 118 insertions(+), 45 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index e5ee8b2be5e..f730f51879e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -23,12 +23,14 @@ void MergeTreeBlockOutputStream::write(const Block & block) storage.delayInsertOrThrowIfNeeded(); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block); + MergeTreeData::DataPartsVector inserted_parts; for (auto & current_block : part_blocks) { Stopwatch watch; MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block); storage.renameTempPartAndAdd(part, &storage.increment); + inserted_parts.push_back(part); PartLog::addNewPart(storage.global_context, part, watch.elapsed()); @@ -36,20 +38,11 @@ void MergeTreeBlockOutputStream::write(const Block & block) { storage.in_memory_merges_throttler.add(part_in_memory->block.bytes(), part_in_memory->rows_count); - auto settings = storage.getSettings(); - if (settings->in_memory_parts_insert_sync) - { - if (!part_in_memory->waitUntilMerged(in_memory_parts_timeout)) - throw Exception("Timeout exceeded while waiting to write part " - + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); - } - else if (storage.merging_mutating_task_handle && !storage.in_memory_merges_throttler.needDelayMerge()) + if (storage.merging_mutating_task_handle && !storage.in_memory_merges_throttler.needDelayMerge()) { storage.in_memory_merges_throttler.reset(); storage.merging_mutating_task_handle->wake(); } - - continue; } else if (storage.merging_mutating_task_handle) { @@ -57,6 +50,17 @@ void MergeTreeBlockOutputStream::write(const Block & block) storage.merging_mutating_task_handle->wake(); } } + + if (storage.getSettings()->in_memory_parts_insert_sync) + { + for (const auto & part : inserted_parts) + { + auto part_in_memory = asInMemoryPart(part); + if (!part_in_memory->waitUntilMerged(in_memory_parts_timeout)) + throw Exception("Timeout exceeded while waiting to write part " + + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); + } + } } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 098416e87ed..aa52d4fdb37 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -245,7 +245,8 @@ MergeTreeData::MergeTreeData( String reason; if (!canUsePolymorphicParts(*settings, &reason) && !reason.empty()) - LOG_WARNING(log, "{} Settings 'min_bytes_for_wide_part' and 'min_bytes_for_wide_part' will be ignored.", reason); + LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " + "'min_rows_for_compact_part' and 'min_bytes_for_compact_part' will be ignored.", reason); } @@ -1592,10 +1593,10 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const { - if (!canUseAdaptiveGranularity()) + const auto settings = getSettings(); + if (!canUsePolymorphicParts(*settings)) return MergeTreeDataPartType::WIDE; - const auto settings = getSettings(); if (bytes_uncompressed < settings->min_bytes_for_compact_part || rows_count < settings->min_rows_for_compact_part) return MergeTreeDataPartType::IN_MEMORY; @@ -1607,10 +1608,10 @@ MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, s MergeTreeDataPartType MergeTreeData::choosePartTypeOnDisk(size_t bytes_uncompressed, size_t rows_count) const { - if (!canUseAdaptiveGranularity()) + const auto settings = getSettings(); + if (!canUsePolymorphicParts(*settings)) return MergeTreeDataPartType::WIDE; - const auto settings = getSettings(); if (bytes_uncompressed < settings->min_bytes_for_wide_part || rows_count < settings->min_rows_for_wide_part) return MergeTreeDataPartType::COMPACT; @@ -3605,11 +3606,15 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S { if (!canUseAdaptiveGranularity()) { - if ((settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0) && out_reason) + if (out_reason && (settings.min_rows_for_wide_part != 0 || settings.min_bytes_for_wide_part != 0 + || settings.min_rows_for_compact_part != 0 || settings.min_bytes_for_compact_part != 0)) { std::ostringstream message; - message << "Table can't create parts with adaptive granularity, but settings min_rows_for_wide_part = " - << settings.min_rows_for_wide_part << ", min_bytes_for_wide_part = " << settings.min_bytes_for_wide_part + message << "Table can't create parts with adaptive granularity, but settings" + << "min_rows_for_wide_part = " << settings.min_rows_for_wide_part + << ", min_bytes_for_wide_part = " << settings.min_bytes_for_wide_part + << ", min_rows_for_compact_part = " << settings.min_rows_for_compact_part + << ", min_bytes_for_compact_part = " << settings.min_bytes_for_compact_part << ". Parts with non-adaptive granularity can be stored only in Wide (default) format."; *out_reason = message.str(); } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 4a0ce0945a5..e3f25d561cd 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -901,7 +901,7 @@ private: /// Check selected parts for movements. Used by ALTER ... MOVE queries. CurrentlyMovingPartsTagger checkPartsForMove(const DataPartsVector & parts, SpacePtr space); - bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason) const; + bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason = nullptr) const; WriteAheadLogPtr write_ahead_log; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index bec9d16209d..e56e069d91a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -95,16 +95,19 @@ void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix) const flushToDisk(storage.getRelativeDataPath(), detached_path); } -bool MergeTreeDataPartInMemory::waitUntilMerged(size_t timeout) const +bool MergeTreeDataPartInMemory::waitUntilMerged(size_t timeout_ms) const { auto lock = storage.lockParts(); - return is_merged.wait_for(lock, std::chrono::milliseconds(timeout), - [this]() { return state == State::Outdated; }); + return is_merged.wait_for(lock, std::chrono::milliseconds(timeout_ms), + [this]() { return state != State::Committed; }); } void MergeTreeDataPartInMemory::notifyMerged() const { - is_merged.notify_one(); + LOG_DEBUG(&Poco::Logger::get("InMemPart"), "notifiedMerged"); + LOG_DEBUG(&Poco::Logger::get("InMemPart"), "state {}", stateString()); + + is_merged.notify_all(); } void MergeTreeDataPartInMemory::renameTo(const String & new_relative_path, bool /* remove_new_dir_if_exists */) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index e48d9b8e201..cd44fefdb22 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -45,7 +45,7 @@ public: void flushToDisk(const String & base_path, const String & new_relative_path) const; - bool waitUntilMerged(size_t timeout) const; + bool waitUntilMerged(size_t timeout_ms) const; void notifyMerged() const; mutable Block block; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index ad70a541611..9bf61d028cf 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -121,7 +121,8 @@ struct MergeTreeSettings : public SettingsCollection static bool isPartFormatSetting(const String & name) { - return name == "min_bytes_for_wide_part" || name == "min_rows_for_wide_part"; + return name == "min_bytes_for_wide_part" || name == "min_rows_for_wide_part" + || name == "min_bytes_for_compact_part" || name == "min_rows_for_compact_part"; } }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 8d8ab831e66..c17a44c2b61 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -31,9 +31,14 @@ namespace ErrorCodes ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( - StorageReplicatedMergeTree & storage_, size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, bool deduplicate_) - : storage(storage_), quorum(quorum_), quorum_timeout_ms(quorum_timeout_ms_), max_parts_per_block(max_parts_per_block_), deduplicate(deduplicate_), - log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) + StorageReplicatedMergeTree & storage_, size_t quorum_, size_t quorum_timeout_ms_, + size_t max_parts_per_block_, size_t insert_in_memory_parts_timeout_ms_, bool deduplicate_) + : storage(storage_), quorum(quorum_) + , quorum_timeout_ms(quorum_timeout_ms_) + , max_parts_per_block(max_parts_per_block_) + , insert_in_memory_parts_timeout_ms(insert_in_memory_parts_timeout_ms_) + , deduplicate(deduplicate_) + , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) { /// The quorum value `1` has the same meaning as if it is disabled. if (quorum == 1) @@ -365,6 +370,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo + zkutil::ZooKeeper::error2string(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } + auto part_in_memory = asInMemoryPart(part); + if (part_in_memory && storage.getSettings()->in_memory_parts_insert_sync) + { + if (!part_in_memory->waitUntilMerged(insert_in_memory_parts_timeout_ms)) + throw Exception("Timeout exceeded while waiting to write part " + + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); + } + if (quorum) { /// We are waiting for quorum to be satisfied. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index b8650c25c7d..9591c5791c5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -24,7 +24,7 @@ class ReplicatedMergeTreeBlockOutputStream : public IBlockOutputStream public: ReplicatedMergeTreeBlockOutputStream(StorageReplicatedMergeTree & storage_, size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, - bool deduplicate_); + size_t insert_in_memory_parts_timeout_ms_, bool deduplicate_); Block getHeader() const override; void writePrefix() override; @@ -58,6 +58,7 @@ private: size_t quorum; size_t quorum_timeout_ms; size_t max_parts_per_block; + size_t insert_in_memory_parts_timeout_ms; bool deduplicate = true; bool last_block_is_duplicate = false; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 882b5593c76..d3877a0ff69 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1085,20 +1085,6 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) try { checkPartChecksumsAndCommit(transaction, part); - - DataPartsVector parts_to_remove_immediatly; - for (const auto & part_ptr : parts) - { - if (auto part_in_memory = asInMemoryPart(part_ptr)) - { - part_in_memory->notifyMerged(); - modifyPartState(part_in_memory, DataPartState::Deleting); - parts_to_remove_immediatly.push_back(part_in_memory); - } - } - - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_immediatly); - removePartsFinally(parts_to_remove_immediatly); } catch (const Exception & e) { @@ -1122,6 +1108,20 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) throw; } + DataPartsVector parts_to_remove_immediatly; + for (const auto & part_ptr : parts) + { + if (auto part_in_memory = asInMemoryPart(part_ptr)) + { + modifyPartState(part_in_memory, DataPartState::Deleting); + part_in_memory->notifyMerged(); + parts_to_remove_immediatly.push_back(part_in_memory); + } + } + + tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_immediatly); + removePartsFinally(parts_to_remove_immediatly); + /** Removing old parts from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts. */ @@ -3138,7 +3138,11 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; return std::make_shared(*this, - query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, deduplicate); + query_settings.insert_quorum, + query_settings.insert_quorum_timeout.totalMilliseconds(), + query_settings.max_partitions_per_insert_block, + query_settings.insert_in_memory_parts_timeout.totalMilliseconds(), + deduplicate); } @@ -3662,7 +3666,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, bool PartsTemporaryRename renamed_parts(*this, "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); - ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, false); /// TODO Allow to use quorum here. + ReplicatedMergeTreeBlockOutputStream output(*this, 0, 0, 0, 0, false); /// TODO Allow to use quorum here. for (size_t i = 0; i < loaded_parts.size(); ++i) { String old_name = loaded_parts[i]->name; diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 362204c307d..145d6aedb19 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -9,6 +9,7 @@ from helpers.test_tools import TSV from helpers.test_tools import assert_eq_with_retry from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager +from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) @@ -90,6 +91,8 @@ def start_cluster(): create_tables('in_memory_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard4") create_tables('wal_table', [node11, node12], [settings_in_memory, settings_in_memory], "shard4") create_tables('restore_table', [node11, node12], [settings_in_memory, settings_in_memory], "shard5") + create_tables('deduplication_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5") + create_tables('sync_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5") yield cluster @@ -422,6 +425,45 @@ def test_in_memory_wal_rotate(start_cluster): assert os.path.exists(wal_file) assert os.path.getsize(wal_file) == 0 +def test_in_memory_deduplication(start_cluster): + for i in range(3): + node9.query("INSERT INTO deduplication_table (date, id, s) VALUES (toDate('2020-03-03'), 1, 'foo')") + node10.query("INSERT INTO deduplication_table (date, id, s) VALUES (toDate('2020-03-03'), 1, 'foo')") + + node9.query("SYSTEM SYNC REPLICA deduplication_table", timeout=20) + node10.query("SYSTEM SYNC REPLICA deduplication_table", timeout=20) + + assert node9.query("SELECT date, id, s FROM deduplication_table") == "2020-03-03\t1\tfoo\n" + assert node10.query("SELECT date, id, s FROM deduplication_table") == "2020-03-03\t1\tfoo\n" + +def test_in_memory_sync_insert(start_cluster): + node9.query("ALTER TABLE sync_table MODIFY SETTING in_memory_parts_insert_sync = 1") + node10.query("ALTER TABLE sync_table MODIFY SETTING in_memory_parts_insert_sync = 1") + node9.query("SYSTEM STOP MERGES sync_table") + node10.query("SYSTEM STOP MERGES sync_table") + + pool = Pool(5) + tasks = [] + for i in range(5): + tasks.append(pool.apply_async(insert_random_data, ('sync_table', node9, 50))) + + time.sleep(5) + assert node9.query("SELECT count() FROM sync_table") == "250\n" + assert node9.query("SELECT part_type, count() FROM system.parts WHERE table = 'sync_table' AND active GROUP BY part_type") == "InMemory\t5\n" + + for task in tasks: + assert not task.ready() + + node9.query("SYSTEM START MERGES sync_table") + node10.query("SYSTEM START MERGES sync_table") + assert_eq_with_retry(node9, "OPTIMIZE TABLE sync_table FINAL SETTINGS optimize_throw_if_noop = 1", "") + + for task in tasks: + task.get() + + assert node9.query("SELECT count() FROM sync_table") == "250\n" + assert node9.query("SELECT part_type, count() FROM system.parts WHERE table = 'sync_table' AND active GROUP BY part_type") == "Compact\t1\n" + def test_polymorphic_parts_index(start_cluster): node1.query(''' CREATE TABLE index_compact(a UInt32, s String) From 8b2232dc5ed2170f03e519c91d16be3e62bcc483 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 22 Jun 2020 16:26:55 +0300 Subject: [PATCH 026/196] Parser for Explain --- src/Parsers/ASTExplainQuery.h | 34 +++++++++++------- src/Parsers/ParserExplainQuery.cpp | 51 +++++++++++++++++++++++++++ src/Parsers/ParserExplainQuery.h | 25 +++++++++++++ src/Parsers/ParserQueryWithOutput.cpp | 30 +++------------- src/Parsers/ParserQueryWithOutput.h | 7 ++-- src/Parsers/ya.make | 1 + 6 files changed, 107 insertions(+), 41 deletions(-) create mode 100644 src/Parsers/ParserExplainQuery.cpp create mode 100644 src/Parsers/ParserExplainQuery.h diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index d7a40a2eb85..34fdb94c735 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -8,45 +8,53 @@ namespace DB /// AST, EXPLAIN or other query with meaning of explanation query instead of execution -class ASTExplainQuery : public IAST +class ASTExplainQuery : public ASTQueryWithOutput { public: enum ExplainKind { - ParsedAST, - AnalyzedSyntax, + ParsedAST, /// 'EXPLAIN AST SELECT ...' + AnalyzedSyntax, /// 'EXPLAIN SYNTAX SELECT ...' + QueryPlan, /// 'EXPLAIN SELECT ...' }; - ASTExplainQuery(ExplainKind kind_) - : kind(kind_) - {} + ASTExplainQuery(ExplainKind kind_, bool old_syntax_) + : kind(kind_), old_syntax(old_syntax_) + { + children.emplace_back(); /// explained query + } - String getID(char delim) const override { return "Explain" + (delim + toString(kind)); } + String getID(char delim) const override { return "Explain" + (delim + toString(kind, old_syntax)); } ExplainKind getKind() const { return kind; } ASTPtr clone() const override { auto res = std::make_shared(*this); res->children.clear(); res->children.push_back(children[0]->clone()); + cloneOutputOptions(*res); return res; } + ASTPtr & getExplainedQuery() { return children.at(0); } + protected: - void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override + void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { - settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind) << (settings.hilite ? hilite_none : "") << " "; + settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind, old_syntax) << (settings.hilite ? hilite_none : "") << " "; children.at(0)->formatImpl(settings, state, frame); } private: ExplainKind kind; + bool old_syntax; /// "EXPLAIN AST" -> "AST", "EXPLAIN SYNTAX" -> "ANALYZE" - static String toString(ExplainKind kind) + static String toString(ExplainKind kind, bool old_syntax) { switch (kind) { - case ParsedAST: return "AST"; - case AnalyzedSyntax: return "ANALYZE"; + case ParsedAST: return old_syntax ? "AST" : "EXPLAIN AST"; + case AnalyzedSyntax: return old_syntax ? "ANALYZE" : "EXPLAIN SYNTAX"; + case QueryPlan: return "EXPLAIN"; } __builtin_unreachable(); diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp new file mode 100644 index 00000000000..72806be1d29 --- /dev/null +++ b/src/Parsers/ParserExplainQuery.cpp @@ -0,0 +1,51 @@ +#include +#include +#include +#include + +namespace DB +{ + +bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ASTExplainQuery::ExplainKind kind; + bool old_syntax = false; + + ParserKeyword s_ast("AST"); + ParserKeyword s_analyze("ANALYZE"); + ParserKeyword s_explain("EXPLAIN"); + ParserKeyword s_syntax("SYNTAX"); + + if (enable_debug_queries && s_ast.ignore(pos, expected)) + { + old_syntax = true; + kind = ASTExplainQuery::ExplainKind::ParsedAST; + } + else if (enable_debug_queries && s_analyze.ignore(pos, expected)) + { + old_syntax = true; + kind = ASTExplainQuery::ExplainKind::AnalyzedSyntax; + } + else if (s_explain.ignore(pos, expected)) + { + kind = ASTExplainQuery::QueryPlan; + + if (s_ast.ignore(pos, expected)) + kind = ASTExplainQuery::ExplainKind::ParsedAST; + else if (s_syntax.ignore(pos, expected)) + kind = ASTExplainQuery::ExplainKind::AnalyzedSyntax; + } + else + return false; + + auto explain_query = std::make_shared(kind, old_syntax); + + ParserSelectWithUnionQuery select_p; + if (!select_p.parse(pos, explain_query->getExplainedQuery(), expected)) + return false; + + node = std::move(explain_query); + return true; +} + +} diff --git a/src/Parsers/ParserExplainQuery.h b/src/Parsers/ParserExplainQuery.h new file mode 100644 index 00000000000..224f466c1a1 --- /dev/null +++ b/src/Parsers/ParserExplainQuery.h @@ -0,0 +1,25 @@ +#pragma once + +#include + +namespace DB +{ + + +class ParserExplainQuery : public IParserBase +{ +public: + explicit ParserExplainQuery(bool enable_debug_queries_ = false) + : enable_debug_queries(enable_debug_queries_) + { + } + +protected: + const char * getName() const override { return "EXPLAIN"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + +private: + bool enable_debug_queries; +}; + +} diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index c7a42b5bdad..80d44e86155 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB @@ -44,21 +45,13 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserShowCreateAccessEntityQuery show_create_access_entity_p; ParserShowGrantsQuery show_grants_p; ParserShowPrivilegesQuery show_privileges_p; + ParserExplainQuery explain_p(enable_debug_queries); ASTPtr query; - ParserKeyword s_ast("AST"); - ParserKeyword s_analyze("ANALYZE"); - bool explain_ast = false; - bool analyze_syntax = false; - - if (enable_explain && s_ast.ignore(pos, expected)) - explain_ast = true; - - if (enable_explain && s_analyze.ignore(pos, expected)) - analyze_syntax = true; - - bool parsed = select_p.parse(pos, query, expected) + bool parsed = + explain_p.parse(pos, query, expected) + || select_p.parse(pos, query, expected) || show_create_access_entity_p.parse(pos, query, expected) /// should be before `show_tables_p` || show_tables_p.parse(pos, query, expected) || table_p.parse(pos, query, expected) @@ -116,19 +109,6 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.children.push_back(query_with_output.settings_ast); } - if (explain_ast) - { - node = std::make_shared(ASTExplainQuery::ParsedAST); - node->children.push_back(query); - } - else if (analyze_syntax) - { - node = std::make_shared(ASTExplainQuery::AnalyzedSyntax); - node->children.push_back(query); - } - else - node = query; - return true; } diff --git a/src/Parsers/ParserQueryWithOutput.h b/src/Parsers/ParserQueryWithOutput.h index 3163bc38437..d0962862c42 100644 --- a/src/Parsers/ParserQueryWithOutput.h +++ b/src/Parsers/ParserQueryWithOutput.h @@ -11,8 +11,9 @@ namespace DB class ParserQueryWithOutput : public IParserBase { public: - ParserQueryWithOutput(bool enable_explain_ = false) - : enable_explain(enable_explain_) + /// enable_debug_queries flag enables queries 'AST SELECT' and 'ANALYZE SELECT' + explicit ParserQueryWithOutput(bool enable_debug_queries_ = false) + : enable_debug_queries(enable_debug_queries_) {} protected: @@ -21,7 +22,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; private: - bool enable_explain; + bool enable_debug_queries; }; } diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index cf1ff9f5a02..c7029426a49 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -84,6 +84,7 @@ SRCS( ParserDictionaryAttributeDeclaration.cpp ParserDropAccessEntityQuery.cpp ParserDropQuery.cpp + ParserExplainQuery.cpp ParserGrantQuery.cpp ParserInsertQuery.cpp ParserKillQueryQuery.cpp From 78d28be8cf9c2c8fa8f7af876f4ce051ce91a5c7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 22 Jun 2020 21:56:53 +0300 Subject: [PATCH 027/196] check in-memory parts, comments and style-fixes --- src/Storages/MergeTree/DataPartsExchange.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 13 ++++++++++++- src/Storages/MergeTree/MergeTreeDataPartInMemory.h | 3 +++ .../MergeTree/MergeTreeDataPartWriterInMemory.cpp | 13 +------------ src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeWriteAheadLog.h | 13 ++++++++++--- src/Storages/MergeTree/checkDataPart.cpp | 13 +++++++++++-- 8 files changed, 44 insertions(+), 25 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 7039951b256..140f4b423b3 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -149,8 +149,8 @@ void Service::sendPartFromDisk(const MergeTreeData::DataPartPtr & part, WriteBuf /// We'll take a list of files from the list of checksums. MergeTreeData::DataPart::Checksums checksums = part->checksums; /// Add files that are not in the checksum list. - checksums.files["checksums.txt"]; - checksums.files["columns.txt"]; + checksums.files["checksums.txt"] = {}; + checksums.files["columns.txt"] = {}; MergeTreeData::DataPart::Checksums data_checksums; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bb3777df69a..7adba8422d0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -859,7 +859,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) part_names_with_disks.emplace_back(it->name(), disk_ptr); /// Create and correctly initialize global WAL object, if it's needed - if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE && settings->in_memory_parts_enable_wal) + if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal) { write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); for (auto && part : write_ahead_log->restore()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index e56e069d91a..1c6adfb40ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -126,10 +126,21 @@ void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName each_columns_size[column.name].data_uncompressed += block.getByName(column.name).column->byteSize(); } +IMergeTreeDataPart::Checksum MergeTreeDataPartInMemory::calculateBlockChecksum() const +{ + SipHash hash; + IMergeTreeDataPart::Checksum checksum; + for (const auto & column : block) + column.column->updateHashFast(hash); + + checksum.uncompressed_size = block.bytes(); + hash.get128(checksum.uncompressed_hash.first, checksum.uncompressed_hash.second); + return checksum; +} + DataPartInMemoryPtr asInMemoryPart(const MergeTreeDataPartPtr & part) { return std::dynamic_pointer_cast(part); } - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index cd44fefdb22..bb08687e37d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -48,6 +48,9 @@ public: bool waitUntilMerged(size_t timeout_ms) const; void notifyMerged() const; + /// Returns hash of parts's block + Checksum calculateBlockChecksum() const; + mutable Block block; private: diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index 2c50d5baee0..67165b9b365 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -69,14 +69,6 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl } } -static MergeTreeDataPartChecksum createUncompressedChecksum(size_t size, SipHash & hash) -{ - MergeTreeDataPartChecksum checksum; - checksum.uncompressed_size = size; - hash.get128(checksum.uncompressed_hash.first, checksum.uncompressed_hash.second); - return checksum; -} - void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart::Checksums & checksums) { /// If part is empty we still need to initialize block by empty columns. @@ -84,10 +76,7 @@ void MergeTreeDataPartWriterInMemory::finishDataSerialization(IMergeTreeDataPart for (const auto & column : columns_list) part_in_memory->block.insert(ColumnWithTypeAndName{column.type, column.name}); - SipHash hash; - for (const auto & column : part_in_memory->block) - column.column->updateHashFast(hash); - checksums.files["data.bin"] = createUncompressedChecksum(part_in_memory->block.bytes(), hash); + checksums.files["data.bin"] = part_in_memory->calculateBlockChecksum(); } } diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index b26bdf67b4f..f44e0c5f107 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -53,7 +53,7 @@ void MergeTreeWriteAheadLog::addPart(const Block & block, const String & part_na auto max_wal_bytes = storage.getSettings()->write_ahead_log_max_bytes; if (out->count() > max_wal_bytes) - rotate(); + rotate(lock); } void MergeTreeWriteAheadLog::dropPart(const String & part_name) @@ -65,7 +65,7 @@ void MergeTreeWriteAheadLog::dropPart(const String & part_name) writeStringBinary(part_name, *out); } -void MergeTreeWriteAheadLog::rotate() +void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) { String new_name = String(WAL_FILE_NAME) + "_" + toString(min_block_number) + "_" @@ -138,8 +138,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() /// But if it contains any part rotate and save them. if (max_block_number == -1) disk->remove(path); - else if (name == DEFAULT_WAL_FILE) - rotate(); + else if (name == DEFAULT_WAL_FILE_NAME) + rotate(lock); break; } diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index e33cc8d534b..1e6dfa58e7b 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -10,6 +10,13 @@ namespace DB class MergeTreeData; +/** WAL stores addditions and removals of data parts in in-memory format. + * Format of data in WAL: + * - version + * - type of action (ADD or DROP) + * - part name + * - part's block in Native format. (for ADD action) + */ class MergeTreeWriteAheadLog { public: @@ -22,10 +29,10 @@ public: constexpr static auto WAL_FILE_NAME = "wal"; constexpr static auto WAL_FILE_EXTENSION = ".bin"; - constexpr static auto DEFAULT_WAL_FILE = "wal.bin"; + constexpr static auto DEFAULT_WAL_FILE_NAME = "wal.bin"; MergeTreeWriteAheadLog(const MergeTreeData & storage_, const DiskPtr & disk_, - const String & name = DEFAULT_WAL_FILE); + const String & name = DEFAULT_WAL_FILE_NAME); void addPart(const Block & block, const String & part_name); void dropPart(const String & part_name); @@ -36,7 +43,7 @@ public: private: void init(); - void rotate(); + void rotate(const std::lock_guard & lock); const MergeTreeData & storage; DiskPtr disk; diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 8395a7461a7..793bddc88c0 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -161,13 +162,21 @@ IMergeTreeDataPart::Checksums checkDataPart( return checksums_data; } +IMergeTreeDataPart::Checksums checkDataPartInMemory(const DataPartInMemoryPtr & data_part) +{ + IMergeTreeDataPart::Checksums data_checksums; + data_checksums.files["data.bin"] = data_part->calculateBlockChecksum(); + data_part->checksums.checkEqual(data_checksums, true); + return data_checksums; +} + IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, bool require_checksums, std::function is_cancelled) { - if (!data_part->isStoredOnDisk()) - return data_part->checksums; + if (auto part_in_memory = asInMemoryPart(data_part)) + return checkDataPartInMemory(part_in_memory); return checkDataPart( data_part->volume->getDisk(), From 814d78af298b74cae0f830a33ab53dc66f4794d6 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Tue, 23 Jun 2020 16:42:52 +0300 Subject: [PATCH 028/196] Better namings --- src/Functions/FunctionsLogical.cpp | 44 +++++++++++++++--------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index ade2fe960b0..07351f6f2c9 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -42,7 +42,7 @@ using UInt8Container = ColumnUInt8::Container; using UInt8ColumnPtrs = std::vector; -MutableColumnPtr convertFromTernaryData(const UInt8Container & ternary_data, const bool make_nullable) +MutableColumnPtr buildColumnFromTernaryData(const UInt8Container & ternary_data, const bool make_nullable) { const size_t rows_count = ternary_data.size(); @@ -63,7 +63,7 @@ MutableColumnPtr convertFromTernaryData(const UInt8Container & ternary_data, con } template -bool tryConvertColumnToUInt8(const IColumn * column, UInt8Container & res) +bool tryConvertColumnToBool(const IColumn * column, UInt8Container & res) { const auto col = checkAndGetColumn>(column); if (!col) @@ -76,17 +76,17 @@ bool tryConvertColumnToUInt8(const IColumn * column, UInt8Container & res) return true; } -void convertColumnToUInt8(const IColumn * column, UInt8Container & res) +void convertAnyColumnToBool(const IColumn * column, UInt8Container & res) { - if (!tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res) && - !tryConvertColumnToUInt8(column, res)) + if (!tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res) && + !tryConvertColumnToBool(column, res)) throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN); } @@ -119,7 +119,7 @@ static bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res, Func && func) } template -inline bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res) +inline bool extractConstColumnsAsBool(ColumnRawPtrs & in, UInt8 & res) { return extractConstColumns( in, res, @@ -131,7 +131,7 @@ inline bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res) } template -inline bool extractConstColumnsTernary(ColumnRawPtrs & in, UInt8 & res_3v) +inline bool extractConstColumnsAsTernary(ColumnRawPtrs & in, UInt8 & res_3v) { return extractConstColumns( in, res_3v, @@ -188,7 +188,7 @@ private: /// A helper class used by AssociativeGenericApplierImpl /// Allows for on-the-fly conversion of any data type into intermediate ternary representation -using ValueGetter = std::function; +using TernaryValueGetter = std::function; template struct ValueGetterBuilderImpl; @@ -196,7 +196,7 @@ struct ValueGetterBuilderImpl; template struct ValueGetterBuilderImpl { - static ValueGetter build(const IColumn * x) + static TernaryValueGetter build(const IColumn * x) { if (const auto * nullable_column = typeid_cast(x)) { @@ -218,7 +218,7 @@ struct ValueGetterBuilderImpl template <> struct ValueGetterBuilderImpl<> { - static ValueGetter build(const IColumn * x) + static TernaryValueGetter build(const IColumn * x) { throw Exception( std::string("Unknown numeric column of type: ") + demangle(typeid(x).name()), @@ -253,7 +253,7 @@ public: } private: - const ValueGetter val_getter; + const TernaryValueGetter val_getter; const AssociativeGenericApplierImpl next; }; @@ -271,7 +271,7 @@ public: inline ResultValueType apply(const size_t i) const { return val_getter(i); } private: - const ValueGetter val_getter; + const TernaryValueGetter val_getter; }; @@ -338,7 +338,7 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn if (has_consts && (arguments.empty() || Op::isSaturatedValue(const_3v_value))) { result_info.column = ColumnConst::create( - convertFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()), + buildColumnFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()), input_rows_count ); return; @@ -349,7 +349,7 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn OperationApplier::apply(arguments, result_column->getData(), has_consts); - result_info.column = convertFromTernaryData(result_column->getData(), result_info.type->isNullable()); + result_info.column = buildColumnFromTernaryData(result_column->getData(), result_info.type->isNullable()); } @@ -447,7 +447,7 @@ static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & re else { auto converted_column = ColumnUInt8::create(input_rows_count); - convertColumnToUInt8(column, converted_column->getData()); + convertAnyColumnToBool(column, converted_column->getData()); uint8_args.push_back(converted_column.get()); converted_columns_holder.emplace_back(std::move(converted_column)); } From 8c3f496826cc97e3c01f5c652c679481ff71e567 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 24 Jun 2020 01:15:50 +0300 Subject: [PATCH 029/196] add test with alters and in-memory parts --- .../MergeTree/MergeTreeWriteAheadLog.cpp | 5 +++- .../test_polymorphic_parts/test.py | 29 ++++++++++++++++++- 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index f44e0c5f107..38967873319 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -32,7 +32,10 @@ MergeTreeWriteAheadLog::MergeTreeWriteAheadLog( void MergeTreeWriteAheadLog::init() { out = disk->writeFile(path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append); - block_out = std::make_unique(*out, 0, storage.getSampleBlock()); + + /// Small hack: in NativeBlockOutputStream header is used only in `getHeader` method. + /// To avoid complex logic of changing it during ALTERs we leave it empty. + block_out = std::make_unique(*out, 0, Block{}); min_block_number = std::numeric_limits::max(); max_block_number = -1; } diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 145d6aedb19..7a282a7e09c 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -71,7 +71,7 @@ node6 = cluster.add_instance('node6', config_dir='configs', main_configs=['confi 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) +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) node11 = cluster.add_instance('node11', config_dir="configs", main_configs=['configs/do_not_merge.xml'], with_zookeeper=True, stay_alive=True) @@ -93,6 +93,7 @@ def start_cluster(): create_tables('restore_table', [node11, node12], [settings_in_memory, settings_in_memory], "shard5") create_tables('deduplication_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5") create_tables('sync_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5") + create_tables('alters_table', [node9, node10], [settings_in_memory, settings_in_memory], "shard5") yield cluster @@ -464,6 +465,32 @@ def test_in_memory_sync_insert(start_cluster): assert node9.query("SELECT count() FROM sync_table") == "250\n" assert node9.query("SELECT part_type, count() FROM system.parts WHERE table = 'sync_table' AND active GROUP BY part_type") == "Compact\t1\n" +# Checks that restoring from WAL works after table schema changed +def test_in_memory_alters(start_cluster): + def check_parts_type(parts_num): + assert node9.query("SELECT part_type, count() FROM system.parts WHERE table = 'alters_table' \ + AND active GROUP BY part_type") == "InMemory\t{}\n".format(parts_num) + + node9.query("INSERT INTO alters_table (date, id, s) VALUES (toDate('2020-10-10'), 1, 'ab'), (toDate('2020-10-10'), 2, 'cd')") + node9.query("ALTER TABLE alters_table ADD COLUMN col1 UInt32") + node9.restart_clickhouse(kill=True) + + expected = "1\tab\t0\n2\tcd\t0\n" + assert node9.query("SELECT id, s, col1 FROM alters_table") == expected + check_parts_type(1) + + node9.query("INSERT INTO alters_table (date, id, col1) VALUES (toDate('2020-10-10'), 3, 100)") + node9.query("ALTER TABLE alters_table MODIFY COLUMN col1 String") + node9.query("ALTER TABLE alters_table DROP COLUMN s") + node9.restart_clickhouse(kill=True) + + check_parts_type(2) + with pytest.raises(Exception): + node9.query("SELECT id, s, col1 FROM alters_table") + + expected = expected = "1\t0_foo\n2\t0_foo\n3\t100_foo\n" + assert node9.query("SELECT id, col1 || '_foo' FROM alters_table") + def test_polymorphic_parts_index(start_cluster): node1.query(''' CREATE TABLE index_compact(a UInt32, s String) From bfc8ddbd5cd5a486d9b7cf94ae3634d4c498164d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 25 Jun 2020 23:40:07 +0300 Subject: [PATCH 030/196] Added performance test to resemble questionable benchmark --- tests/performance/pre_limit_no_sorting.xml | 2 -- tests/performance/prewhere.xml | 3 -- tests/performance/questdb_sum_double.xml | 36 +++++++++++++++++++ .../vectorize_aggregation_combinators.xml | 4 --- 4 files changed, 36 insertions(+), 9 deletions(-) create mode 100644 tests/performance/questdb_sum_double.xml diff --git a/tests/performance/pre_limit_no_sorting.xml b/tests/performance/pre_limit_no_sorting.xml index a1e50f736b8..73cc7010d2c 100644 --- a/tests/performance/pre_limit_no_sorting.xml +++ b/tests/performance/pre_limit_no_sorting.xml @@ -1,5 +1,3 @@ - SELECT sum(number) FROM (select number from system.numbers_mt limit 1000000000) - diff --git a/tests/performance/prewhere.xml b/tests/performance/prewhere.xml index 43ce52d8bd4..39bf36e6234 100644 --- a/tests/performance/prewhere.xml +++ b/tests/performance/prewhere.xml @@ -1,7 +1,4 @@ - - - hits_10m_single diff --git a/tests/performance/questdb_sum_double.xml b/tests/performance/questdb_sum_double.xml new file mode 100644 index 00000000000..2eb353d0374 --- /dev/null +++ b/tests/performance/questdb_sum_double.xml @@ -0,0 +1,36 @@ + + + 4 + 1 + 2000000000 + 10000000000 + + + + + format + + Memory + MergeTree ORDER BY tuple() + + + + type + + Float64 + Float64 NULL + Float32 + Float32 NULL + Int32 + Int32 NULL + + + + + CREATE TABLE zz (x Float64) ENGINE {engine} + INSERT INTO zz SELECT rand() FROM numbers(1000000000) + + SELECT sum(x) FROM zz + + DROP TABLE IF EXISTS zz + diff --git a/tests/performance/vectorize_aggregation_combinators.xml b/tests/performance/vectorize_aggregation_combinators.xml index 47ac0719bb5..697665650a4 100644 --- a/tests/performance/vectorize_aggregation_combinators.xml +++ b/tests/performance/vectorize_aggregation_combinators.xml @@ -1,8 +1,4 @@ - - - - 1 From 23a3a8642627264910f6cf169f5c3487f1092c23 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 25 Jun 2020 23:43:03 +0300 Subject: [PATCH 031/196] Update questdb_sum_double.xml --- tests/performance/questdb_sum_double.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/questdb_sum_double.xml b/tests/performance/questdb_sum_double.xml index 2eb353d0374..4a9ac61699a 100644 --- a/tests/performance/questdb_sum_double.xml +++ b/tests/performance/questdb_sum_double.xml @@ -8,7 +8,7 @@ - format + engine Memory MergeTree ORDER BY tuple() @@ -27,7 +27,7 @@ - CREATE TABLE zz (x Float64) ENGINE {engine} + CREATE TABLE zz (x {type}) ENGINE {engine} INSERT INTO zz SELECT rand() FROM numbers(1000000000) SELECT sum(x) FROM zz From 079d29ea198534082dd0f34efd74a1ea6ab08b99 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 26 Jun 2020 08:37:41 +0300 Subject: [PATCH 032/196] boop the CI From c8832769f6c70cb7094a29f9e058632ed3024e8c Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 26 Jun 2020 08:44:42 +0300 Subject: [PATCH 033/196] Fuzzer --- programs/client/CMakeLists.txt | 1 + programs/client/Client.cpp | 156 +++++++++++ programs/client/QueryFuzzer.cpp | 460 ++++++++++++++++++++++++++++++++ programs/client/QueryFuzzer.h | 49 ++++ 4 files changed, 666 insertions(+) create mode 100644 programs/client/QueryFuzzer.cpp create mode 100644 programs/client/QueryFuzzer.h diff --git a/programs/client/CMakeLists.txt b/programs/client/CMakeLists.txt index 6ded6a94f3a..8ec91a10257 100644 --- a/programs/client/CMakeLists.txt +++ b/programs/client/CMakeLists.txt @@ -1,6 +1,7 @@ set (CLICKHOUSE_CLIENT_SOURCES Client.cpp ConnectionParameters.cpp + QueryFuzzer.cpp Suggest.cpp ) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index b0371550903..c06944fe4a3 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1,5 +1,6 @@ #include "TestHint.h" #include "ConnectionParameters.h" +#include "QueryFuzzer.h" #include "Suggest.h" #if USE_REPLXX @@ -212,6 +213,9 @@ private: ConnectionParameters connection_parameters; + QueryFuzzer fuzzer; + int query_fuzzer_runs; + void initialize(Poco::Util::Application & self) override { Poco::Util::Application::initialize(self); @@ -768,10 +772,17 @@ private: if (!config().has("multiquery")) { + assert(!query_fuzzer_runs); processTextAsSingleQuery(text); return true; } + if (query_fuzzer_runs) + { + processWithFuzzing(text); + return true; + } + return processMultiQuery(text); } @@ -870,6 +881,100 @@ private: } + // Returns whether we can continue. + bool processWithFuzzing(const String & text) + { + /// Several queries separated by ';'. + /// INSERT data is ended by the end of line, not ';'. + + const char * begin = text.data(); + const char * end = begin + text.size(); + + while (begin < end) + { + // Skip whitespace before the query + while (isWhitespaceASCII(*begin) || *begin == ';') + { + ++begin; + } + + const auto this_query_begin = begin; + ASTPtr orig_ast = parseQuery(begin, end, true); + + if (!orig_ast) + { + // Can't continue after a parsing error + return false; + } + + auto as_insert = orig_ast->as(); + if (as_insert && as_insert->data) + { + // INSERT data is ended by newline + as_insert->end = find_first_symbols<'\n'>(as_insert->data, end); + begin = as_insert->end; + } + + full_query = text.substr(this_query_begin - text.data(), + begin - text.data()); + + ASTPtr fuzz_base = orig_ast; + for (int fuzz_step = 0; fuzz_step < query_fuzzer_runs; fuzz_step++) + { + ASTPtr ast_to_process; + try + { + auto base_before_fuzz = fuzz_base->formatForErrorMessage(); + ast_to_process = fuzz_base->clone(); + fuzzer.fuzzMain(ast_to_process); + auto base_after_fuzz = fuzz_base->formatForErrorMessage(); + + // Debug AST cloning errors. + assert(base_before_fuzz == base_after_fuzz); + + auto fuzzed_text = ast_to_process->formatForErrorMessage(); + if (fuzz_step > 0 && fuzzed_text == base_before_fuzz) + { + fprintf(stderr, "got boring ast\n"); + continue; + } + + parsed_query = ast_to_process; + query_to_send = parsed_query->formatForErrorMessage(); + + processParsedSingleQuery(); + } + catch (...) + { + last_exception_received_from_server = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); + received_exception_from_server = true; + std::cerr << "Error on processing query: " << ast_to_process->formatForErrorMessage() << std::endl << last_exception_received_from_server->message(); + } + + if (received_exception_from_server && !ignore_error) + { + // fuzz again + fprintf(stderr, "got error, will fuzz again\n"); + continue; + } + else if (ast_to_process->formatForErrorMessage().size() > 500) + { + // ast too long, please no; start from original ast + fprintf(stderr, "current ast too long, won't elaborate\n"); + fuzz_base = orig_ast; + } + else + { + // fuzz starting from this successful query + fprintf(stderr, "using this ast as etalon\n"); + fuzz_base = ast_to_process; + } + } + } + + return true; + } + void processTextAsSingleQuery(const String & text_) { full_query = text_; @@ -1894,6 +1999,7 @@ public: ("highlight", po::value()->default_value(true), "enable or disable basic syntax highlight in interactive command line") ("log-level", po::value(), "client log level") ("server_logs_file", po::value(), "put server logs into specified file") + ("query-fuzzer-runs", po::value()->default_value(0), "query fuzzer runs") ; Settings cmd_settings; @@ -2045,6 +2151,12 @@ public: if (options.count("highlight")) config().setBool("highlight", options["highlight"].as()); + if ((query_fuzzer_runs = options["query-fuzzer-runs"].as())) + { + // Fuzzer implies multiquery + config().setBool("multiquery", true); + } + argsToConfig(common_arguments, config(), 100); clearPasswordFromCommandLine(argc, argv); @@ -2056,10 +2168,54 @@ public: #pragma GCC diagnostic ignored "-Wunused-function" #pragma GCC diagnostic ignored "-Wmissing-declarations" +using signal_function = void(int, siginfo_t*, void*); + +/// Setup signal handlers. +static void add_signal_handler(const std::vector & signals, signal_function handler) +{ + struct sigaction sa; + memset(&sa, 0, sizeof(sa)); + sa.sa_sigaction = handler; + sa.sa_flags = SA_SIGINFO; + + { +#if defined(OS_DARWIN) + sigemptyset(&sa.sa_mask); + for (auto signal : signals) + sigaddset(&sa.sa_mask, signal); +#else + if (sigemptyset(&sa.sa_mask)) + throw Poco::Exception("Cannot set signal handler."); + + for (auto signal : signals) + if (sigaddset(&sa.sa_mask, signal)) + throw Poco::Exception("Cannot set signal handler."); +#endif + + for (auto signal : signals) + if (sigaction(signal, &sa, nullptr)) + throw Poco::Exception("Cannot set signal handler."); + } +}; + +/** Handler for "fault" or diagnostic signals. */ +static void signalHandler(int sig, siginfo_t * /*info*/, void * context) +{ + const ucontext_t signal_context = *reinterpret_cast(context); + const StackTrace stack_trace(signal_context); + std::cerr << fmt::format("Received signal {} at: {}", sig, + stack_trace.toString()) << std::endl; + + signal(sig, SIG_DFL); + raise(sig); +} + int mainEntryClickHouseClient(int argc, char ** argv) { try { + add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, + SIGPIPE}, signalHandler); DB::Client client; client.init(argc, argv); return client.run(); diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp new file mode 100644 index 00000000000..b31653b666b --- /dev/null +++ b/programs/client/QueryFuzzer.cpp @@ -0,0 +1,460 @@ +#include "QueryFuzzer.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + +Field QueryFuzzer::getRandomField(int type) +{ + switch(type) + { + case 0: + { + static constexpr Int64 values[] + = {-2, -1, 0, 1, 2, 3, 7, 10, 100, 255, 256, 257, 1023, 1024, + 1025, 65535, 65536, 65537, 1024 * 1024 - 1, 1024 * 1024, + 1024 * 1024 + 1, INT64_MIN, INT64_MAX}; + return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; + } + case 1: + { + static constexpr float values[] + = {NAN, INFINITY, -INFINITY, 0., 0.0001, 0.5, 0.9999, + 1., 1.0001, 2., 10.0001, 100.0001, 1000.0001}; + return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; + } + case 2: + { + static constexpr Int64 values[] + = {-2, -1, 0, 1, 2, 3, 7, 10, 100, 255, 256, 257, 1023, 1024, + 1025, 65535, 65536, 65537, 1024 * 1024 - 1, 1024 * 1024, + 1024 * 1024 + 1, INT64_MIN, INT64_MAX}; + static constexpr UInt64 scales[] = {0, 1, 2, 10}; + return DecimalField( + values[fuzz_rand() % (sizeof(values) / sizeof(*values))], + scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))] + ); + } + default: + assert(false); + } +} + +Field QueryFuzzer::fuzzField(Field field) +{ + const auto type = field.getType(); + + int type_index = -1; + + if (type == Field::Types::Int64 + || type == Field::Types::UInt64) + { + type_index = 0; + } + else if (type == Field::Types::Float64) + { + type_index = 1; + } + else if (type == Field::Types::Decimal32 + || type == Field::Types::Decimal64 + || type == Field::Types::Decimal128) + { + type_index = 2; + } + + if (fuzz_rand() % 20 == 0) + { + return Null{}; + } + + if (type_index >= 0) + { + if (fuzz_rand() % 20 == 0) + { + // Change type sometimes, but not often, because it mostly leads to + // boring errors. + type_index = fuzz_rand() % 3; + } + return getRandomField(type_index); + } + + if (type == Field::Types::String) + { + auto & str = field.get(); + UInt64 action = fuzz_rand() % 10; + switch (action) + { + case 0: + str = ""; + break; + case 1: + str = str + str; + break; + case 2: + str = str + str + str + str; + break; + case 4: + if (str.size() > 0) + { + str[fuzz_rand() % str.size()] = '\0'; + } + break; + default: + // Do nothing + break; + } + } + else if (type == Field::Types::Array || type == Field::Types::Tuple) + { + auto & arr = field.reinterpret(); + + if (fuzz_rand() % 5 == 0 && arr.size() > 0) + { + size_t pos = fuzz_rand() % arr.size(); + arr.erase(arr.begin() + pos); + fprintf(stderr, "erased\n"); + } + + if (fuzz_rand() % 5 == 0) + { + if (!arr.empty()) + { + size_t pos = fuzz_rand() % arr.size(); + arr.insert(arr.begin() + pos, fuzzField(arr[pos])); + fprintf(stderr, "inserted (pos %zd)\n", pos); + } + else + { + arr.insert(arr.begin(), getRandomField(0)); + fprintf(stderr, "inserted (0)\n"); + } + + } + + for (auto & element : arr) + { + element = fuzzField(element); + } + } + + return field; +} + +ASTPtr QueryFuzzer::getRandomColumnLike() +{ + if (column_like.empty()) + { + return nullptr; + } + + ASTPtr new_ast = column_like[fuzz_rand() % column_like.size()]->clone(); + new_ast->setAlias(""); + + return new_ast; +} + +void QueryFuzzer::replaceWithColumnLike(ASTPtr & ast) +{ + if (column_like.empty()) + { + return; + } + + std::string old_alias = ast->tryGetAlias(); + ast = getRandomColumnLike(); + ast->setAlias(old_alias); +} + +void QueryFuzzer::replaceWithTableLike(ASTPtr & ast) +{ + if (table_like.empty()) + { + return; + } + + ASTPtr new_ast = table_like[fuzz_rand() % table_like.size()]->clone(); + + std::string old_alias = ast->tryGetAlias(); + new_ast->setAlias(old_alias); + + ast = new_ast; +} + +void QueryFuzzer::fuzzColumnLikeExpressionList(ASTPtr ast) +{ + if (!ast) + { + return; + } + + auto * impl = assert_cast(ast.get()); + if (fuzz_rand() % 50 == 0 && impl->children.size() > 1) + { + // Don't remove last element -- this leads to questionable + // constructs such as empty select. + impl->children.erase(impl->children.begin() + + fuzz_rand() % impl->children.size()); + } + if (fuzz_rand() % 50 == 0) + { + auto pos = impl->children.empty() + ? impl->children.begin() + : impl->children.begin() + fuzz_rand() % impl->children.size(); + auto col = getRandomColumnLike(); + if (col) + { + impl->children.insert(pos, col); + } + else + { + fprintf(stderr, "no random col!\n"); + } + } + + /* + fuzz(impl->children); + */ +} + +void QueryFuzzer::fuzz(ASTs & asts) +{ + for (auto & ast : asts) + { + fuzz(ast); + } +} + +void QueryFuzzer::fuzz(ASTPtr & ast) +{ + if (!ast) + return; + + //fprintf(stderr, "name: %s\n", demangle(typeid(*ast).name()).c_str()); + + if (auto * with_union = typeid_cast(ast.get())) + { + fuzz(with_union->list_of_selects); + } + else if (auto * tables = typeid_cast(ast.get())) + { + fuzz(tables->children); + } + else if (auto * tables_element = typeid_cast(ast.get())) + { + fuzz(tables_element->table_join); + fuzz(tables_element->table_expression); + fuzz(tables_element->array_join); + } + else if (auto * table_expr = typeid_cast(ast.get())) + { + fuzz(table_expr->database_and_table_name); + fuzz(table_expr->subquery); + fuzz(table_expr->table_function); + } + else if (auto * expr_list = typeid_cast(ast.get())) + { + fuzz(expr_list->children); + } + else if (auto * fn = typeid_cast(ast.get())) + { + fuzzColumnLikeExpressionList(fn->arguments); + fuzzColumnLikeExpressionList(fn->parameters); + + fuzz(fn->children); + } + else if (auto * select = typeid_cast(ast.get())) + { + fuzzColumnLikeExpressionList(select->select()); + fuzzColumnLikeExpressionList(select->groupBy()); + + fuzz(select->children); + } + else if (auto * literal = typeid_cast(ast.get())) + { + // Only change the queries sometimes. + int r = fuzz_rand() % 10; + if (r == 0) + { + literal->value = fuzzField(literal->value); + } + else if (r == 1) + { + /* replace with a random function? */ + } + else if (r == 2) + { + /* replace with something column-like */ + replaceWithColumnLike(ast); + } + } + else + { + fuzz(ast->children); + } + + /* + if (auto * with_alias = dynamic_cast(ast.get())) + { + int dice = fuzz_rand() % 20; + if (dice == 0) + { + with_alias->alias = aliases[fuzz_rand() % aliases.size()]; + } + else if (dice < 5) + { + with_alias->alias = ""; + } + } + */ +} + +void QueryFuzzer::collectFuzzInfoMain(const ASTPtr ast) +{ + collectFuzzInfoRecurse(ast); + + /* + with_alias.clear(); + for (const auto & [name, value] : with_alias_map) + { + with_alias.push_back(value); + //fprintf(stderr, "alias %s\n", value->formatForErrorMessage().c_str()); + } + */ + + aliases.clear(); + for (const auto & alias : aliases_set) + { + aliases.push_back(alias); + //fprintf(stderr, "alias %s\n", alias.c_str()); + } + + column_like.clear(); + for (const auto & [name, value] : column_like_map) + { + column_like.push_back(value); + //fprintf(stderr, "column %s\n", name.c_str()); + } + + table_like.clear(); + for (const auto & [name, value] : table_like_map) + { + table_like.push_back(value); + //fprintf(stderr, "table %s\n", name.c_str()); + } +} + +void QueryFuzzer::addTableLike(const ASTPtr ast) +{ + if (table_like_map.size() > 1000) + { + return; + } + + const auto name = ast->formatForErrorMessage(); + if (name.size() < 200) + { + table_like_map.insert({name, ast}); + } +} + +void QueryFuzzer::addColumnLike(const ASTPtr ast) +{ + if (column_like_map.size() > 1000) + { + return; + } + + const auto name = ast->formatForErrorMessage(); + if (name.size() < 200) + { + column_like_map.insert({name, ast}); + } +} + +void QueryFuzzer::collectFuzzInfoRecurse(const ASTPtr ast) +{ + if (auto * impl = dynamic_cast(ast.get())) + { + if (aliases_set.size() < 1000) + { + aliases_set.insert(impl->alias); + } + } + + if (typeid_cast(ast.get())) + { + addColumnLike(ast); + } + else if (typeid_cast(ast.get())) + { + addColumnLike(ast); + } + else if (typeid_cast(ast.get())) + { + addColumnLike(ast); + } + else if (typeid_cast(ast.get())) + { + addTableLike(ast); + } + else if (typeid_cast(ast.get())) + { + addTableLike(ast); + } + + for (const auto & child : ast->children) + { + collectFuzzInfoRecurse(child); + } +} + +void QueryFuzzer::fuzzMain(ASTPtr & ast) +{ + /* + std::cerr << "before: " << std::endl; + ast->dumpTree(std::cerr); + */ + + collectFuzzInfoMain(ast); + fuzz(ast); + + /* + std::cerr << "after: " << std::endl; + ast->dumpTree(std::cerr); + */ + + std::cout << std::endl; + formatAST(*ast, std::cout); + std::cout << std::endl << std::endl; +} + +} // namespace DB diff --git a/programs/client/QueryFuzzer.h b/programs/client/QueryFuzzer.h new file mode 100644 index 00000000000..4b8a713bc4c --- /dev/null +++ b/programs/client/QueryFuzzer.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include + +#include +#include +#include + +namespace DB +{ + +struct QueryFuzzer +{ + //pcg64 fuzz_rand{static_cast(rand())}; + pcg64 fuzz_rand{clock_gettime_ns()}; + + // Collection of asts with alias. + /* + std::unordered_map with_alias_map; + std::vector with_alias; + */ + + std::unordered_set aliases_set; + std::vector aliases; + + std::unordered_map column_like_map; + std::vector column_like; + + std::unordered_map table_like_map; + std::vector table_like; + + Field getRandomField(int type); + Field fuzzField(Field field); + ASTPtr getRandomColumnLike(); + void replaceWithColumnLike(ASTPtr & ast); + void replaceWithTableLike(ASTPtr & ast); + void fuzzColumnLikeExpressionList(ASTPtr ast); + void fuzz(ASTs & asts); + void fuzz(ASTPtr & ast); + void collectFuzzInfoMain(const ASTPtr ast); + void addTableLike(const ASTPtr ast); + void addColumnLike(const ASTPtr ast); + void collectFuzzInfoRecurse(const ASTPtr ast); + void fuzzMain(ASTPtr & ast); +}; + +} From 3843661981c474ffe94d7795fb59db4d6f0194c2 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 26 Jun 2020 12:43:19 +0300 Subject: [PATCH 034/196] Update questdb_sum_double.xml --- tests/performance/questdb_sum_double.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/questdb_sum_double.xml b/tests/performance/questdb_sum_double.xml index 4a9ac61699a..ce65df3903f 100644 --- a/tests/performance/questdb_sum_double.xml +++ b/tests/performance/questdb_sum_double.xml @@ -27,10 +27,10 @@ - CREATE TABLE zz (x {type}) ENGINE {engine} - INSERT INTO zz SELECT rand() FROM numbers(1000000000) + CREATE TABLE `zz_{type}_{engine}` (x {type}) ENGINE {engine} + INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(1000000000) - SELECT sum(x) FROM zz + SELECT sum(x) FROM `zz_{type}_{engine}` - DROP TABLE IF EXISTS zz + DROP TABLE IF EXISTS `zz_{type}_{engine}` From 6f1824f0ea23521e60cba05df0a89e8eb7268f06 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Jun 2020 14:30:23 +0300 Subject: [PATCH 035/196] Correct merge with master --- src/Storages/MergeTree/DataPartsExchange.cpp | 11 +++--- src/Storages/MergeTree/DataPartsExchange.h | 2 ++ src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- .../MergeTree/IMergeTreeDataPartWriter.cpp | 4 +-- .../MergeTree/IMergeTreeDataPartWriter.h | 1 + .../MergeTree/MergeTreeBlockOutputStream.cpp | 3 -- .../MergeTree/MergeTreeBlockOutputStream.h | 4 ++- src/Storages/MergeTree/MergeTreeData.cpp | 26 ++++++++------ src/Storages/MergeTree/MergeTreeData.h | 8 ++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 17 +++++---- .../MergeTree/MergeTreeDataPartInMemory.h | 7 ++-- .../MergeTreeDataPartWriterCompact.cpp | 4 --- .../MergeTreeDataPartWriterInMemory.cpp | 3 +- .../MergeTreeDataPartWriterInMemory.h | 1 + .../MergeTreeDataPartWriterOnDisk.cpp | 7 ++-- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 1 + src/Storages/MergeTree/MergeTreePartition.cpp | 6 ++-- src/Storages/MergeTree/MergeTreePartition.h | 5 ++- .../MergeTree/MergeTreeReaderInMemory.cpp | 3 +- .../MergeTree/MergeTreeReaderInMemory.h | 1 + .../MergeTree/MergeTreeWriteAheadLog.cpp | 10 +++--- .../MergeTree/MergeTreeWriteAheadLog.h | 2 +- src/Storages/StorageMergeTree.cpp | 15 ++++---- src/Storages/StorageReplicatedMergeTree.cpp | 36 ++++++++++--------- src/Storages/StorageReplicatedMergeTree.h | 4 +-- 27 files changed, 106 insertions(+), 81 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 879edb8fa7c..ac0f0bf5214 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -137,11 +137,12 @@ void Service::processQuery(const Poco::Net::HTMLForm & params, ReadBuffer & /*bo void Service::sendPartFromMemory(const MergeTreeData::DataPartPtr & part, WriteBuffer & out) { + auto metadata_snapshot = data.getInMemoryMetadataPtr(); auto part_in_memory = asInMemoryPart(part); if (!part_in_memory) throw Exception("Part " + part->name + " is not stored in memory", ErrorCodes::LOGICAL_ERROR); - NativeBlockOutputStream block_out(out, 0, data.getSampleBlock()); + NativeBlockOutputStream block_out(out, 0, metadata_snapshot->getSampleBlock()); part->checksums.write(out); block_out.write(part_in_memory->block); } @@ -202,6 +203,7 @@ MergeTreeData::DataPartPtr Service::findPart(const String & name) } MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( + const StorageMetadataPtr & metadata_snapshot, const String & part_name, const String & replica_path, const String & host, @@ -280,12 +282,13 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) readStringBinary(part_type, in); - return part_type == "InMemory" ? downloadPartToMemory(part_name, std::move(reservation), in) + return part_type == "InMemory" ? downloadPartToMemory(part_name, metadata_snapshot, std::move(reservation), in) : downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix_, std::move(reservation), in); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( const String & part_name, + const StorageMetadataPtr & metadata_snapshot, ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in) { @@ -303,9 +306,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( new_data_part->is_temp = true; new_data_part->setColumns(block.getNamesAndTypesList()); new_data_part->minmax_idx.update(block, data.minmax_idx_columns); - new_data_part->partition.create(data, block, 0); + new_data_part->partition.create(metadata_snapshot, block, 0); - MergedBlockOutputStream part_out(new_data_part, block.getNamesAndTypesList(), {}, nullptr); + MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, nullptr); part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_data_part); diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index e6f67677e96..8b4f104267f 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -53,6 +53,7 @@ public: /// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory. MergeTreeData::MutableDataPartPtr fetchPart( + const StorageMetadataPtr & metadata_snapshot, const String & part_name, const String & replica_path, const String & host, @@ -78,6 +79,7 @@ private: MergeTreeData::MutableDataPartPtr downloadPartToMemory( const String & part_name, + const StorageMetadataPtr & metadata_snapshot, ReservationPtr reservation, PooledReadWriteBufferFromHTTP & in); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index e2a68842d22..977874542e8 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -808,7 +808,7 @@ void IMergeTreeDataPart::renameToDetached(const String & prefix) const renameTo(getRelativePathForDetachedPart(prefix), true); } -void IMergeTreeDataPart::makeCloneInDetached(const String & prefix) const +void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const { String destination_path = storage.relative_data_path + getRelativePathForDetachedPart(prefix); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 8e1149871e2..0f0b193464a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -311,7 +311,7 @@ public: virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const; /// Makes clone of a part in detached/ directory via hard links - virtual void makeCloneInDetached(const String & prefix) const; + virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const; /// Makes full clone of part in detached/ on another disk void makeCloneOnDiskDetached(const ReservationPtr & reservation) const; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index d8304ef5db1..143a9920b93 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -6,9 +6,11 @@ namespace DB IMergeTreeDataPartWriter::IMergeTreeDataPartWriter( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot_, const MergeTreeWriterSettings & settings_) : data_part(data_part_) , storage(data_part_->storage) + , metadata_snapshot(metadata_snapshot_) , columns_list(columns_list_) , settings(settings_) , with_final_mark(storage.getSettings()->write_final_mark && settings.can_use_adaptive_granularity){} @@ -36,8 +38,6 @@ Columns IMergeTreeDataPartWriter::releaseIndexColumns() std::make_move_iterator(index_columns.end())); } -} - void IMergeTreeDataPartWriter::next() { current_mark = next_mark; diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index c112aa93221..959d6af9ed2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -22,6 +22,7 @@ public: IMergeTreeDataPartWriter( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot_, const MergeTreeWriterSettings & settings_); IMergeTreeDataPartWriter( diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 3487495b108..6cb79296b09 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -24,9 +24,6 @@ void MergeTreeBlockOutputStream::write(const Block & block) auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot); MergeTreeData::DataPartsVector inserted_parts; -======= - auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, ); ->>>>>>> master for (auto & current_block : part_blocks) { Stopwatch watch; diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h index 94691d50219..3faed068230 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -18,7 +18,9 @@ public: : storage(storage_) , metadata_snapshot(metadata_snapshot_) , max_parts_per_block(max_parts_per_block_) - , in_memory_parts_timeout(in_memory_parts_timeout_) {} + , in_memory_parts_timeout(in_memory_parts_timeout_) + { + } Block getHeader() const override; void write(const Block & block) override; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1b92594ef27..bb34a4bf443 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -658,6 +658,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) { LOG_DEBUG(log, "Loading data parts"); + auto metadata_snapshot = getInMemoryMetadataPtr(); const auto settings = getSettings(); std::vector> part_names_with_disks; MutableDataPartsVector parts_from_wal; @@ -704,13 +705,13 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (it->name() == MergeTreeWriteAheadLog::DEFAULT_WAL_FILE_NAME && settings->in_memory_parts_enable_wal) { write_ahead_log = std::make_shared(*this, disk_ptr, it->name()); - for (auto && part : write_ahead_log->restore()) + for (auto && part : write_ahead_log->restore(metadata_snapshot)) parts_from_wal.push_back(std::move(part)); } else if (startsWith(it->name(), MergeTreeWriteAheadLog::WAL_FILE_NAME)) { MergeTreeWriteAheadLog wal(*this, disk_ptr, it->name()); - for (auto && part : wal.restore()) + for (auto && part : wal.restore(metadata_snapshot)) parts_from_wal.push_back(std::move(part)); } } @@ -1579,9 +1580,9 @@ void MergeTreeData::changeSettings( } } -void MergeTreeData::freezeAll(const String & with_name, const Context & context, TableLockHolder &) +void MergeTreeData::freezeAll(const String & with_name, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableLockHolder &) { - freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, with_name, context); + freezePartitionsByMatcher([] (const DataPartPtr &){ return true; }, metadata_snapshot, with_name, context); } void MergeTreeData::PartsTemporaryRename::addPart(const String & old_name, const String & new_name) @@ -2398,7 +2399,7 @@ void MergeTreeData::removePartContributionToColumnSizes(const DataPartPtr & part } -void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & with_name, const Context & context, TableLockHolder &) +void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context, TableLockHolder &) { std::optional prefix; String partition_id; @@ -2430,6 +2431,7 @@ void MergeTreeData::freezePartition(const ASTPtr & partition_ast, const String & else return part->info.partition_id == partition_id; }, + metadata_snapshot, with_name, context); } @@ -3110,9 +3112,11 @@ MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData( return checkStructureAndGetMergeTreeData(*source_table, src_snapshot, my_snapshot); } -MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(const MergeTreeData::DataPartPtr & src_part, - const String & tmp_part_prefix, - const MergeTreePartInfo & dst_part_info) +MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( + const MergeTreeData::DataPartPtr & src_part, + const String & tmp_part_prefix, + const MergeTreePartInfo & dst_part_info, + const StorageMetadataPtr & metadata_snapshot) { /// Check that the storage policy contains the disk where the src_part is located. bool does_storage_policy_allow_same_disk = false; @@ -3144,7 +3148,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( { const auto & src_relative_data_path = src_part_in_memory->storage.relative_data_path; auto flushed_part_path = src_part_in_memory->getRelativePathForPrefix(tmp_part_prefix); - src_part_in_memory->flushToDisk(src_relative_data_path, flushed_part_path); + src_part_in_memory->flushToDisk(src_relative_data_path, flushed_part_path, metadata_snapshot); src_part_path = src_relative_data_path + flushed_part_path + "/"; } @@ -3207,7 +3211,7 @@ MergeTreeData::PathsWithDisks MergeTreeData::getRelativeDataPathsWithDisks() con return res; } -void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context) +void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context) { String clickhouse_path = Poco::Path(context.getPath()).makeAbsolute().toString(); String default_shadow_path = clickhouse_path + "shadow/"; @@ -3237,7 +3241,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String & String backup_part_path = backup_path + relative_data_path + part->relative_path; if (auto part_in_memory = asInMemoryPart(part)) - part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path); + part_in_memory->flushToDisk(backup_path + relative_data_path, part->relative_path, metadata_snapshot); else localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 358e77a847c..11c20df210f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -512,7 +512,7 @@ public: TableLockHolder & table_lock_holder); /// Freezes all parts. - void freezeAll(const String & with_name, const Context & context, TableLockHolder & table_lock_holder); + void freezeAll(const String & with_name, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableLockHolder & table_lock_holder); /// Should be called if part data is suspected to be corrupted. void reportBrokenPart(const String & name) const @@ -534,7 +534,7 @@ public: * Backup is created in directory clickhouse_dir/shadow/i/, where i - incremental number, * or if 'with_name' is specified - backup is created in directory with specified name. */ - void freezePartition(const ASTPtr & partition, const String & with_name, const Context & context, TableLockHolder & table_lock_holder); + void freezePartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context, TableLockHolder & table_lock_holder); public: @@ -567,7 +567,7 @@ public: MergeTreeData & checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const; MergeTreeData::MutableDataPartPtr cloneAndLoadDataPartOnSameDisk( - const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info); + const MergeTreeData::DataPartPtr & src_part, const String & tmp_part_prefix, const MergeTreePartInfo & dst_part_info, const StorageMetadataPtr & metadata_snapshot); virtual std::vector getMutationsStatus() const = 0; @@ -829,7 +829,7 @@ protected: /// Common part for |freezePartition()| and |freezeAll()|. using MatcherFn = std::function; - void freezePartitionsByMatcher(MatcherFn matcher, const String & with_name, const Context & context); + void freezePartitionsByMatcher(MatcherFn matcher, const StorageMetadataPtr & metadata_snapshot, const String & with_name, const Context & context); bool canReplacePartition(const DataPartPtr & src_part) const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 83bc608b729..b7ef924c564 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1024,7 +1024,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor if (source_part->isStoredOnDisk() && !isStorageTouchedByMutations(storage_from_source_part, metadata_snapshot, commands_for_part, context_for_reading)) { LOG_TRACE(log, "Part {} doesn't change up to mutation version {}", source_part->name, future_part.part_info.mutation); - return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info); + return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info, metadata_snapshot); } else { diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 1c6adfb40ca..2efba227f70 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -38,6 +38,7 @@ MergeTreeDataPartInMemory::MergeTreeDataPartInMemory( IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( const NamesAndTypesList & columns_to_read, + const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, UncompressedCache * /* uncompressed_cache */, MarkCache * /* mark_cache */, @@ -47,21 +48,23 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, mark_ranges, reader_settings); + ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & /* indices_to_recalc */, const CompressionCodecPtr & /* default_codec */, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & /* computed_index_granularity */) const { auto ptr = std::static_pointer_cast(shared_from_this()); - return std::make_unique(ptr, columns_list, writer_settings); + return std::make_unique( + ptr, columns_list, metadata_snapshot, writer_settings); } -void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const String & new_relative_path) const +void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const { const auto & disk = volume->getDisk(); String destination_path = base_path + new_relative_path; @@ -82,17 +85,17 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri disk->createDirectories(destination_path); auto compression_codec = storage.global_context.chooseCompressionCodec(0, 0); - auto indices = MergeTreeIndexFactory::instance().getMany(storage.getSecondaryIndices()); - MergedBlockOutputStream out(new_data_part, columns, indices, compression_codec); + auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); + MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec); out.writePrefix(); out.write(block); out.writeSuffixAndFinalizePart(new_data_part); } -void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix) const +void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const { String detached_path = getRelativePathForDetachedPart(prefix); - flushToDisk(storage.getRelativeDataPath(), detached_path); + flushToDisk(storage.getRelativeDataPath(), detached_path, metadata_snapshot); } bool MergeTreeDataPartInMemory::waitUntilMerged(size_t timeout_ms) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index bb08687e37d..eb053d3dc9a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -23,15 +23,16 @@ public: MergeTreeReaderPtr getReader( const NamesAndTypesList & columns, + const StorageMetadataPtr & metadata_snapshot, const MarkRanges & mark_ranges, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const MergeTreeReaderSettings & reader_settings_, const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const override; - MergeTreeWriterPtr getWriter( const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & writer_settings, @@ -41,9 +42,9 @@ public: bool hasColumnFiles(const String & column_name, const IDataType & /* type */) const override { return !!getColumnPosition(column_name); } String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) const override; - void makeCloneInDetached(const String & prefix) const override; + void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const override; - void flushToDisk(const String & base_path, const String & new_relative_path) const; + void flushToDisk(const String & base_path, const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const; bool waitUntilMerged(size_t timeout_ms) const; void notifyMerged() const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index a92094d81bc..f2ac7555af8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -15,10 +15,6 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) -======= - : IMergeTreeDataPartWriter( - data_part_, columns_list_, indices_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) ->>>>>>> master { using DataPart = MergeTreeDataPartCompact; String data_file_name = DataPart::DATA_FILE_NAME; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp index 67165b9b365..a7486158737 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.cpp @@ -13,8 +13,9 @@ namespace ErrorCodes MergeTreeDataPartWriterInMemory::MergeTreeDataPartWriterInMemory( const DataPartInMemoryPtr & part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot_, const MergeTreeWriterSettings & settings_) - : IMergeTreeDataPartWriter(part_, columns_list_, settings_) + : IMergeTreeDataPartWriter(part_, columns_list_, metadata_snapshot_, settings_) , part_in_memory(part_) {} void MergeTreeDataPartWriterInMemory::write( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h index d7732d99267..92e4228a90d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterInMemory.h @@ -11,6 +11,7 @@ public: MergeTreeDataPartWriterInMemory( const DataPartInMemoryPtr & part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot, const MergeTreeWriterSettings & settings_); /// You can write only one block. In-memory part can be written only at INSERT. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 69c1c0f6117..6f4585b819d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -65,13 +65,14 @@ void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPa MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list_, + const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter(data_part_, - columns_list_, indices_to_recalc_, + columns_list_, metadata_snapshot_, indices_to_recalc_, index_granularity_, settings_) , part_path(data_part_->getFullRelativePath()) , marks_file_extension(marks_file_extension_) @@ -156,7 +157,7 @@ void MergeTreeDataPartWriterOnDisk::fillIndexGranularity(size_t index_granularit void MergeTreeDataPartWriterOnDisk::initPrimaryIndex() { - if (storage.hasPrimaryKey()) + if (metadata_snapshot->hasPrimaryKey()) { index_file_stream = data_part->volume->getDisk()->writeFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite); index_stream = std::make_unique(*index_file_stream); @@ -216,7 +217,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializePrimaryIndex(const Bloc while (index_mark < total_marks && current_row < rows) { - if (storage.hasPrimaryKey()) + if (metadata_snapshot->hasPrimaryKey()) { for (size_t j = 0; j < primary_columns_num; ++j) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 5827c9fff62..8598a5143cb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -64,6 +64,7 @@ public: MergeTreeDataPartWriterOnDisk( const MergeTreeData::DataPartPtr & data_part_, const NamesAndTypesList & columns_list, + const StorageMetadataPtr & metadata_snapshot_, const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index fe3bc37d26f..4a846f63b7c 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -158,12 +158,12 @@ void MergeTreePartition::store(const Block & partition_key_sample, const DiskPtr checksums.files["partition.dat"].file_hash = out_hashing.getHash(); } -void MergeTreePartition::create(const MergeTreeData & storage, Block block, size_t row) +void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row) { - if (!storage.hasPartitionKey()) + if (!metadata_snapshot->hasPartitionKey()) return; - const auto & partition_key = storage.getPartitionKey(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); partition_key.expression->execute(block); size_t partition_columns_num = partition_key.sample_block.columns(); value.resize(partition_columns_num); diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index d91022f655f..947fb3ec504 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -12,6 +12,9 @@ class Block; class MergeTreeData; struct FormatSettings; struct MergeTreeDataPartChecksums; +struct StorageInMemoryMetadata; + +using StorageMetadataPtr = std::shared_ptr; /// This class represents a partition value of a single part and encapsulates its loading/storing logic. struct MergeTreePartition @@ -37,7 +40,7 @@ public: void assign(const MergeTreePartition & other) { value.assign(other.value); } - void create(const MergeTreeData & storage, Block block, size_t row); + void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row); }; } diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 924198b49f1..83f1ac5b86a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -18,9 +18,10 @@ namespace ErrorCodes MergeTreeReaderInMemory::MergeTreeReaderInMemory( DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, + const StorageMetadataPtr & metadata_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_) - : IMergeTreeReader(data_part_, std::move(columns_), + : IMergeTreeReader(data_part_, std::move(columns_), metadata_snapshot_, nullptr, nullptr, std::move(mark_ranges_), std::move(settings_), {}) , part_in_memory(std::move(data_part_)) diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.h b/src/Storages/MergeTree/MergeTreeReaderInMemory.h index 196fc53725a..4526b19c4a8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.h +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.h @@ -17,6 +17,7 @@ public: MergeTreeReaderInMemory( DataPartInMemoryPtr data_part_, NamesAndTypesList columns_, + const StorageMetadataPtr & metadata_snapshot_, MarkRanges mark_ranges_, MergeTreeReaderSettings settings_); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 38967873319..eda8579c76a 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -78,7 +78,7 @@ void MergeTreeWriteAheadLog::rotate(const std::lock_guard &) init(); } -MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() +MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const StorageMetadataPtr & metadata_snapshot) { std::lock_guard lock(write_mutex); @@ -151,12 +151,12 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore() if (action_type == ActionType::ADD_PART) { - MergedBlockOutputStream part_out(part, block.getNamesAndTypesList(), {}, nullptr); + MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, nullptr); part->minmax_idx.update(block, storage.minmax_idx_columns); - part->partition.create(storage, block, 0); - if (storage.hasSortingKey()) - storage.getSortingKey().expression->execute(block); + part->partition.create(metadata_snapshot, block, 0); + if (metadata_snapshot->hasSortingKey()) + metadata_snapshot->getSortingKey().expression->execute(block); part_out.writePrefix(); part_out.write(block); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h index 1e6dfa58e7b..2cc3c2b4181 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.h +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.h @@ -36,7 +36,7 @@ public: void addPart(const Block & block, const String & part_name); void dropPart(const String & part_name); - std::vector restore(); + std::vector restore(const StorageMetadataPtr & metadata_snapshot); using MinMaxBlockNumber = std::pair; static std::optional tryParseMinMaxBlockNumber(const String & filename); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 721c7321a25..e5a5fcdb091 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -204,7 +204,9 @@ BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Sto { const auto & settings = context.getSettingsRef(); - return std::make_shared(*this, metadata_snapshot, settings.max_partitions_per_insert_block, settings.insert_in_memory_parts_timeout.totalMilliseconds()); + return std::make_shared( + *this, metadata_snapshot, settings.max_partitions_per_insert_block, + settings.insert_in_memory_parts_timeout.totalMilliseconds()); } void StorageMergeTree::checkTableCanBeDropped() const @@ -1091,14 +1093,14 @@ void StorageMergeTree::alterPartition( case PartitionCommand::FREEZE_PARTITION: { auto lock = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - freezePartition(command.partition, command.with_name, context, lock); + freezePartition(command.partition, metadata_snapshot, command.with_name, context, lock); } break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { auto lock = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - freezeAll(command.with_name, context, lock); + freezeAll(command.with_name, metadata_snapshot, context, lock); } break; @@ -1115,6 +1117,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons /// This protects against "revival" of data for a removed partition after completion of merge. auto merge_blocker = merger_mutator.merges_blocker.cancel(); + auto metadata_snapshot = getInMemoryMetadataPtr(); String partition_id = getPartitionIDFromQuery(partition, context); /// TODO: should we include PreComitted parts like in Replicated case? @@ -1128,7 +1131,7 @@ void StorageMergeTree::dropPartition(const ASTPtr & partition, bool detach, cons for (const auto & part : parts_to_remove) { LOG_INFO(log, "Detaching {}", part->relative_path); - part->makeCloneInDetached(""); + part->makeCloneInDetached("", metadata_snapshot); } } @@ -1189,7 +1192,7 @@ void StorageMergeTree::replacePartitionFrom(const StoragePtr & source_table, con Int64 temp_index = insert_increment.get(); MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - dst_parts.emplace_back(cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info)); + dst_parts.emplace_back(cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, my_metadata_snapshot)); } /// ATTACH empty part set @@ -1273,7 +1276,7 @@ void StorageMergeTree::movePartitionToTable(const StoragePtr & dest_table, const Int64 temp_index = insert_increment.get(); MergeTreePartInfo dst_part_info(partition_id, temp_index, temp_index, src_part->info.level); - dst_parts.emplace_back(dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info)); + dst_parts.emplace_back(dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot)); } /// empty part set diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8ccee0febcf..a4e3c2db43c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1532,6 +1532,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) /// Looking for covering part. After that entry.actual_new_part_name may be filled. String replica = findReplicaHavingCoveringPart(entry, true); const auto storage_settings_ptr = getSettings(); + auto metadata_snapshot = getInMemoryMetadataPtr(); static std::atomic_uint total_fetches {0}; if (storage_settings_ptr->replicated_max_parallel_fetches && total_fetches >= storage_settings_ptr->replicated_max_parallel_fetches) @@ -1664,7 +1665,7 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry) try { String part_name = entry.actual_new_part_name.empty() ? entry.new_part_name : entry.actual_new_part_name; - if (!fetchPart(part_name, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) + if (!fetchPart(part_name, metadata_snapshot, zookeeper_path + "/replicas/" + replica, false, entry.quorum)) return false; } catch (Exception & e) @@ -1728,6 +1729,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) /// And, if you do not, the parts will come to life after the server is restarted. /// Therefore, we use all data parts. + auto metadata_snapshot = getInMemoryMetadataPtr(); DataPartsVector parts_to_remove; { auto data_parts_lock = lockParts(); @@ -1740,7 +1742,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) for (const auto & part : parts_to_remove) { LOG_INFO(log, "Detaching {}", part->relative_path); - part->makeCloneInDetached(""); + part->makeCloneInDetached("", metadata_snapshot); } } @@ -1763,6 +1765,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) { Stopwatch watch; auto & entry_replace = *entry.replace_range_entry; + auto metadata_snapshot = getInMemoryMetadataPtr(); MergeTreePartInfo drop_range = MergeTreePartInfo::fromPartName(entry_replace.drop_range_part_name, format_version); /// Range with only one block has special meaning ATTACH PARTITION @@ -2002,7 +2005,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) throw Exception("Checksums of " + part_desc->src_table_part->name + " is suddenly changed", ErrorCodes::UNFINISHED); part_desc->res_part = cloneAndLoadDataPartOnSameDisk( - part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info); + part_desc->src_table_part, TMP_PREFIX + "clone_", part_desc->new_part_info, metadata_snapshot); } else if (!part_desc->replica.empty()) { @@ -2015,7 +2018,8 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (interserver_scheme != address.scheme) throw Exception("Interserver schemas are different '" + interserver_scheme + "' != '" + address.scheme + "', can't fetch part from " + address.host, ErrorCodes::LOGICAL_ERROR); - part_desc->res_part = fetcher.fetchPart(part_desc->found_new_part_name, source_replica_path, + part_desc->res_part = fetcher.fetchPart( + metadata_snapshot, part_desc->found_new_part_name, source_replica_path, address.host, address.replication_port, timeouts, user, password, interserver_scheme, false, TMP_PREFIX + "fetch_"); /// TODO: check columns_version of fetched part @@ -3075,7 +3079,7 @@ void StorageReplicatedMergeTree::cleanLastPartNode(const String & partition_id) } -bool StorageReplicatedMergeTree::fetchPart(const String & part_name, +bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, const String & source_replica_path, bool to_detached, size_t quorum) { const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -3162,7 +3166,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, { get_part = [&, part_to_clone]() { - return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info); + return cloneAndLoadDataPartOnSameDisk(part_to_clone, "tmp_clone_", part_info, metadata_snapshot); }; } else @@ -3180,7 +3184,7 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, ErrorCodes::LOGICAL_ERROR); return fetcher.fetchPart( - part_name, source_replica_path, + metadata_snapshot, part_name, source_replica_path, address.host, address.replication_port, timeouts, user_password.first, user_password.second, interserver_scheme, to_detached); }; @@ -3909,20 +3913,20 @@ void StorageReplicatedMergeTree::alterPartition( break; case PartitionCommand::FETCH_PARTITION: - fetchPartition(command.partition, command.from_zookeeper_path, query_context); + fetchPartition(command.partition, metadata_snapshot, command.from_zookeeper_path, query_context); break; case PartitionCommand::FREEZE_PARTITION: { auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); - freezePartition(command.partition, command.with_name, query_context, lock); + freezePartition(command.partition, metadata_snapshot, command.with_name, query_context, lock); } break; case PartitionCommand::FREEZE_ALL_PARTITIONS: { auto lock = lockForShare(query_context.getCurrentQueryId(), query_context.getSettingsRef().lock_acquire_timeout); - freezeAll(command.with_name, query_context, lock); + freezeAll(command.with_name, metadata_snapshot, query_context, lock); } break; } @@ -4518,7 +4522,7 @@ void StorageReplicatedMergeTree::getReplicaDelays(time_t & out_absolute_delay, t } -void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const String & from_, const Context & query_context) +void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from_, const Context & query_context) { String partition_id = getPartitionIDFromQuery(partition, query_context); @@ -4658,7 +4662,7 @@ void StorageReplicatedMergeTree::fetchPartition(const ASTPtr & partition, const { try { - fetchPart(part, best_replica_path, true, 0); + fetchPart(part, metadata_snapshot, best_replica_path, true, 0); } catch (const DB::Exception & e) { @@ -5120,8 +5124,8 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( LOG_TRACE(log, "Deleted {} deduplication block IDs in partition ID {}", to_delete_futures.size(), partition_id); } -void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, - const Context & context) +void StorageReplicatedMergeTree::replacePartitionFrom( + const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & context) { /// First argument is true, because we possibly will add new data to current table. auto lock1 = lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); @@ -5201,7 +5205,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom(const StoragePtr & source_ UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info); + auto dst_part = cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, metadata_snapshot); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); @@ -5380,7 +5384,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta UInt64 index = lock->getNumber(); MergeTreePartInfo dst_part_info(partition_id, index, index, src_part->info.level); - auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info); + auto dst_part = dest_table_storage->cloneAndLoadDataPartOnSameDisk(src_part, TMP_PREFIX, dst_part_info, dest_metadata_snapshot); src_parts.emplace_back(src_part); dst_parts.emplace_back(dst_part); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index a60ee626369..e466e807066 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -474,7 +474,7 @@ private: * If quorum != 0, then the node for tracking the quorum is updated. * Returns false if part is already fetching right now. */ - bool fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum); + bool fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, const String & replica_path, bool to_detached, size_t quorum); /// Required only to avoid races between executeLogEntry and fetchPartition std::unordered_set currently_fetching_parts; @@ -533,7 +533,7 @@ private: void attachPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, bool part, const Context & query_context); void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, const Context & query_context); void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, const Context & query_context); - void fetchPartition(const ASTPtr & partition, const String & from, const Context & query_context); + void fetchPartition(const ASTPtr & partition, const StorageMetadataPtr & metadata_snapshot, const String & from, const Context & query_context); /// Check granularity of already existing replicated table in zookeeper if it exists /// return true if it's fixed From 6caf2e7a3f248a7a00db363ee47535c1925f31b9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Jun 2020 14:38:37 +0300 Subject: [PATCH 036/196] Fix protocol check --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index ac0f0bf5214..0fe2a1adbbf 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -279,7 +279,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( } String part_type = "Wide"; - if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_SIZE) + if (server_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_TYPE) readStringBinary(part_type, in); return part_type == "InMemory" ? downloadPartToMemory(part_name, metadata_snapshot, std::move(reservation), in) From 9710a6770492cc5985c757f65fb1314e14a0772b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Jun 2020 15:27:12 +0300 Subject: [PATCH 037/196] Fix style check --- src/Storages/MergeTree/MergeTreeData.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bb34a4bf443..f4511219b04 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -58,7 +58,6 @@ #include #include #include -#include namespace ProfileEvents From 2efca8e397ce2fd4466ea0e2da9813f7615768f9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 26 Jun 2020 20:54:50 +0300 Subject: [PATCH 038/196] Update questdb_sum_double.xml --- tests/performance/questdb_sum_double.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/performance/questdb_sum_double.xml b/tests/performance/questdb_sum_double.xml index ce65df3903f..d865fe9782f 100644 --- a/tests/performance/questdb_sum_double.xml +++ b/tests/performance/questdb_sum_double.xml @@ -1,6 +1,7 @@ 4 + 20G 1 2000000000 10000000000 From e1481ccfc2d6719f33bb0ecd49186750afc97670 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 22 Jun 2020 17:37:42 +0300 Subject: [PATCH 039/196] Add initial explain. --- src/Interpreters/InterpreterExplainQuery.cpp | 44 +- src/Interpreters/InterpreterSelectQuery.cpp | 8 +- src/Parsers/ASTExplainQuery.h | 2 +- src/Parsers/ParserExplainQuery.cpp | 2 +- src/Parsers/ParserQueryWithOutput.cpp | 11 + src/Processors/QueryPlan/QueryPlan.cpp | 53 ++ src/Processors/QueryPlan/QueryPlan.h | 4 + .../00597_push_down_predicate.reference | 581 +++++++++++++++++- .../00826_cross_to_inner_join.reference | 72 ++- .../00849_multiple_comma_join.reference | 266 +++++++- .../00849_multiple_comma_join_2.reference | 209 ++++++- .../0_stateless/00908_analyze_query.reference | 5 +- .../01029_early_constant_folding.reference | 15 +- .../01056_predicate_optimizer_bugs.reference | 149 ++++- ...76_predicate_optimizer_with_view.reference | 44 +- .../01083_cross_to_inner_with_like.reference | 24 +- .../01300_group_by_other_keys.reference | 71 ++- ...01300_group_by_other_keys_having.reference | 45 +- ..._duplicate_order_by_and_distinct.reference | 52 +- ...egate_functions_of_group_by_keys.reference | 74 ++- .../01322_any_input_optimize.reference | 3 +- 21 files changed, 1608 insertions(+), 126 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 4890287e81e..c79cc93409a 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -12,14 +12,20 @@ #include #include #include +#include #include #include - +#include namespace DB { +namespace ErrorCodes +{ + extern const int INCORRECT_QUERY; +} + namespace { struct ExplainAnalyzedSyntaxMatcher @@ -79,10 +85,31 @@ Block InterpreterExplainQuery::getSampleBlock() return block; } +static void fillColumn(IColumn & column, const std::string & str) +{ + size_t start = 0; + size_t end = 0; + size_t size = str.size(); + + while (end < size) + { + if (str[end] == '\n') + { + column.insertData(str.data() + start, end - start); + start = end + 1; + } + + ++end; + } + + if (start < end) + column.insertData(str.data() + start, end - start); +} BlockInputStreamPtr InterpreterExplainQuery::executeImpl() { const auto & ast = query->as(); + Block sample_block = getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); @@ -99,8 +126,21 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() ast.children.at(0)->format(IAST::FormatSettings(ss, false)); } + else if (ast.getKind() == ASTExplainQuery::QueryPlan) + { + if (!dynamic_cast(ast.getExplainedQuery().get())) + throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY); - res_columns[0]->insert(ss.str()); + QueryPlan plan; + + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions()); + interpreter.buildQueryPlan(plan); + + WriteBufferFromOStream buffer(ss); + plan.explain(buffer); + } + + fillColumn(*res_columns[0], ss.str()); return std::make_shared(sample_block.cloneWithColumns(std::move(res_columns))); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 042ab9e093c..7fbfdf23959 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -968,7 +968,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu */ if (!expressions.first_stage && !expressions.need_aggregate && !(query.group_by_with_totals && !aggregate_final)) - executeMergeSorted(query_plan, "before ORDER BY"); + executeMergeSorted(query_plan, "for ORDER BY"); else /// Otherwise, just sort. executeOrder(query_plan, query_info.input_order_info); } @@ -1596,7 +1596,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo limit, SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode)); - partial_sorting->setStepDescription("Sort each block before ORDER BY"); + partial_sorting->setStepDescription("Sort each block for ORDER BY"); query_plan.addStep(std::move(partial_sorting)); /// Merge the sorted blocks. @@ -1607,11 +1607,11 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo settings.max_bytes_before_external_sort, context->getTemporaryVolume(), settings.min_free_disk_space_for_temporary_data); - merge_sorting_step->setStepDescription("Merge sorted blocks before ORDER BY"); + merge_sorting_step->setStepDescription("Merge sorted blocks for ORDER BY"); query_plan.addStep(std::move(merge_sorting_step)); /// If there are several streams, we merge them into one - executeMergeSorted(query_plan, output_order_descr, limit, "before ORDER BY"); + executeMergeSorted(query_plan, output_order_descr, limit, "for ORDER BY"); } diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index 34fdb94c735..1e3aca25700 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -35,7 +35,7 @@ public: return res; } - ASTPtr & getExplainedQuery() { return children.at(0); } + const ASTPtr & getExplainedQuery() const { return children.at(0); } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index 72806be1d29..7c6e8e0085e 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -41,7 +41,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected auto explain_query = std::make_shared(kind, old_syntax); ParserSelectWithUnionQuery select_p; - if (!select_p.parse(pos, explain_query->getExplainedQuery(), expected)) + if (!select_p.parse(pos, explain_query->children.at(0), expected)) return false; node = std::move(explain_query); diff --git a/src/Parsers/ParserQueryWithOutput.cpp b/src/Parsers/ParserQueryWithOutput.cpp index 80d44e86155..6c356955932 100644 --- a/src/Parsers/ParserQueryWithOutput.cpp +++ b/src/Parsers/ParserQueryWithOutput.cpp @@ -109,6 +109,17 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec query_with_output.children.push_back(query_with_output.settings_ast); } + if (auto * ast = query->as()) + { + /// Set default format TSV, because output is a single string column. + if (!ast->format) + { + ast->format = std::make_shared("TSV"); + ast->children.push_back(ast->format); + } + } + + node = std::move(query); return true; } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index d5ac5d0572c..f350aa4b00b 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include namespace DB @@ -11,6 +12,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +QueryPlan::QueryPlan() = default; QueryPlan::~QueryPlan() = default; void QueryPlan::checkInitialized() const @@ -173,4 +175,55 @@ void QueryPlan::addInterpreterContext(std::shared_ptr context) interpreter_context.emplace_back(std::move(context)); } +void QueryPlan::explain(WriteBuffer & buffer) +{ + checkInitialized(); + + size_t ident = 2; + + struct Frame + { + Node * node; + bool is_description_printed = false; + size_t next_child = 0; + }; + + std::stack stack; + stack.push(Frame{.node = root}); + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (!frame.is_description_printed) + { + std::string prefix((stack.size() - 1) * ident, ' '); + buffer.write(prefix.data(), prefix.size()); + + auto name = frame.node->step->getName(); + buffer.write(name.data(), name.size()); + + auto description = frame.node->step->getStepDescription(); + if (!description.empty()) + { + buffer.write(" (", 2); + buffer.write(description.data(), description.size()); + buffer.write(')'); + } + + buffer.write('\n'); + + frame.is_description_printed = true; + } + + if (frame.next_child < frame.node->children.size()) + { + stack.push(Frame{frame.node->children[frame.next_child]}); + ++frame.next_child; + } + else + stack.pop(); + } +} + } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 45dfd6cf601..6312f2e4180 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -15,11 +15,13 @@ class QueryPipeline; using QueryPipelinePtr = std::unique_ptr; class Context; +class WriteBuffer; /// A tree of query steps. class QueryPlan { public: + QueryPlan(); ~QueryPlan(); void unitePlans(QueryPlanStepPtr step, std::vector plans); @@ -31,6 +33,8 @@ public: QueryPipelinePtr buildQueryPipeline(); + void explain(WriteBuffer & buffer); + /// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines. /// TODO: make it in a better way. void setMaxThreads(size_t max_threads_) { max_threads = max_threads_; } diff --git a/tests/queries/0_stateless/00597_push_down_predicate.reference b/tests/queries/0_stateless/00597_push_down_predicate.reference index 829c5a1577e..1798c727088 100644 --- a/tests/queries/0_stateless/00597_push_down_predicate.reference +++ b/tests/queries/0_stateless/00597_push_down_predicate.reference @@ -4,61 +4,582 @@ 1 2000-01-01 1 test string 1 1 -------Forbid push down------- -SELECT count()\nFROM \n(\n SELECT \n [number] AS a,\n [number * 2] AS b\n FROM system.numbers\n LIMIT 1\n) AS t\nARRAY JOIN \n a,\n b\nWHERE NOT ignore(a + b) +SELECT count() +FROM +( + SELECT + [number] AS a, + [number * 2] AS b + FROM system.numbers + LIMIT 1 +) AS t +ARRAY JOIN + a, + b +WHERE NOT ignore(a + b) 1 -SELECT \n a,\n b\nFROM \n(\n SELECT 1 AS a\n)\nANY LEFT JOIN \n(\n SELECT \n 1 AS a,\n 1 AS b\n) USING (a)\nWHERE b = 0 -SELECT \n a,\n b\nFROM \n(\n SELECT \n 1 AS a,\n 1 AS b\n)\nANY RIGHT JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0 -SELECT \n a,\n b\nFROM \n(\n SELECT 1 AS a\n)\nANY FULL OUTER JOIN \n(\n SELECT \n 1 AS a,\n 1 AS b\n) USING (a)\nWHERE b = 0 -SELECT \n a,\n b\nFROM \n(\n SELECT \n 1 AS a,\n 1 AS b\n)\nANY FULL OUTER JOIN \n(\n SELECT 1 AS a\n) USING (a)\nWHERE b = 0 +SELECT + a, + b +FROM +( + SELECT 1 AS a +) +ANY LEFT JOIN +( + SELECT + 1 AS a, + 1 AS b +) USING (a) +WHERE b = 0 +SELECT + a, + b +FROM +( + SELECT + 1 AS a, + 1 AS b +) +ANY RIGHT JOIN +( + SELECT 1 AS a +) USING (a) +WHERE b = 0 +SELECT + a, + b +FROM +( + SELECT 1 AS a +) +ANY FULL OUTER JOIN +( + SELECT + 1 AS a, + 1 AS b +) USING (a) +WHERE b = 0 +SELECT + a, + b +FROM +( + SELECT + 1 AS a, + 1 AS b +) +ANY FULL OUTER JOIN +( + SELECT 1 AS a +) USING (a) +WHERE b = 0 -------Need push down------- -SELECT toString(value) AS value\nFROM \n(\n SELECT 1 AS value\n) +SELECT toString(value) AS value +FROM +( + SELECT 1 AS value +) 1 -SELECT id\nFROM \n(\n SELECT 1 AS id\n UNION ALL\n SELECT 2 AS `--predicate_optimizer_0`\n WHERE 0\n)\nWHERE id = 1 +SELECT id +FROM +( + SELECT 1 AS id + UNION ALL + SELECT 2 AS `--predicate_optimizer_0` + WHERE 0 +) +WHERE id = 1 1 -SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1 +SELECT id +FROM +( + SELECT arrayJoin([1, 2, 3]) AS id + WHERE id = 1 +) +WHERE id = 1 1 -SELECT id\nFROM \n(\n SELECT arrayJoin([1, 2, 3]) AS id\n WHERE id = 1\n)\nWHERE id = 1 +SELECT id +FROM +( + SELECT arrayJoin([1, 2, 3]) AS id + WHERE id = 1 +) +WHERE id = 1 1 -SELECT \n id,\n subquery\nFROM \n(\n SELECT \n 1 AS id,\n CAST(1, \'UInt8\') AS subquery\n) +SELECT + id, + subquery +FROM +( + SELECT + 1 AS id, + CAST(1, \'UInt8\') AS subquery +) 1 1 -SELECT \n a,\n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a,\n b\n FROM test_00597\n HAVING a = 3\n)\nWHERE a = 3 +SELECT + a, + b +FROM +( + SELECT + toUInt64(sum(id) AS b) AS a, + b + FROM test_00597 + HAVING a = 3 +) +WHERE a = 3 3 3 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n name,\n value,\n min(id) AS id\n FROM test_00597\n GROUP BY \n date,\n name,\n value\n HAVING id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + name, + value, + min(id) AS id + FROM test_00597 + GROUP BY + date, + name, + value + HAVING id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n a,\n b\nFROM \n(\n SELECT \n toUInt64(sum(id) AS b) AS a,\n b\n FROM test_00597 AS table_alias\n HAVING b = 3\n) AS outer_table_alias\nWHERE b = 3 +SELECT + a, + b +FROM +( + SELECT + toUInt64(sum(id) AS b) AS a, + b + FROM test_00597 AS table_alias + HAVING b = 3 +) AS outer_table_alias +WHERE b = 3 3 3 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n )\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM + ( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + ) + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n ) AS b\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM + ( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + ) AS b + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n )\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM + ( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + ) + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n) AS b\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 +) AS b +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n ) AS a\n WHERE id = 1\n) AS b\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM + ( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + ) AS a + WHERE id = 1 +) AS b +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n id,\n date,\n value\nFROM \n(\n SELECT \n id,\n date,\n min(value) AS value\n FROM test_00597\n WHERE id = 1\n GROUP BY \n id,\n date\n)\nWHERE id = 1 +SELECT + id, + date, + value +FROM +( + SELECT + id, + date, + min(value) AS value + FROM test_00597 + WHERE id = 1 + GROUP BY + id, + date +) +WHERE id = 1 1 2000-01-01 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n UNION ALL\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + UNION ALL + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value,\n date,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n)\nANY LEFT JOIN \n(\n SELECT id\n FROM test_00597\n) USING (id)\nWHERE id = 1 +SELECT + date, + id, + name, + value, + date, + name, + value +FROM +( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 +) +ANY LEFT JOIN +( + SELECT id + FROM test_00597 +) USING (id) +WHERE id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 -SELECT \n id,\n date,\n name,\n value\nFROM \n(\n SELECT toInt8(1) AS id\n)\nANY LEFT JOIN \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n) AS test_00597 USING (id)\nWHERE value = 1 +SELECT + id, + date, + name, + value +FROM +( + SELECT toInt8(1) AS id +) +ANY LEFT JOIN +( + SELECT + date, + id, + name, + value + FROM test_00597 +) AS test_00597 USING (id) +WHERE value = 1 1 2000-01-01 test string 1 1 -SELECT value\nFROM \n(\n SELECT toInt8(1) AS id\n)\nANY LEFT JOIN test_00597 AS b USING (id)\nWHERE value = 1 +SELECT value +FROM +( + SELECT toInt8(1) AS id +) +ANY LEFT JOIN test_00597 AS b USING (id) +WHERE value = 1 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value,\n date,\n name,\n value\n FROM \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n )\n ANY LEFT JOIN \n (\n SELECT id\n FROM test_00597\n ) USING (id)\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT + date, + id, + name, + value, + date, + name, + value + FROM + ( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + ) + ANY LEFT JOIN + ( + SELECT id + FROM test_00597 + ) USING (id) + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value,\n b.date,\n b.name,\n b.value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n)\nANY LEFT JOIN \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n) AS b USING (id)\nWHERE b.id = 1 +SELECT + date, + id, + name, + value, + b.date, + b.name, + b.value +FROM +( + SELECT + date, + id, + name, + value + FROM test_00597 +) +ANY LEFT JOIN +( + SELECT + date, + id, + name, + value + FROM test_00597 +) AS b USING (id) +WHERE b.id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 -SELECT \n id,\n date,\n name,\n value\nFROM \n(\n SELECT \n toInt8(1) AS id,\n toDate(\'2000-01-01\') AS date\n FROM system.numbers\n LIMIT 1\n)\nANY LEFT JOIN \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n) AS b USING (date, id)\nWHERE b.date = toDate(\'2000-01-01\') +SELECT + id, + date, + name, + value +FROM +( + SELECT + toInt8(1) AS id, + toDate(\'2000-01-01\') AS date + FROM system.numbers + LIMIT 1 +) +ANY LEFT JOIN +( + SELECT + date, + id, + name, + value + FROM test_00597 +) AS b USING (date, id) +WHERE b.date = toDate(\'2000-01-01\') 1 2000-01-01 test string 1 1 -SELECT \n date,\n id,\n name,\n value,\n `b.date`,\n `b.id`,\n `b.name`,\n `b.value`\nFROM \n(\n SELECT \n date,\n id,\n name,\n value,\n b.date,\n b.id,\n b.name,\n b.value\n FROM \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n ) AS a\n ANY LEFT JOIN \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n ) AS b ON id = b.id\n WHERE id = 1\n)\nWHERE id = 1 +SELECT + date, + id, + name, + value, + `b.date`, + `b.id`, + `b.name`, + `b.value` +FROM +( + SELECT + date, + id, + name, + value, + b.date, + b.id, + b.name, + b.value + FROM + ( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + ) AS a + ANY LEFT JOIN + ( + SELECT + date, + id, + name, + value + FROM test_00597 + ) AS b ON id = b.id + WHERE id = 1 +) +WHERE id = 1 2000-01-01 1 test string 1 1 2000-01-01 1 test string 1 1 -SELECT \n date,\n id,\n name,\n value,\n r.date,\n r.name,\n r.value\nFROM \n(\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n)\nSEMI LEFT JOIN \n(\n SELECT \n date,\n id,\n name,\n value\n FROM \n (\n SELECT \n date,\n id,\n name,\n value\n FROM test_00597\n WHERE id = 1\n )\n WHERE id = 1\n) AS r USING (id)\nWHERE r.id = 1 +SELECT + date, + id, + name, + value, + r.date, + r.name, + r.value +FROM +( + SELECT + date, + id, + name, + value + FROM test_00597 +) +SEMI LEFT JOIN +( + SELECT + date, + id, + name, + value + FROM + ( + SELECT + date, + id, + name, + value + FROM test_00597 + WHERE id = 1 + ) + WHERE id = 1 +) AS r USING (id) +WHERE r.id = 1 2000-01-01 1 test string 1 1 2000-01-01 test string 1 1 diff --git a/tests/queries/0_stateless/00826_cross_to_inner_join.reference b/tests/queries/0_stateless/00826_cross_to_inner_join.reference index 2a4b1487f20..6e5cbdcab4e 100644 --- a/tests/queries/0_stateless/00826_cross_to_inner_join.reference +++ b/tests/queries/0_stateless/00826_cross_to_inner_join.reference @@ -35,18 +35,74 @@ comma nullable 1 1 1 1 2 2 1 2 cross -SELECT \n a,\n b,\n t2_00826.a,\n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a +SELECT + a, + b, + t2_00826.a, + t2_00826.b +FROM t1_00826 +ALL INNER JOIN t2_00826 ON a = t2_00826.a +WHERE a = t2_00826.a cross nullable -SELECT \n a,\n b,\n t2_00826.a,\n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.a\nWHERE a = t2_00826.a +SELECT + a, + b, + t2_00826.a, + t2_00826.b +FROM t1_00826 +ALL INNER JOIN t2_00826 ON a = t2_00826.a +WHERE a = t2_00826.a cross nullable vs not nullable -SELECT \n a,\n b,\n t2_00826.a,\n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON a = t2_00826.b\nWHERE a = t2_00826.b +SELECT + a, + b, + t2_00826.a, + t2_00826.b +FROM t1_00826 +ALL INNER JOIN t2_00826 ON a = t2_00826.b +WHERE a = t2_00826.b cross self -SELECT \n a,\n b,\n y.a,\n y.b\nFROM t1_00826 AS x\nALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b)\nWHERE (a = y.a) AND (b = y.b) +SELECT + a, + b, + y.a, + y.b +FROM t1_00826 AS x +ALL INNER JOIN t1_00826 AS y ON (a = y.a) AND (b = y.b) +WHERE (a = y.a) AND (b = y.b) cross one table expr -SELECT \n a,\n b,\n t2_00826.a,\n t2_00826.b\nFROM t1_00826\nCROSS JOIN t2_00826\nWHERE a = b +SELECT + a, + b, + t2_00826.a, + t2_00826.b +FROM t1_00826 +CROSS JOIN t2_00826 +WHERE a = b cross multiple ands -SELECT \n a,\n b,\n t2_00826.a,\n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) +SELECT + a, + b, + t2_00826.a, + t2_00826.b +FROM t1_00826 +ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b) +WHERE (a = t2_00826.a) AND (b = t2_00826.b) cross and inside and -SELECT \n a,\n b,\n t2_00826.a,\n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) +SELECT + a, + b, + t2_00826.a, + t2_00826.b +FROM t1_00826 +ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (a = t2_00826.a) AND (a = t2_00826.a) AND (b = t2_00826.b) +WHERE (a = t2_00826.a) AND ((a = t2_00826.a) AND ((a = t2_00826.a) AND (b = t2_00826.b))) cross split conjunction -SELECT \n a,\n b,\n t2_00826.a,\n t2_00826.b\nFROM t1_00826\nALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b)\nWHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0) +SELECT + a, + b, + t2_00826.a, + t2_00826.b +FROM t1_00826 +ALL INNER JOIN t2_00826 ON (a = t2_00826.a) AND (b = t2_00826.b) +WHERE (a = t2_00826.a) AND (b = t2_00826.b) AND (a >= 1) AND (t2_00826.b > 0) diff --git a/tests/queries/0_stateless/00849_multiple_comma_join.reference b/tests/queries/0_stateless/00849_multiple_comma_join.reference index 0f7d28b65a0..5a5a90cbdf2 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join.reference +++ b/tests/queries/0_stateless/00849_multiple_comma_join.reference @@ -1,18 +1,254 @@ -SELECT a\nFROM t1_00849\nCROSS JOIN t2_00849 -SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON a = t2_00849.a\nWHERE a = t2_00849.a -SELECT a\nFROM t1_00849\nALL INNER JOIN t2_00849 ON b = t2_00849.b\nWHERE b = t2_00849.b -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a AS `--t2_00849.a`,\n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n) AS `--.s`\nALL INNER JOIN t3_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`,\n b AS `--t1_00849.b`,\n t2_00849.a,\n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n) AS `--.s`\nALL INNER JOIN t3_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n b,\n `--t2_00849.a`,\n `t2_00849.b`,\n a AS `--t3_00849.a`,\n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a AS `--t2_00849.a`,\n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t1_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n `--t1_00849.b`,\n `t2_00849.a`,\n `--t2_00849.b`,\n a,\n b AS `--t3_00849.b`\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b AS `--t1_00849.b`,\n t2_00849.a,\n t2_00849.b AS `--t2_00849.b`\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t1_00849.b` = b\nWHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n b,\n `--t2_00849.a`,\n `t2_00849.b`,\n a AS `--t3_00849.a`,\n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a AS `--t2_00849.a`,\n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t2_00849.a` = a\nWHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n b,\n `--t2_00849.a`,\n `t2_00849.b`,\n a AS `--t3_00849.a`,\n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a AS `--t2_00849.a`,\n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`)\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n b,\n `--t2_00849.a`,\n `t2_00849.b`,\n a AS `--t3_00849.a`,\n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a AS `--t2_00849.a`,\n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n CROSS JOIN t3_00849\n) AS `--.s`\nALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`)\nWHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n b,\n `--t2_00849.a`,\n `t2_00849.b`,\n a AS `--t3_00849.a`,\n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a AS `--t2_00849.a`,\n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n ) AS `--.s`\n ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a`\n) AS `--.s`\nALL INNER JOIN t4_00849 ON `--t3_00849.a` = a\nWHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n b,\n `t2_00849.a`,\n `t2_00849.b`,\n a,\n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a,\n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n CROSS JOIN t3_00849\n) AS `--.s`\nCROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n `--t1_00849.a`,\n b,\n `t2_00849.a`,\n `t2_00849.b`,\n a,\n t3_00849.b\n FROM \n (\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a,\n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n ) AS `--.s`\n CROSS JOIN t3_00849\n) AS `--.s`\nCROSS JOIN t4_00849 -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a,\n t2_00849.b\n FROM t1_00849\n CROSS JOIN t2_00849\n) AS `--.s`\nCROSS JOIN t3_00849 -SELECT `--t1_00849.a` AS `t1_00849.a`\nFROM \n(\n SELECT \n a AS `--t1_00849.a`,\n b,\n t2_00849.a AS `--t2_00849.a`,\n t2_00849.b\n FROM t1_00849\n ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a`\n) AS `--.s`\nCROSS JOIN t3_00849 +SELECT a +FROM t1_00849 +CROSS JOIN t2_00849 +SELECT a +FROM t1_00849 +ALL INNER JOIN t2_00849 ON a = t2_00849.a +WHERE a = t2_00849.a +SELECT a +FROM t1_00849 +ALL INNER JOIN t2_00849 ON b = t2_00849.b +WHERE b = t2_00849.b +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a AS `--t2_00849.a`, + t2_00849.b + FROM t1_00849 + ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` +) AS `--.s` +ALL INNER JOIN t3_00849 ON `--t1_00849.a` = a +WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + a AS `--t1_00849.a`, + b AS `--t1_00849.b`, + t2_00849.a, + t2_00849.b AS `--t2_00849.b` + FROM t1_00849 + ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b` +) AS `--.s` +ALL INNER JOIN t3_00849 ON `--t1_00849.b` = b +WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = b) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + b, + `--t2_00849.a`, + `t2_00849.b`, + a AS `--t3_00849.a`, + t3_00849.b + FROM + ( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a AS `--t2_00849.a`, + t2_00849.b + FROM t1_00849 + ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` + ) AS `--.s` + ALL INNER JOIN t3_00849 ON `--t1_00849.a` = `--t3_00849.a` +) AS `--.s` +ALL INNER JOIN t4_00849 ON `--t1_00849.a` = a +WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t1_00849.a` = `--t3_00849.a`) AND (`--t1_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + `--t1_00849.b`, + `t2_00849.a`, + `--t2_00849.b`, + a, + b AS `--t3_00849.b` + FROM + ( + SELECT + a AS `--t1_00849.a`, + b AS `--t1_00849.b`, + t2_00849.a, + t2_00849.b AS `--t2_00849.b` + FROM t1_00849 + ALL INNER JOIN t2_00849 ON `--t1_00849.b` = `--t2_00849.b` + ) AS `--.s` + ALL INNER JOIN t3_00849 ON `--t1_00849.b` = `--t3_00849.b` +) AS `--.s` +ALL INNER JOIN t4_00849 ON `--t1_00849.b` = b +WHERE (`--t1_00849.b` = `--t2_00849.b`) AND (`--t1_00849.b` = `--t3_00849.b`) AND (`--t1_00849.b` = b) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + b, + `--t2_00849.a`, + `t2_00849.b`, + a AS `--t3_00849.a`, + t3_00849.b + FROM + ( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a AS `--t2_00849.a`, + t2_00849.b + FROM t1_00849 + ALL INNER JOIN t2_00849 ON `--t2_00849.a` = `--t1_00849.a` + ) AS `--.s` + ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a` +) AS `--.s` +ALL INNER JOIN t4_00849 ON `--t2_00849.a` = a +WHERE (`--t2_00849.a` = `--t1_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t2_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + b, + `--t2_00849.a`, + `t2_00849.b`, + a AS `--t3_00849.a`, + t3_00849.b + FROM + ( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a AS `--t2_00849.a`, + t2_00849.b + FROM t1_00849 + CROSS JOIN t2_00849 + ) AS `--.s` + ALL INNER JOIN t3_00849 ON (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) +) AS `--.s` +ALL INNER JOIN t4_00849 ON `--t3_00849.a` = a +WHERE (`--t3_00849.a` = `--t1_00849.a`) AND (`--t3_00849.a` = `--t2_00849.a`) AND (`--t3_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + b, + `--t2_00849.a`, + `t2_00849.b`, + a AS `--t3_00849.a`, + t3_00849.b + FROM + ( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a AS `--t2_00849.a`, + t2_00849.b + FROM t1_00849 + CROSS JOIN t2_00849 + ) AS `--.s` + CROSS JOIN t3_00849 +) AS `--.s` +ALL INNER JOIN t4_00849 ON (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) +WHERE (a = `--t1_00849.a`) AND (a = `--t2_00849.a`) AND (a = `--t3_00849.a`) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + b, + `--t2_00849.a`, + `t2_00849.b`, + a AS `--t3_00849.a`, + t3_00849.b + FROM + ( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a AS `--t2_00849.a`, + t2_00849.b + FROM t1_00849 + ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` + ) AS `--.s` + ALL INNER JOIN t3_00849 ON `--t2_00849.a` = `--t3_00849.a` +) AS `--.s` +ALL INNER JOIN t4_00849 ON `--t3_00849.a` = a +WHERE (`--t1_00849.a` = `--t2_00849.a`) AND (`--t2_00849.a` = `--t3_00849.a`) AND (`--t3_00849.a` = a) +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + b, + `t2_00849.a`, + `t2_00849.b`, + a, + t3_00849.b + FROM + ( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a, + t2_00849.b + FROM t1_00849 + CROSS JOIN t2_00849 + ) AS `--.s` + CROSS JOIN t3_00849 +) AS `--.s` +CROSS JOIN t4_00849 +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + `--t1_00849.a`, + b, + `t2_00849.a`, + `t2_00849.b`, + a, + t3_00849.b + FROM + ( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a, + t2_00849.b + FROM t1_00849 + CROSS JOIN t2_00849 + ) AS `--.s` + CROSS JOIN t3_00849 +) AS `--.s` +CROSS JOIN t4_00849 +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a, + t2_00849.b + FROM t1_00849 + CROSS JOIN t2_00849 +) AS `--.s` +CROSS JOIN t3_00849 +SELECT `--t1_00849.a` AS `t1_00849.a` +FROM +( + SELECT + a AS `--t1_00849.a`, + b, + t2_00849.a AS `--t2_00849.a`, + t2_00849.b + FROM t1_00849 + ALL INNER JOIN t2_00849 ON `--t1_00849.a` = `--t2_00849.a` +) AS `--.s` +CROSS JOIN t3_00849 SELECT * FROM t1, t2 1 1 1 1 1 1 1 \N diff --git a/tests/queries/0_stateless/00849_multiple_comma_join_2.reference b/tests/queries/0_stateless/00849_multiple_comma_join_2.reference index f2e832123e0..e08d6ff1192 100644 --- a/tests/queries/0_stateless/00849_multiple_comma_join_2.reference +++ b/tests/queries/0_stateless/00849_multiple_comma_join_2.reference @@ -1,18 +1,197 @@ -SELECT a\nFROM t1\nCROSS JOIN t2 -SELECT a\nFROM t1\nALL INNER JOIN t2 ON a = t2.a\nWHERE a = t2.a -SELECT a\nFROM t1\nALL INNER JOIN t2 ON b = t2.b\nWHERE b = t2.b -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n a AS `--t1.a`,\n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n) AS `--.s`\nALL INNER JOIN t3 ON `--t1.a` = a\nWHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = a) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n b AS `--t1.b`,\n a AS `--t1.a`,\n t2.b AS `--t2.b`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.b` = `--t2.b`\n) AS `--.s`\nALL INNER JOIN t3 ON `--t1.b` = b\nWHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = b) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`,\n `--t2.a`,\n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`,\n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t1.a` = `--t3.a`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t1.a` = a\nWHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = `--t3.a`) AND (`--t1.a` = a) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.b`,\n `--t1.a`,\n `--t2.b`,\n b AS `--t3.b`\n FROM \n (\n SELECT \n b AS `--t1.b`,\n a AS `--t1.a`,\n t2.b AS `--t2.b`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.b` = `--t2.b`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t1.b` = `--t3.b`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t1.b` = b\nWHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = `--t3.b`) AND (`--t1.b` = b) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`,\n `--t2.a`,\n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`,\n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t2.a` = `--t1.a`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t2.a` = `--t3.a`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t2.a` = a\nWHERE (`--t2.a` = `--t1.a`) AND (`--t2.a` = `--t3.a`) AND (`--t2.a` = a) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`,\n `--t2.a`,\n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`,\n t2.a AS `--t2.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n ALL INNER JOIN t3 ON (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`)\n) AS `--.s`\nALL INNER JOIN t4 ON `--t3.a` = a\nWHERE (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`) AND (`--t3.a` = a) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`,\n `--t2.a`,\n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`,\n t2.a AS `--t2.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n CROSS JOIN t3\n) AS `--.s`\nALL INNER JOIN t4 ON (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`)\nWHERE (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n `--t1.a`,\n `--t2.a`,\n a AS `--t3.a`\n FROM \n (\n SELECT \n a AS `--t1.a`,\n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n ) AS `--.s`\n ALL INNER JOIN t3 ON `--t2.a` = `--t3.a`\n) AS `--.s`\nALL INNER JOIN t4 ON `--t3.a` = a\nWHERE (`--t1.a` = `--t2.a`) AND (`--t2.a` = `--t3.a`) AND (`--t3.a` = a) -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT `--t1.a`\n FROM \n (\n SELECT a AS `--t1.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n CROSS JOIN t3\n) AS `--.s`\nCROSS JOIN t4 -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT `--t1.a`\n FROM \n (\n SELECT a AS `--t1.a`\n FROM t1\n CROSS JOIN t2\n ) AS `--.s`\n CROSS JOIN t3\n) AS `--.s`\nCROSS JOIN t4 -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT a AS `--t1.a`\n FROM t1\n CROSS JOIN t2\n) AS `--.s`\nCROSS JOIN t3 -SELECT `--t1.a` AS `t1.a`\nFROM \n(\n SELECT \n a AS `--t1.a`,\n t2.a AS `--t2.a`\n FROM t1\n ALL INNER JOIN t2 ON `--t1.a` = `--t2.a`\n) AS `--.s`\nCROSS JOIN t3 +SELECT a +FROM t1 +CROSS JOIN t2 +SELECT a +FROM t1 +ALL INNER JOIN t2 ON a = t2.a +WHERE a = t2.a +SELECT a +FROM t1 +ALL INNER JOIN t2 ON b = t2.b +WHERE b = t2.b +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + a AS `--t1.a`, + t2.a AS `--t2.a` + FROM t1 + ALL INNER JOIN t2 ON `--t1.a` = `--t2.a` +) AS `--.s` +ALL INNER JOIN t3 ON `--t1.a` = a +WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = a) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + b AS `--t1.b`, + a AS `--t1.a`, + t2.b AS `--t2.b` + FROM t1 + ALL INNER JOIN t2 ON `--t1.b` = `--t2.b` +) AS `--.s` +ALL INNER JOIN t3 ON `--t1.b` = b +WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = b) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + `--t1.a`, + `--t2.a`, + a AS `--t3.a` + FROM + ( + SELECT + a AS `--t1.a`, + t2.a AS `--t2.a` + FROM t1 + ALL INNER JOIN t2 ON `--t1.a` = `--t2.a` + ) AS `--.s` + ALL INNER JOIN t3 ON `--t1.a` = `--t3.a` +) AS `--.s` +ALL INNER JOIN t4 ON `--t1.a` = a +WHERE (`--t1.a` = `--t2.a`) AND (`--t1.a` = `--t3.a`) AND (`--t1.a` = a) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + `--t1.b`, + `--t1.a`, + `--t2.b`, + b AS `--t3.b` + FROM + ( + SELECT + b AS `--t1.b`, + a AS `--t1.a`, + t2.b AS `--t2.b` + FROM t1 + ALL INNER JOIN t2 ON `--t1.b` = `--t2.b` + ) AS `--.s` + ALL INNER JOIN t3 ON `--t1.b` = `--t3.b` +) AS `--.s` +ALL INNER JOIN t4 ON `--t1.b` = b +WHERE (`--t1.b` = `--t2.b`) AND (`--t1.b` = `--t3.b`) AND (`--t1.b` = b) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + `--t1.a`, + `--t2.a`, + a AS `--t3.a` + FROM + ( + SELECT + a AS `--t1.a`, + t2.a AS `--t2.a` + FROM t1 + ALL INNER JOIN t2 ON `--t2.a` = `--t1.a` + ) AS `--.s` + ALL INNER JOIN t3 ON `--t2.a` = `--t3.a` +) AS `--.s` +ALL INNER JOIN t4 ON `--t2.a` = a +WHERE (`--t2.a` = `--t1.a`) AND (`--t2.a` = `--t3.a`) AND (`--t2.a` = a) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + `--t1.a`, + `--t2.a`, + a AS `--t3.a` + FROM + ( + SELECT + a AS `--t1.a`, + t2.a AS `--t2.a` + FROM t1 + CROSS JOIN t2 + ) AS `--.s` + ALL INNER JOIN t3 ON (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`) +) AS `--.s` +ALL INNER JOIN t4 ON `--t3.a` = a +WHERE (`--t3.a` = `--t1.a`) AND (`--t3.a` = `--t2.a`) AND (`--t3.a` = a) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + `--t1.a`, + `--t2.a`, + a AS `--t3.a` + FROM + ( + SELECT + a AS `--t1.a`, + t2.a AS `--t2.a` + FROM t1 + CROSS JOIN t2 + ) AS `--.s` + CROSS JOIN t3 +) AS `--.s` +ALL INNER JOIN t4 ON (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`) +WHERE (a = `--t1.a`) AND (a = `--t2.a`) AND (a = `--t3.a`) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + `--t1.a`, + `--t2.a`, + a AS `--t3.a` + FROM + ( + SELECT + a AS `--t1.a`, + t2.a AS `--t2.a` + FROM t1 + ALL INNER JOIN t2 ON `--t1.a` = `--t2.a` + ) AS `--.s` + ALL INNER JOIN t3 ON `--t2.a` = `--t3.a` +) AS `--.s` +ALL INNER JOIN t4 ON `--t3.a` = a +WHERE (`--t1.a` = `--t2.a`) AND (`--t2.a` = `--t3.a`) AND (`--t3.a` = a) +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT `--t1.a` + FROM + ( + SELECT a AS `--t1.a` + FROM t1 + CROSS JOIN t2 + ) AS `--.s` + CROSS JOIN t3 +) AS `--.s` +CROSS JOIN t4 +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT `--t1.a` + FROM + ( + SELECT a AS `--t1.a` + FROM t1 + CROSS JOIN t2 + ) AS `--.s` + CROSS JOIN t3 +) AS `--.s` +CROSS JOIN t4 +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT a AS `--t1.a` + FROM t1 + CROSS JOIN t2 +) AS `--.s` +CROSS JOIN t3 +SELECT `--t1.a` AS `t1.a` +FROM +( + SELECT + a AS `--t1.a`, + t2.a AS `--t2.a` + FROM t1 + ALL INNER JOIN t2 ON `--t1.a` = `--t2.a` +) AS `--.s` +CROSS JOIN t3 SELECT * FROM t1, t2 1 1 1 1 1 1 1 \N diff --git a/tests/queries/0_stateless/00908_analyze_query.reference b/tests/queries/0_stateless/00908_analyze_query.reference index 66db6f5a2e4..0305f528b25 100644 --- a/tests/queries/0_stateless/00908_analyze_query.reference +++ b/tests/queries/0_stateless/00908_analyze_query.reference @@ -1 +1,4 @@ -SELECT \n a,\n b\nFROM a +SELECT + a, + b +FROM a diff --git a/tests/queries/0_stateless/01029_early_constant_folding.reference b/tests/queries/0_stateless/01029_early_constant_folding.reference index 399f7f5d3b9..7e2f6c7ce76 100644 --- a/tests/queries/0_stateless/01029_early_constant_folding.reference +++ b/tests/queries/0_stateless/01029_early_constant_folding.reference @@ -1,5 +1,12 @@ -SELECT 1\nWHERE 0 SELECT 1 -SELECT 1\nWHERE 0 -SELECT 1\nWHERE 1 IN (\n(\n SELECT arrayJoin([1, 2, 3])\n) AS subquery) -SELECT 1\nWHERE NOT ignore() +WHERE 0 +SELECT 1 +SELECT 1 +WHERE 0 +SELECT 1 +WHERE 1 IN ( +( + SELECT arrayJoin([1, 2, 3]) +) AS subquery) +SELECT 1 +WHERE NOT ignore() diff --git a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference index c797226d832..fd9d96bdf5f 100644 --- a/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference +++ b/tests/queries/0_stateless/01056_predicate_optimizer_bugs.reference @@ -1,19 +1,125 @@ -SELECT \n k,\n v,\n d,\n i\nFROM \n(\n SELECT \n t.1 AS k,\n t.2 AS v,\n runningDifference(v) AS d,\n runningDifference(cityHash64(t.1)) AS i\n FROM \n (\n SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t\n )\n)\nWHERE i = 0 +SELECT + k, + v, + d, + i +FROM +( + SELECT + t.1 AS k, + t.2 AS v, + runningDifference(v) AS d, + runningDifference(cityHash64(t.1)) AS i + FROM + ( + SELECT arrayJoin([(\'a\', 1), (\'a\', 2), (\'a\', 3), (\'b\', 11), (\'b\', 13), (\'b\', 15)]) AS t + ) +) +WHERE i = 0 a 1 0 0 a 2 1 0 a 3 1 0 b 13 2 0 b 15 2 0 -SELECT \n co,\n co2,\n co3,\n num\nFROM \n(\n SELECT \n co,\n co2,\n co3,\n count() AS num\n FROM \n (\n SELECT \n 1 AS co,\n 2 AS co2,\n 3 AS co3\n )\n GROUP BY \n co,\n co2,\n co3\n WITH CUBE\n HAVING (co2 != 2) AND (co != 0)\n)\nWHERE (co != 0) AND (co2 != 2) +SELECT + co, + co2, + co3, + num +FROM +( + SELECT + co, + co2, + co3, + count() AS num + FROM + ( + SELECT + 1 AS co, + 2 AS co2, + 3 AS co3 + ) + GROUP BY + co, + co2, + co3 + WITH CUBE + HAVING (co2 != 2) AND (co != 0) +) +WHERE (co != 0) AND (co2 != 2) 1 0 3 1 1 0 0 1 -SELECT alias AS name\nFROM \n(\n SELECT name AS alias\n FROM system.settings\n WHERE alias = \'enable_optimize_predicate_expression\'\n)\nANY INNER JOIN \n(\n SELECT name\n FROM system.settings\n) USING (name)\nWHERE name = \'enable_optimize_predicate_expression\' +SELECT alias AS name +FROM +( + SELECT name AS alias + FROM system.settings + WHERE alias = \'enable_optimize_predicate_expression\' +) +ANY INNER JOIN +( + SELECT name + FROM system.settings +) USING (name) +WHERE name = \'enable_optimize_predicate_expression\' enable_optimize_predicate_expression 1 val11 val21 val31 -SELECT ccc\nFROM \n(\n SELECT 1 AS ccc\n WHERE 0\n UNION ALL\n SELECT ccc\n FROM \n (\n SELECT 2 AS ccc\n )\n ANY INNER JOIN \n (\n SELECT 2 AS ccc\n ) USING (ccc)\n WHERE ccc > 1\n)\nWHERE ccc > 1 +SELECT ccc +FROM +( + SELECT 1 AS ccc + WHERE 0 + UNION ALL + SELECT ccc + FROM + ( + SELECT 2 AS ccc + ) + ANY INNER JOIN + ( + SELECT 2 AS ccc + ) USING (ccc) + WHERE ccc > 1 +) +WHERE ccc > 1 2 -SELECT \n ts,\n id,\n id_b,\n b.ts,\n b.id,\n id_c\nFROM \n(\n SELECT \n ts,\n id,\n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON b.id = id_b\nWHERE ts <= toDateTime(\'1970-01-01 03:00:00\') -SELECT \n ts AS `--a.ts`,\n id AS `--a.id`,\n id_b AS `--a.id_b`,\n b.ts AS `--b.ts`,\n b.id AS `--b.id`,\n id_c AS `--b.id_c`\nFROM \n(\n SELECT \n ts,\n id,\n id_b\n FROM A\n WHERE ts <= toDateTime(\'1970-01-01 03:00:00\')\n) AS a\nALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b`\nWHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\') +SELECT + ts, + id, + id_b, + b.ts, + b.id, + id_c +FROM +( + SELECT + ts, + id, + id_b + FROM A + WHERE ts <= toDateTime(\'1970-01-01 03:00:00\') +) AS a +ALL LEFT JOIN B AS b ON b.id = id_b +WHERE ts <= toDateTime(\'1970-01-01 03:00:00\') +SELECT + ts AS `--a.ts`, + id AS `--a.id`, + id_b AS `--a.id_b`, + b.ts AS `--b.ts`, + b.id AS `--b.id`, + id_c AS `--b.id_c` +FROM +( + SELECT + ts, + id, + id_b + FROM A + WHERE ts <= toDateTime(\'1970-01-01 03:00:00\') +) AS a +ALL LEFT JOIN B AS b ON `--b.id` = `--a.id_b` +WHERE `--a.ts` <= toDateTime(\'1970-01-01 03:00:00\') 2 3 3 4 4 5 @@ -22,8 +128,33 @@ SELECT \n ts AS `--a.ts`,\n id AS `--a.id`,\n id_b AS `--a.id_b`,\n 4 0 2 3 4 5 -SELECT dummy\nFROM \n(\n SELECT dummy\n FROM system.one\n WHERE arrayMap(x -> (x + 1), [dummy]) = [1]\n)\nWHERE arrayMap(x -> (x + 1), [dummy]) = [1] +SELECT dummy +FROM +( + SELECT dummy + FROM system.one + WHERE arrayMap(x -> (x + 1), [dummy]) = [1] +) +WHERE arrayMap(x -> (x + 1), [dummy]) = [1] 0 -SELECT \n id,\n value,\n value_1\nFROM \n(\n SELECT \n 1 AS id,\n 2 AS value\n)\nALL INNER JOIN \n(\n SELECT \n 1 AS id,\n 3 AS value_1\n) USING (id)\nWHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6] +SELECT + id, + value, + value_1 +FROM +( + SELECT + 1 AS id, + 2 AS value +) +ALL INNER JOIN +( + SELECT + 1 AS id, + 3 AS value_1 +) USING (id) +WHERE arrayMap(x -> ((x + value) + value_1), [1]) = [6] 1 2 3 -SELECT dummy\nFROM system.one\nWHERE (dummy > 0) AND (dummy < 0) +SELECT dummy +FROM system.one +WHERE (dummy > 0) AND (dummy < 0) diff --git a/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference b/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference index 5cc0a546e27..d6426f679c5 100644 --- a/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference +++ b/tests/queries/0_stateless/01076_predicate_optimizer_with_view.reference @@ -1,4 +1,40 @@ -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 1\n) AS test_view\nWHERE id = 1 -SELECT \n date,\n id,\n name,\n value\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 2\n) AS test_view\nWHERE id = 2 -SELECT id\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 1\n) AS test_view\nWHERE id = 1 -SELECT id\nFROM \n(\n SELECT *\n FROM default.test\n HAVING id = 1\n) AS s\nWHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT * + FROM default.test + HAVING id = 1 +) AS test_view +WHERE id = 1 +SELECT + date, + id, + name, + value +FROM +( + SELECT * + FROM default.test + HAVING id = 2 +) AS test_view +WHERE id = 2 +SELECT id +FROM +( + SELECT * + FROM default.test + HAVING id = 1 +) AS test_view +WHERE id = 1 +SELECT id +FROM +( + SELECT * + FROM default.test + HAVING id = 1 +) AS s +WHERE id = 1 diff --git a/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference b/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference index e6ebffcae9c..5491e82c7d3 100644 --- a/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference +++ b/tests/queries/0_stateless/01083_cross_to_inner_with_like.reference @@ -1,3 +1,21 @@ -SELECT \n k,\n r.k,\n name\nFROM n\nALL INNER JOIN r ON k = r.k\nWHERE (k = r.k) AND (name = \'A\') -SELECT \n k,\n r.k,\n name\nFROM n\nALL INNER JOIN r ON k = r.k\nWHERE (k = r.k) AND (name LIKE \'A%\') -SELECT \n k,\n r.k,\n name\nFROM n\nALL INNER JOIN r ON k = r.k\nWHERE (k = r.k) AND (name NOT LIKE \'A%\') +SELECT + k, + r.k, + name +FROM n +ALL INNER JOIN r ON k = r.k +WHERE (k = r.k) AND (name = \'A\') +SELECT + k, + r.k, + name +FROM n +ALL INNER JOIN r ON k = r.k +WHERE (k = r.k) AND (name LIKE \'A%\') +SELECT + k, + r.k, + name +FROM n +ALL INNER JOIN r ON k = r.k +WHERE (k = r.k) AND (name NOT LIKE \'A%\') diff --git a/tests/queries/0_stateless/01300_group_by_other_keys.reference b/tests/queries/0_stateless/01300_group_by_other_keys.reference index 52a5e7f0002..7f0661ec4fe 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys.reference @@ -22,11 +22,35 @@ 3465736.595946905 3465735.2096525617 3465735.9027997428 -SELECT max(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n number % 2,\n number % 3,\n ((number % 2) + (number % 3)) % 2\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY number % 5\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n (number % 2) * (number % 3),\n number % 3\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n number % 3,\n number % 2\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n (number % 2) % 3,\n number % 2\nORDER BY k ASC +SELECT max(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + number % 2, + number % 3, + ((number % 2) + (number % 3)) % 2 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY number % 5 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + (number % 2) * (number % 3), + number % 3 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + number % 3, + number % 2 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + (number % 2) % 3, + number % 2 +ORDER BY k ASC 6931467.646716369 6931468.33986355 6931469.0330107305 @@ -51,8 +75,35 @@ SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n (number 3465736.595946905 3465735.2096525617 3465735.9027997428 -SELECT max(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n number % 2,\n number % 3,\n ((number % 2) + (number % 3)) % 2\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n number % 5,\n (number % 5) * (number % 5)\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n (number % 2) * (number % 3),\n number % 3\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n (number % 2) * (number % 3),\n number % 3,\n number % 2\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n (number % 2) % 3,\n number % 2\nORDER BY k ASC +SELECT max(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + number % 2, + number % 3, + ((number % 2) + (number % 3)) % 2 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + number % 5, + (number % 5) * (number % 5) +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + (number % 2) * (number % 3), + number % 3 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + (number % 2) * (number % 3), + number % 3, + number % 2 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + (number % 2) % 3, + number % 2 +ORDER BY k ASC diff --git a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference index d7c04e64df5..67262193df5 100644 --- a/tests/queries/0_stateless/01300_group_by_other_keys_having.reference +++ b/tests/queries/0_stateless/01300_group_by_other_keys_having.reference @@ -7,9 +7,23 @@ 0 1 4 -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n number % 3,\n number % 2\nHAVING avg(log(2) * number) > 3465735.3\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nWHERE ((number % 5) * (number % 5)) < 5\nGROUP BY number % 5\nORDER BY k ASC -SELECT (number % 5) * (number % 5) AS k\nFROM numbers(10000000)\nWHERE ((number % 5) * (number % 5)) < 5\nGROUP BY number % 5\nORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + number % 3, + number % 2 +HAVING avg(log(2) * number) > 3465735.3 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +WHERE ((number % 5) * (number % 5)) < 5 +GROUP BY number % 5 +ORDER BY k ASC +SELECT (number % 5) * (number % 5) AS k +FROM numbers(10000000) +WHERE ((number % 5) * (number % 5)) < 5 +GROUP BY number % 5 +ORDER BY k ASC 3465735.9027997246 3465735.902799725 3465736.595946905 @@ -19,6 +33,25 @@ SELECT (number % 5) * (number % 5) AS k\nFROM numbers(10000000)\nWHERE ((number 0 1 4 -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nGROUP BY \n (number % 2) * (number % 3),\n number % 3,\n number % 2\nHAVING avg(log(2) * number) > 3465735.3\nORDER BY k ASC -SELECT avg(log(2) * number) AS k\nFROM numbers(10000000)\nWHERE ((number % 5) * (number % 5)) < 5\nGROUP BY \n number % 5,\n (number % 5) * (number % 5)\nORDER BY k ASC -SELECT (number % 5) * (number % 5) AS k\nFROM numbers(10000000)\nWHERE ((number % 5) * (number % 5)) < 5\nGROUP BY \n number % 5,\n (number % 5) * (number % 5)\nORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +GROUP BY + (number % 2) * (number % 3), + number % 3, + number % 2 +HAVING avg(log(2) * number) > 3465735.3 +ORDER BY k ASC +SELECT avg(log(2) * number) AS k +FROM numbers(10000000) +WHERE ((number % 5) * (number % 5)) < 5 +GROUP BY + number % 5, + (number % 5) * (number % 5) +ORDER BY k ASC +SELECT (number % 5) * (number % 5) AS k +FROM numbers(10000000) +WHERE ((number % 5) * (number % 5)) < 5 +GROUP BY + number % 5, + (number % 5) * (number % 5) +ORDER BY k ASC diff --git a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference b/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference index 208f3d1abe5..5e6fdfcdefb 100644 --- a/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference +++ b/tests/queries/0_stateless/01305_duplicate_order_by_and_distinct.reference @@ -1,14 +1,58 @@ -SELECT number\nFROM \n(\n SELECT number\n FROM \n (\n SELECT DISTINCT number\n FROM numbers(3)\n )\n)\nORDER BY number ASC +SELECT number +FROM +( + SELECT number + FROM + ( + SELECT DISTINCT number + FROM numbers(3) + ) +) +ORDER BY number ASC 0 1 2 -SELECT DISTINCT number\nFROM \n(\n SELECT DISTINCT number\n FROM \n (\n SELECT DISTINCT number\n FROM numbers(3)\n ORDER BY number ASC\n )\n ORDER BY number ASC\n)\nORDER BY number ASC +SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM + ( + SELECT DISTINCT number + FROM numbers(3) + ORDER BY number ASC + ) + ORDER BY number ASC +) +ORDER BY number ASC 0 1 2 -SELECT number\nFROM \n(\n SELECT DISTINCT number\n FROM \n (\n SELECT DISTINCT number % 2 AS number\n FROM numbers(3)\n )\n)\nORDER BY number ASC +SELECT number +FROM +( + SELECT DISTINCT number + FROM + ( + SELECT DISTINCT number % 2 AS number + FROM numbers(3) + ) +) +ORDER BY number ASC 0 1 -SELECT DISTINCT number\nFROM \n(\n SELECT DISTINCT number\n FROM \n (\n SELECT DISTINCT number % 2 AS number\n FROM numbers(3)\n ORDER BY number ASC\n )\n ORDER BY number ASC\n)\nORDER BY number ASC +SELECT DISTINCT number +FROM +( + SELECT DISTINCT number + FROM + ( + SELECT DISTINCT number % 2 AS number + FROM numbers(3) + ORDER BY number ASC + ) + ORDER BY number ASC +) +ORDER BY number ASC 0 1 diff --git a/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference b/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference index 0ca6bee85a3..875a6753f84 100644 --- a/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference +++ b/tests/queries/0_stateless/01321_aggregate_functions_of_group_by_keys.reference @@ -47,10 +47,39 @@ 24 0 0 -SELECT \n number % 2 AS a,\n number % 3 AS b\nFROM numbers(10000000)\nGROUP BY \n number % 2,\n number % 3\nORDER BY \n min(number % 2) AS a ASC,\n max(number % 3) AS b ASC -SELECT \n number % 2 AS a,\n number % 3 AS b\nFROM numbers(10000000)\nGROUP BY \n number % 2,\n number % 3\nORDER BY \n any(number % 2) AS a ASC,\n anyLast(number % 3) AS b ASC -SELECT (number % 5) * (number % 7) AS a\nFROM numbers(10000000)\nGROUP BY \n number % 7,\n number % 5\nORDER BY max((number % 5) * (number % 7)) AS a ASC -SELECT foo\nFROM \n(\n SELECT number AS foo\n FROM numbers(1)\n GROUP BY number\n) +SELECT + number % 2 AS a, + number % 3 AS b +FROM numbers(10000000) +GROUP BY + number % 2, + number % 3 +ORDER BY + min(number % 2) AS a ASC, + max(number % 3) AS b ASC +SELECT + number % 2 AS a, + number % 3 AS b +FROM numbers(10000000) +GROUP BY + number % 2, + number % 3 +ORDER BY + any(number % 2) AS a ASC, + anyLast(number % 3) AS b ASC +SELECT (number % 5) * (number % 7) AS a +FROM numbers(10000000) +GROUP BY + number % 7, + number % 5 +ORDER BY max((number % 5) * (number % 7)) AS a ASC +SELECT foo +FROM +( + SELECT number AS foo + FROM numbers(1) + GROUP BY number +) 0 0 0 1 0 2 @@ -99,7 +128,36 @@ SELECT foo\nFROM \n(\n SELECT number AS foo\n FROM numbers(1)\n GROUP B 20 24 0 -SELECT \n min(number % 2) AS a,\n max(number % 3) AS b\nFROM numbers(10000000)\nGROUP BY \n number % 2,\n number % 3\nORDER BY \n a ASC,\n b ASC -SELECT \n any(number % 2) AS a,\n anyLast(number % 3) AS b\nFROM numbers(10000000)\nGROUP BY \n number % 2,\n number % 3\nORDER BY \n a ASC,\n b ASC -SELECT max((number % 5) * (number % 7)) AS a\nFROM numbers(10000000)\nGROUP BY \n number % 7,\n number % 5\nORDER BY a ASC -SELECT foo\nFROM \n(\n SELECT anyLast(number) AS foo\n FROM numbers(1)\n GROUP BY number\n) +SELECT + min(number % 2) AS a, + max(number % 3) AS b +FROM numbers(10000000) +GROUP BY + number % 2, + number % 3 +ORDER BY + a ASC, + b ASC +SELECT + any(number % 2) AS a, + anyLast(number % 3) AS b +FROM numbers(10000000) +GROUP BY + number % 2, + number % 3 +ORDER BY + a ASC, + b ASC +SELECT max((number % 5) * (number % 7)) AS a +FROM numbers(10000000) +GROUP BY + number % 7, + number % 5 +ORDER BY a ASC +SELECT foo +FROM +( + SELECT anyLast(number) AS foo + FROM numbers(1) + GROUP BY number +) diff --git a/tests/queries/0_stateless/01322_any_input_optimize.reference b/tests/queries/0_stateless/01322_any_input_optimize.reference index 5b724453c71..209a4afb3a2 100644 --- a/tests/queries/0_stateless/01322_any_input_optimize.reference +++ b/tests/queries/0_stateless/01322_any_input_optimize.reference @@ -1,2 +1,3 @@ 9 -SELECT any(number) + (any(number) * 2)\nFROM numbers(3, 10) +SELECT any(number) + (any(number) * 2) +FROM numbers(3, 10) From 47d9f4af9c20d7db0e47957555ded0c03330334a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Jun 2020 14:26:06 +0300 Subject: [PATCH 040/196] Header for explain. --- src/Core/ColumnWithTypeAndName.cpp | 7 ++- src/Core/ColumnWithTypeAndName.h | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 64 +++++++++++++++++++- src/Parsers/ASTExplainQuery.h | 23 ++++++- src/Parsers/ParserExplainQuery.cpp | 17 +++++- src/Processors/QueryPlan/QueryPlan.cpp | 62 +++++++++++++------ src/Processors/QueryPlan/QueryPlan.h | 7 ++- 7 files changed, 155 insertions(+), 27 deletions(-) diff --git a/src/Core/ColumnWithTypeAndName.cpp b/src/Core/ColumnWithTypeAndName.cpp index 9acc2d56408..d5b43b3841b 100644 --- a/src/Core/ColumnWithTypeAndName.cpp +++ b/src/Core/ColumnWithTypeAndName.cpp @@ -28,9 +28,12 @@ bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) cons } -void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const +void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out, bool escaped) const { - out << name; + if (escaped) + writeEscapedString(name, out); + else + out << name; if (type) out << ' ' << type->getName(); diff --git a/src/Core/ColumnWithTypeAndName.h b/src/Core/ColumnWithTypeAndName.h index 27b09710258..b2f1a076b76 100644 --- a/src/Core/ColumnWithTypeAndName.h +++ b/src/Core/ColumnWithTypeAndName.h @@ -33,7 +33,7 @@ struct ColumnWithTypeAndName ColumnWithTypeAndName cloneEmpty() const; bool operator==(const ColumnWithTypeAndName & other) const; - void dumpStructure(WriteBuffer & out) const; + void dumpStructure(WriteBuffer & out, bool escaped = false) const; String dumpStructure() const; }; #pragma GCC diagnostic pop diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index c79cc93409a..8c6e0069f9f 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -24,6 +24,8 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_QUERY; + extern const int INVALID_SETTING_VALUE; + extern const int UNKNOWN_SETTING; } namespace @@ -106,9 +108,67 @@ static void fillColumn(IColumn & column, const std::string & str) column.insertData(str.data() + start, end - start); } +using BinarySettings = std::unordered_map; + +BinarySettings checkAndGetSettings(const ASTPtr & ast_settings) +{ + if (!ast_settings) + return {}; + + NameSet supported_settings = {"header"}; + auto get_supported_settings_string = [&supported_settings]() + { + std::string res; + for (const auto & setting : supported_settings) + { + if (!res.empty()) + res += ", "; + + res += setting; + } + + return res; + }; + + BinarySettings settings; + const auto & set_query = ast_settings->as(); + + for (const auto & change : set_query.changes) + { + if (supported_settings.count(change.name) == 0) + throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN query. Supported settings: " + + get_supported_settings_string(), ErrorCodes::UNKNOWN_SETTING); + + if (change.value.getType() != Field::Types::UInt64) + throw Exception("Invalid type " + std::string(change.value.getTypeName()) + " for setting \"" + change.name + + "\" only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE); + + auto value = change.value.get(); + if (value > 1) + throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name + + "\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE); + + settings[change.name] = value; + } + + return settings; +} + +static QueryPlan::ExplainOptions getExplainOptions(const BinarySettings & settings) +{ + QueryPlan::ExplainOptions options; + + auto it = settings.find("header"); + if (it != settings.end()) + options.header = it->second; + + return options; +} + BlockInputStreamPtr InterpreterExplainQuery::executeImpl() { const auto & ast = query->as(); + auto settings = checkAndGetSettings(ast.getSettings()); Block sample_block = getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); @@ -124,7 +184,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() ExplainAnalyzedSyntaxVisitor::Data data{.context = context}; ExplainAnalyzedSyntaxVisitor(data).visit(query); - ast.children.at(0)->format(IAST::FormatSettings(ss, false)); + ast.getExplainedQuery()->format(IAST::FormatSettings(ss, false)); } else if (ast.getKind() == ASTExplainQuery::QueryPlan) { @@ -137,7 +197,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() interpreter.buildQueryPlan(plan); WriteBufferFromOStream buffer(ss); - plan.explain(buffer); + plan.explain(buffer, getExplainOptions(settings)); } fillColumn(*res_columns[0], ss.str()); diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index 1e3aca25700..e813b211f50 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -21,7 +21,6 @@ public: ASTExplainQuery(ExplainKind kind_, bool old_syntax_) : kind(kind_), old_syntax(old_syntax_) { - children.emplace_back(); /// explained query } String getID(char delim) const override { return "Explain" + (delim + toString(kind, old_syntax)); } @@ -35,19 +34,37 @@ public: return res; } - const ASTPtr & getExplainedQuery() const { return children.at(0); } + void setExplainedQuery(ASTPtr query_) + { + children.emplace_back(query_); + query = std::move(query_); + } + + void setSettings(ASTPtr settings_) + { + children.emplace_back(settings_); + ast_settings = std::move(settings_); + } + + const ASTPtr & getExplainedQuery() const { return query; } + const ASTPtr & getSettings() const { return ast_settings; } protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind, old_syntax) << (settings.hilite ? hilite_none : "") << " "; - children.at(0)->formatImpl(settings, state, frame); + ast_settings->formatImpl(settings, state, frame); + settings.ostr << settings.nl_or_ws; + query->formatImpl(settings, state, frame); } private: ExplainKind kind; bool old_syntax; /// "EXPLAIN AST" -> "AST", "EXPLAIN SYNTAX" -> "ANALYZE" + ASTPtr query; + ASTPtr ast_settings; + static String toString(ExplainKind kind, bool old_syntax) { switch (kind) diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index 7c6e8e0085e..892cc2116dc 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -40,10 +41,24 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected auto explain_query = std::make_shared(kind, old_syntax); + { + ASTPtr settings; + ParserSetQuery parser_settings(true); + + auto begin = pos; + if (parser_settings.parse(pos, settings, expected)) + explain_query->setSettings(std::move(settings)); + else + pos = begin; + } + ParserSelectWithUnionQuery select_p; - if (!select_p.parse(pos, explain_query->children.at(0), expected)) + ASTPtr query; + if (!select_p.parse(pos, query, expected)) return false; + explain_query->setExplainedQuery(std::move(query)); + node = std::move(explain_query); return true; } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index f350aa4b00b..7b2a46cbb90 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include namespace DB @@ -175,7 +177,48 @@ void QueryPlan::addInterpreterContext(std::shared_ptr context) interpreter_context.emplace_back(std::move(context)); } -void QueryPlan::explain(WriteBuffer & buffer) + +static void explainStep( + WriteBuffer & buffer, IQueryPlanStep & step, size_t ident, const QueryPlan::ExplainOptions & options) +{ + std::string prefix(ident, ' '); + buffer << prefix; + buffer << step.getName(); + + const auto & description = step.getStepDescription(); + if (!description.empty()) + buffer <<" (" << description << ')'; + + buffer.write('\n'); + + if (options.header) + { + buffer << prefix; + + if (!step.hasOutputStream()) + buffer << "No header"; + else if (!step.getOutputStream().header) + buffer << "Empty header"; + else + { + buffer << "Header: "; + bool first = true; + + for (const auto & elem : step.getOutputStream().header) + { + if (!first) + buffer << ",\n" << prefix << " "; + + first = false; + elem.dumpStructure(buffer, true); + } + } + + buffer.write('\n'); + } +} + +void QueryPlan::explain(WriteBuffer & buffer, const ExplainOptions & options) { checkInitialized(); @@ -197,22 +240,7 @@ void QueryPlan::explain(WriteBuffer & buffer) if (!frame.is_description_printed) { - std::string prefix((stack.size() - 1) * ident, ' '); - buffer.write(prefix.data(), prefix.size()); - - auto name = frame.node->step->getName(); - buffer.write(name.data(), name.size()); - - auto description = frame.node->step->getStepDescription(); - if (!description.empty()) - { - buffer.write(" (", 2); - buffer.write(description.data(), description.size()); - buffer.write(')'); - } - - buffer.write('\n'); - + explainStep(buffer, *frame.node->step, (stack.size() - 1) * ident, options); frame.is_description_printed = true; } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 6312f2e4180..e544ddfc1ee 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -33,7 +33,12 @@ public: QueryPipelinePtr buildQueryPipeline(); - void explain(WriteBuffer & buffer); + struct ExplainOptions + { + bool header = false; + }; + + void explain(WriteBuffer & buffer, const ExplainOptions & options); /// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines. /// TODO: make it in a better way. From 8408c03aff825d2512bf258f22dfb769469cb0c2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Jun 2020 16:52:36 +0300 Subject: [PATCH 041/196] Update explain settings. --- src/Interpreters/InterpreterExplainQuery.cpp | 68 ++++++++++++-------- 1 file changed, 42 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 8c6e0069f9f..b7bbd8a27ed 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes extern const int INCORRECT_QUERY; extern const int INVALID_SETTING_VALUE; extern const int UNKNOWN_SETTING; + extern const int LOGICAL_ERROR; } namespace @@ -108,36 +109,62 @@ static void fillColumn(IColumn & column, const std::string & str) column.insertData(str.data() + start, end - start); } -using BinarySettings = std::unordered_map; - -BinarySettings checkAndGetSettings(const ASTPtr & ast_settings) +namespace { - if (!ast_settings) - return {}; - NameSet supported_settings = {"header"}; - auto get_supported_settings_string = [&supported_settings]() +struct ExplainSettings +{ + QueryPlan::ExplainOptions query_plan_options; + + std::unordered_map> boolean_settings = + { + {"header", query_plan_options.header}, + }; + + bool has(const std::string & name) const + { + return boolean_settings.count(name) > 0; + } + + void setBooleanSetting(const std::string & name, bool value) + { + auto it = boolean_settings.find(name); + if (it == boolean_settings.end()) + throw Exception("Unknown setting for ExplainSettings: " + name, ErrorCodes::LOGICAL_ERROR); + + it->second.get() = value; + } + + std::string getSettingsList() const { std::string res; - for (const auto & setting : supported_settings) + for (const auto & setting : boolean_settings) { if (!res.empty()) res += ", "; - res += setting; + res += setting.first; } return res; - }; + } +}; - BinarySettings settings; +} + +ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) +{ + if (!ast_settings) + return {}; + + ExplainSettings settings; const auto & set_query = ast_settings->as(); for (const auto & change : set_query.changes) { - if (supported_settings.count(change.name) == 0) + if (!settings.has(change.name)) throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN query. Supported settings: " + - get_supported_settings_string(), ErrorCodes::UNKNOWN_SETTING); + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING); if (change.value.getType() != Field::Types::UInt64) throw Exception("Invalid type " + std::string(change.value.getTypeName()) + " for setting \"" + change.name + @@ -148,23 +175,12 @@ BinarySettings checkAndGetSettings(const ASTPtr & ast_settings) throw Exception("Invalid value " + std::to_string(value) + " for setting \"" + change.name + "\". Only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE); - settings[change.name] = value; + settings.setBooleanSetting(change.name, value); } return settings; } -static QueryPlan::ExplainOptions getExplainOptions(const BinarySettings & settings) -{ - QueryPlan::ExplainOptions options; - - auto it = settings.find("header"); - if (it != settings.end()) - options.header = it->second; - - return options; -} - BlockInputStreamPtr InterpreterExplainQuery::executeImpl() { const auto & ast = query->as(); @@ -197,7 +213,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() interpreter.buildQueryPlan(plan); WriteBufferFromOStream buffer(ss); - plan.explain(buffer, getExplainOptions(settings)); + plan.explain(buffer, settings.query_plan_options); } fillColumn(*res_columns[0], ss.str()); From b282a1775fcac494fc1f99758bbcf9fbfe2baef1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Jun 2020 17:11:15 +0300 Subject: [PATCH 042/196] Add description setting. --- src/Interpreters/InterpreterExplainQuery.cpp | 1 + src/Processors/QueryPlan/QueryPlan.cpp | 2 +- src/Processors/QueryPlan/QueryPlan.h | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index b7bbd8a27ed..bbfa83af2e7 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -119,6 +119,7 @@ struct ExplainSettings std::unordered_map> boolean_settings = { {"header", query_plan_options.header}, + {"description", query_plan_options.description}, }; bool has(const std::string & name) const diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 7b2a46cbb90..887bbf557fb 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -186,7 +186,7 @@ static void explainStep( buffer << step.getName(); const auto & description = step.getStepDescription(); - if (!description.empty()) + if (options.description && !description.empty()) buffer <<" (" << description << ')'; buffer.write('\n'); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index e544ddfc1ee..d37b94fea8b 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -36,6 +36,7 @@ public: struct ExplainOptions { bool header = false; + bool description = true; }; void explain(WriteBuffer & buffer, const ExplainOptions & options); From 4ed499c6f011337cb1f7af7e854bf76697b6d4e4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Jun 2020 17:20:57 +0300 Subject: [PATCH 043/196] Fix explain formatting. --- src/Parsers/ASTExplainQuery.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index e813b211f50..802f9b6c356 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -52,8 +52,14 @@ public: protected: void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override { - settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind, old_syntax) << (settings.hilite ? hilite_none : "") << " "; - ast_settings->formatImpl(settings, state, frame); + settings.ostr << (settings.hilite ? hilite_keyword : "") << toString(kind, old_syntax) << (settings.hilite ? hilite_none : ""); + + if (ast_settings) + { + settings.ostr << ' '; + ast_settings->formatImpl(settings, state, frame); + } + settings.ostr << settings.nl_or_ws; query->formatImpl(settings, state, frame); } From d88e19b69d795d8d0c93d4ea0d4c08f9bc4e060e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 23 Jun 2020 19:06:56 +0300 Subject: [PATCH 044/196] Added actions setting for Explain query. --- src/Interpreters/AggregateDescription.cpp | 80 ++++++++++++++++++++ src/Interpreters/AggregateDescription.h | 2 + src/Interpreters/Aggregator.cpp | 27 +++++++ src/Interpreters/Aggregator.h | 3 + src/Interpreters/InterpreterExplainQuery.cpp | 1 + src/Interpreters/ya.make | 1 + src/Processors/QueryPlan/AggregatingStep.cpp | 5 ++ src/Processors/QueryPlan/AggregatingStep.h | 2 + src/Processors/QueryPlan/IQueryPlanStep.h | 3 + src/Processors/QueryPlan/QueryPlan.cpp | 19 +++++ src/Processors/QueryPlan/QueryPlan.h | 1 + 11 files changed, 144 insertions(+) create mode 100644 src/Interpreters/AggregateDescription.cpp diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp new file mode 100644 index 00000000000..ed737bd4fa4 --- /dev/null +++ b/src/Interpreters/AggregateDescription.cpp @@ -0,0 +1,80 @@ +#include +#include + +namespace DB +{ + +Strings AggregateDescription::explain() const +{ + Strings res; + String arguments_pos_str; + for (auto arg : arguments) + { + if (!arguments_pos_str.empty()) + arguments_pos_str += ", "; + + arguments_pos_str += std::to_string(arg); + } + + if (arguments_pos_str.empty()) + arguments_pos_str = "none"; + + res.emplace_back("argument positions: " + arguments_pos_str); + + String arguments_names_str; + for (const auto & arg : argument_names) + { + if (!arguments_names_str.empty()) + arguments_names_str += ", "; + + arguments_names_str += arg; + } + + if (arguments_names_str.empty()) + arguments_names_str = "none"; + + res.emplace_back("arguments: " + arguments_names_str); + res.emplace_back("column_name: " + column_name); + + auto get_params_string = [](const Array & arr) + { + String params_str; + for (const auto & param : arr) + { + if (!params_str.empty()) + params_str += ", "; + + params_str += applyVisitor(FieldVisitorToString(), param); + } + + return params_str; + }; + + if (function) + { + String types_str; + for (const auto & type : function->getArgumentTypes()) + { + if (!types_str.empty()) + types_str += ", "; + + types_str += type->getName(); + } + + auto params_str = get_params_string(function->getParameters()); + if (!params_str.empty()) + params_str = "(" + params_str + ")"; + + res.emplace_back("function: " + function->getName() + params_str + '(' + types_str + ") -> " + + function->getReturnType()->getName()); + } + else + res.emplace_back("function: nullptr"); + + if (!parameters.empty()) + res.emplace_back("parameters: " + get_params_string(parameters)); + + return res; +} + +} diff --git a/src/Interpreters/AggregateDescription.h b/src/Interpreters/AggregateDescription.h index dece93c9093..a17e307451d 100644 --- a/src/Interpreters/AggregateDescription.h +++ b/src/Interpreters/AggregateDescription.h @@ -15,6 +15,8 @@ struct AggregateDescription ColumnNumbers arguments; Names argument_names; /// used if no `arguments` are specified. String column_name; /// What name to use for a column with aggregate function values + + Strings explain() const; /// Get description for EXPLAIN query. }; using AggregateDescriptions = std::vector; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c8165632896..c30c8faac12 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -150,6 +150,33 @@ Block Aggregator::Params::getHeader( return materializeBlock(res); } +Strings Aggregator::Params::explain() const +{ + Strings res; + const auto & header = src_header ? src_header + : intermediate_header; + String keys_str; + for (auto key : keys) + { + if (keys_str.empty()) + keys_str += ", "; + + if (key >= header.columns()) + keys_str += "unknown position " + std::to_string(key); + else + keys_str += src_header.getByPosition(key).name; + } + + res.emplace_back("keys: " + std::move(keys_str)); + + for (const auto & aggregate : aggregates) + { + auto aggregate_strings = aggregate.explain(); + res.insert(res.end(), aggregate_strings.begin(), aggregate_strings.end()); + } + + return res; +} Aggregator::Aggregator(const Params & params_) : params(params_), diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 0546271873b..2b022f98367 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -923,6 +923,9 @@ public: { return getHeader(src_header, intermediate_header, keys, aggregates, final); } + + /// Returns keys and aggregated for EXPLAIN query + Strings explain() const; }; Aggregator(const Params & params_); diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index bbfa83af2e7..45e1d9f13ff 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -120,6 +120,7 @@ struct ExplainSettings { {"header", query_plan_options.header}, {"description", query_plan_options.description}, + {"actions", query_plan_options.actions} }; bool has(const std::string & name) const diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 394c9c36064..3df3cb7f3dc 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -18,6 +18,7 @@ SRCS( ActionsVisitor.cpp addMissingDefaults.cpp addTypeConversionToAST.cpp + AggregateDescription.cpp Aggregator.cpp AnyInputOptimize.cpp ArithmeticOperationsInAgrFuncOptimize.cpp diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 47cb444654c..757e47e1a6e 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -139,4 +139,9 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) pipeline.enableQuotaForCurrentStreams(); } +Strings AggregatingStep::describeActions() const +{ + return params.explain(); +} + } diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index df75eee1b3f..54ea7c1424f 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -28,6 +28,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: Aggregator::Params params; bool final; diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index 4f9e4d3268c..e64219265c5 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -51,6 +51,9 @@ public: const std::string & getStepDescription() const { return step_description; } void setStepDescription(std::string description) { step_description = std::move(description); } + /// Get detailed description of step actions. This is shown in EXPLAIN query with options `actions = 1`. + virtual Strings describeActions() const { return {}; } + protected: DataStreams input_streams; std::optional output_stream; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 887bbf557fb..65b0fc6b33f 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -216,6 +216,25 @@ static void explainStep( buffer.write('\n'); } + + if (options.actions) + { + auto actions = step.describeActions(); + if (!actions.empty()) + { + buffer << "Actions: "; + bool first = true; + + for (auto & action : actions) + { + if (!first) + buffer << ",\n" << prefix << " "; + + first = false; + buffer << action; + } + } + } } void QueryPlan::explain(WriteBuffer & buffer, const ExplainOptions & options) diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index d37b94fea8b..7a6efc1c207 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -37,6 +37,7 @@ public: { bool header = false; bool description = true; + bool actions = false; }; void explain(WriteBuffer & buffer, const ExplainOptions & options); From ca01094f4b7dbdb18c2bc4b16e595fc946ed13b4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 24 Jun 2020 15:09:01 +0300 Subject: [PATCH 045/196] More actions description. --- src/Core/SortDescription.cpp | 41 ++++++++++++ src/Core/SortDescription.h | 3 + src/Core/ya.make | 1 + src/Interpreters/AggregateDescription.cpp | 62 ++++++++++--------- src/Interpreters/Aggregator.cpp | 24 +++++-- src/Interpreters/InterpreterSelectQuery.cpp | 4 +- src/Processors/QueryPlan/ConvertingStep.cpp | 30 +++++++++ src/Processors/QueryPlan/ConvertingStep.h | 2 + src/Processors/QueryPlan/CreatingSetsStep.cpp | 17 +++++ src/Processors/QueryPlan/CreatingSetsStep.h | 2 + src/Processors/QueryPlan/DistinctStep.cpp | 14 +++++ src/Processors/QueryPlan/DistinctStep.h | 2 + src/Processors/QueryPlan/ExpressionStep.cpp | 20 ++++++ src/Processors/QueryPlan/ExpressionStep.h | 4 ++ src/Processors/QueryPlan/FillingStep.cpp | 5 ++ src/Processors/QueryPlan/FillingStep.h | 2 + src/Processors/QueryPlan/FilterStep.cpp | 12 ++++ src/Processors/QueryPlan/FilterStep.h | 2 + .../QueryPlan/FinishSortingStep.cpp | 13 ++++ src/Processors/QueryPlan/FinishSortingStep.h | 2 + src/Processors/QueryPlan/LimitByStep.cpp | 19 ++++++ src/Processors/QueryPlan/LimitByStep.h | 2 + src/Processors/QueryPlan/LimitStep.cpp | 26 ++++++++ src/Processors/QueryPlan/LimitStep.h | 2 + src/Processors/QueryPlan/MergeSortingStep.cpp | 11 ++++ src/Processors/QueryPlan/MergeSortingStep.h | 2 + .../QueryPlan/MergingAggregatedStep.cpp | 5 ++ .../QueryPlan/MergingAggregatedStep.h | 2 + .../QueryPlan/MergingSortedStep.cpp | 10 +++ src/Processors/QueryPlan/MergingSortedStep.h | 2 + .../{OffsetsStep.cpp => OffsetStep.cpp} | 11 +++- .../QueryPlan/{OffsetsStep.h => OffsetStep.h} | 8 ++- .../QueryPlan/PartialSortingStep.cpp | 10 +++ src/Processors/QueryPlan/PartialSortingStep.h | 2 + src/Processors/QueryPlan/QueryPlan.cpp | 13 +--- src/Processors/QueryPlan/TotalsHavingStep.cpp | 31 ++++++++++ src/Processors/QueryPlan/TotalsHavingStep.h | 2 + .../Transforms/ConvertingTransform.h | 2 + src/Processors/ya.make | 2 +- 39 files changed, 368 insertions(+), 56 deletions(-) create mode 100644 src/Core/SortDescription.cpp rename src/Processors/QueryPlan/{OffsetsStep.cpp => OffsetStep.cpp} (70%) rename src/Processors/QueryPlan/{OffsetsStep.h => OffsetStep.h} (51%) diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp new file mode 100644 index 00000000000..e27c39dbb5f --- /dev/null +++ b/src/Core/SortDescription.cpp @@ -0,0 +1,41 @@ +#include +#include + +namespace DB +{ + +String dumpSortDescription(const SortDescription & description, const Block & header) +{ + String res; + + for (const auto & desc : description) + { + if (!res.empty()) + res += ", "; + + if (!desc.column_name.empty()) + res += desc.column_name; + else + { + if (desc.column_number < header.columns()) + res += header.getByPosition(desc.column_number).name; + else + res += "?"; + + res += " (pos " + std::to_string(desc.column_number) + ")"; + } + + if (desc.direction > 0) + res += " ASC"; + else + res += " DESC"; + + if (desc.with_fill) + res += " WITH FILL"; + } + + return res; +} + +} + diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index 86e4bb573ed..a155032edfe 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -71,4 +71,7 @@ struct SortColumnDescription /// Description of the sorting rule for several columns. using SortDescription = std::vector; +class Block; +String dumpSortDescription(const SortDescription & description, const Block & header); + } diff --git a/src/Core/ya.make b/src/Core/ya.make index 06fed2dc257..14d609dfa96 100644 --- a/src/Core/ya.make +++ b/src/Core/ya.make @@ -20,6 +20,7 @@ SRCS( NamesAndTypes.cpp Settings.cpp SettingsCollection.cpp + SortDescription.cpp ) END() diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp index ed737bd4fa4..53ecd3159e2 100644 --- a/src/Interpreters/AggregateDescription.cpp +++ b/src/Interpreters/AggregateDescription.cpp @@ -7,34 +7,8 @@ namespace DB Strings AggregateDescription::explain() const { Strings res; - String arguments_pos_str; - for (auto arg : arguments) - { - if (!arguments_pos_str.empty()) - arguments_pos_str += ", "; - arguments_pos_str += std::to_string(arg); - } - - if (arguments_pos_str.empty()) - arguments_pos_str = "none"; - - res.emplace_back("argument positions: " + arguments_pos_str); - - String arguments_names_str; - for (const auto & arg : argument_names) - { - if (!arguments_names_str.empty()) - arguments_names_str += ", "; - - arguments_names_str += arg; - } - - if (arguments_names_str.empty()) - arguments_names_str = "none"; - - res.emplace_back("arguments: " + arguments_names_str); - res.emplace_back("column_name: " + column_name); + res.emplace_back(column_name); auto get_params_string = [](const Array & arr) { @@ -65,14 +39,42 @@ Strings AggregateDescription::explain() const if (!params_str.empty()) params_str = "(" + params_str + ")"; - res.emplace_back("function: " + function->getName() + params_str + '(' + types_str + ") -> " + + res.emplace_back(" Function: " + function->getName() + params_str + '(' + types_str + ") -> " + function->getReturnType()->getName()); } else - res.emplace_back("function: nullptr"); + res.emplace_back(" Function: nullptr"); if (!parameters.empty()) - res.emplace_back("parameters: " + get_params_string(parameters)); + res.emplace_back(" Parameters: " + get_params_string(parameters)); + + String arguments_names_str; + for (const auto & arg : argument_names) + { + if (!arguments_names_str.empty()) + arguments_names_str += ", "; + + arguments_names_str += arg; + } + + if (arguments_names_str.empty()) + arguments_names_str = "none"; + + res.emplace_back(" Arguments: " + arguments_names_str); + + String arguments_pos_str; + for (auto arg : arguments) + { + if (!arguments_pos_str.empty()) + arguments_pos_str += ", "; + + arguments_pos_str += std::to_string(arg); + } + + if (arguments_pos_str.empty()) + arguments_pos_str = "none"; + + res.emplace_back(" Argument positions: " + arguments_pos_str); return res; } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index c30c8faac12..8c20698f42a 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -158,21 +158,33 @@ Strings Aggregator::Params::explain() const String keys_str; for (auto key : keys) { - if (keys_str.empty()) + if (!keys_str.empty()) keys_str += ", "; if (key >= header.columns()) keys_str += "unknown position " + std::to_string(key); else - keys_str += src_header.getByPosition(key).name; + keys_str += header.getByPosition(key).name; } - res.emplace_back("keys: " + std::move(keys_str)); + res.emplace_back("Keys: " + std::move(keys_str)); - for (const auto & aggregate : aggregates) + if (!aggregates.empty()) { - auto aggregate_strings = aggregate.explain(); - res.insert(res.end(), aggregate_strings.begin(), aggregate_strings.end()); + bool first = true; + for (const auto & aggregate : aggregates) + { + auto aggregate_strings = aggregate.explain(); + for (const auto & aggregate_str : aggregate_strings) + { + if (first) + res.emplace_back("Aggregates: " + aggregate_str); + else + res.emplace_back(" " + aggregate_str); + + first = false; + } + } } return res; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7fbfdf23959..401e5583859 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -53,7 +53,7 @@ #include #include #include -#include +#include #include #include @@ -1792,7 +1792,7 @@ void InterpreterSelectQuery::executeOffset(QueryPlan & query_plan) UInt64 limit_offset; std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context); - auto offsets_step = std::make_unique(query_plan.getCurrentDataStream(), limit_offset); + auto offsets_step = std::make_unique(query_plan.getCurrentDataStream(), limit_offset); query_plan.addStep(std::move(offsets_step)); } } diff --git a/src/Processors/QueryPlan/ConvertingStep.cpp b/src/Processors/QueryPlan/ConvertingStep.cpp index 99287c50b90..3c5238d4fd5 100644 --- a/src/Processors/QueryPlan/ConvertingStep.cpp +++ b/src/Processors/QueryPlan/ConvertingStep.cpp @@ -46,4 +46,34 @@ void ConvertingStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings ConvertingStep::describeActions() const +{ + const auto & header = input_streams[0].header; + auto conversion = ConvertingTransform(header, result_header, ConvertingTransform::MatchColumnsMode::Name) + .getConversion(); + + Strings res; + + auto get_description = [](const ColumnWithTypeAndName & elem, bool is_const) + { + return elem.name + " " + elem.type->getName() + (is_const ? " Const" : ""); + }; + + for (size_t i = 0; i < conversion.size(); ++i) + { + const auto & from = header.getByPosition(conversion[i]); + const auto & to = result_header.getByPosition(i); + + bool from_const = from.column && isColumnConst(*from.column); + bool to_const = to.column && isColumnConst(*to.column); + + if (from.name == to.name && from.type->equals(*to.type) && from_const == to_const) + res.emplace_back(get_description(from, from_const)); + else + res.emplace_back(get_description(to, to_const) + " <- " + get_description(from, from_const)); + } + + return res; +} + } diff --git a/src/Processors/QueryPlan/ConvertingStep.h b/src/Processors/QueryPlan/ConvertingStep.h index 540deece246..bf0840b1f96 100644 --- a/src/Processors/QueryPlan/ConvertingStep.h +++ b/src/Processors/QueryPlan/ConvertingStep.h @@ -13,6 +13,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: Block result_header; }; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 4480fd53f32..2f47c7abbb3 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -35,4 +35,21 @@ void CreatingSetsStep::transformPipeline(QueryPipeline & pipeline) pipeline.addCreatingSetsTransform(std::move(creating_sets)); } +Strings CreatingSetsStep::describeActions() const +{ + Strings res; + for (const auto & set : subqueries_for_sets) + { + String str; + if (set.second.set) + str += "Set: "; + else if (set.second.join) + str += "Join: "; + + str += set.first; + } + + return res; +} + } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index d3c4db30502..7f05de4b889 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -19,6 +19,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: SubqueriesForSets subqueries_for_sets; SizeLimits network_transfer_limits; diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 0e1cab637fa..1a558669289 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -65,4 +65,18 @@ void DistinctStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings DistinctStep::describeActions() const +{ + String res; + for (const auto & column : columns) + { + if (!res.empty()) + res += ", "; + + res += column; + } + + return {"Columns: " + res}; +} + } diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index 5ec6e683bba..8af909c60da 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -19,6 +19,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: SizeLimits set_size_limits; UInt64 limit_hint; diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index 75c07554318..5d765c6fd3d 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -59,6 +59,21 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline) }); } +static Strings getActionsDescription(const ExpressionActionsPtr & expression) +{ + Strings res; + for (const auto & action : expression->getActions()) + res.emplace_back((res.empty() ? "Actions: " + : " ") + action.toString()); + + return res; +} + +Strings ExpressionStep::describeActions() const +{ + return getActionsDescription(expression); +} + InflatingExpressionStep::InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_, bool default_totals_) : ITransformingStep( input_stream_, @@ -88,4 +103,9 @@ void InflatingExpressionStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings InflatingExpressionStep::describeActions() const +{ + return getActionsDescription(expression); +} + } diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index 4f268944c95..c6685e5e863 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -15,6 +15,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: ExpressionActionsPtr expression; bool default_totals; /// See ExpressionTransform @@ -29,6 +31,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: ExpressionActionsPtr expression; bool default_totals; /// See ExpressionTransform diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index 80dba794943..b4a2a6d2963 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -27,4 +27,9 @@ void FillingStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings FillingStep::describeActions() const +{ + return {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)}; +} + } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index e7ec7ab17d7..80f7e00dcf7 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -14,6 +14,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: SortDescription sort_description; }; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 38ab4471e53..7bef168cd90 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -55,4 +55,16 @@ void FilterStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings FilterStep::describeActions() const +{ + Strings res; + res.emplace_back("Filter column: " + filter_column_name); + + for (const auto & action : expression->getActions()) + res.emplace_back((res.size() == 1 ? "Actions: " + : " ") + action.toString()); + + return res; +} + } diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index faadd41a58d..3f2f8b431f9 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -19,6 +19,8 @@ public: String getName() const override { return "Filter"; } void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: ExpressionActionsPtr expression; String filter_column_name; diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index 4b0e6d6a66d..bb0b0816374 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -69,4 +69,17 @@ void FinishSortingStep::transformPipeline(QueryPipeline & pipeline) } } +Strings FinishSortingStep::describeActions() const +{ + Strings res = { + "Prefix sort description: " + dumpSortDescription(prefix_description, input_streams.front().header), + "Result sort description: " + dumpSortDescription(result_description, input_streams.front().header) + }; + + if (limit) + res.emplace_back("Limit " + std::to_string(limit)); + + return res; +} + } diff --git a/src/Processors/QueryPlan/FinishSortingStep.h b/src/Processors/QueryPlan/FinishSortingStep.h index 43bdf261e97..cc7dfe1388b 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.h +++ b/src/Processors/QueryPlan/FinishSortingStep.h @@ -19,6 +19,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: SortDescription prefix_description; SortDescription result_description; diff --git a/src/Processors/QueryPlan/LimitByStep.cpp b/src/Processors/QueryPlan/LimitByStep.cpp index e18df84258e..05d346f9ae6 100644 --- a/src/Processors/QueryPlan/LimitByStep.cpp +++ b/src/Processors/QueryPlan/LimitByStep.cpp @@ -37,4 +37,23 @@ void LimitByStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings LimitByStep::describeActions() const +{ + Strings res; + String columns_str; + for (const auto & column : columns) + { + if (!columns_str.empty()) + columns_str += ", "; + + columns_str += column; + } + + return { + "Columns: " + columns_str, + "Length " + std::to_string(group_length), + "Offset " + std::to_string(group_offset), + }; +} + } diff --git a/src/Processors/QueryPlan/LimitByStep.h b/src/Processors/QueryPlan/LimitByStep.h index 744918cb836..b12c85b9179 100644 --- a/src/Processors/QueryPlan/LimitByStep.h +++ b/src/Processors/QueryPlan/LimitByStep.h @@ -15,6 +15,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: size_t group_length; size_t group_offset; diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index 4b7928bfc5a..f6c2c2dee44 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -35,4 +35,30 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline) pipeline.addPipe({std::move(transform)}); } +Strings LimitStep::describeActions() const +{ + Strings res; + res.emplace_back("Limit " + std::to_string(limit)); + res.emplace_back("Offset " + std::to_string(offset)); + + if (with_ties || always_read_till_end) + { + String str; + if (with_ties) + str += "WITH TIES"; + + if (always_read_till_end) + { + if (!str.empty()) + str += ", "; + + str += "Reads all data"; + } + + res.emplace_back(str); + } + + return res; +} + } diff --git a/src/Processors/QueryPlan/LimitStep.h b/src/Processors/QueryPlan/LimitStep.h index 4a12e8f6705..2bb0f537817 100644 --- a/src/Processors/QueryPlan/LimitStep.h +++ b/src/Processors/QueryPlan/LimitStep.h @@ -20,6 +20,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: size_t limit; size_t offset; diff --git a/src/Processors/QueryPlan/MergeSortingStep.cpp b/src/Processors/QueryPlan/MergeSortingStep.cpp index 4c8f265c2ca..d61a6e721ea 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.cpp +++ b/src/Processors/QueryPlan/MergeSortingStep.cpp @@ -48,4 +48,15 @@ void MergeSortingStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings MergeSortingStep::describeActions() const +{ + Strings res = {"Sort description: " + dumpSortDescription(description, input_streams.front().header)}; + + if (limit) + res.emplace_back("Limit " + std::to_string(limit)); + + return res; +} + + } diff --git a/src/Processors/QueryPlan/MergeSortingStep.h b/src/Processors/QueryPlan/MergeSortingStep.h index 3d12bda3139..49645180cf0 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.h +++ b/src/Processors/QueryPlan/MergeSortingStep.h @@ -24,6 +24,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: SortDescription description; size_t max_merged_block_size; diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index 459a0b90040..de085c44b8f 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -63,4 +63,9 @@ void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline) pipeline.enableQuotaForCurrentStreams(); } +Strings MergingAggregatedStep::describeActions() const +{ + return params->params.explain(); +} + } diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 51a907285df..8b22495b5ac 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -22,6 +22,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: AggregatingTransformParamsPtr params; bool memory_efficient_aggregation; diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp index 9c4bf874510..a362a2b0c25 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ b/src/Processors/QueryPlan/MergingSortedStep.cpp @@ -46,4 +46,14 @@ void MergingSortedStep::transformPipeline(QueryPipeline & pipeline) } } +Strings MergingSortedStep::describeActions() const +{ + Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)}; + + if (limit) + res.emplace_back("Limit " + std::to_string(limit)); + + return res; +} + } diff --git a/src/Processors/QueryPlan/MergingSortedStep.h b/src/Processors/QueryPlan/MergingSortedStep.h index 920073da8cb..40803cef0cc 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.h +++ b/src/Processors/QueryPlan/MergingSortedStep.h @@ -20,6 +20,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: SortDescription sort_description; size_t max_block_size; diff --git a/src/Processors/QueryPlan/OffsetsStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp similarity index 70% rename from src/Processors/QueryPlan/OffsetsStep.cpp rename to src/Processors/QueryPlan/OffsetStep.cpp index e09a169f4bd..27da67c2be2 100644 --- a/src/Processors/QueryPlan/OffsetsStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -13,13 +13,13 @@ static ITransformingStep::DataStreamTraits getTraits() }; } -OffsetsStep::OffsetsStep(const DataStream & input_stream_, size_t offset_) +OffsetStep::OffsetStep(const DataStream & input_stream_, size_t offset_) : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , offset(offset_) { } -void OffsetsStep::transformPipeline(QueryPipeline & pipeline) +void OffsetStep::transformPipeline(QueryPipeline & pipeline) { pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr { @@ -30,4 +30,9 @@ void OffsetsStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings OffsetStep::describeActions() const +{ + return {"Offset " + std::to_string(offset)}; +} + } diff --git a/src/Processors/QueryPlan/OffsetsStep.h b/src/Processors/QueryPlan/OffsetStep.h similarity index 51% rename from src/Processors/QueryPlan/OffsetsStep.h rename to src/Processors/QueryPlan/OffsetStep.h index 83f0c43dd7d..167b65f6fd8 100644 --- a/src/Processors/QueryPlan/OffsetsStep.h +++ b/src/Processors/QueryPlan/OffsetStep.h @@ -5,15 +5,17 @@ namespace DB { -class OffsetsStep : public ITransformingStep +class OffsetStep : public ITransformingStep { public: - OffsetsStep(const DataStream & input_stream_, size_t offset_); + OffsetStep(const DataStream & input_stream_, size_t offset_); - String getName() const override { return "Offsets"; } + String getName() const override { return "Offset"; } void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: size_t offset; }; diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp index c8be58eb324..767eebf6f01 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ b/src/Processors/QueryPlan/PartialSortingStep.cpp @@ -50,4 +50,14 @@ void PartialSortingStep::transformPipeline(QueryPipeline & pipeline) }); } +Strings PartialSortingStep::describeActions() const +{ + Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)}; + + if (limit) + res.emplace_back("Limit " + std::to_string(limit)); + + return res; +} + } diff --git a/src/Processors/QueryPlan/PartialSortingStep.h b/src/Processors/QueryPlan/PartialSortingStep.h index c4967e8ec30..fd2526c23aa 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.h +++ b/src/Processors/QueryPlan/PartialSortingStep.h @@ -19,6 +19,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: SortDescription sort_description; UInt64 limit; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 65b0fc6b33f..035216abb27 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -207,7 +207,7 @@ static void explainStep( for (const auto & elem : step.getOutputStream().header) { if (!first) - buffer << ",\n" << prefix << " "; + buffer << "\n" << prefix << " "; first = false; elem.dumpStructure(buffer, true); @@ -222,17 +222,8 @@ static void explainStep( auto actions = step.describeActions(); if (!actions.empty()) { - buffer << "Actions: "; - bool first = true; - for (auto & action : actions) - { - if (!first) - buffer << ",\n" << prefix << " "; - - first = false; - buffer << action; - } + buffer << prefix << action << '\n'; } } } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index ec3788bc7d3..70b35ee7f94 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -44,4 +45,34 @@ void TotalsHavingStep::transformPipeline(QueryPipeline & pipeline) pipeline.addTotalsHavingTransform(std::move(totals_having)); } +static String totalsModeToString(TotalsMode totals_mode, double auto_include_threshold) +{ + switch (totals_mode) + { + case TotalsMode::BEFORE_HAVING: + return "before_having"; + case TotalsMode::AFTER_HAVING_INCLUSIVE: + return "after_having_inclusive"; + case TotalsMode::AFTER_HAVING_EXCLUSIVE: + return "after_having_exclusive"; + case TotalsMode::AFTER_HAVING_AUTO: + return "after_having_auto threshold " + std::to_string(auto_include_threshold); + } + + __builtin_unreachable(); +} + +Strings TotalsHavingStep::describeActions() const +{ + Strings res; + res.emplace_back("Filter column: " + filter_column_name); + res.emplace_back("Mode: " + totalsModeToString(totals_mode, auto_include_threshold)); + + for (const auto & action : expression->getActions()) + res.emplace_back((res.size() == 2 ? "Actions: " + : " ") + action.toString()); + + return res; +} + } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 52cc936f622..e2dd2f4dd55 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -25,6 +25,8 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + Strings describeActions() const override; + private: bool overflow_row; ExpressionActionsPtr expression; diff --git a/src/Processors/Transforms/ConvertingTransform.h b/src/Processors/Transforms/ConvertingTransform.h index 45a6688c07a..b4b42dcb6ea 100644 --- a/src/Processors/Transforms/ConvertingTransform.h +++ b/src/Processors/Transforms/ConvertingTransform.h @@ -35,6 +35,8 @@ public: String getName() const override { return "Converting"; } + const ColumnNumbers & getConversion() const { return conversion; } + protected: void transform(Chunk & chunk) override; diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 4e6ec2372da..550aa9ad7da 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -157,7 +157,7 @@ SRCS( QueryPlan/MergeSortingStep.cpp QueryPlan/MergingAggregatedStep.cpp QueryPlan/MergingSortedStep.cpp - QueryPlan/OffsetsStep.cpp + QueryPlan/OffsetStep.cpp QueryPlan/PartialSortingStep.cpp QueryPlan/UnionStep.cpp QueryPlan/ReadFromPreparedSource.cpp From 5a678d74c264f8281f6d3324aa1861531c28c186 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Jun 2020 12:39:17 +0300 Subject: [PATCH 046/196] Add explain pipeline. --- src/Interpreters/InterpreterExplainQuery.cpp | 15 ++- src/Parsers/ASTExplainQuery.h | 2 + src/Parsers/ParserExplainQuery.cpp | 3 + src/Processors/IProcessor.h | 15 +++ src/Processors/QueryPipeline.cpp | 113 +++++++++++++----- src/Processors/QueryPipeline.h | 45 ++++++- .../QueryPlan/AddingDelayedStreamStep.cpp | 1 + src/Processors/QueryPlan/AggregatingStep.cpp | 28 ++++- src/Processors/QueryPlan/AggregatingStep.h | 8 ++ src/Processors/QueryPlan/IQueryPlanStep.cpp | 38 ++++++ src/Processors/QueryPlan/IQueryPlanStep.h | 17 +++ src/Processors/QueryPlan/ISourceStep.cpp | 7 ++ src/Processors/QueryPlan/ISourceStep.h | 6 + .../QueryPlan/ITransformingStep.cpp | 18 ++- src/Processors/QueryPlan/ITransformingStep.h | 9 +- src/Processors/QueryPlan/QueryPlan.cpp | 49 +++++++- src/Processors/QueryPlan/QueryPlan.h | 1 + .../QueryPlan/ReadFromStorageStep.cpp | 8 ++ .../QueryPlan/ReadFromStorageStep.h | 3 + src/Processors/QueryPlan/UnionStep.cpp | 9 ++ src/Processors/QueryPlan/UnionStep.h | 3 + 21 files changed, 359 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 45e1d9f13ff..26784c1f623 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include @@ -217,6 +216,20 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() WriteBufferFromOStream buffer(ss); plan.explain(buffer, settings.query_plan_options); } + else if (ast.getKind() == ASTExplainQuery::QueryPipeline) + { + if (!dynamic_cast(ast.getExplainedQuery().get())) + throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY); + + QueryPlan plan; + + InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions()); + interpreter.buildQueryPlan(plan); + plan.buildQueryPipeline(); + + WriteBufferFromOStream buffer(ss); + plan.explainPipeline(buffer); + } fillColumn(*res_columns[0], ss.str()); diff --git a/src/Parsers/ASTExplainQuery.h b/src/Parsers/ASTExplainQuery.h index 802f9b6c356..0c376e270d4 100644 --- a/src/Parsers/ASTExplainQuery.h +++ b/src/Parsers/ASTExplainQuery.h @@ -16,6 +16,7 @@ public: ParsedAST, /// 'EXPLAIN AST SELECT ...' AnalyzedSyntax, /// 'EXPLAIN SYNTAX SELECT ...' QueryPlan, /// 'EXPLAIN SELECT ...' + QueryPipeline, /// 'EXPLAIN PIPELINE ...' }; ASTExplainQuery(ExplainKind kind_, bool old_syntax_) @@ -78,6 +79,7 @@ private: case ParsedAST: return old_syntax ? "AST" : "EXPLAIN AST"; case AnalyzedSyntax: return old_syntax ? "ANALYZE" : "EXPLAIN SYNTAX"; case QueryPlan: return "EXPLAIN"; + case QueryPipeline: return "EXPLAIN PIPELINE"; } __builtin_unreachable(); diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index 892cc2116dc..dae9f63e911 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -16,6 +16,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_analyze("ANALYZE"); ParserKeyword s_explain("EXPLAIN"); ParserKeyword s_syntax("SYNTAX"); + ParserKeyword s_pipeline("PIPELINE"); if (enable_debug_queries && s_ast.ignore(pos, expected)) { @@ -35,6 +36,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected kind = ASTExplainQuery::ExplainKind::ParsedAST; else if (s_syntax.ignore(pos, expected)) kind = ASTExplainQuery::ExplainKind::AnalyzedSyntax; + else if (s_pipeline.ignore(pos, expected)) + kind = ASTExplainQuery::ExplainKind::QueryPipeline; } else return false; diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index a9bd73d8026..7c9f94916bf 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -15,6 +15,8 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +class IQueryPlanStep; + class IProcessor; using ProcessorPtr = std::shared_ptr; using Processors = std::vector; @@ -288,6 +290,16 @@ public: void enableQuota() { has_quota = true; } bool hasQuota() const { return has_quota; } + /// Step of QueryPlan from which processor was created. + void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0) + { + query_plan_step = step; + query_plan_step_group = group; + } + + IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; } + size_t getQueryPlanStepGroup() const { return query_plan_step_group; } + protected: virtual void onCancel() {} @@ -299,6 +311,9 @@ private: size_t stream_number = NO_STREAM; bool has_quota = false; + + IQueryPlanStep * query_plan_step; + size_t query_plan_step_group; }; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 7ad7bddb104..8927c75329a 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -69,7 +69,8 @@ void QueryPipeline::init(Pipe pipe) init(std::move(pipes)); } -static OutputPort * uniteExtremes(const std::vector & ports, const Block & header, Processors & processors) +static OutputPort * uniteExtremes(const std::vector & ports, const Block & header, + QueryPipeline::ProcessorsContainer & processors) { /// Here we calculate extremes for extremes in case we unite several pipelines. /// Example: select number from numbers(2) union all select number from numbers(3) @@ -90,14 +91,15 @@ static OutputPort * uniteExtremes(const std::vector & ports, const connect(resize->getOutputs().front(), extremes->getInputPort()); connect(extremes->getOutputPort(), sink->getPort()); - processors.emplace_back(std::move(resize)); - processors.emplace_back(std::move(extremes)); - processors.emplace_back(std::move(sink)); + processors.emplace(std::move(resize)); + processors.emplace(std::move(extremes)); + processors.emplace(std::move(sink)); return extremes_port; } -static OutputPort * uniteTotals(const std::vector & ports, const Block & header, Processors & processors) +static OutputPort * uniteTotals(const std::vector & ports, const Block & header, + QueryPipeline::ProcessorsContainer & processors) { /// Calculate totals fro several streams. /// Take totals from first sources which has any, skip others. @@ -115,8 +117,8 @@ static OutputPort * uniteTotals(const std::vector & ports, const B connect(concat->getOutputs().front(), limit->getInputPort()); - processors.emplace_back(std::move(concat)); - processors.emplace_back(std::move(limit)); + processors.emplace(std::move(concat)); + processors.emplace(std::move(limit)); return totals_port; } @@ -167,8 +169,7 @@ void QueryPipeline::init(Pipes pipes) } streams.addStream(&pipe.getPort(), pipe.maxParallelStreams()); - auto cur_processors = std::move(pipe).detachProcessors(); - processors.insert(processors.end(), cur_processors.begin(), cur_processors.end()); + processors.emplace(std::move(pipe).detachProcessors()); } if (!totals.empty()) @@ -242,7 +243,7 @@ void QueryPipeline::addSimpleTransformImpl(const TProcessorGetter & getter) { connect(*stream, transform->getInputs().front()); stream = &transform->getOutputs().front(); - processors.emplace_back(std::move(transform)); + processors.emplace(std::move(transform)); } }; @@ -293,7 +294,7 @@ void QueryPipeline::setSinks(const ProcessorGetterWithStreamKind & getter) transform = std::make_shared(stream->getHeader()); connect(*stream, transform->getInputs().front()); - processors.emplace_back(std::move(transform)); + processors.emplace(std::move(transform)); }; for (auto & stream : streams) @@ -339,7 +340,7 @@ void QueryPipeline::addPipe(Processors pipe) header = output.getHeader(); } - processors.insert(processors.end(), pipe.begin(), pipe.end()); + processors.emplace(pipe); current_header = std::move(header); } @@ -352,7 +353,7 @@ void QueryPipeline::addDelayedStream(ProcessorPtr source) IProcessor::PortNumbers delayed_streams = { streams.size() }; streams.addStream(&source->getOutputs().front(), 0); - processors.emplace_back(std::move(source)); + processors.emplace(std::move(source)); auto processor = std::make_shared(current_header, streams.size(), delayed_streams); addPipe({ std::move(processor) }); @@ -383,7 +384,7 @@ void QueryPipeline::resize(size_t num_streams, bool force, bool strict) for (auto & output : resize->getOutputs()) streams.addStream(&output, 0); - processors.emplace_back(std::move(resize)); + processors.emplace(std::move(resize)); } void QueryPipeline::enableQuotaForCurrentStreams() @@ -412,7 +413,7 @@ void QueryPipeline::addTotalsHavingTransform(ProcessorPtr transform) streams.assign({ &outputs.front() }); totals_having_port = &outputs.back(); current_header = outputs.front().getHeader(); - processors.emplace_back(std::move(transform)); + processors.emplace(std::move(transform)); } void QueryPipeline::addDefaultTotals() @@ -434,7 +435,7 @@ void QueryPipeline::addDefaultTotals() auto source = std::make_shared(current_header, Chunk(std::move(columns), 1)); totals_having_port = &source->getPort(); - processors.emplace_back(source); + processors.emplace(std::move(source)); } void QueryPipeline::addTotals(ProcessorPtr source) @@ -448,7 +449,7 @@ void QueryPipeline::addTotals(ProcessorPtr source) assertBlocksHaveEqualStructure(current_header, source->getOutputs().front().getHeader(), "QueryPipeline"); totals_having_port = &source->getOutputs().front(); - processors.emplace_back(std::move(source)); + processors.emplace(std::move(source)); } void QueryPipeline::dropTotalsAndExtremes() @@ -457,7 +458,7 @@ void QueryPipeline::dropTotalsAndExtremes() { auto null_sink = std::make_shared(port->getHeader()); connect(*port, null_sink->getPort()); - processors.emplace_back(std::move(null_sink)); + processors.emplace(std::move(null_sink)); port = nullptr; }; @@ -486,7 +487,7 @@ void QueryPipeline::addExtremesTransform() stream = &transform->getOutputPort(); extremes.push_back(&transform->getExtremesPort()); - processors.emplace_back(std::move(transform)); + processors.emplace(std::move(transform)); } if (extremes.size() == 1) @@ -510,8 +511,8 @@ void QueryPipeline::addCreatingSetsTransform(ProcessorPtr transform) connect(*streams.back(), concat->getInputs().back()); streams.assign({ &concat->getOutputs().front() }); - processors.emplace_back(std::move(transform)); - processors.emplace_back(std::move(concat)); + processors.emplace(std::move(transform)); + processors.emplace(std::move(concat)); } void QueryPipeline::setOutputFormat(ProcessorPtr output) @@ -538,17 +539,17 @@ void QueryPipeline::setOutputFormat(ProcessorPtr output) { auto null_source = std::make_shared(totals.getHeader()); totals_having_port = &null_source->getPort(); - processors.emplace_back(std::move(null_source)); + processors.emplace(std::move(null_source)); } if (!extremes_port) { auto null_source = std::make_shared(extremes.getHeader()); extremes_port = &null_source->getPort(); - processors.emplace_back(std::move(null_source)); + processors.emplace(std::move(null_source)); } - processors.emplace_back(std::move(output)); + processors.emplace(std::move(output)); connect(*streams.front(), main); connect(*totals_having_port, totals); @@ -587,6 +588,7 @@ void QueryPipeline::unitePipelines( { auto & pipeline = *pipeline_ptr; pipeline.checkInitialized(); + pipeline.processors.setCollectedProcessors(processors.getCollectedProcessors()); if (!pipeline.isCompleted()) { @@ -604,7 +606,7 @@ void QueryPipeline::unitePipelines( connect(*pipeline.extremes_port, converting->getInputPort()); extremes.push_back(&converting->getOutputPort()); - processors.push_back(std::move(converting)); + processors.emplace(std::move(converting)); } /// Take totals only from first port. @@ -615,10 +617,13 @@ void QueryPipeline::unitePipelines( connect(*pipeline.totals_having_port, converting->getInputPort()); totals.push_back(&converting->getOutputPort()); - processors.push_back(std::move(converting)); + processors.emplace(std::move(converting)); } - processors.insert(processors.end(), pipeline.processors.begin(), pipeline.processors.end()); + auto * collector = processors.setCollectedProcessors(nullptr); + processors.emplace(pipeline.processors.detach()); + processors.setCollectedProcessors(collector); + streams.addStreams(pipeline.streams); table_locks.insert(table_locks.end(), std::make_move_iterator(pipeline.table_locks.begin()), std::make_move_iterator(pipeline.table_locks.end())); @@ -649,7 +654,7 @@ void QueryPipeline::unitePipelines( void QueryPipeline::setProgressCallback(const ProgressCallback & callback) { - for (auto & processor : processors) + for (auto & processor : processors.get()) { if (auto * source = dynamic_cast(processor.get())) source->setProgressCallback(callback); @@ -663,7 +668,7 @@ void QueryPipeline::setProcessListElement(QueryStatus * elem) { process_list_element = elem; - for (auto & processor : processors) + for (auto & processor : processors.get()) { if (auto * source = dynamic_cast(processor.get())) source->setProcessListElement(elem); @@ -775,7 +780,7 @@ Pipe QueryPipeline::getPipe() && Pipes QueryPipeline::getPipes() && { - Pipe pipe(std::move(processors), streams.at(0), totals_having_port, extremes_port); + Pipe pipe(processors.detach(), streams.at(0), totals_having_port, extremes_port); pipe.max_parallel_streams = streams.maxParallelStreams(); for (auto & lock : table_locks) @@ -807,7 +812,7 @@ PipelineExecutorPtr QueryPipeline::execute() if (!isCompleted()) throw Exception("Cannot execute pipeline because it is not completed.", ErrorCodes::LOGICAL_ERROR); - return std::make_shared(processors, process_list_element); + return std::make_shared(processors.get(), process_list_element); } QueryPipeline & QueryPipeline::operator= (QueryPipeline && rhs) @@ -837,4 +842,50 @@ QueryPipeline & QueryPipeline::operator= (QueryPipeline && rhs) return *this; } +void QueryPipeline::ProcessorsContainer::emplace(ProcessorPtr processor) +{ + if (collected_processors) + collected_processors->emplace_back(processor); + + processors.emplace_back(std::move(processor)); +} + +void QueryPipeline::ProcessorsContainer::emplace(Processors processors_) +{ + for (auto & processor : processors_) + emplace(std::move(processor)); +} + +Processors * QueryPipeline::ProcessorsContainer::setCollectedProcessors(Processors * collected_processors_) +{ + if (collected_processors && collected_processors_) + throw Exception("Cannot set collected processors to QueryPipeline because " + "another one object was already created for current pipeline." , ErrorCodes::LOGICAL_ERROR); + + std::swap(collected_processors, collected_processors_); + return collected_processors_; +} + +QueryPipelineProcessorsCollector::QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_) + : pipeline(pipeline_), step(step_) +{ + pipeline.processors.setCollectedProcessors(&processors); +} + +QueryPipelineProcessorsCollector::~QueryPipelineProcessorsCollector() +{ + pipeline.processors.setCollectedProcessors(nullptr); +} + +Processors QueryPipelineProcessorsCollector::detachProcessors(size_t group) +{ + Processors res; + res.swap(processors); + + for (auto & processor : processors) + processor->setQueryPlanStep(step, group); + + return res; +} + } diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 6d9409ffc47..19979721e0e 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -19,6 +19,8 @@ class Context; class IOutputFormat; +class QueryPipelineProcessorsCollector; + class QueryPipeline { private: @@ -70,6 +72,26 @@ private: }; public: + + class ProcessorsContainer + { + public: + bool empty() const { return processors.empty(); } + void emplace(ProcessorPtr processor); + void emplace(Processors processors_); + Processors * getCollectedProcessors() const; + Processors * setCollectedProcessors(Processors * collected_processors); + Processors & get() { return processors; } + Processors detach() { return std::move(processors); } + private: + /// All added processors. + Processors processors; + + /// If is set, all newly created processors will be added to this too. + /// It is needed for debug. See QueryPipelineProcessorsCollector below. + Processors * collected_processors = nullptr; + }; + QueryPipeline() = default; QueryPipeline(QueryPipeline &&) = default; ~QueryPipeline() = default; @@ -135,6 +157,8 @@ public: void enableQuotaForCurrentStreams(); + /// Unite several pipelines together. Result pipeline would have common_header structure. + /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. void unitePipelines(std::vector> pipelines, const Block & common_header); PipelineExecutorPtr execute(); @@ -185,8 +209,7 @@ private: /// Common header for each stream. Block current_header; - /// All added processors. - Processors processors; + ProcessorsContainer processors; /// Port for each independent "stream". Streams streams; @@ -214,6 +237,24 @@ private: void addSimpleTransformImpl(const TProcessorGetter & getter); void initRowsBeforeLimit(); + + friend class QueryPipelineProcessorsCollector; +}; + +/// This is a small class which collects newly added processors to QueryPipeline. +/// Pipeline must live longer that this class. +class QueryPipelineProcessorsCollector +{ +public: + explicit QueryPipelineProcessorsCollector(QueryPipeline & pipeline_, IQueryPlanStep * step_ = nullptr); + ~QueryPipelineProcessorsCollector(); + + Processors detachProcessors(size_t group = 0); + +private: + QueryPipeline & pipeline; + IQueryPlanStep * step; + Processors processors; }; } diff --git a/src/Processors/QueryPlan/AddingDelayedStreamStep.cpp b/src/Processors/QueryPlan/AddingDelayedStreamStep.cpp index 522094cb790..7f872fbd72c 100644 --- a/src/Processors/QueryPlan/AddingDelayedStreamStep.cpp +++ b/src/Processors/QueryPlan/AddingDelayedStreamStep.cpp @@ -22,6 +22,7 @@ AddingDelayedStreamStep::AddingDelayedStreamStep( void AddingDelayedStreamStep::transformPipeline(QueryPipeline & pipeline) { + source->setQueryPlanStep(this); pipeline.addDelayedStream(source); } diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 757e47e1a6e..21d9b9d7825 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -25,7 +25,7 @@ AggregatingStep::AggregatingStep( bool storage_has_evenly_distributed_read_, InputOrderInfoPtr group_by_info_, SortDescription group_by_sort_description_) - : ITransformingStep(input_stream_, params_.getHeader(final_), getTraits()) + : ITransformingStep(input_stream_, params_.getHeader(final_), getTraits(), false) , params(std::move(params_)) , final(std::move(final_)) , max_block_size(max_block_size_) @@ -39,6 +39,8 @@ AggregatingStep::AggregatingStep( void AggregatingStep::transformPipeline(QueryPipeline & pipeline) { + QueryPipelineProcessorsCollector collector(pipeline, this); + /// Forget about current totals and extremes. They will be calculated again after aggregation if needed. pipeline.dropTotalsAndExtremes(); @@ -74,6 +76,8 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) return std::make_shared(header, transform_params, group_by_sort_description, max_block_size, many_data, counter++); }); + aggregating_in_order = collector.detachProcessors(0); + for (auto & column_description : group_by_sort_description) { if (!column_description.column_name.empty()) @@ -90,6 +94,7 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) max_block_size); pipeline.addPipe({ std::move(transform) }); + aggregating_sorted = collector.detachProcessors(1); } else { @@ -97,6 +102,8 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) { return std::make_shared(header, transform_params, group_by_sort_description, max_block_size); }); + + aggregating_in_order = collector.detachProcessors(0); } pipeline.addSimpleTransform([&](const Block & header) @@ -104,6 +111,8 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) return std::make_shared(header, transform_params); }); + finalizing = collector.detachProcessors(2); + pipeline.enableQuotaForCurrentStreams(); return; } @@ -125,6 +134,8 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) }); pipeline.resize(1); + + aggregating = collector.detachProcessors(0); } else { @@ -134,6 +145,8 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) { return std::make_shared(header, transform_params); }); + + aggregating = collector.detachProcessors(0); } pipeline.enableQuotaForCurrentStreams(); @@ -144,4 +157,17 @@ Strings AggregatingStep::describeActions() const return params.explain(); } +void AggregatingStep::describePipeline(FormatSettings & settings) const +{ + if (!aggregating.empty()) + IQueryPlanStep::describePipeline(aggregating, settings); + else + { + /// Processors are printed in reverse order. + IQueryPlanStep::describePipeline(finalizing, settings); + IQueryPlanStep::describePipeline(aggregating_sorted, settings); + IQueryPlanStep::describePipeline(aggregating_in_order, settings); + } +} + } diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 54ea7c1424f..933fad5e1bf 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -29,6 +29,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; Strings describeActions() const override; + void describePipeline(FormatSettings & settings) const override; private: Aggregator::Params params; @@ -41,6 +42,13 @@ private: InputOrderInfoPtr group_by_info; SortDescription group_by_sort_description; + + Processors aggregating_in_order; + Processors aggregating_sorted; + Processors finalizing; + + Processors aggregating; + }; } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.cpp b/src/Processors/QueryPlan/IQueryPlanStep.cpp index f25d17188ea..2a50f571483 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.cpp +++ b/src/Processors/QueryPlan/IQueryPlanStep.cpp @@ -1,4 +1,6 @@ #include +#include +#include namespace DB { @@ -16,4 +18,40 @@ const DataStream & IQueryPlanStep::getOutputStream() const return *output_stream; } +static void doDescribeProcessor(const IProcessor & processor, size_t count, IQueryPlanStep::FormatSettings & settings) +{ + settings.out << String(settings.offset, settings.ident_char) << processor.getName(); + if (count > 1) + settings.out << " x " << std::to_string(count); + + size_t num_inputs = processor.getInputs().size(); + size_t num_outputs = processor.getOutputs().size(); + if (num_inputs != 1 || num_outputs != 1) + settings.out << " " << std::to_string(num_inputs) << " -> " << std::to_string(num_outputs); + + settings.out << '\n'; + settings.offset += settings.ident; +} + +void IQueryPlanStep::describePipeline(const Processors & processors, FormatSettings & settings) +{ + const IProcessor * prev = nullptr; + size_t count = 0; + + for (auto it = processors.rbegin(); it != processors.rend(); ++it) + { + if (prev && prev->getName() != (*it)->getName()) + { + doDescribeProcessor(*prev, count, settings); + count = 0; + } + + ++count; + prev = it->get(); + } + + if (prev) + doDescribeProcessor(*prev, count, settings); +} + } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index e64219265c5..27e81554590 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -8,6 +8,10 @@ class QueryPipeline; using QueryPipelinePtr = std::unique_ptr; using QueryPipelines = std::vector; +class IProcessor; +using ProcessorPtr = std::shared_ptr; +using Processors = std::vector; + /// Description of data stream. class DataStream { @@ -51,15 +55,28 @@ public: const std::string & getStepDescription() const { return step_description; } void setStepDescription(std::string description) { step_description = std::move(description); } + struct FormatSettings + { + WriteBuffer & out; + size_t offset = 0; + const size_t ident = 2; + const char ident_char = ' '; + }; + /// Get detailed description of step actions. This is shown in EXPLAIN query with options `actions = 1`. virtual Strings describeActions() const { return {}; } + /// Get description of processors added in current step. Should be called after updatePipeline(). + virtual void describePipeline(FormatSettings & /*settings*/) const {} + protected: DataStreams input_streams; std::optional output_stream; /// Text description about what current step does. std::string step_description; + + static void describePipeline(const Processors & processors, FormatSettings & settings); }; using QueryPlanStepPtr = std::unique_ptr; diff --git a/src/Processors/QueryPlan/ISourceStep.cpp b/src/Processors/QueryPlan/ISourceStep.cpp index 9909a777267..cf68104f18c 100644 --- a/src/Processors/QueryPlan/ISourceStep.cpp +++ b/src/Processors/QueryPlan/ISourceStep.cpp @@ -12,8 +12,15 @@ ISourceStep::ISourceStep(DataStream output_stream_) QueryPipelinePtr ISourceStep::updatePipeline(QueryPipelines) { auto pipeline = std::make_unique(); + QueryPipelineProcessorsCollector collector(*pipeline, this); initializePipeline(*pipeline); + processors = collector.detachProcessors(); return pipeline; } +void ISourceStep::describePipeline(FormatSettings & settings) const +{ + IQueryPlanStep::describePipeline(processors, settings); +} + } diff --git a/src/Processors/QueryPlan/ISourceStep.h b/src/Processors/QueryPlan/ISourceStep.h index 8373a34b57f..673139b6a31 100644 --- a/src/Processors/QueryPlan/ISourceStep.h +++ b/src/Processors/QueryPlan/ISourceStep.h @@ -12,6 +12,12 @@ public: QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override; virtual void initializePipeline(QueryPipeline & pipeline) = 0; + + void describePipeline(FormatSettings & settings) const override; + +private: + /// We collect processors got after pipeline transformation. + Processors processors; }; } diff --git a/src/Processors/QueryPlan/ITransformingStep.cpp b/src/Processors/QueryPlan/ITransformingStep.cpp index bf4afdfccd4..877bb4d01ba 100644 --- a/src/Processors/QueryPlan/ITransformingStep.cpp +++ b/src/Processors/QueryPlan/ITransformingStep.cpp @@ -4,7 +4,8 @@ namespace DB { -ITransformingStep::ITransformingStep(DataStream input_stream, Block output_header, DataStreamTraits traits) +ITransformingStep::ITransformingStep(DataStream input_stream, Block output_header, DataStreamTraits traits, bool collect_processors_) + : collect_processors(collect_processors_) { input_streams.emplace_back(std::move(input_stream)); output_stream = DataStream{.header = std::move(output_header)}; @@ -18,8 +19,21 @@ ITransformingStep::ITransformingStep(DataStream input_stream, Block output_heade QueryPipelinePtr ITransformingStep::updatePipeline(QueryPipelines pipelines) { - transformPipeline(*pipelines.front()); + if (collect_processors) + { + QueryPipelineProcessorsCollector collector(*pipelines.front(), this); + transformPipeline(*pipelines.front()); + processors = collector.detachProcessors(); + } + else + transformPipeline(*pipelines.front()); + return std::move(pipelines.front()); } +void ITransformingStep::describePipeline(FormatSettings & settings) const +{ + IQueryPlanStep::describePipeline(processors, settings); +} + } diff --git a/src/Processors/QueryPlan/ITransformingStep.h b/src/Processors/QueryPlan/ITransformingStep.h index f18b6ba3c8d..a375a283c3f 100644 --- a/src/Processors/QueryPlan/ITransformingStep.h +++ b/src/Processors/QueryPlan/ITransformingStep.h @@ -12,11 +12,18 @@ public: bool preserves_distinct_columns; }; - ITransformingStep(DataStream input_stream, Block output_header, DataStreamTraits traits); + ITransformingStep(DataStream input_stream, Block output_header, DataStreamTraits traits, bool collect_processors_ = true); QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override; virtual void transformPipeline(QueryPipeline & pipeline) = 0; + + void describePipeline(FormatSettings & settings) const override; + +private: + /// We collect processors got after pipeline transformation. + Processors processors; + bool collect_processors; }; } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 035216abb27..f78f2d038be 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -2,7 +2,6 @@ #include #include #include -#include #include #include @@ -264,4 +263,52 @@ void QueryPlan::explain(WriteBuffer & buffer, const ExplainOptions & options) } } +static void explainPipelineStep(IQueryPlanStep & step, IQueryPlanStep::FormatSettings & settings) +{ + settings.out << String(settings.offset, settings.ident_char) << "(" << step.getName() << ")\n"; + size_t current_offset = settings.offset; + step.describePipeline(settings); + if (current_offset == settings.offset) + settings.offset += settings.ident; +} + +void QueryPlan::explainPipeline(WriteBuffer & buffer) +{ + checkInitialized(); + + IQueryPlanStep::FormatSettings settings{.out = buffer}; + + struct Frame + { + Node * node; + size_t offset = 0; + bool is_description_printed = false; + size_t next_child = 0; + }; + + std::stack stack; + stack.push(Frame{.node = root}); + + while (!stack.empty()) + { + auto & frame = stack.top(); + + if (!frame.is_description_printed) + { + settings.offset = frame.offset; + explainPipelineStep(*frame.node->step, settings); + frame.offset = settings.offset; + frame.is_description_printed = true; + } + + if (frame.next_child < frame.node->children.size()) + { + stack.push(Frame{frame.node->children[frame.next_child], frame.offset}); + ++frame.next_child; + } + else + stack.pop(); + } +} + } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 7a6efc1c207..abea4a39786 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -41,6 +41,7 @@ public: }; void explain(WriteBuffer & buffer, const ExplainOptions & options); + void explainPipeline(WriteBuffer & buffer); /// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines. /// TODO: make it in a better way. diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.cpp b/src/Processors/QueryPlan/ReadFromStorageStep.cpp index 83b8682e09c..8fd32a4f98f 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.cpp +++ b/src/Processors/QueryPlan/ReadFromStorageStep.cpp @@ -66,6 +66,7 @@ ReadFromStorageStep::ReadFromStorageStep( } pipeline = std::make_unique(); + QueryPipelineProcessorsCollector collector(*pipeline, this); /// Table lock is stored inside pipeline here. pipeline->addTableLock(table_lock); @@ -122,6 +123,8 @@ ReadFromStorageStep::ReadFromStorageStep( pipeline->addInterpreterContext(std::move(context)); pipeline->addStorageHolder(std::move(storage)); + processors = collector.detachProcessors(); + output_stream = DataStream{.header = pipeline->getHeader()}; } @@ -132,4 +135,9 @@ QueryPipelinePtr ReadFromStorageStep::updatePipeline(QueryPipelines) return std::move(pipeline); } +void ReadFromStorageStep::describePipeline(FormatSettings & settings) const +{ + IQueryPlanStep::describePipeline(processors, settings); +} + } diff --git a/src/Processors/QueryPlan/ReadFromStorageStep.h b/src/Processors/QueryPlan/ReadFromStorageStep.h index 230e5acc1e0..0d058c5fc57 100644 --- a/src/Processors/QueryPlan/ReadFromStorageStep.h +++ b/src/Processors/QueryPlan/ReadFromStorageStep.h @@ -34,6 +34,8 @@ public: QueryPipelinePtr updatePipeline(QueryPipelines) override; + void describePipeline(FormatSettings & settings) const override; + private: TableStructureReadLockHolder table_lock; SelectQueryOptions options; @@ -47,6 +49,7 @@ private: size_t max_streams; QueryPipelinePtr pipeline; + Processors processors; }; } diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index c39a2fcafda..5e5ec6dc670 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -19,9 +19,12 @@ UnionStep::UnionStep(DataStreams input_streams_, Block result_header, size_t max QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines) { auto pipeline = std::make_unique(); + QueryPipelineProcessorsCollector collector(*pipeline, this); + if (pipelines.empty()) { pipeline->init(Pipe(std::make_shared(output_stream->header))); + processors = collector.detachProcessors(); return pipeline; } @@ -35,7 +38,13 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines) pipeline->setMaxThreads(std::min(num_pipelines, max_threads)); } + processors = collector.detachProcessors(); return pipeline; } +void UnionStep::describePipeline(FormatSettings & settings) const +{ + IQueryPlanStep::describePipeline(processors, settings); +} + } diff --git a/src/Processors/QueryPlan/UnionStep.h b/src/Processors/QueryPlan/UnionStep.h index 2c3d17b2e82..089c621e86a 100644 --- a/src/Processors/QueryPlan/UnionStep.h +++ b/src/Processors/QueryPlan/UnionStep.h @@ -14,9 +14,12 @@ public: QueryPipelinePtr updatePipeline(QueryPipelines pipelines) override; + void describePipeline(FormatSettings & settings) const override; + private: Block header; size_t max_threads; + Processors processors; }; } From 79bcc8309ca075b5c0e688e8285bb7dc905711f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jun 2020 16:58:28 +0300 Subject: [PATCH 047/196] Header for pipeline. --- src/Interpreters/InterpreterExplainQuery.cpp | 42 +++++++++++---- src/Processors/QueryPipeline.h | 2 +- src/Processors/QueryPlan/IQueryPlanStep.cpp | 54 ++++++++++++++++++++ src/Processors/QueryPlan/IQueryPlanStep.h | 1 + src/Processors/QueryPlan/QueryPlan.cpp | 8 +-- src/Processors/QueryPlan/QueryPlan.h | 15 ++++-- 6 files changed, 103 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 26784c1f623..9c256675fe4 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -111,9 +111,10 @@ static void fillColumn(IColumn & column, const std::string & str) namespace { -struct ExplainSettings +struct QueryPlanSettings { - QueryPlan::ExplainOptions query_plan_options; + QueryPlan::ExplainPlanOptions query_plan_options; + constexpr static char name[] = "PLAN"; std::unordered_map> boolean_settings = { @@ -121,6 +122,23 @@ struct ExplainSettings {"description", query_plan_options.description}, {"actions", query_plan_options.actions} }; +}; + +struct QueryPipelineSettings +{ + QueryPlan::ExplainPipelineOptions query_pipeline_options; + constexpr static char name[] = "PIPELINE"; + + std::unordered_map> boolean_settings = + { + {"header", query_pipeline_options.header}, + }; +}; + +template +struct ExplainSettings : public Settings +{ + using Settings::boolean_settings; bool has(const std::string & name) const { @@ -151,21 +169,20 @@ struct ExplainSettings } }; -} - -ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) +template +ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) { if (!ast_settings) return {}; - ExplainSettings settings; + ExplainSettings settings; const auto & set_query = ast_settings->as(); for (const auto & change : set_query.changes) { if (!settings.has(change.name)) - throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN query. Supported settings: " + - settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING); + throw Exception("Unknown setting \"" + change.name + "\" for EXPLAIN " + Settings::name + " query. " + "Supported settings: " + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING); if (change.value.getType() != Field::Types::UInt64) throw Exception("Invalid type " + std::string(change.value.getTypeName()) + " for setting \"" + change.name + @@ -182,10 +199,11 @@ ExplainSettings checkAndGetSettings(const ASTPtr & ast_settings) return settings; } +} + BlockInputStreamPtr InterpreterExplainQuery::executeImpl() { const auto & ast = query->as(); - auto settings = checkAndGetSettings(ast.getSettings()); Block sample_block = getSampleBlock(); MutableColumns res_columns = sample_block.cloneEmptyColumns(); @@ -208,19 +226,21 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (!dynamic_cast(ast.getExplainedQuery().get())) throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY); + auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions()); interpreter.buildQueryPlan(plan); WriteBufferFromOStream buffer(ss); - plan.explain(buffer, settings.query_plan_options); + plan.explainPlan(buffer, settings.query_plan_options); } else if (ast.getKind() == ASTExplainQuery::QueryPipeline) { if (!dynamic_cast(ast.getExplainedQuery().get())) throw Exception("Only SELECT is supported for EXPLAIN query", ErrorCodes::INCORRECT_QUERY); + auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions()); @@ -228,7 +248,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() plan.buildQueryPipeline(); WriteBufferFromOStream buffer(ss); - plan.explainPipeline(buffer); + plan.explainPipeline(buffer, settings.query_pipeline_options); } fillColumn(*res_columns[0], ss.str()); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 19979721e0e..77dce2de206 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -79,7 +79,7 @@ public: bool empty() const { return processors.empty(); } void emplace(ProcessorPtr processor); void emplace(Processors processors_); - Processors * getCollectedProcessors() const; + Processors * getCollectedProcessors() const { return collected_processors; } Processors * setCollectedProcessors(Processors * collected_processors); Processors & get() { return processors; } Processors detach() { return std::move(processors); } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.cpp b/src/Processors/QueryPlan/IQueryPlanStep.cpp index 2a50f571483..e1f44b1e4e4 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.cpp +++ b/src/Processors/QueryPlan/IQueryPlanStep.cpp @@ -18,6 +18,38 @@ const DataStream & IQueryPlanStep::getOutputStream() const return *output_stream; } +static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep::FormatSettings & settings) +{ + String prefix(settings.offset, settings.ident_char); + prefix += "Header"; + + if (count > 1) + prefix += " x " + std::to_string(count) + " "; + + prefix += ": "; + + settings.out << prefix; + + if (!header) + { + settings.out << " empty\n"; + return; + } + + prefix.assign(prefix.size(), settings.ident_char); + bool first = true; + + for (const auto & elem : header) + { + if (!first) + settings.out << prefix; + + first = false; + elem.dumpStructure(settings.out, true); + settings.out << '\n'; + } +} + static void doDescribeProcessor(const IProcessor & processor, size_t count, IQueryPlanStep::FormatSettings & settings) { settings.out << String(settings.offset, settings.ident_char) << processor.getName(); @@ -30,6 +62,28 @@ static void doDescribeProcessor(const IProcessor & processor, size_t count, IQue settings.out << " " << std::to_string(num_inputs) << " -> " << std::to_string(num_outputs); settings.out << '\n'; + + if (settings.write_header) + { + const Block * last_header = nullptr; + size_t num_equal_headers = 0; + + for (const auto & port : processor.getOutputs()) + { + if (last_header && !blocksHaveEqualStructure(*last_header, port.getHeader())) + { + doDescribeHeader(*last_header, num_equal_headers, settings); + num_equal_headers = 0; + } + + ++num_equal_headers; + last_header = &port.getHeader(); + } + + if (last_header) + doDescribeHeader(*last_header, num_equal_headers, settings); + } + settings.offset += settings.ident; } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index 27e81554590..f57d47109e5 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -61,6 +61,7 @@ public: size_t offset = 0; const size_t ident = 2; const char ident_char = ' '; + const bool write_header = false; }; /// Get detailed description of step actions. This is shown in EXPLAIN query with options `actions = 1`. diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index f78f2d038be..53e97d6715c 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -178,7 +178,7 @@ void QueryPlan::addInterpreterContext(std::shared_ptr context) static void explainStep( - WriteBuffer & buffer, IQueryPlanStep & step, size_t ident, const QueryPlan::ExplainOptions & options) + WriteBuffer & buffer, IQueryPlanStep & step, size_t ident, const QueryPlan::ExplainPlanOptions & options) { std::string prefix(ident, ' '); buffer << prefix; @@ -227,7 +227,7 @@ static void explainStep( } } -void QueryPlan::explain(WriteBuffer & buffer, const ExplainOptions & options) +void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options) { checkInitialized(); @@ -272,11 +272,11 @@ static void explainPipelineStep(IQueryPlanStep & step, IQueryPlanStep::FormatSet settings.offset += settings.ident; } -void QueryPlan::explainPipeline(WriteBuffer & buffer) +void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptions & options) { checkInitialized(); - IQueryPlanStep::FormatSettings settings{.out = buffer}; + IQueryPlanStep::FormatSettings settings{.out = buffer, .write_header = options.header}; struct Frame { diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index abea4a39786..b96c281063a 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -33,15 +33,24 @@ public: QueryPipelinePtr buildQueryPipeline(); - struct ExplainOptions + struct ExplainPlanOptions { + /// Add output header to step. bool header = false; + /// Add description of step. bool description = true; + /// Add detailed information about step actions. bool actions = false; }; - void explain(WriteBuffer & buffer, const ExplainOptions & options); - void explainPipeline(WriteBuffer & buffer); + struct ExplainPipelineOptions + { + /// Show header of output ports. + bool header = false; + }; + + void explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options); + void explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptions & options); /// Set upper limit for the recommend number of threads. Will be applied to the newly-created pipelines. /// TODO: make it in a better way. From f4e944cd35a961b75c8f75f42d1ae6710ebbe1f8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jun 2020 18:07:49 +0300 Subject: [PATCH 048/196] Graph for pipeline. --- src/Interpreters/InterpreterExplainQuery.cpp | 23 ++++++++++++++++++-- src/Processors/QueryPipeline.h | 4 ++++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 9c256675fe4..91fa31c1d04 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { @@ -114,6 +115,7 @@ namespace struct QueryPlanSettings { QueryPlan::ExplainPlanOptions query_plan_options; + constexpr static char name[] = "PLAN"; std::unordered_map> boolean_settings = @@ -127,11 +129,14 @@ struct QueryPlanSettings struct QueryPipelineSettings { QueryPlan::ExplainPipelineOptions query_pipeline_options; + bool graph = false; + constexpr static char name[] = "PIPELINE"; std::unordered_map> boolean_settings = { {"header", query_pipeline_options.header}, + {"graph", graph}, }; }; @@ -212,10 +217,16 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (ast.getKind() == ASTExplainQuery::ParsedAST) { + if (ast.getSettings()) + throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING); + dumpAST(ast, ss); } else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax) { + if (ast.getSettings()) + throw Exception("Settings are not supported for EXPLAIN SYNTAX query.", ErrorCodes::UNKNOWN_SETTING); + ExplainAnalyzedSyntaxVisitor::Data data{.context = context}; ExplainAnalyzedSyntaxVisitor(data).visit(query); @@ -245,10 +256,18 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions()); interpreter.buildQueryPlan(plan); - plan.buildQueryPipeline(); + auto pipeline = plan.buildQueryPipeline(); WriteBufferFromOStream buffer(ss); - plan.explainPipeline(buffer, settings.query_pipeline_options); + + if (settings.graph) + { + printPipeline(pipeline->getProcessors(), buffer); + } + else + { + plan.explainPipeline(buffer, settings.query_pipeline_options); + } } fillColumn(*res_columns[0], ss.str()); diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index 77dce2de206..1edb16e70be 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -82,6 +82,7 @@ public: Processors * getCollectedProcessors() const { return collected_processors; } Processors * setCollectedProcessors(Processors * collected_processors); Processors & get() { return processors; } + const Processors & get() const { return processors; } Processors detach() { return std::move(processors); } private: /// All added processors. @@ -196,6 +197,9 @@ public: Pipe getPipe() &&; Pipes getPipes() &&; + /// Get internal processors. + const Processors & getProcessors() const { return processors.get(); } + private: /// Destruction order: processors, header, locks, temporary storages, local contexts From 2c02bfac754a537d674dd3184f6c80c5602c95cf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Jun 2020 20:56:33 +0300 Subject: [PATCH 049/196] Compact pipeline. --- src/Interpreters/InterpreterExplainQuery.cpp | 7 +- src/Processors/IProcessor.h | 4 +- src/Processors/QueryPipeline.cpp | 5 +- src/Processors/printPipeline.cpp | 164 +++++++++++++++++++ src/Processors/printPipeline.h | 10 +- src/Processors/ya.make | 1 + 6 files changed, 183 insertions(+), 8 deletions(-) create mode 100644 src/Processors/printPipeline.cpp diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 91fa31c1d04..9b71ae39a8e 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -130,6 +130,7 @@ struct QueryPipelineSettings { QueryPlan::ExplainPipelineOptions query_pipeline_options; bool graph = false; + bool compact = true; constexpr static char name[] = "PIPELINE"; @@ -137,6 +138,7 @@ struct QueryPipelineSettings { {"header", query_pipeline_options.header}, {"graph", graph}, + {"compact", compact}, }; }; @@ -262,7 +264,10 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (settings.graph) { - printPipeline(pipeline->getProcessors(), buffer); + if (settings.compact) + printPipelineCompact(pipeline->getProcessors(), buffer, settings.query_pipeline_options.header); + else + printPipeline(pipeline->getProcessors(), buffer); } else { diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 7c9f94916bf..e9148dd5075 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -312,8 +312,8 @@ private: bool has_quota = false; - IQueryPlanStep * query_plan_step; - size_t query_plan_step_group; + IQueryPlanStep * query_plan_step = nullptr; + size_t query_plan_step_group = 0; }; diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 8927c75329a..fa04082c82f 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -879,12 +879,11 @@ QueryPipelineProcessorsCollector::~QueryPipelineProcessorsCollector() Processors QueryPipelineProcessorsCollector::detachProcessors(size_t group) { - Processors res; - res.swap(processors); - for (auto & processor : processors) processor->setQueryPlanStep(step, group); + Processors res; + res.swap(processors); return res; } diff --git a/src/Processors/printPipeline.cpp b/src/Processors/printPipeline.cpp new file mode 100644 index 00000000000..76c84218fb8 --- /dev/null +++ b/src/Processors/printPipeline.cpp @@ -0,0 +1,164 @@ +#include +#include +#include +#include + +namespace DB +{ + +void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header) +{ + struct Node; + struct Key + { + size_t group; + IQueryPlanStep * step; + std::string name; + + auto getTuple() const { return std::forward_as_tuple(group, step, name); } + + bool operator<(const Key & other) const + { + return getTuple() < other.getTuple(); + } + }; + + struct EdgeData + { + Block header; + size_t count; + }; + + using Edge = std::vector; + + struct Node + { + size_t id = 0; + std::map edges = {}; + std::vector agents = {}; + }; + + std::map graph; + auto get_key = [](const IProcessor & processor) + { + return Key{processor.getQueryPlanStepGroup(), processor.getQueryPlanStep(), processor.getName()}; + }; + + for (const auto & processor : processors) + { + auto res = graph.emplace(get_key(*processor), Node()); + res.first->second.agents.emplace_back(processor.get()); + + if (res.second) + res.first->second.id = graph.size(); + } + + Block empty_header; + + for (const auto & processor : processors) + { + auto & from = graph[get_key(*processor)]; + + for (auto & port : processor->getOutputs()) + { + if (!port.isConnected()) + continue; + + auto & to = graph[get_key(port.getInputPort().getProcessor())]; + auto & edge = from.edges[&to]; + const auto & header = with_header ? port.getHeader() : empty_header; + + bool found = false; + for (auto & item : edge) + { + if (blocksHaveEqualStructure(header, item.header)) + { + found = true; + ++item.count; + break; + } + } + + if (!found) + edge.emplace_back(EdgeData{header, 1}); + } + } + + std::map> steps_map; + + for (const auto & item : graph) + steps_map[item.first.step].emplace_back(&item.second); + + out << "digraph\n{\n"; + out << " rankdir=\"LR\";\n"; + out << " { node [shape = box]\n"; + + /// Nodes // TODO quoting and escaping + size_t next_step = 0; + for (const auto & item : steps_map) + { + if (item.first != nullptr) + { + out << " subgraph cluster_" << next_step << " {\n"; + out << " label =\"" << item.first->getName() << "\";\n"; + out << " style=filled;\n"; + out << " color=lightgrey;\n"; + out << " node [style=filled,color=white];\n"; + out << " { rank = same;\n"; + + ++next_step; + } + + for (const auto & node : item.second) + { + const auto & processor = node->agents.front(); + out << " n" << node->id << " [label=\"" << processor->getName(); + + if (node->agents.size() > 1) + out << " x " << node->agents.size(); + + const auto & description = processor->getDescription(); + if (!description.empty()) + out << ' ' << description; + + out << "\"];\n"; + } + + if (item.first != nullptr) + { + out << " }\n"; + out << " }\n"; + } + } + + out << " }\n"; + + /// Edges + for (const auto & item : graph) + { + for (const auto & edge : item.second.edges) + { + for (const auto & data : edge.second) + { + out << " n" << item.second.id << " -> " << "n" << edge.first->id << " [label=\""; + + if (data.count > 1) + out << "x " << data.count; + + if (with_header) + { + for (const auto & elem : data.header) + { + out << "\n"; + elem.dumpStructure(out); + } + } + + out << "\"];\n"; + } + } + } + out << "}\n"; +} + +} diff --git a/src/Processors/printPipeline.h b/src/Processors/printPipeline.h index ce7306ec4cf..a6d134a5c17 100644 --- a/src/Processors/printPipeline.h +++ b/src/Processors/printPipeline.h @@ -15,6 +15,8 @@ template void printPipeline(const Processors & processors, const Statuses & statuses, WriteBuffer & out) { out << "digraph\n{\n"; + out << " rankdir=\"LR\";\n"; + out << " { node [shape = box]\n"; auto get_proc_id = [](const IProcessor & proc) -> UInt64 { @@ -26,7 +28,7 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri /// Nodes // TODO quoting and escaping for (const auto & processor : processors) { - out << "n" << get_proc_id(*processor) << "[label=\"" << processor->getName() << processor->getDescription(); + out << " n" << get_proc_id(*processor) << "[label=\"" << processor->getName() << processor->getDescription(); if (statuses_iter != statuses.end()) { @@ -37,6 +39,8 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri out << "\"];\n"; } + out << " }\n"; + /// Edges for (const auto & processor : processors) { @@ -48,7 +52,7 @@ void printPipeline(const Processors & processors, const Statuses & statuses, Wri const IProcessor & curr = *processor; const IProcessor & next = port.getInputPort().getProcessor(); - out << "n" << get_proc_id(curr) << " -> n" << get_proc_id(next) << ";\n"; + out << " n" << get_proc_id(curr) << " -> n" << get_proc_id(next) << ";\n"; } } out << "}\n"; @@ -60,4 +64,6 @@ void printPipeline(const Processors & processors, WriteBuffer & out) printPipeline(processors, std::vector(), out); } +void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header); + } diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 550aa9ad7da..3488cf534d3 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -105,6 +105,7 @@ SRCS( OffsetTransform.cpp Pipe.cpp Port.cpp + printPipeline.cpp QueryPipeline.cpp ResizeProcessor.cpp Sources/DelayedSource.cpp From f1f98d7582ab5e0ef46bffa90d08e4acf9439ff7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 27 Jun 2020 12:14:38 +0300 Subject: [PATCH 050/196] More comments. --- src/Interpreters/InterpreterExplainQuery.cpp | 3 +++ src/Processors/printPipeline.cpp | 19 ++++++++++++++++--- src/Processors/printPipeline.h | 4 ++++ 3 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 9b71ae39a8e..322b298f181 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -88,6 +88,7 @@ Block InterpreterExplainQuery::getSampleBlock() return block; } +/// Split str by line feed and write as separate row to ColumnString. static void fillColumn(IColumn & column, const std::string & str) { size_t start = 0; @@ -112,6 +113,8 @@ static void fillColumn(IColumn & column, const std::string & str) namespace { +/// Settings. Different for each explain type. + struct QueryPlanSettings { QueryPlan::ExplainPlanOptions query_plan_options; diff --git a/src/Processors/printPipeline.cpp b/src/Processors/printPipeline.cpp index 76c84218fb8..ee7231c45fb 100644 --- a/src/Processors/printPipeline.cpp +++ b/src/Processors/printPipeline.cpp @@ -9,6 +9,8 @@ namespace DB void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header) { struct Node; + + /// Group by processors name, QueryPlanStep and group in this step. struct Key { size_t group; @@ -23,6 +25,7 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool } }; + /// Group ports by header. struct EdgeData { Block header; @@ -39,22 +42,26 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool }; std::map graph; + auto get_key = [](const IProcessor & processor) { return Key{processor.getQueryPlanStepGroup(), processor.getQueryPlanStep(), processor.getName()}; }; + /// Fill nodes. for (const auto & processor : processors) { auto res = graph.emplace(get_key(*processor), Node()); - res.first->second.agents.emplace_back(processor.get()); + auto & node = res.first->second; + node.agents.emplace_back(processor.get()); if (res.second) - res.first->second.id = graph.size(); + node.id = graph.size(); } Block empty_header; + /// Fill edges. for (const auto & processor : processors) { auto & from = graph[get_key(*processor)]; @@ -66,8 +73,12 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool auto & to = graph[get_key(port.getInputPort().getProcessor())]; auto & edge = from.edges[&to]; - const auto & header = with_header ? port.getHeader() : empty_header; + /// Use empty header for each edge if with_header is false. + const auto & header = with_header ? port.getHeader() + : empty_header; + + /// Group by header. bool found = false; for (auto & item : edge) { @@ -84,6 +95,7 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool } } + /// Group processors by it's QueryPlanStep. std::map> steps_map; for (const auto & item : graph) @@ -97,6 +109,7 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool size_t next_step = 0; for (const auto & item : steps_map) { + /// Use separate clusters for each step. if (item.first != nullptr) { out << " subgraph cluster_" << next_step << " {\n"; diff --git a/src/Processors/printPipeline.h b/src/Processors/printPipeline.h index a6d134a5c17..9497bc3cc3c 100644 --- a/src/Processors/printPipeline.h +++ b/src/Processors/printPipeline.h @@ -64,6 +64,10 @@ void printPipeline(const Processors & processors, WriteBuffer & out) printPipeline(processors, std::vector(), out); } +/// Prints pipeline in compact representation. +/// Group processors by it's name, QueryPlanStep and QueryPlanStepGroup. +/// If QueryPlanStep wasn't set for processor, representation may be not correct. +/// If with_header is set, prints block header for each edge. void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool with_header); } From 5b1596cdb210b0b2ccac2ad2fe5ea309efe94d99 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 27 Jun 2020 17:02:24 +0300 Subject: [PATCH 051/196] Refactor some code. --- src/Core/ColumnWithTypeAndName.cpp | 7 +- src/Core/ColumnWithTypeAndName.h | 2 +- src/Core/SortDescription.cpp | 26 ++--- src/Core/SortDescription.h | 2 +- src/Interpreters/AggregateDescription.cpp | 105 +++++++++++------- src/Interpreters/AggregateDescription.h | 2 +- src/Interpreters/Aggregator.cpp | 50 ++++----- src/Interpreters/Aggregator.h | 2 +- src/Processors/QueryPlan/AggregatingStep.cpp | 4 +- src/Processors/QueryPlan/AggregatingStep.h | 2 +- src/Processors/QueryPlan/ConvertingStep.cpp | 25 +++-- src/Processors/QueryPlan/ConvertingStep.h | 2 +- src/Processors/QueryPlan/CreatingSetsStep.cpp | 16 +-- src/Processors/QueryPlan/CreatingSetsStep.h | 2 +- src/Processors/QueryPlan/DistinctStep.cpp | 26 +++-- src/Processors/QueryPlan/DistinctStep.h | 2 +- src/Processors/QueryPlan/ExpressionStep.cpp | 25 +++-- src/Processors/QueryPlan/ExpressionStep.h | 4 +- src/Processors/QueryPlan/FillingStep.cpp | 7 +- src/Processors/QueryPlan/FillingStep.h | 2 +- src/Processors/QueryPlan/FilterStep.cpp | 18 +-- src/Processors/QueryPlan/FilterStep.h | 2 +- .../QueryPlan/FinishSortingStep.cpp | 20 ++-- src/Processors/QueryPlan/FinishSortingStep.h | 2 +- src/Processors/QueryPlan/IQueryPlanStep.cpp | 2 +- src/Processors/QueryPlan/IQueryPlanStep.h | 2 +- src/Processors/QueryPlan/LimitByStep.cpp | 34 +++--- src/Processors/QueryPlan/LimitByStep.h | 4 +- src/Processors/QueryPlan/LimitStep.cpp | 23 ++-- src/Processors/QueryPlan/LimitStep.h | 2 +- src/Processors/QueryPlan/MergeSortingStep.cpp | 13 ++- src/Processors/QueryPlan/MergeSortingStep.h | 2 +- .../QueryPlan/MergingAggregatedStep.cpp | 4 +- .../QueryPlan/MergingAggregatedStep.h | 2 +- .../QueryPlan/MergingSortedStep.cpp | 13 +-- src/Processors/QueryPlan/MergingSortedStep.h | 2 +- src/Processors/QueryPlan/OffsetStep.cpp | 5 +- src/Processors/QueryPlan/OffsetStep.h | 2 +- .../QueryPlan/PartialSortingStep.cpp | 13 +-- src/Processors/QueryPlan/PartialSortingStep.h | 2 +- src/Processors/QueryPlan/QueryPlan.cpp | 42 ++++--- src/Processors/QueryPlan/TotalsHavingStep.cpp | 20 ++-- src/Processors/QueryPlan/TotalsHavingStep.h | 2 +- 43 files changed, 299 insertions(+), 245 deletions(-) diff --git a/src/Core/ColumnWithTypeAndName.cpp b/src/Core/ColumnWithTypeAndName.cpp index d5b43b3841b..9acc2d56408 100644 --- a/src/Core/ColumnWithTypeAndName.cpp +++ b/src/Core/ColumnWithTypeAndName.cpp @@ -28,12 +28,9 @@ bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) cons } -void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out, bool escaped) const +void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const { - if (escaped) - writeEscapedString(name, out); - else - out << name; + out << name; if (type) out << ' ' << type->getName(); diff --git a/src/Core/ColumnWithTypeAndName.h b/src/Core/ColumnWithTypeAndName.h index b2f1a076b76..27b09710258 100644 --- a/src/Core/ColumnWithTypeAndName.h +++ b/src/Core/ColumnWithTypeAndName.h @@ -33,7 +33,7 @@ struct ColumnWithTypeAndName ColumnWithTypeAndName cloneEmpty() const; bool operator==(const ColumnWithTypeAndName & other) const; - void dumpStructure(WriteBuffer & out, bool escaped = false) const; + void dumpStructure(WriteBuffer & out) const; String dumpStructure() const; }; #pragma GCC diagnostic pop diff --git a/src/Core/SortDescription.cpp b/src/Core/SortDescription.cpp index e27c39dbb5f..4a5952c3bc2 100644 --- a/src/Core/SortDescription.cpp +++ b/src/Core/SortDescription.cpp @@ -1,40 +1,40 @@ #include #include +#include namespace DB { -String dumpSortDescription(const SortDescription & description, const Block & header) +void dumpSortDescription(const SortDescription & description, const Block & header, WriteBuffer & out) { - String res; + bool first = true; for (const auto & desc : description) { - if (!res.empty()) - res += ", "; + if (!first) + out << ", "; + first = false; if (!desc.column_name.empty()) - res += desc.column_name; + out << desc.column_name; else { if (desc.column_number < header.columns()) - res += header.getByPosition(desc.column_number).name; + out << header.getByPosition(desc.column_number).name; else - res += "?"; + out << "?"; - res += " (pos " + std::to_string(desc.column_number) + ")"; + out << " (pos " << desc.column_number << ")"; } if (desc.direction > 0) - res += " ASC"; + out << " ASC"; else - res += " DESC"; + out << " DESC"; if (desc.with_fill) - res += " WITH FILL"; + out << " WITH FILL"; } - - return res; } } diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index a155032edfe..f51ee0ba135 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -72,6 +72,6 @@ struct SortColumnDescription using SortDescription = std::vector; class Block; -String dumpSortDescription(const SortDescription & description, const Block & header); +void dumpSortDescription(const SortDescription & description, const Block & header, WriteBuffer & out); } diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp index 53ecd3159e2..35057516f91 100644 --- a/src/Interpreters/AggregateDescription.cpp +++ b/src/Interpreters/AggregateDescription.cpp @@ -1,82 +1,101 @@ #include #include +#include namespace DB { -Strings AggregateDescription::explain() const +void AggregateDescription::explain(WriteBuffer & out, size_t ident) const { - Strings res; + String prefix(ident, ' '); - res.emplace_back(column_name); + out << prefix << column_name << '\n'; - auto get_params_string = [](const Array & arr) + auto dump_params = [&](const Array & arr) { - String params_str; + bool first = true; for (const auto & param : arr) { - if (!params_str.empty()) - params_str += ", "; + if (!first) + out << ", "; - params_str += applyVisitor(FieldVisitorToString(), param); + first = false; + + out << applyVisitor(FieldVisitorToString(), param); } - - return params_str; }; if (function) { - String types_str; - for (const auto & type : function->getArgumentTypes()) - { - if (!types_str.empty()) - types_str += ", "; + out << prefix << " Function: " << function->getName(); - types_str += type->getName(); + const auto & params = function->getParameters(); + if (!params.empty()) + { + out << "("; + dump_params(params); + out << ")"; } - auto params_str = get_params_string(function->getParameters()); - if (!params_str.empty()) - params_str = "(" + params_str + ")"; + out << "("; - res.emplace_back(" Function: " + function->getName() + params_str + '(' + types_str + ") -> " + - function->getReturnType()->getName()); + bool first = true; + for (const auto & type : function->getArgumentTypes()) + { + if (!first) + out << ", "; + first = false; + + out << type->getName(); + } + + out << ")\n"; } else - res.emplace_back(" Function: nullptr"); + out << prefix << " Function: nullptr\n"; if (!parameters.empty()) - res.emplace_back(" Parameters: " + get_params_string(parameters)); - - String arguments_names_str; - for (const auto & arg : argument_names) { - if (!arguments_names_str.empty()) - arguments_names_str += ", "; - - arguments_names_str += arg; + out << prefix << " Parameters: "; + dump_params(parameters); + out << '\n'; } - if (arguments_names_str.empty()) - arguments_names_str = "none"; + out << prefix << " Arguments: "; - res.emplace_back(" Arguments: " + arguments_names_str); - - String arguments_pos_str; - for (auto arg : arguments) + if (argument_names.empty()) + out << "none\n"; + else { - if (!arguments_pos_str.empty()) - arguments_pos_str += ", "; + bool first = true; + for (const auto & arg : argument_names) + { + if (!first) + out << ", "; + first = false; - arguments_pos_str += std::to_string(arg); + out << arg; + } + out << "\n"; } - if (arguments_pos_str.empty()) - arguments_pos_str = "none"; + out << prefix << " Argument positions: "; - res.emplace_back(" Argument positions: " + arguments_pos_str); + if (arguments.empty()) + out << "none\n"; + else + { + bool first = true; + for (auto arg : arguments) + { + if (!first) + out << ", "; + first = false; - return res; + out << arg; + } + out << '\n'; + } } } diff --git a/src/Interpreters/AggregateDescription.h b/src/Interpreters/AggregateDescription.h index a17e307451d..6038b224821 100644 --- a/src/Interpreters/AggregateDescription.h +++ b/src/Interpreters/AggregateDescription.h @@ -16,7 +16,7 @@ struct AggregateDescription Names argument_names; /// used if no `arguments` are specified. String column_name; /// What name to use for a column with aggregate function values - Strings explain() const; /// Get description for EXPLAIN query. + void explain(WriteBuffer & out, size_t ident) const; /// Get description for EXPLAIN query. }; using AggregateDescriptions = std::vector; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 785dc88bffa..fdbf0587f78 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -29,6 +29,7 @@ #include #include #include +#include namespace ProfileEvents @@ -150,44 +151,41 @@ Block Aggregator::Params::getHeader( return materializeBlock(res); } -Strings Aggregator::Params::explain() const +void Aggregator::Params::explain(WriteBuffer & out, size_t ident) const { Strings res; const auto & header = src_header ? src_header : intermediate_header; - String keys_str; - for (auto key : keys) + + String prefix(ident, ' '); + { - if (!keys_str.empty()) - keys_str += ", "; + /// Dump keys. + out << prefix << "Keys: "; - if (key >= header.columns()) - keys_str += "unknown position " + std::to_string(key); - else - keys_str += header.getByPosition(key).name; + bool first = true; + for (auto key : keys) + { + if (!first) + out << ", "; + first = false; + + if (key >= header.columns()) + out << "unknown position " << key; + else + out << header.getByPosition(key).name; + } + + out << '\n'; } - res.emplace_back("Keys: " + std::move(keys_str)); - if (!aggregates.empty()) { - bool first = true; + out << prefix << "Aggregates:\n"; + for (const auto & aggregate : aggregates) - { - auto aggregate_strings = aggregate.explain(); - for (const auto & aggregate_str : aggregate_strings) - { - if (first) - res.emplace_back("Aggregates: " + aggregate_str); - else - res.emplace_back(" " + aggregate_str); - - first = false; - } - } + aggregate.explain(out, ident + 4); } - - return res; } Aggregator::Aggregator(const Params & params_) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 03706834783..1ae0898d9ca 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -925,7 +925,7 @@ public: } /// Returns keys and aggregated for EXPLAIN query - Strings explain() const; + void explain(WriteBuffer & out, size_t ident) const; }; Aggregator(const Params & params_); diff --git a/src/Processors/QueryPlan/AggregatingStep.cpp b/src/Processors/QueryPlan/AggregatingStep.cpp index 21d9b9d7825..0bb85b68e41 100644 --- a/src/Processors/QueryPlan/AggregatingStep.cpp +++ b/src/Processors/QueryPlan/AggregatingStep.cpp @@ -152,9 +152,9 @@ void AggregatingStep::transformPipeline(QueryPipeline & pipeline) pipeline.enableQuotaForCurrentStreams(); } -Strings AggregatingStep::describeActions() const +void AggregatingStep::describeActions(FormatSettings & settings) const { - return params.explain(); + params.explain(settings.out, settings.offset); } void AggregatingStep::describePipeline(FormatSettings & settings) const diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 933fad5e1bf..fbed0e3cec6 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -28,7 +28,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings &) const override; void describePipeline(FormatSettings & settings) const override; private: diff --git a/src/Processors/QueryPlan/ConvertingStep.cpp b/src/Processors/QueryPlan/ConvertingStep.cpp index 3c5238d4fd5..79e230445e5 100644 --- a/src/Processors/QueryPlan/ConvertingStep.cpp +++ b/src/Processors/QueryPlan/ConvertingStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -46,19 +47,19 @@ void ConvertingStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings ConvertingStep::describeActions() const +void ConvertingStep::describeActions(FormatSettings & settings) const { const auto & header = input_streams[0].header; auto conversion = ConvertingTransform(header, result_header, ConvertingTransform::MatchColumnsMode::Name) .getConversion(); - Strings res; - - auto get_description = [](const ColumnWithTypeAndName & elem, bool is_const) + auto dump_description = [&](const ColumnWithTypeAndName & elem, bool is_const) { - return elem.name + " " + elem.type->getName() + (is_const ? " Const" : ""); + settings.out << elem.name << ' ' << elem.type->getName() << (is_const ? " Const" : "") << '\n'; }; + String prefix(settings.offset, ' '); + for (size_t i = 0; i < conversion.size(); ++i) { const auto & from = header.getByPosition(conversion[i]); @@ -67,13 +68,19 @@ Strings ConvertingStep::describeActions() const bool from_const = from.column && isColumnConst(*from.column); bool to_const = to.column && isColumnConst(*to.column); + settings.out << prefix; + if (from.name == to.name && from.type->equals(*to.type) && from_const == to_const) - res.emplace_back(get_description(from, from_const)); + dump_description(from, from_const); else - res.emplace_back(get_description(to, to_const) + " <- " + get_description(from, from_const)); + { + dump_description(to, to_const); + settings.out << " <- "; + dump_description(from, from_const); + } + + settings.out << '\n'; } - - return res; } } diff --git a/src/Processors/QueryPlan/ConvertingStep.h b/src/Processors/QueryPlan/ConvertingStep.h index bf0840b1f96..fd3241f9863 100644 --- a/src/Processors/QueryPlan/ConvertingStep.h +++ b/src/Processors/QueryPlan/ConvertingStep.h @@ -13,7 +13,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: Block result_header; diff --git a/src/Processors/QueryPlan/CreatingSetsStep.cpp b/src/Processors/QueryPlan/CreatingSetsStep.cpp index 2f47c7abbb3..7d78fc88bad 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.cpp +++ b/src/Processors/QueryPlan/CreatingSetsStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -35,21 +36,20 @@ void CreatingSetsStep::transformPipeline(QueryPipeline & pipeline) pipeline.addCreatingSetsTransform(std::move(creating_sets)); } -Strings CreatingSetsStep::describeActions() const +void CreatingSetsStep::describeActions(FormatSettings & settings) const { - Strings res; + String prefix(settings.offset, ' '); + for (const auto & set : subqueries_for_sets) { - String str; + settings.out << prefix; if (set.second.set) - str += "Set: "; + settings.out << "Set: "; else if (set.second.join) - str += "Join: "; + settings.out << "Join: "; - str += set.first; + settings.out << set.first << '\n'; } - - return res; } } diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index 7f05de4b889..91ee975c3c8 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -19,7 +19,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: SubqueriesForSets subqueries_for_sets; diff --git a/src/Processors/QueryPlan/DistinctStep.cpp b/src/Processors/QueryPlan/DistinctStep.cpp index 1a558669289..d9f3f70cfc2 100644 --- a/src/Processors/QueryPlan/DistinctStep.cpp +++ b/src/Processors/QueryPlan/DistinctStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -65,18 +66,27 @@ void DistinctStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings DistinctStep::describeActions() const +void DistinctStep::describeActions(FormatSettings & settings) const { - String res; - for (const auto & column : columns) - { - if (!res.empty()) - res += ", "; + String prefix(settings.offset, ' '); + settings.out << prefix << "Columns: "; - res += column; + if (columns.empty()) + settings.out << "none"; + else + { + bool first = true; + for (const auto & column : columns) + { + if (!first) + settings.out << ", "; + first = false; + + settings.out << column; + } } - return {"Columns: " + res}; + settings.out << '\n'; } } diff --git a/src/Processors/QueryPlan/DistinctStep.h b/src/Processors/QueryPlan/DistinctStep.h index 8af909c60da..ae0ca564d21 100644 --- a/src/Processors/QueryPlan/DistinctStep.h +++ b/src/Processors/QueryPlan/DistinctStep.h @@ -19,7 +19,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: SizeLimits set_size_limits; diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index ef9b4ffd5eb..dff57202c9c 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -50,19 +51,23 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline) }); } -static Strings getActionsDescription(const ExpressionActionsPtr & expression) +static void doDescribeActions(const ExpressionActionsPtr & expression, IQueryPlanStep::FormatSettings & settings) { - Strings res; - for (const auto & action : expression->getActions()) - res.emplace_back((res.empty() ? "Actions: " - : " ") + action.toString()); + String prefix(settings.offset, ' '); + bool first = true; - return res; + for (const auto & action : expression->getActions()) + { + settings.out << prefix << (first ? "Actions: " + : " "); + first = false; + settings.out << action.toString() << '\n'; + } } -Strings ExpressionStep::describeActions() const +void ExpressionStep::describeActions(FormatSettings & settings) const { - return getActionsDescription(expression); + doDescribeActions(expression, settings); } InflatingExpressionStep::InflatingExpressionStep(const DataStream & input_stream_, ExpressionActionsPtr expression_) @@ -93,9 +98,9 @@ void InflatingExpressionStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings InflatingExpressionStep::describeActions() const +void InflatingExpressionStep::describeActions(FormatSettings & settings) const { - return getActionsDescription(expression); + doDescribeActions(expression, settings); } } diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index 2222dc12c3e..c1d2153363a 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -20,7 +20,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: ExpressionActionsPtr expression; @@ -37,7 +37,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: ExpressionActionsPtr expression; diff --git a/src/Processors/QueryPlan/FillingStep.cpp b/src/Processors/QueryPlan/FillingStep.cpp index b4a2a6d2963..c5085c2b735 100644 --- a/src/Processors/QueryPlan/FillingStep.cpp +++ b/src/Processors/QueryPlan/FillingStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -27,9 +28,11 @@ void FillingStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings FillingStep::describeActions() const +void FillingStep::describeActions(FormatSettings & settings) const { - return {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)}; + settings.out << String(settings.offset, ' '); + dumpSortDescription(sort_description, input_streams.front().header, settings.out); + settings.out << '\n'; } } diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 80f7e00dcf7..9ff5c7fa425 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -14,7 +14,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: SortDescription sort_description; diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 7bef168cd90..7359ef335fd 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -55,16 +56,19 @@ void FilterStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings FilterStep::describeActions() const +void FilterStep::describeActions(FormatSettings & settings) const { - Strings res; - res.emplace_back("Filter column: " + filter_column_name); + String prefix(settings.offset, ' '); + settings.out << prefix << "Filter column: " << filter_column_name << '\n'; + bool first = true; for (const auto & action : expression->getActions()) - res.emplace_back((res.size() == 1 ? "Actions: " - : " ") + action.toString()); - - return res; + { + settings.out << prefix << (first ? "Actions: " + : " "); + first = false; + settings.out << action.toString() << '\n'; + } } } diff --git a/src/Processors/QueryPlan/FilterStep.h b/src/Processors/QueryPlan/FilterStep.h index 3f2f8b431f9..baf42bb5dc4 100644 --- a/src/Processors/QueryPlan/FilterStep.h +++ b/src/Processors/QueryPlan/FilterStep.h @@ -19,7 +19,7 @@ public: String getName() const override { return "Filter"; } void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: ExpressionActionsPtr expression; diff --git a/src/Processors/QueryPlan/FinishSortingStep.cpp b/src/Processors/QueryPlan/FinishSortingStep.cpp index bb0b0816374..a0f51198722 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.cpp +++ b/src/Processors/QueryPlan/FinishSortingStep.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -69,17 +70,20 @@ void FinishSortingStep::transformPipeline(QueryPipeline & pipeline) } } -Strings FinishSortingStep::describeActions() const +void FinishSortingStep::describeActions(FormatSettings & settings) const { - Strings res = { - "Prefix sort description: " + dumpSortDescription(prefix_description, input_streams.front().header), - "Result sort description: " + dumpSortDescription(result_description, input_streams.front().header) - }; + String prefix(settings.offset, ' '); + + settings.out << prefix << "Prefix sort description: "; + dumpSortDescription(prefix_description, input_streams.front().header, settings.out); + settings.out << '\n'; + + settings.out << prefix << "Result sort description: "; + dumpSortDescription(result_description, input_streams.front().header, settings.out); + settings.out << '\n'; if (limit) - res.emplace_back("Limit " + std::to_string(limit)); - - return res; + settings.out << prefix << "Limit " << limit << '\n'; } } diff --git a/src/Processors/QueryPlan/FinishSortingStep.h b/src/Processors/QueryPlan/FinishSortingStep.h index cc7dfe1388b..49b64e64c7e 100644 --- a/src/Processors/QueryPlan/FinishSortingStep.h +++ b/src/Processors/QueryPlan/FinishSortingStep.h @@ -19,7 +19,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: SortDescription prefix_description; diff --git a/src/Processors/QueryPlan/IQueryPlanStep.cpp b/src/Processors/QueryPlan/IQueryPlanStep.cpp index e1f44b1e4e4..34639740b6b 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.cpp +++ b/src/Processors/QueryPlan/IQueryPlanStep.cpp @@ -45,7 +45,7 @@ static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep: settings.out << prefix; first = false; - elem.dumpStructure(settings.out, true); + elem.dumpStructure(settings.out); settings.out << '\n'; } } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index f57d47109e5..fcaef1ce9d3 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -65,7 +65,7 @@ public: }; /// Get detailed description of step actions. This is shown in EXPLAIN query with options `actions = 1`. - virtual Strings describeActions() const { return {}; } + virtual void describeActions(FormatSettings & /*settings*/) const {} /// Get description of processors added in current step. Should be called after updatePipeline(). virtual void describePipeline(FormatSettings & /*settings*/) const {} diff --git a/src/Processors/QueryPlan/LimitByStep.cpp b/src/Processors/QueryPlan/LimitByStep.cpp index 05d346f9ae6..ce42ffffc28 100644 --- a/src/Processors/QueryPlan/LimitByStep.cpp +++ b/src/Processors/QueryPlan/LimitByStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -37,23 +38,30 @@ void LimitByStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings LimitByStep::describeActions() const +void LimitByStep::describeActions(FormatSettings & settings) const { - Strings res; - String columns_str; - for (const auto & column : columns) - { - if (!columns_str.empty()) - columns_str += ", "; + String prefix(settings.offset, ' '); - columns_str += column; + settings.out << prefix << "Columns: "; + + if (columns.empty()) + settings.out << "none\n"; + else + { + bool first = true; + for (const auto & column : columns) + { + if (!first) + settings.out << ", "; + first = false; + + settings.out << column; + } + settings.out << '\n'; } - return { - "Columns: " + columns_str, - "Length " + std::to_string(group_length), - "Offset " + std::to_string(group_offset), - }; + settings.out << prefix << "Length " << group_length << '\n'; + settings.out << prefix << "Offset " << group_offset << '\n'; } } diff --git a/src/Processors/QueryPlan/LimitByStep.h b/src/Processors/QueryPlan/LimitByStep.h index b12c85b9179..22d228e89ed 100644 --- a/src/Processors/QueryPlan/LimitByStep.h +++ b/src/Processors/QueryPlan/LimitByStep.h @@ -15,7 +15,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: size_t group_length; @@ -24,5 +24,3 @@ private: }; } - - diff --git a/src/Processors/QueryPlan/LimitStep.cpp b/src/Processors/QueryPlan/LimitStep.cpp index f6c2c2dee44..9e44a2b7f01 100644 --- a/src/Processors/QueryPlan/LimitStep.cpp +++ b/src/Processors/QueryPlan/LimitStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -35,30 +36,30 @@ void LimitStep::transformPipeline(QueryPipeline & pipeline) pipeline.addPipe({std::move(transform)}); } -Strings LimitStep::describeActions() const +void LimitStep::describeActions(FormatSettings & settings) const { - Strings res; - res.emplace_back("Limit " + std::to_string(limit)); - res.emplace_back("Offset " + std::to_string(offset)); + String prefix(settings.offset, ' '); + settings.out << prefix << "Limit " << limit << '\n'; + settings.out << prefix << "Offset " << offset << '\n'; if (with_ties || always_read_till_end) { + settings.out << prefix; + String str; if (with_ties) - str += "WITH TIES"; + settings.out << "WITH TIES"; if (always_read_till_end) { - if (!str.empty()) - str += ", "; + if (!with_ties) + settings.out << ", "; - str += "Reads all data"; + settings.out << "Reads all data"; } - res.emplace_back(str); + settings.out << '\n'; } - - return res; } } diff --git a/src/Processors/QueryPlan/LimitStep.h b/src/Processors/QueryPlan/LimitStep.h index 2bb0f537817..299948f4a5a 100644 --- a/src/Processors/QueryPlan/LimitStep.h +++ b/src/Processors/QueryPlan/LimitStep.h @@ -20,7 +20,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: size_t limit; diff --git a/src/Processors/QueryPlan/MergeSortingStep.cpp b/src/Processors/QueryPlan/MergeSortingStep.cpp index d61a6e721ea..bc5d4af7691 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.cpp +++ b/src/Processors/QueryPlan/MergeSortingStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -48,15 +49,15 @@ void MergeSortingStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings MergeSortingStep::describeActions() const +void MergeSortingStep::describeActions(FormatSettings & settings) const { - Strings res = {"Sort description: " + dumpSortDescription(description, input_streams.front().header)}; + String prefix(settings.offset, ' '); + settings.out << prefix << "Sort description: "; + dumpSortDescription(description, input_streams.front().header, settings.out); + settings.out << '\n'; if (limit) - res.emplace_back("Limit " + std::to_string(limit)); - - return res; + settings.out << prefix << "Limit " << limit << '\n'; } - } diff --git a/src/Processors/QueryPlan/MergeSortingStep.h b/src/Processors/QueryPlan/MergeSortingStep.h index 49645180cf0..ba1066776c4 100644 --- a/src/Processors/QueryPlan/MergeSortingStep.h +++ b/src/Processors/QueryPlan/MergeSortingStep.h @@ -24,7 +24,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: SortDescription description; diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.cpp b/src/Processors/QueryPlan/MergingAggregatedStep.cpp index de085c44b8f..72f696d45ab 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.cpp +++ b/src/Processors/QueryPlan/MergingAggregatedStep.cpp @@ -63,9 +63,9 @@ void MergingAggregatedStep::transformPipeline(QueryPipeline & pipeline) pipeline.enableQuotaForCurrentStreams(); } -Strings MergingAggregatedStep::describeActions() const +void MergingAggregatedStep::describeActions(FormatSettings & settings) const { - return params->params.explain(); + return params->params.explain(settings.out, settings.offset); } } diff --git a/src/Processors/QueryPlan/MergingAggregatedStep.h b/src/Processors/QueryPlan/MergingAggregatedStep.h index 8b22495b5ac..31ef3b75baa 100644 --- a/src/Processors/QueryPlan/MergingAggregatedStep.h +++ b/src/Processors/QueryPlan/MergingAggregatedStep.h @@ -22,7 +22,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: AggregatingTransformParamsPtr params; diff --git a/src/Processors/QueryPlan/MergingSortedStep.cpp b/src/Processors/QueryPlan/MergingSortedStep.cpp index a362a2b0c25..a4a9fc4281f 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.cpp +++ b/src/Processors/QueryPlan/MergingSortedStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -46,14 +47,12 @@ void MergingSortedStep::transformPipeline(QueryPipeline & pipeline) } } -Strings MergingSortedStep::describeActions() const +void MergingSortedStep::describeActions(FormatSettings & settings) const { - Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)}; - - if (limit) - res.emplace_back("Limit " + std::to_string(limit)); - - return res; + String prefix(settings.offset, ' '); + settings.out << prefix << "Sort description: "; + dumpSortDescription(sort_description, input_streams.front().header, settings.out); + settings.out << '\n'; } } diff --git a/src/Processors/QueryPlan/MergingSortedStep.h b/src/Processors/QueryPlan/MergingSortedStep.h index 40803cef0cc..b378abcbe58 100644 --- a/src/Processors/QueryPlan/MergingSortedStep.h +++ b/src/Processors/QueryPlan/MergingSortedStep.h @@ -20,7 +20,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: SortDescription sort_description; diff --git a/src/Processors/QueryPlan/OffsetStep.cpp b/src/Processors/QueryPlan/OffsetStep.cpp index 27da67c2be2..9e8a1b2c48f 100644 --- a/src/Processors/QueryPlan/OffsetStep.cpp +++ b/src/Processors/QueryPlan/OffsetStep.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -30,9 +31,9 @@ void OffsetStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings OffsetStep::describeActions() const +void OffsetStep::describeActions(FormatSettings & settings) const { - return {"Offset " + std::to_string(offset)}; + settings.out << String(settings.offset, ' ') << "Offset " << offset << '\n'; } } diff --git a/src/Processors/QueryPlan/OffsetStep.h b/src/Processors/QueryPlan/OffsetStep.h index 167b65f6fd8..38b92b79f9a 100644 --- a/src/Processors/QueryPlan/OffsetStep.h +++ b/src/Processors/QueryPlan/OffsetStep.h @@ -14,7 +14,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: size_t offset; diff --git a/src/Processors/QueryPlan/PartialSortingStep.cpp b/src/Processors/QueryPlan/PartialSortingStep.cpp index 767eebf6f01..cbc1b4b1b90 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.cpp +++ b/src/Processors/QueryPlan/PartialSortingStep.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB { @@ -50,14 +51,12 @@ void PartialSortingStep::transformPipeline(QueryPipeline & pipeline) }); } -Strings PartialSortingStep::describeActions() const +void PartialSortingStep::describeActions(FormatSettings & settings) const { - Strings res = {"Sort description: " + dumpSortDescription(sort_description, input_streams.front().header)}; - - if (limit) - res.emplace_back("Limit " + std::to_string(limit)); - - return res; + String prefix(settings.offset, ' '); + settings.out << prefix << "Sort description: "; + dumpSortDescription(sort_description, input_streams.front().header, settings.out); + settings.out << '\n'; } } diff --git a/src/Processors/QueryPlan/PartialSortingStep.h b/src/Processors/QueryPlan/PartialSortingStep.h index fd2526c23aa..9308ee83906 100644 --- a/src/Processors/QueryPlan/PartialSortingStep.h +++ b/src/Processors/QueryPlan/PartialSortingStep.h @@ -19,7 +19,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: SortDescription sort_description; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 53e97d6715c..2d3936d109a 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -178,60 +178,55 @@ void QueryPlan::addInterpreterContext(std::shared_ptr context) static void explainStep( - WriteBuffer & buffer, IQueryPlanStep & step, size_t ident, const QueryPlan::ExplainPlanOptions & options) + const IQueryPlanStep & step, + IQueryPlanStep::FormatSettings & settings, + const QueryPlan::ExplainPlanOptions & options) { - std::string prefix(ident, ' '); - buffer << prefix; - buffer << step.getName(); + std::string prefix(settings.offset, ' '); + settings.out << prefix; + settings.out << step.getName(); const auto & description = step.getStepDescription(); if (options.description && !description.empty()) - buffer <<" (" << description << ')'; + settings.out <<" (" << description << ')'; - buffer.write('\n'); + settings.out.write('\n'); if (options.header) { - buffer << prefix; + settings.out << prefix; if (!step.hasOutputStream()) - buffer << "No header"; + settings.out << "No header"; else if (!step.getOutputStream().header) - buffer << "Empty header"; + settings.out << "Empty header"; else { - buffer << "Header: "; + settings.out << "Header: "; bool first = true; for (const auto & elem : step.getOutputStream().header) { if (!first) - buffer << "\n" << prefix << " "; + settings.out << "\n" << prefix << " "; first = false; - elem.dumpStructure(buffer, true); + elem.dumpStructure(settings.out); } } - buffer.write('\n'); + settings.out.write('\n'); } if (options.actions) - { - auto actions = step.describeActions(); - if (!actions.empty()) - { - for (auto & action : actions) - buffer << prefix << action << '\n'; - } - } + step.describeActions(settings); } void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & options) { checkInitialized(); - size_t ident = 2; + IQueryPlanStep::FormatSettings settings{.out = buffer, .write_header = options.header}; struct Frame { @@ -249,7 +244,8 @@ void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & opt if (!frame.is_description_printed) { - explainStep(buffer, *frame.node->step, (stack.size() - 1) * ident, options); + settings.offset = (stack.size() - 1) * settings.ident; + explainStep(*frame.node->step, settings, options); frame.is_description_printed = true; } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 70b35ee7f94..fd658e9eb70 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -62,17 +63,20 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr __builtin_unreachable(); } -Strings TotalsHavingStep::describeActions() const +void TotalsHavingStep::describeActions(FormatSettings & settings) const { - Strings res; - res.emplace_back("Filter column: " + filter_column_name); - res.emplace_back("Mode: " + totalsModeToString(totals_mode, auto_include_threshold)); + String prefix(settings.offset, ' '); + settings.out << prefix << "Filter column: " << filter_column_name << '\n'; + settings.out << prefix << "Mode: " << totalsModeToString(totals_mode, auto_include_threshold) << '\n'; + bool first = true; for (const auto & action : expression->getActions()) - res.emplace_back((res.size() == 2 ? "Actions: " - : " ") + action.toString()); - - return res; + { + settings.out << prefix << (first ? "Actions: " + : " "); + first = false; + settings.out << action.toString() << '\n'; + } } } diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index e2dd2f4dd55..c50278cf0f3 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -25,7 +25,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; - Strings describeActions() const override; + void describeActions(FormatSettings & settings) const override; private: bool overflow_row; From ff47f8303453adcbce1044ae7deb66c075f25ba8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 27 Jun 2020 17:18:23 +0300 Subject: [PATCH 052/196] Refactor some code. --- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 322b298f181..ad2c77e37a3 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -225,7 +225,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() if (ast.getSettings()) throw Exception("Settings are not supported for EXPLAIN AST query.", ErrorCodes::UNKNOWN_SETTING); - dumpAST(ast, ss); + dumpAST(*ast.getExplainedQuery(), ss); } else if (ast.getKind() == ASTExplainQuery::AnalyzedSyntax) { From 90fb5716cfbe47716602183147605a8dc55da84e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 27 Jun 2020 18:30:19 +0300 Subject: [PATCH 053/196] Fix build. --- src/Interpreters/InterpreterExplainQuery.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index ad2c77e37a3..cf74be26fc8 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -150,16 +150,16 @@ struct ExplainSettings : public Settings { using Settings::boolean_settings; - bool has(const std::string & name) const + bool has(const std::string & name_) const { - return boolean_settings.count(name) > 0; + return boolean_settings.count(name_) > 0; } - void setBooleanSetting(const std::string & name, bool value) + void setBooleanSetting(const std::string & name_, bool value) { - auto it = boolean_settings.find(name); + auto it = boolean_settings.find(name_); if (it == boolean_settings.end()) - throw Exception("Unknown setting for ExplainSettings: " + name, ErrorCodes::LOGICAL_ERROR); + throw Exception("Unknown setting for ExplainSettings: " + name_, ErrorCodes::LOGICAL_ERROR); it->second.get() = value; } From 897ed2f3a11c6dd6cc369778e1e4a5c28ef1116a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 27 Jun 2020 18:30:54 +0300 Subject: [PATCH 054/196] Fix tests. --- .../01259_combinator_distinct.reference | 6 +- ...ithmetic_operations_in_aggr_func.reference | 504 ++++++++++++++++-- 2 files changed, 460 insertions(+), 50 deletions(-) diff --git a/tests/queries/0_stateless/01259_combinator_distinct.reference b/tests/queries/0_stateless/01259_combinator_distinct.reference index 94e4f7d0d17..3196d78ee44 100644 --- a/tests/queries/0_stateless/01259_combinator_distinct.reference +++ b/tests/queries/0_stateless/01259_combinator_distinct.reference @@ -10,6 +10,8 @@ 24 27 2 -SELECT uniqExactIf(number % 10, (number % 5) = 2)\nFROM numbers(10000) +SELECT uniqExactIf(number % 10, (number % 5) = 2) +FROM numbers(10000) 9 -SELECT sumDistinctIf(number % 10, (number % 5) = 2)\nFROM numbers(10000) +SELECT sumDistinctIf(number % 10, (number % 5) = 2) +FROM numbers(10000) diff --git a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference index ba265f2a7ce..eeba2646046 100644 --- a/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference +++ b/tests/queries/0_stateless/01271_optimize_arithmetic_operations_in_aggr_func.reference @@ -1,51 +1,459 @@ -SELECT \n sum(n + 1),\n sum(1 + n),\n sum(n - 1),\n sum(1 - n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(n) * 2,\n 2 * sum(n),\n sum(n) / 2,\n sum(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(n) + 1,\n 1 + min(n),\n min(n) - 1,\n 1 - min(n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(n) * 2,\n 2 * min(n),\n min(n) / 2,\n min(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(n) + 1,\n 1 + max(n),\n max(n) - 1,\n 1 - max(n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(n) * 2,\n 2 * max(n),\n max(n) / 2,\n max(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(n + -1),\n sum(-1 + n),\n sum(n - -1),\n sum(-1 - n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(n) * -2,\n -2 * sum(n),\n sum(n) / -2,\n sum(-1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(n) + -1,\n -1 + min(n),\n min(n) - -1,\n -1 - min(n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(n) * -2,\n -2 * max(n),\n max(n) / -2,\n min(-1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(n) + -1,\n -1 + max(n),\n max(n) - -1,\n -1 - max(n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(n) * -2,\n -2 * min(n),\n min(n) / -2,\n max(-1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(abs(2) + 1),\n sum(abs(2) + n),\n sum(n - abs(2)),\n sum(1 - abs(2))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(abs(2)) * 2,\n sum(abs(2) * n),\n sum(n / abs(2)),\n sum(1 / abs(2))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(abs(2)) + 1,\n min(abs(2) + n),\n min(n - abs(2)),\n 1 - min(abs(2))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(abs(2)) * 2,\n min(abs(2) * n),\n min(n / abs(2)),\n min(1 / abs(2))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(abs(2)) + 1,\n max(abs(2) + n),\n max(n - abs(2)),\n 1 - max(abs(2))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(abs(2)) * 2,\n max(abs(2) * n),\n max(n / abs(2)),\n max(1 / abs(2))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(abs(n) + 1),\n sum(abs(n) + n),\n sum(n - abs(n)),\n sum(1 - abs(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(abs(n)) * 2,\n sum(abs(n) * n),\n sum(n / abs(n)),\n sum(1 / abs(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(abs(n)) + 1,\n min(abs(n) + n),\n min(n - abs(n)),\n 1 - min(abs(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(abs(n)) * 2,\n min(abs(n) * n),\n min(n / abs(n)),\n min(1 / abs(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(abs(n)) + 1,\n max(abs(n) + n),\n max(n - abs(n)),\n 1 - max(abs(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(abs(n)) * 2,\n max(abs(n) * n),\n max(n / abs(n)),\n max(1 / abs(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum((n * n) + 1),\n sum(1 + (n * n)),\n sum((n * n) - 1),\n sum(1 - (n * n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(n * n) * 2,\n sum((2 * n) * n),\n sum(n * n) / 2,\n sum((1 / n) * n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(n * n) + 1,\n 1 + min(n * n),\n min(n * n) - 1,\n 1 - min(n * n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n min(n * n) * 2,\n min((2 * n) * n),\n min(n * n) / 2,\n min((1 / n) * n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(n * n) + 1,\n 1 + max(n * n),\n max(n * n) - 1,\n 1 - max(n * n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n max(n * n) * 2,\n max((2 * n) * n),\n max(n * n) / 2,\n max((1 / n) * n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum((1 + n) + 1),\n sum((1 + 1) + n),\n sum((1 + n) - 1),\n sum((1 + 1) - n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum(1 + (n * 2)),\n sum(1 + (2 * n)),\n sum(1 + (n / 2)),\n sum(1 + (1 / n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n (1 + min(n)) + 1,\n min((1 + 1) + n),\n (1 + min(n)) - 1,\n min((1 + 1) - n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n 1 + min(n * 2),\n 1 + min(2 * n),\n 1 + min(n / 2),\n 1 + min(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n (1 + max(n)) + 1,\n max((1 + 1) + n),\n (1 + max(n)) - 1,\n max((1 + 1) - n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n 1 + max(n * 2),\n 1 + max(2 * n),\n 1 + max(n / 2),\n 1 + max(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n sum((n + -1) + -1),\n sum((-1 + n) + -1),\n sum((n - -1) + -1),\n sum((-1 - n) + -1)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n (sum(n) * -2) * -1,\n (-2 * sum(n)) * -1,\n (sum(n) / -2) / -1,\n sum(-1 / n) / -1\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n (min(n) + -1) + -1,\n (-1 + min(n)) + -1,\n (min(n) - -1) + -1,\n (-1 - min(n)) + -1\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n (min(n) * -2) * -1,\n (-2 * min(n)) * -1,\n (min(n) / -2) / -1,\n max(-1 / n) / -1\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n (max(n) + -1) + -1,\n (-1 + max(n)) + -1,\n (max(n) - -1) + -1,\n (-1 - max(n)) + -1\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT \n (max(n) * -2) * -1,\n (-2 * max(n)) * -1,\n (max(n) / -2) / -1,\n min(-1 / n) / -1\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT ((sum(n + 1) + sum(1 + n)) + sum(n - 1)) + sum(1 - n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT (((sum(n) * 2) + (2 * sum(n))) + (sum(n) / 2)) + sum(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT (((min(n) + 1) + (1 + min(n))) + (min(n) - 1)) + (1 - min(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT (((min(n) * 2) + (2 * min(n))) + (min(n) / 2)) + min(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT (((max(n) + 1) + (1 + max(n))) + (max(n) - 1)) + (1 - max(n))\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) -SELECT (((max(n) * 2) + (2 * max(n))) + (max(n) / 2)) + max(1 / n)\nFROM \n(\n SELECT number AS n\n FROM numbers(10)\n) +SELECT + sum(n + 1), + sum(1 + n), + sum(n - 1), + sum(1 - n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(n) * 2, + 2 * sum(n), + sum(n) / 2, + sum(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(n) + 1, + 1 + min(n), + min(n) - 1, + 1 - min(n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(n) * 2, + 2 * min(n), + min(n) / 2, + min(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(n) + 1, + 1 + max(n), + max(n) - 1, + 1 - max(n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(n) * 2, + 2 * max(n), + max(n) / 2, + max(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(n + -1), + sum(-1 + n), + sum(n - -1), + sum(-1 - n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(n) * -2, + -2 * sum(n), + sum(n) / -2, + sum(-1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(n) + -1, + -1 + min(n), + min(n) - -1, + -1 - min(n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(n) * -2, + -2 * max(n), + max(n) / -2, + min(-1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(n) + -1, + -1 + max(n), + max(n) - -1, + -1 - max(n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(n) * -2, + -2 * min(n), + min(n) / -2, + max(-1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(abs(2) + 1), + sum(abs(2) + n), + sum(n - abs(2)), + sum(1 - abs(2)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(abs(2)) * 2, + sum(abs(2) * n), + sum(n / abs(2)), + sum(1 / abs(2)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(abs(2)) + 1, + min(abs(2) + n), + min(n - abs(2)), + 1 - min(abs(2)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(abs(2)) * 2, + min(abs(2) * n), + min(n / abs(2)), + min(1 / abs(2)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(abs(2)) + 1, + max(abs(2) + n), + max(n - abs(2)), + 1 - max(abs(2)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(abs(2)) * 2, + max(abs(2) * n), + max(n / abs(2)), + max(1 / abs(2)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(abs(n) + 1), + sum(abs(n) + n), + sum(n - abs(n)), + sum(1 - abs(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(abs(n)) * 2, + sum(abs(n) * n), + sum(n / abs(n)), + sum(1 / abs(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(abs(n)) + 1, + min(abs(n) + n), + min(n - abs(n)), + 1 - min(abs(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(abs(n)) * 2, + min(abs(n) * n), + min(n / abs(n)), + min(1 / abs(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(abs(n)) + 1, + max(abs(n) + n), + max(n - abs(n)), + 1 - max(abs(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(abs(n)) * 2, + max(abs(n) * n), + max(n / abs(n)), + max(1 / abs(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum((n * n) + 1), + sum(1 + (n * n)), + sum((n * n) - 1), + sum(1 - (n * n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(n * n) * 2, + sum((2 * n) * n), + sum(n * n) / 2, + sum((1 / n) * n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(n * n) + 1, + 1 + min(n * n), + min(n * n) - 1, + 1 - min(n * n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + min(n * n) * 2, + min((2 * n) * n), + min(n * n) / 2, + min((1 / n) * n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(n * n) + 1, + 1 + max(n * n), + max(n * n) - 1, + 1 - max(n * n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + max(n * n) * 2, + max((2 * n) * n), + max(n * n) / 2, + max((1 / n) * n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum((1 + n) + 1), + sum((1 + 1) + n), + sum((1 + n) - 1), + sum((1 + 1) - n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum(1 + (n * 2)), + sum(1 + (2 * n)), + sum(1 + (n / 2)), + sum(1 + (1 / n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + (1 + min(n)) + 1, + min((1 + 1) + n), + (1 + min(n)) - 1, + min((1 + 1) - n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + 1 + min(n * 2), + 1 + min(2 * n), + 1 + min(n / 2), + 1 + min(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + (1 + max(n)) + 1, + max((1 + 1) + n), + (1 + max(n)) - 1, + max((1 + 1) - n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + 1 + max(n * 2), + 1 + max(2 * n), + 1 + max(n / 2), + 1 + max(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + sum((n + -1) + -1), + sum((-1 + n) + -1), + sum((n - -1) + -1), + sum((-1 - n) + -1) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + (sum(n) * -2) * -1, + (-2 * sum(n)) * -1, + (sum(n) / -2) / -1, + sum(-1 / n) / -1 +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + (min(n) + -1) + -1, + (-1 + min(n)) + -1, + (min(n) - -1) + -1, + (-1 - min(n)) + -1 +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + (min(n) * -2) * -1, + (-2 * min(n)) * -1, + (min(n) / -2) / -1, + max(-1 / n) / -1 +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + (max(n) + -1) + -1, + (-1 + max(n)) + -1, + (max(n) - -1) + -1, + (-1 - max(n)) + -1 +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT + (max(n) * -2) * -1, + (-2 * max(n)) * -1, + (max(n) / -2) / -1, + min(-1 / n) / -1 +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT ((sum(n + 1) + sum(1 + n)) + sum(n - 1)) + sum(1 - n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT (((sum(n) * 2) + (2 * sum(n))) + (sum(n) / 2)) + sum(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT (((min(n) + 1) + (1 + min(n))) + (min(n) - 1)) + (1 - min(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT (((min(n) * 2) + (2 * min(n))) + (min(n) / 2)) + min(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT (((max(n) + 1) + (1 + max(n))) + (max(n) - 1)) + (1 - max(n)) +FROM +( + SELECT number AS n + FROM numbers(10) +) +SELECT (((max(n) * 2) + (2 * max(n))) + (max(n) / 2)) + max(1 / n) +FROM +( + SELECT number AS n + FROM numbers(10) +) 55 55 35 -35 90 90 22.5 inf 1 1 -1 1 From c6f5a4e07480c2b9042108fe1ecce37991cc2880 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 27 Jun 2020 19:17:03 +0300 Subject: [PATCH 055/196] Update questdb_sum_double.xml --- tests/performance/questdb_sum_double.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/questdb_sum_double.xml b/tests/performance/questdb_sum_double.xml index d865fe9782f..c65f5f9e464 100644 --- a/tests/performance/questdb_sum_double.xml +++ b/tests/performance/questdb_sum_double.xml @@ -4,7 +4,7 @@ 20G 1 2000000000 - 10000000000 + 10G From a43cb93be537a596c63a9e18ffbbb4c148ff3710 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 29 Jun 2020 18:46:50 +0300 Subject: [PATCH 056/196] remove questionable functionality --- .../MergeTree/MergeTreeBlockOutputStream.cpp | 13 ------------ .../MergeTree/MergeTreeBlockOutputStream.h | 4 +--- .../MergeTree/MergeTreeDataPartInMemory.cpp | 15 -------------- .../MergeTree/MergeTreeDataPartInMemory.h | 3 --- src/Storages/MergeTree/MergeTreeSettings.h | 1 - .../ReplicatedMergeTreeBlockOutputStream.cpp | 10 ---------- .../ReplicatedMergeTreeBlockOutputStream.h | 2 -- src/Storages/StorageMergeTree.cpp | 20 +------------------ src/Storages/StorageReplicatedMergeTree.cpp | 17 +--------------- 9 files changed, 3 insertions(+), 82 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 6cb79296b09..7457feac207 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -23,14 +23,12 @@ void MergeTreeBlockOutputStream::write(const Block & block) storage.delayInsertOrThrowIfNeeded(); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot); - MergeTreeData::DataPartsVector inserted_parts; for (auto & current_block : part_blocks) { Stopwatch watch; MergeTreeData::MutableDataPartPtr part = storage.writer.writeTempPart(current_block, metadata_snapshot); storage.renameTempPartAndAdd(part, &storage.increment); - inserted_parts.push_back(part); PartLog::addNewPart(storage.global_context, part, watch.elapsed()); @@ -50,17 +48,6 @@ void MergeTreeBlockOutputStream::write(const Block & block) storage.merging_mutating_task_handle->signalReadyToRun(); } } - - if (storage.getSettings()->in_memory_parts_insert_sync) - { - for (const auto & part : inserted_parts) - { - auto part_in_memory = asInMemoryPart(part); - if (!part_in_memory->waitUntilMerged(in_memory_parts_timeout)) - throw Exception("Timeout exceeded while waiting to write part " - + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); - } - } } } diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h index 3faed068230..8aae7f3e625 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -14,11 +14,10 @@ class StorageMergeTree; class MergeTreeBlockOutputStream : public IBlockOutputStream { public: - MergeTreeBlockOutputStream(StorageMergeTree & storage_, const StorageMetadataPtr metadata_snapshot_, size_t max_parts_per_block_, size_t in_memory_parts_timeout_) + MergeTreeBlockOutputStream(StorageMergeTree & storage_, const StorageMetadataPtr metadata_snapshot_, size_t max_parts_per_block_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , max_parts_per_block(max_parts_per_block_) - , in_memory_parts_timeout(in_memory_parts_timeout_) { } @@ -29,7 +28,6 @@ private: StorageMergeTree & storage; StorageMetadataPtr metadata_snapshot; size_t max_parts_per_block; - size_t in_memory_parts_timeout; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 2efba227f70..9b2a8f6a88a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -98,21 +98,6 @@ void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix, const flushToDisk(storage.getRelativeDataPath(), detached_path, metadata_snapshot); } -bool MergeTreeDataPartInMemory::waitUntilMerged(size_t timeout_ms) const -{ - auto lock = storage.lockParts(); - return is_merged.wait_for(lock, std::chrono::milliseconds(timeout_ms), - [this]() { return state != State::Committed; }); -} - -void MergeTreeDataPartInMemory::notifyMerged() const -{ - LOG_DEBUG(&Poco::Logger::get("InMemPart"), "notifiedMerged"); - LOG_DEBUG(&Poco::Logger::get("InMemPart"), "state {}", stateString()); - - is_merged.notify_all(); -} - void MergeTreeDataPartInMemory::renameTo(const String & new_relative_path, bool /* remove_new_dir_if_exists */) const { relative_path = new_relative_path; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index eb053d3dc9a..db1f12a6f63 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -46,9 +46,6 @@ public: void flushToDisk(const String & base_path, const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const; - bool waitUntilMerged(size_t timeout_ms) const; - void notifyMerged() const; - /// Returns hash of parts's block Checksum calculateBlockChecksum() const; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 87a31a21cf4..98a9e24db2b 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -34,7 +34,6 @@ struct MergeTreeSettings : public SettingsCollection M(SettingUInt64, min_bytes_for_compact_part, 0, "Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ M(SettingUInt64, min_rows_for_compact_part, 0, "Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(SettingBool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ - M(SettingBool, in_memory_parts_insert_sync, false, "If true and in-memory parts are enabled, insert will wait while part will persist on disk in result of merge", 0) \ M(SettingUInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ \ /** Merge settings. */ \ diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index a8fadc34a84..a06feca4033 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -36,14 +36,12 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, - size_t insert_in_memory_parts_timeout_ms_, bool deduplicate_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , quorum(quorum_) , quorum_timeout_ms(quorum_timeout_ms_) , max_parts_per_block(max_parts_per_block_) - , insert_in_memory_parts_timeout_ms(insert_in_memory_parts_timeout_ms_) , deduplicate(deduplicate_) , log(&Poco::Logger::get(storage.getLogName() + " (Replicated OutputStream)")) { @@ -378,14 +376,6 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } - auto part_in_memory = asInMemoryPart(part); - if (part_in_memory && storage.getSettings()->in_memory_parts_insert_sync) - { - if (!part_in_memory->waitUntilMerged(insert_in_memory_parts_timeout_ms)) - throw Exception("Timeout exceeded while waiting to write part " - + part->name + " on disk", ErrorCodes::TIMEOUT_EXCEEDED); - } - if (quorum) { /// We are waiting for quorum to be satisfied. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 7af18bbf52b..ac169d248c2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -28,7 +28,6 @@ public: size_t quorum_, size_t quorum_timeout_ms_, size_t max_parts_per_block_, - size_t insert_in_memory_parts_timeout_ms_, bool deduplicate_); Block getHeader() const override; @@ -64,7 +63,6 @@ private: size_t quorum; size_t quorum_timeout_ms; size_t max_parts_per_block; - size_t insert_in_memory_parts_timeout_ms; bool deduplicate = true; bool last_block_is_duplicate = false; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index e5a5fcdb091..68cae13c85c 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -205,8 +205,7 @@ BlockOutputStreamPtr StorageMergeTree::write(const ASTPtr & /*query*/, const Sto const auto & settings = context.getSettingsRef(); return std::make_shared( - *this, metadata_snapshot, settings.max_partitions_per_insert_block, - settings.insert_in_memory_parts_timeout.totalMilliseconds()); + *this, metadata_snapshot, settings.max_partitions_per_insert_block); } void StorageMergeTree::checkTableCanBeDropped() const @@ -696,23 +695,6 @@ bool StorageMergeTree::merge( merging_tagger->reserved_space, deduplicate, force_ttl); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); - - DataPartsVector parts_to_remove_immediately; - { - auto lock = lockParts(); - for (const auto & part : future_part.parts) - { - if (auto part_in_memory = asInMemoryPart(part)) - { - part_in_memory->notifyMerged(); - modifyPartState(part_in_memory, DataPartState::Deleting); - parts_to_remove_immediately.push_back(part_in_memory); - } - } - } - - removePartsFinally(parts_to_remove_immediately); - merging_tagger->is_successful = true; write_part_log({}); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a4e3c2db43c..81142bea1ca 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1371,20 +1371,6 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) throw; } - DataPartsVector parts_to_remove_immediatly; - for (const auto & part_ptr : parts) - { - if (auto part_in_memory = asInMemoryPart(part_ptr)) - { - modifyPartState(part_in_memory, DataPartState::Deleting); - part_in_memory->notifyMerged(); - parts_to_remove_immediatly.push_back(part_in_memory); - } - } - - tryRemovePartsFromZooKeeperWithRetries(parts_to_remove_immediatly); - removePartsFinally(parts_to_remove_immediatly); - /** Removing old parts from ZK and from the disk is delayed - see ReplicatedMergeTreeCleanupThread, clearOldParts. */ @@ -3490,7 +3476,6 @@ BlockOutputStreamPtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, *this, metadata_snapshot, query_settings.insert_quorum, query_settings.insert_quorum_timeout.totalMilliseconds(), query_settings.max_partitions_per_insert_block, - query_settings.insert_in_memory_parts_timeout.totalMilliseconds(), deduplicate); } @@ -4045,7 +4030,7 @@ void StorageReplicatedMergeTree::attachPartition(const ASTPtr & partition, const PartsTemporaryRename renamed_parts(*this, "detached/"); MutableDataPartsVector loaded_parts = tryLoadPartsToAttach(partition, attach_part, query_context, renamed_parts); - ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, 0, false); /// TODO Allow to use quorum here. + ReplicatedMergeTreeBlockOutputStream output(*this, metadata_snapshot, 0, 0, 0, false); /// TODO Allow to use quorum here. for (size_t i = 0; i < loaded_parts.size(); ++i) { String old_name = loaded_parts[i]->name; From b3a7a2777979043c48a7e2b725bff57dccaa2b49 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 29 Jun 2020 23:23:34 +0300 Subject: [PATCH 057/196] Update arrows. --- src/Interpreters/AggregateDescription.cpp | 2 +- src/Processors/QueryPlan/ConvertingStep.cpp | 2 +- src/Processors/QueryPlan/IQueryPlanStep.cpp | 6 +++--- src/Processors/printPipeline.cpp | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp index 35057516f91..f6f2e9287a7 100644 --- a/src/Interpreters/AggregateDescription.cpp +++ b/src/Interpreters/AggregateDescription.cpp @@ -49,7 +49,7 @@ void AggregateDescription::explain(WriteBuffer & out, size_t ident) const out << type->getName(); } - out << ")\n"; + out << ") → " << function->getReturnType()->getName() << "\n"; } else out << prefix << " Function: nullptr\n"; diff --git a/src/Processors/QueryPlan/ConvertingStep.cpp b/src/Processors/QueryPlan/ConvertingStep.cpp index 79e230445e5..f63bfc8095d 100644 --- a/src/Processors/QueryPlan/ConvertingStep.cpp +++ b/src/Processors/QueryPlan/ConvertingStep.cpp @@ -75,7 +75,7 @@ void ConvertingStep::describeActions(FormatSettings & settings) const else { dump_description(to, to_const); - settings.out << " <- "; + settings.out << " ← "; dump_description(from, from_const); } diff --git a/src/Processors/QueryPlan/IQueryPlanStep.cpp b/src/Processors/QueryPlan/IQueryPlanStep.cpp index 34639740b6b..ec75564e175 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.cpp +++ b/src/Processors/QueryPlan/IQueryPlanStep.cpp @@ -24,7 +24,7 @@ static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep: prefix += "Header"; if (count > 1) - prefix += " x " + std::to_string(count) + " "; + prefix += " × " + std::to_string(count) + " "; prefix += ": "; @@ -54,12 +54,12 @@ static void doDescribeProcessor(const IProcessor & processor, size_t count, IQue { settings.out << String(settings.offset, settings.ident_char) << processor.getName(); if (count > 1) - settings.out << " x " << std::to_string(count); + settings.out << " × " << std::to_string(count); size_t num_inputs = processor.getInputs().size(); size_t num_outputs = processor.getOutputs().size(); if (num_inputs != 1 || num_outputs != 1) - settings.out << " " << std::to_string(num_inputs) << " -> " << std::to_string(num_outputs); + settings.out << " " << std::to_string(num_inputs) << " → " << std::to_string(num_outputs); settings.out << '\n'; diff --git a/src/Processors/printPipeline.cpp b/src/Processors/printPipeline.cpp index ee7231c45fb..5cdab1ed3ff 100644 --- a/src/Processors/printPipeline.cpp +++ b/src/Processors/printPipeline.cpp @@ -128,7 +128,7 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool out << " n" << node->id << " [label=\"" << processor->getName(); if (node->agents.size() > 1) - out << " x " << node->agents.size(); + out << " × " << node->agents.size(); const auto & description = processor->getDescription(); if (!description.empty()) @@ -156,7 +156,7 @@ void printPipelineCompact(const Processors & processors, WriteBuffer & out, bool out << " n" << item.second.id << " -> " << "n" << edge.first->id << " [label=\""; if (data.count > 1) - out << "x " << data.count; + out << "× " << data.count; if (with_header) { From 53e955c6dde16f5e8ab769a87bc770b2bb5aefc4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 29 Jun 2020 23:36:18 +0300 Subject: [PATCH 058/196] several fixes --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++-- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartCompact.h | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 3 +- .../MergeTree/MergeTreeDataPartInMemory.h | 4 +-- .../MergeTree/MergeTreeDataPartWide.cpp | 2 +- .../MergeTree/MergeTreeDataPartWide.h | 2 +- .../test_polymorphic_parts/test.py | 28 ------------------- 10 files changed, 13 insertions(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 977874542e8..e5f9273123c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -903,7 +903,7 @@ void IMergeTreeDataPart::calculateColumnsSizesOnDisk() if (getColumns().empty() || checksums.empty()) throw Exception("Cannot calculate columns sizes when columns or checksums are not initialized", ErrorCodes::LOGICAL_ERROR); - calculateEachColumnSizesOnDisk(columns_sizes, total_columns_size); + calculateEachColumnSizes(columns_sizes, total_columns_size); } ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const IDataType & /* type */) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 0f0b193464a..a4784767047 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -351,7 +351,7 @@ protected: /// Fill each_columns_size and total_size with sizes from columns files on /// disk using columns and checksums. - virtual void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0; + virtual void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const = 0; String getRelativePathForDetachedPart(const String & prefix) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f4511219b04..633e9b4d05d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -932,8 +932,8 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) if (settings->in_memory_parts_enable_wal && !write_ahead_log) { - auto disk = makeEmptyReservationOnLargestDisk()->getDisk(); - write_ahead_log = std::make_shared(*this, std::move(disk)); + auto reservation = reserveSpace(settings->write_ahead_log_max_bytes); + write_ahead_log = std::make_shared(*this, reservation->getDisk()); } calculateColumnSizesImpl(); @@ -1020,7 +1020,8 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) if (part.unique() && /// Grab only parts that are not used by anyone (SELECTs for example). ((part_remove_time < now && - now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) || force)) + now - part_remove_time > getSettings()->old_parts_lifetime.totalSeconds()) || force + || isInMemoryPart(part))) /// Remove in-memory parts immediatly to not store excessive data in RAM { parts_to_delete.emplace_back(it); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index adb8146572e..972d8e8a435 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -75,7 +75,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( } -void MergeTreeDataPartCompact::calculateEachColumnSizesOnDisk(ColumnSizeByName & /*each_columns_size*/, ColumnSize & total_size) const +void MergeTreeDataPartCompact::calculateEachColumnSizes(ColumnSizeByName & /*each_columns_size*/, ColumnSize & total_size) const { auto bin_checksum = checksums.files.find(DATA_FILE_NAME_WITH_EXTENSION); if (bin_checksum != checksums.files.end()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 7d5c8628570..2f2a2f537aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -68,7 +68,7 @@ private: void loadIndexGranularity() override; /// Compact parts doesn't support per column size, only total size - void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 9b2a8f6a88a..42672511166 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -103,8 +103,7 @@ void MergeTreeDataPartInMemory::renameTo(const String & new_relative_path, bool relative_path = new_relative_path; } -/// Calculates uncompressed sizes in memory. -void MergeTreeDataPartInMemory::calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const +void MergeTreeDataPartInMemory::calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const { auto it = checksums.files.find("data.bin"); if (it != checksums.files.end()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index db1f12a6f63..f90de0195e2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -54,8 +54,8 @@ public: private: mutable std::condition_variable is_merged; - /// Compact parts doesn't support per column size, only total size - void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + /// Calculates uncompressed sizes in memory. + void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; }; using DataPartInMemoryPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index e9383bc917d..b33a3d4645d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -229,7 +229,7 @@ String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & colum return filename; } -void MergeTreeDataPartWide::calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const +void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const { std::unordered_set processed_substreams; for (const NameAndTypePair & column : columns) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 8d8b6fa678b..aa8c3aedea7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -64,7 +64,7 @@ private: ColumnSize getColumnSizeImpl(const String & name, const IDataType & type, std::unordered_set * processed_substreams) const; - void calculateEachColumnSizesOnDisk(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; + void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; }; } diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 7a282a7e09c..072223e6380 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -437,34 +437,6 @@ def test_in_memory_deduplication(start_cluster): assert node9.query("SELECT date, id, s FROM deduplication_table") == "2020-03-03\t1\tfoo\n" assert node10.query("SELECT date, id, s FROM deduplication_table") == "2020-03-03\t1\tfoo\n" -def test_in_memory_sync_insert(start_cluster): - node9.query("ALTER TABLE sync_table MODIFY SETTING in_memory_parts_insert_sync = 1") - node10.query("ALTER TABLE sync_table MODIFY SETTING in_memory_parts_insert_sync = 1") - node9.query("SYSTEM STOP MERGES sync_table") - node10.query("SYSTEM STOP MERGES sync_table") - - pool = Pool(5) - tasks = [] - for i in range(5): - tasks.append(pool.apply_async(insert_random_data, ('sync_table', node9, 50))) - - time.sleep(5) - assert node9.query("SELECT count() FROM sync_table") == "250\n" - assert node9.query("SELECT part_type, count() FROM system.parts WHERE table = 'sync_table' AND active GROUP BY part_type") == "InMemory\t5\n" - - for task in tasks: - assert not task.ready() - - node9.query("SYSTEM START MERGES sync_table") - node10.query("SYSTEM START MERGES sync_table") - assert_eq_with_retry(node9, "OPTIMIZE TABLE sync_table FINAL SETTINGS optimize_throw_if_noop = 1", "") - - for task in tasks: - task.get() - - assert node9.query("SELECT count() FROM sync_table") == "250\n" - assert node9.query("SELECT part_type, count() FROM system.parts WHERE table = 'sync_table' AND active GROUP BY part_type") == "Compact\t1\n" - # Checks that restoring from WAL works after table schema changed def test_in_memory_alters(start_cluster): def check_parts_type(parts_num): From 5c417f45b8956060bfc6ebda442d305a660db4da Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 30 Jun 2020 12:25:23 +0300 Subject: [PATCH 059/196] streaming wip --- programs/client/Client.cpp | 94 ++++++++++++++++++++++++++++++++++---- src/Client/Connection.h | 2 + 2 files changed, 87 insertions(+), 9 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index c06944fe4a3..ee98d6c3335 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -41,6 +41,7 @@ #include #include #include +#include #include #include #include @@ -663,7 +664,14 @@ private: else { query_id = config().getString("query_id", ""); - nonInteractive(); + if (query_fuzzer_runs) + { + nonInteractiveWithFuzzing(); + } + else + { + nonInteractive(); + } /// If exception code isn't zero, we should return non-zero return code anyway. if (last_exception_received_from_server) @@ -765,6 +773,68 @@ private: processQueryText(text); } + void nonInteractiveWithFuzzing() + { + if (config().has("query")) + { + // Poco configuration should not process substitutions in form of + // ${...} inside query + processWithFuzzing(config().getRawString("query")); + return; + } + + /// If 'query' parameter is not set, read a query from stdin. + /// The query is read entirely into memory (streaming is disabled). + ReadBufferFromFileDescriptor in(STDIN_FILENO); + std::string text; + while (!in.eof()) + { + readStringInto(text, in); + // Append the separator as well + if (!in.eof()) + { + text.append(1, *in.position()); + ++in.position(); + } + + fprintf(stderr, "will now parse '%s'\n", text.c_str()); + + const auto new_end = processWithFuzzing(text); + + if (new_end > &text[0]) + { + const auto rest_size = text.size() - (new_end - &text[0]); + + fprintf(stderr, "total %zd, rest %zd\n", text.size(), rest_size); + + memcpy(&text[0], new_end, rest_size); + text.resize(rest_size); + } + else + { + fprintf(stderr, "total %zd, can't parse\n", text.size()); + } + + if (!connection->isConnected()) + { + // Uh-oh... + std::cerr << "Lost connection to the server." << std::endl; + last_exception_received_from_server.reset(new Exception(210, "~")); + return; + } + + if (text.size() > 1024 * 1024) + { + // Some pathological situation where the text is larger than 1MB + // and we still cannot parse a single query in it. Abort. + std::cerr << "Read too much text and still can't parse a query." + " Aborting." << std::endl; + last_exception_received_from_server.reset(new Exception(1, "~")); + return; + } + } + } + bool processQueryText(const String & text) { if (exit_strings.end() != exit_strings.find(trim(text, [](char c){ return isWhitespaceASCII(c) || c == ';'; }))) @@ -881,8 +951,8 @@ private: } - // Returns whether we can continue. - bool processWithFuzzing(const String & text) + // Returns the last position we could parse. + const char * processWithFuzzing(const String & text) { /// Several queries separated by ';'. /// INSERT data is ended by the end of line, not ';'. @@ -904,7 +974,7 @@ private: if (!orig_ast) { // Can't continue after a parsing error - return false; + return begin; } auto as_insert = orig_ast->as(); @@ -972,7 +1042,7 @@ private: } } - return true; + return end; } void processTextAsSingleQuery(const String & text_) @@ -2153,8 +2223,13 @@ public: if ((query_fuzzer_runs = options["query-fuzzer-runs"].as())) { - // Fuzzer implies multiquery + // Fuzzer implies multiquery. config().setBool("multiquery", true); + + // Ignore errors in parsing queries. + // TODO stop using parseQuery. + config().setBool("ignore-error", true); + ignore_error = true; } argsToConfig(common_arguments, config(), 100); @@ -2165,9 +2240,6 @@ public: } -#pragma GCC diagnostic ignored "-Wunused-function" -#pragma GCC diagnostic ignored "-Wmissing-declarations" - using signal_function = void(int, siginfo_t*, void*); /// Setup signal handlers. @@ -2210,12 +2282,16 @@ static void signalHandler(int sig, siginfo_t * /*info*/, void * context) raise(sig); } +#pragma GCC diagnostic ignored "-Wunused-function" +#pragma GCC diagnostic ignored "-Wmissing-declarations" + int mainEntryClickHouseClient(int argc, char ** argv) { try { add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE}, signalHandler); + DB::Client client; client.init(argc, argv); return client.run(); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index f93fa7871e5..7019778a2c9 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -170,6 +170,8 @@ public: /// If not connected yet, or if connection is broken - then connect. If cannot connect - throw an exception. void forceConnected(const ConnectionTimeouts & timeouts); + bool isConnected() const { return connected; } + TablesStatusResponse getTablesStatus(const ConnectionTimeouts & timeouts, const TablesStatusRequest & request); From d45697a1b575dc0abded4777773c8c5ef9ba4968 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 30 Jun 2020 21:47:12 +0300 Subject: [PATCH 060/196] in-memory-parts: lazy creation of WAL --- .../MergeTree/MergeTreeBlockOutputStream.cpp | 5 ----- src/Storages/MergeTree/MergeTreeData.cpp | 19 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 3 ++- 3 files changed, 12 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 7457feac207..744faa0767e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -7,11 +7,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int TIMEOUT_EXCEEDED; -} - Block MergeTreeBlockOutputStream::getHeader() const { return metadata_snapshot->getSampleBlock(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 633e9b4d05d..1f28f886882 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -930,12 +930,6 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) } } - if (settings->in_memory_parts_enable_wal && !write_ahead_log) - { - auto reservation = reserveSpace(settings->write_ahead_log_max_bytes); - write_ahead_log = std::make_shared(*this, reservation->getDisk()); - } - calculateColumnSizesImpl(); LOG_DEBUG(log, "Loaded data parts ({} items)", data_parts_indexes.size()); @@ -1851,8 +1845,8 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect if (part->state != IMergeTreeDataPart::State::Outdated) modifyPartState(part, IMergeTreeDataPart::State::Outdated); - if (isInMemoryPart(part) && write_ahead_log) - write_ahead_log->dropPart(part->name); + if (isInMemoryPart(part) && getSettings()->in_memory_parts_enable_wal) + getWriteAheadLog()->dropPart(part->name); } } @@ -3513,8 +3507,15 @@ MergeTreeData::AlterConversions MergeTreeData::getAlterConversionsForPart(const return result; } -MergeTreeData::WriteAheadLogPtr MergeTreeData::getWriteAheadLog() const +MergeTreeData::WriteAheadLogPtr MergeTreeData::getWriteAheadLog() { + std::lock_guard lock(write_ahead_log_mutex); + if (!write_ahead_log) + { + auto reservation = reserveSpace(getSettings()->write_ahead_log_max_bytes); + write_ahead_log = std::make_shared(*this, reservation->getDisk()); + } + return write_ahead_log; } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 11c20df210f..02a2bfc4815 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -641,7 +641,7 @@ public: bool isPartInTTLDestination(const TTLDescription & ttl, const IMergeTreeDataPart & part) const; using WriteAheadLogPtr = std::shared_ptr; - WriteAheadLogPtr getWriteAheadLog() const; + WriteAheadLogPtr getWriteAheadLog(); MergeTreeDataFormatVersion format_version; @@ -904,6 +904,7 @@ private: bool canUsePolymorphicParts(const MergeTreeSettings & settings, String * out_reason = nullptr) const; + std::mutex write_ahead_log_mutex; WriteAheadLogPtr write_ahead_log; }; From 68c9896fa2a228ce1395a98041dafaf93add940a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 Jul 2020 00:15:24 +0300 Subject: [PATCH 061/196] fix arcadia build --- src/Storages/ya.make | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 4ca1dad1b1b..18e56a30b24 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -81,6 +81,11 @@ SRCS( MergeTree/ReplicatedMergeTreeTableMetadata.cpp MergeTree/SimpleMergeSelector.cpp MergeTree/TTLMergeSelector.cpp + MergeTree/MergeTreeDataPartInMemory.cpp + MergeTree/MergeTreeDataPartWriterInMemory.cpp + MergeTree/MergeTreeDataPartWriterOnDisk.cpp + MergeTree/MergeTreeReaderInMemory.cpp + MergeTree/MergeTreeWriteAheadLog.cpp System/attachSystemTables.cpp System/StorageSystemAggregateFunctionCombinators.cpp System/StorageSystemAsynchronousMetrics.cpp From 924f1dfc467f87766590a59d6961314705726762 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 2 Jul 2020 11:40:41 +0300 Subject: [PATCH 062/196] Allow to parse EXPLAIN PLAN. --- src/Parsers/ParserExplainQuery.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Parsers/ParserExplainQuery.cpp b/src/Parsers/ParserExplainQuery.cpp index dae9f63e911..c6792d6094b 100644 --- a/src/Parsers/ParserExplainQuery.cpp +++ b/src/Parsers/ParserExplainQuery.cpp @@ -17,6 +17,7 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_explain("EXPLAIN"); ParserKeyword s_syntax("SYNTAX"); ParserKeyword s_pipeline("PIPELINE"); + ParserKeyword s_plan("PLAN"); if (enable_debug_queries && s_ast.ignore(pos, expected)) { @@ -38,6 +39,8 @@ bool ParserExplainQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected kind = ASTExplainQuery::ExplainKind::AnalyzedSyntax; else if (s_pipeline.ignore(pos, expected)) kind = ASTExplainQuery::ExplainKind::QueryPipeline; + else if (s_plan.ignore(pos, expected)) + kind = ASTExplainQuery::ExplainKind::QueryPlan; } else return false; From 22aabf8da3ff5ef58f5db4a67aa56d7a489fc539 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jul 2020 17:58:13 +0200 Subject: [PATCH 063/196] Adding a simple example of using TestFlows. --- tests/testflows/README.md | 55 +++ tests/testflows/example/__init__.py | 0 .../example/configs/clickhouse/common.xml | 6 + .../configs/clickhouse/config.d/logs.xml | 17 + .../configs/clickhouse/config.d/ports.xml | 5 + .../configs/clickhouse/config.d/remote.xml | 107 +++++ .../configs/clickhouse/config.d/ssl.xml | 17 + .../configs/clickhouse/config.d/storage.xml | 20 + .../configs/clickhouse/config.d/zookeeper.xml | 10 + .../example/configs/clickhouse/config.xml | 436 ++++++++++++++++++ .../configs/clickhouse/ssl/dhparam.pem | 8 + .../example/configs/clickhouse/ssl/server.crt | 19 + .../example/configs/clickhouse/ssl/server.key | 28 ++ .../example/configs/clickhouse/users.xml | 133 ++++++ .../configs/clickhouse1/config.d/macros.xml | 8 + .../docker-compose/clickhouse-service.yml | 28 ++ .../example/docker-compose/docker-compose.yml | 31 ++ .../docker-compose/zookeeper-service.yml | 18 + tests/testflows/example/regression.py | 26 ++ .../example/requirements/__init__.py | 1 + .../example/requirements/requirements.py | 9 + tests/testflows/example/tests/example.py | 20 + tests/testflows/helpers/argparser.py | 13 + tests/testflows/helpers/cluster.py | 272 +++++++++++ tests/testflows/regression.py | 19 + 25 files changed, 1306 insertions(+) create mode 100644 tests/testflows/README.md create mode 100644 tests/testflows/example/__init__.py create mode 100644 tests/testflows/example/configs/clickhouse/common.xml create mode 100644 tests/testflows/example/configs/clickhouse/config.d/logs.xml create mode 100644 tests/testflows/example/configs/clickhouse/config.d/ports.xml create mode 100644 tests/testflows/example/configs/clickhouse/config.d/remote.xml create mode 100644 tests/testflows/example/configs/clickhouse/config.d/ssl.xml create mode 100644 tests/testflows/example/configs/clickhouse/config.d/storage.xml create mode 100644 tests/testflows/example/configs/clickhouse/config.d/zookeeper.xml create mode 100644 tests/testflows/example/configs/clickhouse/config.xml create mode 100644 tests/testflows/example/configs/clickhouse/ssl/dhparam.pem create mode 100644 tests/testflows/example/configs/clickhouse/ssl/server.crt create mode 100644 tests/testflows/example/configs/clickhouse/ssl/server.key create mode 100644 tests/testflows/example/configs/clickhouse/users.xml create mode 100644 tests/testflows/example/configs/clickhouse1/config.d/macros.xml create mode 100644 tests/testflows/example/docker-compose/clickhouse-service.yml create mode 100644 tests/testflows/example/docker-compose/docker-compose.yml create mode 100644 tests/testflows/example/docker-compose/zookeeper-service.yml create mode 100755 tests/testflows/example/regression.py create mode 100644 tests/testflows/example/requirements/__init__.py create mode 100644 tests/testflows/example/requirements/requirements.py create mode 100644 tests/testflows/example/tests/example.py create mode 100644 tests/testflows/helpers/argparser.py create mode 100644 tests/testflows/helpers/cluster.py create mode 100755 tests/testflows/regression.py diff --git a/tests/testflows/README.md b/tests/testflows/README.md new file mode 100644 index 00000000000..645ad541795 --- /dev/null +++ b/tests/testflows/README.md @@ -0,0 +1,55 @@ +## ClickHouse Tests in [TestFlows] + +This directory contains integration tests written using [TestFlows] +that involves several ClickHouse instances, custom configs, ZooKeeper, etc. + +## Supported environment + +* [Ubuntu] 18.04 +* [Python 3] >= 3.6 + +## Prerequisites + +* [Docker] [install](https://docs.docker.com/compose/install/) +* [Docker Compose] [install](https://docs.docker.com/engine/install/) +* [TestFlows] [install](https://testflows.com/handbook/#Installation) + +## Running tests locally + +You can run tests locally by passing `--local` and `--clickhouse-binary-path` to the `regression.py`. + +* `--local` specifies that regression will be run locally +* `--clickhouse-binary-path` specifies the path to the ClickHouse binary that will be used during the regression run + +> Note: you can pass `-h` or `--help` argument to the `regression.py` to see a help message. +> +> ```bash +> python3 regression.py -h +> ``` + +> Note: make sure that the ClickHouse binary has correct permissions. +> If you are using `/usr/bin/clickhouse` its owner and group is set to `root:root` by default +> and it needs to be changed to `clickhouse:clickhouse`. You can change the owner and the group +> using the following command. +> +> ```bash +> sudo chown clickhouse:clickhouse /usr/bin/clickhouse +> ``` + +Using the default ClickHouse installation and its server binary at `/usr/bin/clickhouse`, you can run +regressions locally using the following command. + +```bash +python3 regression.py --local --clickhouse-binary-path "/usr/bin/clickhouse" +``` + +## Output Verbosity + +You can control verbosity of the output by specifying the output format with `-o` or `--output` option. +See `--help` for more details. + +[Python 3]: https://www.python.org/ +[Ubuntu]: https://ubuntu.com/ +[TestFlows]: https://testflows.com +[Docker]: https://www.docker.com/ +[Docker Compose]: https://docs.docker.com/compose/ diff --git a/tests/testflows/example/__init__.py b/tests/testflows/example/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/testflows/example/configs/clickhouse/common.xml b/tests/testflows/example/configs/clickhouse/common.xml new file mode 100644 index 00000000000..df952b28c82 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/common.xml @@ -0,0 +1,6 @@ + + Europe/Moscow + 0.0.0.0 + /var/lib/clickhouse/ + /var/lib/clickhouse/tmp/ + diff --git a/tests/testflows/example/configs/clickhouse/config.d/logs.xml b/tests/testflows/example/configs/clickhouse/config.d/logs.xml new file mode 100644 index 00000000000..bdf1bbc11c1 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/config.d/logs.xml @@ -0,0 +1,17 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + + system + part_log
+ 500 +
+
diff --git a/tests/testflows/example/configs/clickhouse/config.d/ports.xml b/tests/testflows/example/configs/clickhouse/config.d/ports.xml new file mode 100644 index 00000000000..fbc6cea74c0 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/config.d/ports.xml @@ -0,0 +1,5 @@ + + + 8443 + 9440 + \ No newline at end of file diff --git a/tests/testflows/example/configs/clickhouse/config.d/remote.xml b/tests/testflows/example/configs/clickhouse/config.d/remote.xml new file mode 100644 index 00000000000..51be2a6e8e3 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/config.d/remote.xml @@ -0,0 +1,107 @@ + + + + + + true + + clickhouse1 + 9000 + + + clickhouse2 + 9000 + + + clickhouse3 + 9000 + + + + + + + true + + clickhouse1 + 9440 + 1 + + + clickhouse2 + 9440 + 1 + + + clickhouse3 + 9440 + 1 + + + + + + + clickhouse1 + 9000 + + + + + clickhouse2 + 9000 + + + + + clickhouse3 + 9000 + + + + + + + clickhouse1 + 9440 + 1 + + + + + clickhouse2 + 9440 + 1 + + + + + clickhouse3 + 9440 + 1 + + + + + diff --git a/tests/testflows/example/configs/clickhouse/config.d/ssl.xml b/tests/testflows/example/configs/clickhouse/config.d/ssl.xml new file mode 100644 index 00000000000..ca65ffd5e04 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/config.d/ssl.xml @@ -0,0 +1,17 @@ + + + + /etc/clickhouse-server/ssl/server.crt + /etc/clickhouse-server/ssl/server.key + none + true + + + true + none + + AcceptCertificateHandler + + + + diff --git a/tests/testflows/example/configs/clickhouse/config.d/storage.xml b/tests/testflows/example/configs/clickhouse/config.d/storage.xml new file mode 100644 index 00000000000..618fd6b6d24 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/config.d/storage.xml @@ -0,0 +1,20 @@ + + + + + + 1024 + + + + + + + default + + + + + + + diff --git a/tests/testflows/example/configs/clickhouse/config.d/zookeeper.xml b/tests/testflows/example/configs/clickhouse/config.d/zookeeper.xml new file mode 100644 index 00000000000..96270e7b645 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/config.d/zookeeper.xml @@ -0,0 +1,10 @@ + + + + + zookeeper + 2181 + + 15000 + + diff --git a/tests/testflows/example/configs/clickhouse/config.xml b/tests/testflows/example/configs/clickhouse/config.xml new file mode 100644 index 00000000000..d34d2c35253 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/config.xml @@ -0,0 +1,436 @@ + + + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + + 8123 + 9000 + + + + + + + + + /etc/clickhouse-server/server.crt + /etc/clickhouse-server/server.key + + /etc/clickhouse-server/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + + true + true + sslv2,sslv3 + true + + + + RejectCertificateHandler + + + + + + + + + 9009 + + + + + + + + + + + + + + + + + + + + 4096 + 3 + + + 100 + + + + + + 8589934592 + + + 5368709120 + + + + /var/lib/clickhouse/ + + + /var/lib/clickhouse/tmp/ + + + /var/lib/clickhouse/user_files/ + + + /var/lib/clickhouse/access/ + + + users.xml + + + default + + + + + + default + + + + + + + + + false + + + + + + + + localhost + 9000 + + + + + + + localhost + 9000 + + + + + localhost + 9000 + + + + + + + localhost + 9440 + 1 + + + + + + + localhost + 9000 + + + + + localhost + 1 + + + + + + + + + + + + + + + + + 3600 + + + + 3600 + + + 60 + + + + + + + + + + system + query_log
+ + toYYYYMM(event_date) + + 7500 +
+ + + + system + trace_log
+ + toYYYYMM(event_date) + 7500 +
+ + + + system + query_thread_log
+ toYYYYMM(event_date) + 7500 +
+ + + + + + + + + + + + + + + + *_dictionary.xml + + + + + + + + + + /clickhouse/task_queue/ddl + + + + + + + + + + + + + + + + click_cost + any + + 0 + 3600 + + + 86400 + 60 + + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + + + /var/lib/clickhouse/format_schemas/ + + + +
diff --git a/tests/testflows/example/configs/clickhouse/ssl/dhparam.pem b/tests/testflows/example/configs/clickhouse/ssl/dhparam.pem new file mode 100644 index 00000000000..2e6cee0798d --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/ssl/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/testflows/example/configs/clickhouse/ssl/server.crt b/tests/testflows/example/configs/clickhouse/ssl/server.crt new file mode 100644 index 00000000000..7ade2d96273 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/ssl/server.crt @@ -0,0 +1,19 @@ +-----BEGIN CERTIFICATE----- +MIIC/TCCAeWgAwIBAgIJANjx1QSR77HBMA0GCSqGSIb3DQEBCwUAMBQxEjAQBgNV +BAMMCWxvY2FsaG9zdDAgFw0xODA3MzAxODE2MDhaGA8yMjkyMDUxNDE4MTYwOFow +FDESMBAGA1UEAwwJbG9jYWxob3N0MIIBIjANBgkqhkiG9w0BAQEFAAOCAQ8AMIIB +CgKCAQEAs9uSo6lJG8o8pw0fbVGVu0tPOljSWcVSXH9uiJBwlZLQnhN4SFSFohfI +4K8U1tBDTnxPLUo/V1K9yzoLiRDGMkwVj6+4+hE2udS2ePTQv5oaMeJ9wrs+5c9T +4pOtlq3pLAdm04ZMB1nbrEysceVudHRkQbGHzHp6VG29Fw7Ga6YpqyHQihRmEkTU +7UCYNA+Vk7aDPdMS/khweyTpXYZimaK9f0ECU3/VOeG3fH6Sp2X6FN4tUj/aFXEj +sRmU5G2TlYiSIUMF2JPdhSihfk1hJVALrHPTU38SOL+GyyBRWdNcrIwVwbpvsvPg +pryMSNxnpr0AK0dFhjwnupIv5hJIOQIDAQABo1AwTjAdBgNVHQ4EFgQUjPLb3uYC +kcamyZHK4/EV8jAP0wQwHwYDVR0jBBgwFoAUjPLb3uYCkcamyZHK4/EV8jAP0wQw +DAYDVR0TBAUwAwEB/zANBgkqhkiG9w0BAQsFAAOCAQEAM/ocuDvfPus/KpMVD51j +4IdlU8R0vmnYLQ+ygzOAo7+hUWP5j0yvq4ILWNmQX6HNvUggCgFv9bjwDFhb/5Vr +85ieWfTd9+LTjrOzTw4avdGwpX9G+6jJJSSq15tw5ElOIFb/qNA9O4dBiu8vn03C +L/zRSXrARhSqTW5w/tZkUcSTT+M5h28+Lgn9ysx4Ff5vi44LJ1NnrbJbEAIYsAAD ++UA+4MBFKx1r6hHINULev8+lCfkpwIaeS8RL+op4fr6kQPxnULw8wT8gkuc8I4+L +P9gg/xDHB44T3ADGZ5Ib6O0DJaNiToO6rnoaaxs0KkotbvDWvRoxEytSbXKoYjYp +0g== +-----END CERTIFICATE----- diff --git a/tests/testflows/example/configs/clickhouse/ssl/server.key b/tests/testflows/example/configs/clickhouse/ssl/server.key new file mode 100644 index 00000000000..f0fb61ac443 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/ssl/server.key @@ -0,0 +1,28 @@ +-----BEGIN PRIVATE KEY----- +MIIEvQIBADANBgkqhkiG9w0BAQEFAASCBKcwggSjAgEAAoIBAQCz25KjqUkbyjyn +DR9tUZW7S086WNJZxVJcf26IkHCVktCeE3hIVIWiF8jgrxTW0ENOfE8tSj9XUr3L +OguJEMYyTBWPr7j6ETa51LZ49NC/mhox4n3Cuz7lz1Pik62WreksB2bThkwHWdus +TKxx5W50dGRBsYfMenpUbb0XDsZrpimrIdCKFGYSRNTtQJg0D5WTtoM90xL+SHB7 +JOldhmKZor1/QQJTf9U54bd8fpKnZfoU3i1SP9oVcSOxGZTkbZOViJIhQwXYk92F +KKF+TWElUAusc9NTfxI4v4bLIFFZ01ysjBXBum+y8+CmvIxI3GemvQArR0WGPCe6 +ki/mEkg5AgMBAAECggEATrbIBIxwDJOD2/BoUqWkDCY3dGevF8697vFuZKIiQ7PP +TX9j4vPq0DfsmDjHvAPFkTHiTQXzlroFik3LAp+uvhCCVzImmHq0IrwvZ9xtB43f +7Pkc5P6h1l3Ybo8HJ6zRIY3TuLtLxuPSuiOMTQSGRL0zq3SQ5DKuGwkz+kVjHXUN +MR2TECFwMHKQ5VLrC+7PMpsJYyOMlDAWhRfUalxC55xOXTpaN8TxNnwQ8K2ISVY5 +212Jz/a4hn4LdwxSz3Tiu95PN072K87HLWx3EdT6vW4Ge5P/A3y+smIuNAlanMnu +plHBRtpATLiTxZt/n6npyrfQVbYjSH7KWhB8hBHtaQKBgQDh9Cq1c/KtqDtE0Ccr +/r9tZNTUwBE6VP+3OJeKdEdtsfuxjOCkS1oAjgBJiSDOiWPh1DdoDeVZjPKq6pIu +Mq12OE3Doa8znfCXGbkSzEKOb2unKZMJxzrz99kXt40W5DtrqKPNb24CNqTiY8Aa +CjtcX+3weat82VRXvph6U8ltMwKBgQDLxjiQQzNoY7qvg7CwJCjf9qq8jmLK766g +1FHXopqS+dTxDLM8eJSRrpmxGWJvNeNc1uPhsKsKgotqAMdBUQTf7rSTbt4MyoH5 +bUcRLtr+0QTK9hDWMOOvleqNXha68vATkohWYfCueNsC60qD44o8RZAS6UNy3ENq +cM1cxqe84wKBgQDKkHutWnooJtajlTxY27O/nZKT/HA1bDgniMuKaz4R4Gr1PIez +on3YW3V0d0P7BP6PWRIm7bY79vkiMtLEKdiKUGWeyZdo3eHvhDb/3DCawtau8L2K +GZsHVp2//mS1Lfz7Qh8/L/NedqCQ+L4iWiPnZ3THjjwn3CoZ05ucpvrAMwKBgB54 +nay039MUVq44Owub3KDg+dcIU62U+cAC/9oG7qZbxYPmKkc4oL7IJSNecGHA5SbU +2268RFdl/gLz6tfRjbEOuOHzCjFPdvAdbysanpTMHLNc6FefJ+zxtgk9sJh0C4Jh +vxFrw9nTKKzfEl12gQ1SOaEaUIO0fEBGbe8ZpauRAoGAMAlGV+2/K4ebvAJKOVTa +dKAzQ+TD2SJmeR1HZmKDYddNqwtZlzg3v4ZhCk4eaUmGeC1Bdh8MDuB3QQvXz4Dr +vOIP4UVaOr+uM+7TgAgVnP4/K6IeJGzUDhX93pmpWhODfdu/oojEKVcpCojmEmS1 +KCBtmIrQLqzMpnBpLNuSY+Q= +-----END PRIVATE KEY----- diff --git a/tests/testflows/example/configs/clickhouse/users.xml b/tests/testflows/example/configs/clickhouse/users.xml new file mode 100644 index 00000000000..86b2cd9e1e3 --- /dev/null +++ b/tests/testflows/example/configs/clickhouse/users.xml @@ -0,0 +1,133 @@ + + + + + + + + 10000000000 + + + 0 + + + random + + + + + 1 + + + + + + + + + + + + + ::/0 + + + + default + + + default + + + 1 + + + + + + + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/tests/testflows/example/configs/clickhouse1/config.d/macros.xml b/tests/testflows/example/configs/clickhouse1/config.d/macros.xml new file mode 100644 index 00000000000..6cdcc1b440c --- /dev/null +++ b/tests/testflows/example/configs/clickhouse1/config.d/macros.xml @@ -0,0 +1,8 @@ + + + + clickhouse1 + 01 + 01 + + diff --git a/tests/testflows/example/docker-compose/clickhouse-service.yml b/tests/testflows/example/docker-compose/clickhouse-service.yml new file mode 100644 index 00000000000..ed345bd7b04 --- /dev/null +++ b/tests/testflows/example/docker-compose/clickhouse-service.yml @@ -0,0 +1,28 @@ +version: '2.3' + +services: + clickhouse: + image: yandex/clickhouse-integration-test + expose: + - "9000" + - "9009" + - "8123" + volumes: + - "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.d:/etc/clickhouse-server/config.d" + - "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.d/:/etc/clickhouse-server/users.d" + - "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/ssl:/etc/clickhouse-server/ssl" + - "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/config.xml:/etc/clickhouse-server/config.xml" + - "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse/users.xml:/etc/clickhouse-server/users.xml" + - "${CLICKHOUSE_TESTS_SERVER_BIN_PATH:-/usr/bin/clickhouse}:/usr/bin/clickhouse" + - "${CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH:-/usr/bin/clickhouse-odbc-bridge}:/usr/bin/clickhouse-odbc-bridge" + entrypoint: bash -c "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" + healthcheck: + test: clickhouse client --query='select 1' + interval: 3s + timeout: 5s + retries: 40 + start_period: 2s + cap_add: + - SYS_PTRACE + security_opt: + - label:disable diff --git a/tests/testflows/example/docker-compose/docker-compose.yml b/tests/testflows/example/docker-compose/docker-compose.yml new file mode 100644 index 00000000000..e7e57386dc4 --- /dev/null +++ b/tests/testflows/example/docker-compose/docker-compose.yml @@ -0,0 +1,31 @@ +version: '2.3' + +services: + zookeeper: + extends: + file: zookeeper-service.yml + service: zookeeper + + clickhouse1: + extends: + file: clickhouse-service.yml + service: clickhouse + hostname: clickhouse1 + volumes: + - "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/database/:/var/lib/clickhouse/" + - "${CLICKHOUSE_TESTS_DIR}/_instances/clickhouse1/logs/:/var/log/clickhouse-server/" + - "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse1/config.d:/etc/clickhouse-server/config.d" + - "${CLICKHOUSE_TESTS_DIR}/configs/clickhouse1/users.d:/etc/clickhouse-server/users.d" + depends_on: + zookeeper: + condition: service_healthy + + # dummy service which does nothing, but allows to postpone + # 'docker-compose up -d' till all dependecies will go healthy + all_services_ready: + image: hello-world + depends_on: + clickhouse1: + condition: service_healthy + zookeeper: + condition: service_healthy diff --git a/tests/testflows/example/docker-compose/zookeeper-service.yml b/tests/testflows/example/docker-compose/zookeeper-service.yml new file mode 100644 index 00000000000..f3df33358be --- /dev/null +++ b/tests/testflows/example/docker-compose/zookeeper-service.yml @@ -0,0 +1,18 @@ +version: '2.3' + +services: + zookeeper: + image: zookeeper:3.4.12 + expose: + - "2181" + environment: + ZOO_TICK_TIME: 500 + ZOO_MY_ID: 1 + healthcheck: + test: echo stat | nc localhost 2181 + interval: 3s + timeout: 2s + retries: 5 + start_period: 2s + security_opt: + - label:disable diff --git a/tests/testflows/example/regression.py b/tests/testflows/example/regression.py new file mode 100755 index 00000000000..2c0a778d39b --- /dev/null +++ b/tests/testflows/example/regression.py @@ -0,0 +1,26 @@ +#!/usr/bin/env python3 +import sys +from testflows.core import * + +append_path(sys.path, "..") + +from helpers.cluster import Cluster +from helpers.argparser import argparser + +@TestFeature +@Name("example") +@ArgumentParser(argparser) +def regression(self, local, clickhouse_binary_path): + """Simple example of how you can use TestFlows to test ClickHouse. + """ + nodes = { + "clickhouse": ("clickhouse1",), + } + + with Cluster(local, clickhouse_binary_path, nodes=nodes) as cluster: + self.context.cluster = cluster + + Scenario(run=load("example.tests.example", "scenario")) + +if main(): + regression() diff --git a/tests/testflows/example/requirements/__init__.py b/tests/testflows/example/requirements/__init__.py new file mode 100644 index 00000000000..02f7d430154 --- /dev/null +++ b/tests/testflows/example/requirements/__init__.py @@ -0,0 +1 @@ +from .requirements import * diff --git a/tests/testflows/example/requirements/requirements.py b/tests/testflows/example/requirements/requirements.py new file mode 100644 index 00000000000..9a0b4922f19 --- /dev/null +++ b/tests/testflows/example/requirements/requirements.py @@ -0,0 +1,9 @@ +from testflows.core import Requirement + +RQ_ClickHouse_Select_1 = Requirement( + name='RQ.ClickHouse.Select.1', + version='1.0', + description=( + '[ClickHouse] SHALL return `1` when user sends a simple `SELECT 1` query.\n' + ) + ) diff --git a/tests/testflows/example/tests/example.py b/tests/testflows/example/tests/example.py new file mode 100644 index 00000000000..8559f290cc4 --- /dev/null +++ b/tests/testflows/example/tests/example.py @@ -0,0 +1,20 @@ +from testflows.core import * +from testflows.asserts import error + +from example.requirements import * + +@TestScenario +@Name("select 1") +@Requirements( + RQ_ClickHouse_Select_1("1.0") +) +def scenario(self, node="clickhouse1"): + """Check that ClickHouse returns 1 when user executes `SELECT 1` query. + """ + node = self.context.cluster.node(node) + + with When("I execute query select 1"): + r = node.query("SELECT 1").output.strip() + + with Then("the result should be 1"): + assert r == "1", error() diff --git a/tests/testflows/helpers/argparser.py b/tests/testflows/helpers/argparser.py new file mode 100644 index 00000000000..34b91453876 --- /dev/null +++ b/tests/testflows/helpers/argparser.py @@ -0,0 +1,13 @@ +import os + +def argparser(parser): + """Default argument parser for regressions. + """ + parser.add_argument("--local", + action="store_true", + help="run regression in local mode", default=False) + + parser.add_argument("--clickhouse-binary-path", + type=str, dest="clickhouse_binary_path", + help="path to ClickHouse binary, default: /usr/bin/clickhouse", metavar="path", + default=os.getenv("CLICKHOUSE_TESTS_SERVER_BIN_PATH", "/usr/bin/clickhouse")) diff --git a/tests/testflows/helpers/cluster.py b/tests/testflows/helpers/cluster.py new file mode 100644 index 00000000000..e087b3b5b9d --- /dev/null +++ b/tests/testflows/helpers/cluster.py @@ -0,0 +1,272 @@ +import os +import time +import inspect +import threading +import tempfile + +from testflows.core import * +from testflows.asserts import error +from testflows.connect import Shell + +class QueryRuntimeException(Exception): + """Exception during query execution on the server. + """ + pass + +class Node(object): + """Generic cluster node. + """ + config_d_dir = "/etc/clickhouse-server/config.d/" + + def __init__(self, cluster, name): + self.cluster = cluster + self.name = name + + def repr(self): + return f"Node(name='{self.name}')" + + def restart(self, timeout=120, safe=True): + """Restart node. + """ + with self.cluster.lock: + for key in list(self.cluster._bash.keys()): + if key.endswith(f"-{self.name}"): + shell = self.cluster._bash.pop(key) + shell.__exit__(None, None, None) + + self.cluster.command(None, f'{self.cluster.docker_compose} restart {self.name}', timeout=timeout) + + def command(self, *args, **kwargs): + return self.cluster.command(self.name, *args, **kwargs) + +class ClickHouseNode(Node): + """Node with ClickHouse server. + """ + def wait_healthy(self, timeout=120): + with By(f"waiting until container {self.name} is healthy"): + start_time = time.time() + while True: + if self.query("select 1", no_checks=1, timeout=120, steps=False).exitcode == 0: + break + if time.time() - start_time < timeout: + time.sleep(2) + continue + assert False, "container is not healthy" + + def restart(self, timeout=120, safe=True): + """Restart node. + """ + if safe: + self.query("SYSTEM STOP MOVES") + self.query("SYSTEM STOP MERGES") + self.query("SYSTEM FLUSH LOGS") + with By("waiting for 5 sec for moves and merges to stop"): + time.sleep(5) + with And("forcing to sync everything to disk"): + self.command("sync", timeout=30) + + with self.cluster.lock: + for key in list(self.cluster._bash.keys()): + if key.endswith(f"-{self.name}"): + shell = self.cluster._bash.pop(key) + shell.__exit__(None, None, None) + + self.cluster.command(None, f'{self.cluster.docker_compose} restart {self.name}', timeout=timeout) + + self.wait_healthy(timeout) + + def query(self, sql, message=None, exitcode=None, steps=True, no_checks=False, + raise_on_exception=False, step=By, settings=None, *args, **kwargs): + """Execute and check query. + + :param sql: sql query + :param message: expected message that should be in the output, default: None + :param exitcode: expected exitcode, default: None + """ + if len(sql) > 1024: + with tempfile.NamedTemporaryFile("w", encoding="utf-8") as query: + query.write(sql) + query.flush() + command = f"cat \"{query.name}\" | {self.cluster.docker_compose} exec -T {self.name} clickhouse client -n" + for setting in settings or []: + name, value = setting + command += f" --{name} \"{value}\"" + description = f""" + echo -e \"{sql[:100]}...\" > {query.name} + {command} + """ + with step("executing command", description=description) if steps else NullStep(): + r = self.cluster.bash(None)(command, *args, **kwargs) + else: + command = f"echo -e \"{sql}\" | clickhouse client -n" + for setting in settings or []: + name, value = setting + command += f" --{name} \"{value}\"" + with step("executing command", description=command) if steps else NullStep(): + r = self.cluster.bash(self.name)(command, *args, **kwargs) + + if no_checks: + return r + + if exitcode is not None: + with Then(f"exitcode should be {exitcode}") if steps else NullStep(): + assert r.exitcode == exitcode, error(r.output) + + if message is not None: + with Then(f"output should contain message", description=message) if steps else NullStep(): + assert message in r.output, error(r.output) + + if message is None or "Exception:" not in message: + with Then("check if output has exception") if steps else NullStep(): + if "Exception:" in r.output: + if raise_on_exception: + raise QueryRuntimeException(r.output) + assert False, error(r.output) + + return r + +class Cluster(object): + """Simple object around docker-compose cluster. + """ + def __init__(self, local=False, + clickhouse_binary_path=None, configs_dir=None, + nodes=None, + docker_compose="docker-compose", docker_compose_project_dir=None, + docker_compose_file="docker-compose.yml"): + + self._bash = {} + self.clickhouse_binary_path = clickhouse_binary_path + self.configs_dir = configs_dir + self.local = local + self.nodes = nodes or {} + self.docker_compose = docker_compose + + frame = inspect.currentframe().f_back + caller_dir = os.path.dirname(os.path.abspath(frame.f_globals["__file__"])) + + # auto set configs directory + if self.configs_dir is None: + caller_configs_dir = caller_dir + if os.path.exists(caller_configs_dir): + self.configs_dir = caller_configs_dir + + if not os.path.exists(self.configs_dir): + raise TypeError("configs directory '{self.configs_dir}' does not exist") + + # auto set docker-compose project directory + if docker_compose_project_dir is None: + caller_project_dir = os.path.join(caller_dir, "docker-compose") + if os.path.exists(caller_project_dir): + docker_compose_project_dir = caller_project_dir + + docker_compose_file_path = os.path.join(docker_compose_project_dir or "", docker_compose_file) + + if not os.path.exists(docker_compose_file_path): + raise TypeError("docker compose file '{docker_compose_file_path}' does not exist") + + self.docker_compose += f" --project-directory \"{docker_compose_project_dir}\" --file \"{docker_compose_file_path}\"" + self.lock = threading.Lock() + + def bash(self, node, timeout=60): + """Returns thread-local bash terminal + to a specific node. + + :param node: name of the service + """ + current_thread = threading.current_thread() + id = f"{current_thread.ident}-{node}" + with self.lock: + if self._bash.get(id) is None: + if node is None: + self._bash[id] = Shell().__enter__() + else: + self._bash[id] = Shell(command=[ + "/bin/bash", "--noediting", "-c", f"{self.docker_compose} exec {node} bash --noediting" + ], name=node).__enter__() + self._bash[id].timeout = timeout + return self._bash[id] + + def __enter__(self): + with Given("docker-compose cluster"): + self.up() + return self + + def __exit__(self, type, value, traceback): + try: + with Finally("I clean up"): + self.down() + finally: + with self.lock: + for shell in self._bash.values(): + shell.__exit__(type, value, traceback) + + def node(self, name): + """Get object with node bound methods. + + :param name: name of service name + """ + if name.startswith("clickhouse"): + return ClickHouseNode(self, name) + return Node(self, name) + + def down(self, timeout=120): + """Bring cluster down by executing docker-compose down.""" + try: + bash = self.bash(None) + with self.lock: + # remove and close all not None node terminals + for id in list(self._bash.keys()): + shell = self._bash.pop(id) + if shell is not bash: + shell.__exit__(None, None, None) + else: + self._bash[id] = shell + finally: + return self.command(None, f"{self.docker_compose} down", timeout=timeout) + + def up(self): + if self.local: + with Given("I am running in local mode"): + with Then("check --clickhouse-binary-path is specified"): + assert self.clickhouse_binary_path, "when running in local mode then --clickhouse-binary-path must be specified" + with And("path should exist"): + assert os.path.exists(self.clickhouse_binary_path) + + os.environ["CLICKHOUSE_TESTS_SERVER_BIN_PATH"] = self.clickhouse_binary_path + os.environ["CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH"] = os.path.join(os.path.dirname(self.clickhouse_binary_path), + "clickhouse-odbc-bridge") + os.environ["CLICKHOUSE_TESTS_DIR"] = self.configs_dir + + with Given("docker-compose"): + self.command(None, "env | grep CLICKHOUSE") + cmd = self.command(None, f'{self.docker_compose} up -d 2>&1 | tee', timeout=30 * 60) + else: + with Given("docker-compose"): + cmd = self.command(None, f'{self.docker_compose} up -d --no-recreate 2>&1 | tee') + + with Then("check there are no unhealthy containers"): + assert "is unhealthy" not in cmd.output, error() + + with Then("wait all nodes report healhy"): + for name in self.nodes["clickhouse"]: + self.node(name).wait_healthy() + + def command(self, node, command, message=None, exitcode=None, steps=True, *args, **kwargs): + """Execute and check command. + + :param node: name of the service + :param command: command + :param message: expected message that should be in the output, default: None + :param exitcode: expected exitcode, default: None + :param steps: don't break command into steps, default: True + """ + debug(f"command() {node}, {command}") + with By("executing command", description=command) if steps else NullStep(): + r = self.bash(node)(command, *args, **kwargs) + if exitcode is not None: + with Then(f"exitcode should be {exitcode}") if steps else NullStep(): + assert r.exitcode == exitcode, error(r.output) + if message is not None: + with Then(f"output should contain message", description=message) if steps else NullStep(): + assert message in r.output, error(r.output) + return r diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py new file mode 100755 index 00000000000..0f74b6e82cd --- /dev/null +++ b/tests/testflows/regression.py @@ -0,0 +1,19 @@ +#!/usr/bin/env python3 +import sys +from testflows.core import * + +append_path(sys.path, "."), + +from helpers.argparser import argparser + +@TestModule +@Name("clickhouse") +@ArgumentParser(argparser) +def regression(self, local, clickhouse_binary_path): + """ClickHouse regression. + """ + Feature(test=load("example.regression", "regression"))( + local=local, clickhouse_binary_path=clickhouse_binary_path) + +if main(): + regression() From 321bad14eeb40f542644825d2ba65771d16a38ff Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jul 2020 20:41:28 +0200 Subject: [PATCH 064/196] Adding a template of the SRS with the notes how to use it. --- .../requirements/CH_SRS001_ClickHouse.html | 1439 +++++++++++++++++ .../requirements/CH_SRS001_ClickHouse.md | 136 ++ .../example/requirements/requirements.py | 65 +- tests/testflows/example/tests/example.py | 2 +- 4 files changed, 1637 insertions(+), 5 deletions(-) create mode 100644 tests/testflows/example/requirements/CH_SRS001_ClickHouse.html create mode 100644 tests/testflows/example/requirements/CH_SRS001_ClickHouse.md diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html new file mode 100644 index 00000000000..5aedad25979 --- /dev/null +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html @@ -0,0 +1,1439 @@ + + + + + + + + +

CH-SRS001 ClickHouse Software Requirements Specification Template

+ +

Author: [name of the author]

+ +

Date: [date]

+ +

Table of Contents

+ + + +

Revision History

+ +

This document is stored in an electronic form using Git source control management software.

+ +

Introduction

+ +

This section provides an introduction to the project or the feature. +All SRS documents must be uniquely identified by a number. In this +case this document is identified by the number

+ +
CH-SRS001
+
+ +

The document number must always be used as a prefix to the document title. For example,

+ +
CH-SRSxxx Name of the document
+
+ +

All the requirements must be specified in the Requirements section.

+ +

Table of Contents

+ +

Note that currently the table of contents is generated manually using

+ +
cat CH_SRS001_ClickHouse.md | tfs document toc
+
+ +

command and needs to be updated any time requirement name is changed +or a new requirement is added.

+ +

Generating HTML version

+ +

You can easily generate a pretty HTML version of this document using the command.

+ +
cat CH_SRS001_ClickHouse.md | tfs document convert > CH_SRS001_ClickHouse.html
+
+ +

Generating Python Requirements

+ +

You can convert this SRS into the requirements.py by using the command.

+ +
cat CH_SRS001_ClickHouse.md | tfs requirements generate > requirements.py
+
+ +

Terminology

+ +

You can define terminolgy using the examples below and make you can make them +linkable as SRS by defining the links in the References section.

+ +

SRS

+ +

Software Requirements Specification

+ +

Some term that you will use

+ +

Some description of the term that you would like to use.

+ +

Requirements

+ +

This section includes all the requirements. This section can be structured in any way one sees fit.

+ +

Each requirement is defined by the section that starts with +the following prefix:

+ +
RQ.[document id].[requirement name]
+
+ +

then immediately followed by a one-line block that contains the +the version of the requirement.

+ +

RQ.CH-SRS001.Example short description of the requirement

+ +

version: 1.0

+ +

This is a long description of the requirement that can include any +relevant information.

+ +

The one-line block that follows the requirement defines the version +of the requirement. The version is controlled manually and is used +to indicate material changes to the requirement that would +require tests that cover this requirement to be updated.

+ +

It is a good practice to use requirement names that are broken +up into groups. It is not recommended to use only numbers +because if the requirement must be moved the numbering will not match. +Therefore, the requirement name should start with the group +name which is then followed by a number if any. For example,

+ +
RQ.SRS001.Group.Subgroup.1
+
+ +

To keep names short, try to use abbreviations for the requirement's group name.

+ +

RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement

+ +

version: 1.0

+ +

This an example of a sub-requirement of the RQ.CH-SRS001.Example. +CH_SRS001_Software_Requirements_Template.md

+ +

RQ.CH-SRS001.Example.Select.1

+ +

version: 1.0

+ +

ClickHouse SHALL return 1 when user executes query

+ +
SELECT 1
+
+ +

References

+ + + + + \ No newline at end of file diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md new file mode 100644 index 00000000000..370ff14c2cb --- /dev/null +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md @@ -0,0 +1,136 @@ +# CH-SRS001 ClickHouse Software Requirements Specification Template + +**Author:** [name of the author] + +**Date:** [date] + +## Table of Contents + +* 1 [Revision History](#revision-history) +* 2 [Introduction](#introduction) + * 2.1 [Table of Contents](#table-of-contents) + * 2.2 [Generating HTML version](#generating-html-version) + * 2.3 [Generating Python Requirements](#generating-python-requirements) +* 3 [Terminology](#terminology) + * 3.1 [SRS](#srs) + * 3.2 [Some term that you will use](#some-term-that-you-will-use) +* 4 [Requirements](#requirements) + * 4.1 [RQ.CH-SRS001.Example short description of the requirement ](#rqch-srs001example-short-description-of-the-requirement-) + * 4.2 [RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement](#rqch-srs001examplesubgroup-short-description-of-the-sub-requirement) + * 4.3 [RQ.CH-SRS001.Example.Select.1](#rqch-srs001exampleselect1) +* 5 [References](#references) + +## Revision History + +This document is stored in an electronic form using [Git] source control management software. + +## Introduction + +This section provides an introduction to the project or the feature. +All [SRS] documents must be uniquely identified by a number. In this +case this document is identified by the number + + CH-SRS001 + +The document number must always be used as a prefix to the document title. For example, + + CH-SRSxxx Name of the document + +All the requirements must be specified in the [Requirements](#requirements) section. + +### Table of Contents + +Note that currently the table of contents is generated manually using + +```bash +cat CH_SRS001_ClickHouse.md | tfs document toc +``` + +command and needs to be updated any time requirement name is changed +or a new requirement is added. + +### Generating HTML version + +You can easily generate a pretty HTML version of this document using the command. + +```bash +cat CH_SRS001_ClickHouse.md | tfs document convert > CH_SRS001_ClickHouse.html +``` + +### Generating Python Requirements + +You can convert this [SRS] into the `requirements.py` by using the command. + +```bash +cat CH_SRS001_ClickHouse.md | tfs requirements generate > requirements.py +``` + +## Terminology + +You can define terminolgy using the examples below and make you can make them +linkable as [SRS] by defining the links in the [References](#References) section. + +### SRS + +Software Requirements Specification + +### Some term that you will use + +Some description of the term that you would like to use. + +## Requirements + +This section includes all the requirements. This section can be structured in any way one sees fit. + +Each requirement is defined by the section that starts with +the following prefix: + + RQ.[document id].[requirement name] + +then immediately followed by a one-line block that contains the +the `version` of the requirement. + +### RQ.CH-SRS001.Example short description of the requirement +version: 1.0 + +This is a long description of the requirement that can include any +relevant information. + +The one-line block that follows the requirement defines the `version` +of the requirement. The version is controlled manually and is used +to indicate material changes to the requirement that would +require tests that cover this requirement to be updated. + +It is a good practice to use requirement names that are broken +up into groups. It is not recommended to use only numbers +because if the requirement must be moved the numbering will not match. +Therefore, the requirement name should start with the group +name which is then followed by a number if any. For example, + + RQ.SRS001.Group.Subgroup.1 + +To keep names short, try to use abbreviations for the requirement's group name. + +### RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement +version: 1.0 + +This an example of a sub-requirement of the [RQ.CH-SRS001.Example](#rqch-srs001example). +CH_SRS001_Software_Requirements_Template.md + +### RQ.CH-SRS001.Example.Select.1 +version: 1.0 + +[ClickHouse] SHALL return `1` when user executes query + +```sql +SELECT 1 +``` + +## References + +* **ClickHouse:** https://clickhouse.tech + +[SRS]: #SRS +[Some term that you will use]: #Sometermthatyouwilluse +[ClickHouse]: https://clickhouse.tech +[Git]: https://git-scm.com/ diff --git a/tests/testflows/example/requirements/requirements.py b/tests/testflows/example/requirements/requirements.py index 9a0b4922f19..f890b9a1daf 100644 --- a/tests/testflows/example/requirements/requirements.py +++ b/tests/testflows/example/requirements/requirements.py @@ -1,9 +1,66 @@ +# These requirements were auto generated +# from software requirements specification (SRS) +# document by TestFlows v1.6.200627.1211752. +# Do not edit by hand but re-generate instead +# using 'tfs requirements generate' command. from testflows.core import Requirement -RQ_ClickHouse_Select_1 = Requirement( - name='RQ.ClickHouse.Select.1', +RQ_CH_SRS001_Example_short_description_of_the_requirement_ = Requirement( + name='RQ.CH-SRS001.Example short description of the requirement ', version='1.0', + priority=None, + group=None, + type=None, + uid=None, description=( - '[ClickHouse] SHALL return `1` when user sends a simple `SELECT 1` query.\n' - ) + 'This is a long description of the requirement that can include any\n' + 'relevant information. \n' + '\n' + 'The one-line block that follows the requirement defines the `version` \n' + 'of the requirement. The version is controlled manually and is used\n' + 'to indicate material changes to the requirement that would \n' + 'require tests that cover this requirement to be updated.\n' + '\n' + 'It is a good practice to use requirement names that are broken\n' + 'up into groups. It is not recommended to use only numbers\n' + 'because if the requirement must be moved the numbering will not match.\n' + 'Therefore, the requirement name should start with the group\n' + 'name which is then followed by a number if any. For example,\n' + '\n' + ' RQ.SRS001.Group.Subgroup.1\n' + '\n' + "To keep names short, try to use abbreviations for the requirement's group name.\n" + ), + link=None + ) + +RQ_CH_SRS001_Example_Subgroup_short_description_of_the_sub_requirement = Requirement( + name='RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + 'This an example of a sub-requirement of the [RQ.CH-SRS001.Example](#rqch-srs001example).\n' + 'CH_SRS001_Software_Requirements_Template.md\n' + ), + link=None + ) + +RQ_CH_SRS001_Example_Select_1 = Requirement( + name='RQ.CH-SRS001.Example.Select.1', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL return `1` when user executes query\n' + '\n' + '```sql\n' + 'SELECT 1\n' + '```\n' + ), + link=None ) diff --git a/tests/testflows/example/tests/example.py b/tests/testflows/example/tests/example.py index 8559f290cc4..6586fa80e41 100644 --- a/tests/testflows/example/tests/example.py +++ b/tests/testflows/example/tests/example.py @@ -6,7 +6,7 @@ from example.requirements import * @TestScenario @Name("select 1") @Requirements( - RQ_ClickHouse_Select_1("1.0") + RQ_CH_SRS001_Example_Select_1("1.0") ) def scenario(self, node="clickhouse1"): """Check that ClickHouse returns 1 when user executes `SELECT 1` query. From a233aa257d6bb7a3a1249c25ca0f90703e6aa642 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jul 2020 20:48:17 +0200 Subject: [PATCH 065/196] Small changes to the SRS template. --- .../example/requirements/CH_SRS001_ClickHouse.html | 8 ++++---- .../example/requirements/CH_SRS001_ClickHouse.md | 8 ++++---- tests/testflows/example/requirements/requirements.py | 8 ++++---- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html index 5aedad25979..ec466d16888 100644 --- a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html @@ -1313,8 +1313,8 @@ WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
  • 4 Requirements
  • 5 References
  • @@ -1390,7 +1390,7 @@ the following prefix:

    then immediately followed by a one-line block that contains the the version of the requirement.

    -

    RQ.CH-SRS001.Example short description of the requirement

    +

    RQ.CH-SRS001.Example

    version: 1.0

    @@ -1413,7 +1413,7 @@ name which is then followed by a number if any. For example,

    To keep names short, try to use abbreviations for the requirement's group name.

    -

    RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement

    +

    RQ.CH-SRS001.Example.Subgroup

    version: 1.0

    diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md index 370ff14c2cb..284c6810958 100644 --- a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md @@ -15,8 +15,8 @@ * 3.1 [SRS](#srs) * 3.2 [Some term that you will use](#some-term-that-you-will-use) * 4 [Requirements](#requirements) - * 4.1 [RQ.CH-SRS001.Example short description of the requirement ](#rqch-srs001example-short-description-of-the-requirement-) - * 4.2 [RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement](#rqch-srs001examplesubgroup-short-description-of-the-sub-requirement) + * 4.1 [RQ.CH-SRS001.Example](#rqch-srs001example) + * 4.2 [RQ.CH-SRS001.Example.Subgroup](#rqch-srs001examplesubgroup) * 4.3 [RQ.CH-SRS001.Example.Select.1](#rqch-srs001exampleselect1) * 5 [References](#references) @@ -90,7 +90,7 @@ the following prefix: then immediately followed by a one-line block that contains the the `version` of the requirement. -### RQ.CH-SRS001.Example short description of the requirement +### RQ.CH-SRS001.Example version: 1.0 This is a long description of the requirement that can include any @@ -111,7 +111,7 @@ name which is then followed by a number if any. For example, To keep names short, try to use abbreviations for the requirement's group name. -### RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement +### RQ.CH-SRS001.Example.Subgroup version: 1.0 This an example of a sub-requirement of the [RQ.CH-SRS001.Example](#rqch-srs001example). diff --git a/tests/testflows/example/requirements/requirements.py b/tests/testflows/example/requirements/requirements.py index f890b9a1daf..b42a73c293e 100644 --- a/tests/testflows/example/requirements/requirements.py +++ b/tests/testflows/example/requirements/requirements.py @@ -5,8 +5,8 @@ # using 'tfs requirements generate' command. from testflows.core import Requirement -RQ_CH_SRS001_Example_short_description_of_the_requirement_ = Requirement( - name='RQ.CH-SRS001.Example short description of the requirement ', +RQ_CH_SRS001_Example = Requirement( + name='RQ.CH-SRS001.Example', version='1.0', priority=None, group=None, @@ -34,8 +34,8 @@ RQ_CH_SRS001_Example_short_description_of_the_requirement_ = Requirement( link=None ) -RQ_CH_SRS001_Example_Subgroup_short_description_of_the_sub_requirement = Requirement( - name='RQ.CH-SRS001.Example.Subgroup short description of the sub-requirement', +RQ_CH_SRS001_Example_Subgroup = Requirement( + name='RQ.CH-SRS001.Example.Subgroup', version='1.0', priority=None, group=None, From 464eb70ca4d4602116d15b694a6483cf591ed3dc Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jul 2020 20:52:17 +0200 Subject: [PATCH 066/196] Removing unnecessary text. --- tests/testflows/example/requirements/CH_SRS001_ClickHouse.html | 3 +-- tests/testflows/example/requirements/CH_SRS001_ClickHouse.md | 1 - tests/testflows/example/requirements/requirements.py | 1 - 3 files changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html index ec466d16888..9cfb3564288 100644 --- a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html @@ -1417,8 +1417,7 @@ name which is then followed by a number if any. For example,

    version: 1.0

    -

    This an example of a sub-requirement of the RQ.CH-SRS001.Example. -CH_SRS001_Software_Requirements_Template.md

    +

    This an example of a sub-requirement of the RQ.CH-SRS001.Example.

    RQ.CH-SRS001.Example.Select.1

    diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md index 284c6810958..e4eb2e38c79 100644 --- a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md @@ -115,7 +115,6 @@ To keep names short, try to use abbreviations for the requirement's group name. version: 1.0 This an example of a sub-requirement of the [RQ.CH-SRS001.Example](#rqch-srs001example). -CH_SRS001_Software_Requirements_Template.md ### RQ.CH-SRS001.Example.Select.1 version: 1.0 diff --git a/tests/testflows/example/requirements/requirements.py b/tests/testflows/example/requirements/requirements.py index b42a73c293e..ef28a5fe149 100644 --- a/tests/testflows/example/requirements/requirements.py +++ b/tests/testflows/example/requirements/requirements.py @@ -43,7 +43,6 @@ RQ_CH_SRS001_Example_Subgroup = Requirement( uid=None, description=( 'This an example of a sub-requirement of the [RQ.CH-SRS001.Example](#rqch-srs001example).\n' - 'CH_SRS001_Software_Requirements_Template.md\n' ), link=None ) From 3f9dd7f62bae54ad6a8292bbbf511ae3f66c72e3 Mon Sep 17 00:00:00 2001 From: Vitaliy Zakaznikov Date: Thu, 2 Jul 2020 21:02:15 +0200 Subject: [PATCH 067/196] Fix small typo. --- tests/testflows/example/requirements/CH_SRS001_ClickHouse.html | 2 +- tests/testflows/example/requirements/CH_SRS001_ClickHouse.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html index 9cfb3564288..8c40d7484cd 100644 --- a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.html @@ -1366,7 +1366,7 @@ or a new requirement is added.

    Terminology

    -

    You can define terminolgy using the examples below and make you can make them +

    You can define terminolgy using the examples below and you can make them linkable as SRS by defining the links in the References section.

    SRS

    diff --git a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md index e4eb2e38c79..36445fab6cf 100644 --- a/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md +++ b/tests/testflows/example/requirements/CH_SRS001_ClickHouse.md @@ -67,7 +67,7 @@ cat CH_SRS001_ClickHouse.md | tfs requirements generate > requirements.py ## Terminology -You can define terminolgy using the examples below and make you can make them +You can define terminolgy using the examples below and you can make them linkable as [SRS] by defining the links in the [References](#References) section. ### SRS From 5e8d5bc8c0169fceac5465e5151145c50b2da071 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Jul 2020 11:10:52 +0300 Subject: [PATCH 068/196] Fix overlimiting the nuber of threads for union. --- src/Processors/QueryPipeline.cpp | 11 ++++++++++- src/Processors/QueryPlan/UnionStep.cpp | 9 +-------- 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 7ad7bddb104..965f462f901 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -565,6 +565,11 @@ void QueryPipeline::setOutputFormat(ProcessorPtr output) void QueryPipeline::unitePipelines( std::vector> pipelines, const Block & common_header) { + /// Should we limit the number of threads for united pipeline. True if all pipelines have max_threads != 0. + /// If true, result max_threads will be sum(max_threads). + /// Note: it may be > than settings.max_threads, so we should apply this limit again. + bool limit_max_threads = !initialized() || max_threads != 0; + if (initialized()) { addSimpleTransform([&](const Block & header) @@ -625,9 +630,13 @@ void QueryPipeline::unitePipelines( interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end()); storage_holders.insert(storage_holders.end(), pipeline.storage_holders.begin(), pipeline.storage_holders.end()); - max_threads = std::max(max_threads, pipeline.max_threads); + max_threads += pipeline.max_threads; + limit_max_threads = limit_max_threads && pipeline.max_threads != 0; } + if (!limit_max_threads) + max_threads = 0; + if (!extremes.empty()) { if (extremes.size() == 1) diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index 14a43cac78b..bb19649f9c6 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -27,15 +27,8 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines) return pipeline; } - size_t num_pipelines = pipelines.size(); pipeline->unitePipelines(std::move(pipelines), output_stream->header); - - if (num_pipelines > 1) - { - // nested queries can force 1 thread (due to simplicity) - // but in case of union this cannot be done. - pipeline->setMaxThreads(std::min(num_pipelines, max_threads)); - } + pipeline->limitMaxThreads(max_threads); return pipeline; } From 8cfe5f137aa63e2f5844e7b19f2f8d39b4de4a56 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Jul 2020 11:22:21 +0300 Subject: [PATCH 069/196] Added test. --- .../queries/0_stateless/01358_union_threads_bug.reference | 2 ++ tests/queries/0_stateless/01358_union_threads_bug.sql | 7 +++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/01358_union_threads_bug.reference create mode 100644 tests/queries/0_stateless/01358_union_threads_bug.sql diff --git a/tests/queries/0_stateless/01358_union_threads_bug.reference b/tests/queries/0_stateless/01358_union_threads_bug.reference new file mode 100644 index 00000000000..4397f4e2fdd --- /dev/null +++ b/tests/queries/0_stateless/01358_union_threads_bug.reference @@ -0,0 +1,2 @@ +300 +1 diff --git a/tests/queries/0_stateless/01358_union_threads_bug.sql b/tests/queries/0_stateless/01358_union_threads_bug.sql new file mode 100644 index 00000000000..0c07364f41c --- /dev/null +++ b/tests/queries/0_stateless/01358_union_threads_bug.sql @@ -0,0 +1,7 @@ +set log_queries = 1; +set max_threads = 16; + +SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100); + +system flush logs; +select length(thread_ids) >= 16 from system.query_log where event_date >= today() - 1 and query like '%SELECT count() FROM (SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100 UNION ALL SELECT number FROM numbers_mt(1000000) ORDER BY number DESC LIMIT 100)%' and type = 'QueryFinish' order by query_start_time desc limit 1; From e7e4d29a540d76bdf3ba607925a4bf2b6de559cf Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Fri, 3 Jul 2020 15:52:16 +0300 Subject: [PATCH 070/196] more fuzzing --- programs/client/Client.cpp | 57 ++++++++++++++++++++++++++------- programs/client/QueryFuzzer.cpp | 1 + src/Parsers/parseQuery.cpp | 9 ++++++ 3 files changed, 55 insertions(+), 12 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index f91588faeed..2a500ea277f 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -790,12 +790,32 @@ private: std::string text; while (!in.eof()) { - readStringInto(text, in); - // Append the separator as well - if (!in.eof()) + // Read until ';' + while (!in.eof()) { - text.append(1, *in.position()); - ++in.position(); + char * next_semicolon = find_first_symbols<';'>(in.position(), + in.buffer().end()); + + if (next_semicolon < in.buffer().end()) + { + // Found the semicolon, append it as well. + next_semicolon++; + text.append(in.position(), next_semicolon - in.position()); + in.position() = next_semicolon; + break; + } + + // Didn't find the semicolon and reached the end of buffer. + text.append(in.position(), next_semicolon - in.position()); + in.position() = next_semicolon; + + if (text.size() > 1024 * 1024) + { + // We've read a lot of text and still haven't seen a semicolon. + // Likely some pathological input, just fall through to prevent + // too long loops. + break; + } } fprintf(stderr, "will now parse '%s'\n", text.c_str()); @@ -824,14 +844,15 @@ private: return; } - if (text.size() > 1024 * 1024) + if (text.size() > 4 * 1024) { // Some pathological situation where the text is larger than 1MB // and we still cannot parse a single query in it. Abort. std::cerr << "Read too much text and still can't parse a query." " Aborting." << std::endl; last_exception_received_from_server.reset(new Exception(1, "~")); - return; + // return; + exit(1); } } } @@ -992,6 +1013,9 @@ private: ASTPtr fuzz_base = orig_ast; for (int fuzz_step = 0; fuzz_step < query_fuzzer_runs; fuzz_step++) { + fprintf(stderr, "fuzzing step %d for query at pos %zd\n", + fuzz_step, this_query_begin - text.data()); + ASTPtr ast_to_process; try { @@ -1001,7 +1025,13 @@ private: auto base_after_fuzz = fuzz_base->formatForErrorMessage(); // Debug AST cloning errors. - assert(base_before_fuzz == base_after_fuzz); + if (base_before_fuzz != base_after_fuzz) + { + fprintf(stderr, "base before fuzz: %s\n" + "base after fuzz: %s\n", base_before_fuzz.c_str(), + base_after_fuzz.c_str()); + assert(false); + } auto fuzzed_text = ast_to_process->formatForErrorMessage(); if (fuzz_step > 0 && fuzzed_text == base_before_fuzz) @@ -1030,7 +1060,7 @@ private: } else if (ast_to_process->formatForErrorMessage().size() > 500) { - // ast too long, please no; start from original ast + // ast too long, start from original ast fprintf(stderr, "current ast too long, won't elaborate\n"); fuzz_base = orig_ast; } @@ -1043,7 +1073,7 @@ private: } } - return end; + return begin; } void processTextAsSingleQuery(const String & text_) @@ -1712,8 +1742,11 @@ private: processed_rows += block.rows(); initBlockOutputStream(block); - /// The header block containing zero rows was used to initialize block_out_stream, do not output it. - if (block.rows() != 0) + /// The header block containing zero rows was used to initialize + /// block_out_stream, do not output it. + /// Also do not output too much data if we're fuzzing. + if (block.rows() != 0 + && (query_fuzzer_runs == 0 || processed_rows < 100)) { block_out_stream->write(block); written_first_block = true; diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index b31653b666b..72e7fb1aa9a 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -66,6 +66,7 @@ Field QueryFuzzer::getRandomField(int type) } default: assert(false); + return Null{}; } } diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index dcc587c83ef..7982b46d8b8 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -225,6 +225,15 @@ ASTPtr tryParseQuery( || token_iterator->type == TokenType::Semicolon) { out_error_message = "Empty query"; + // Token iterator skips over comments, so we'll get this error for queries + // like this: + // " + // -- just a comment + // ; + //" + // Advance the position, so that we can use this parser for stream parsing + // even in presence of such queries. + pos = token_iterator->begin; return nullptr; } From 8349afa4adeb6e6c1629bfdc819f3aa338e7864b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 3 Jul 2020 16:38:35 +0300 Subject: [PATCH 071/196] Try fix tests. --- src/Processors/QueryPipeline.cpp | 15 +++++++++++---- src/Processors/QueryPipeline.h | 2 +- src/Processors/QueryPlan/UnionStep.cpp | 3 +-- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 965f462f901..6b121490be7 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -563,12 +563,12 @@ void QueryPipeline::setOutputFormat(ProcessorPtr output) } void QueryPipeline::unitePipelines( - std::vector> pipelines, const Block & common_header) + std::vector> pipelines, const Block & common_header, size_t max_threads_limit) { /// Should we limit the number of threads for united pipeline. True if all pipelines have max_threads != 0. /// If true, result max_threads will be sum(max_threads). /// Note: it may be > than settings.max_threads, so we should apply this limit again. - bool limit_max_threads = !initialized() || max_threads != 0; + bool will_limit_max_threads = !initialized() || max_threads != 0; if (initialized()) { @@ -631,11 +631,18 @@ void QueryPipeline::unitePipelines( storage_holders.insert(storage_holders.end(), pipeline.storage_holders.begin(), pipeline.storage_holders.end()); max_threads += pipeline.max_threads; - limit_max_threads = limit_max_threads && pipeline.max_threads != 0; + will_limit_max_threads = will_limit_max_threads && pipeline.max_threads != 0; + + /// If one of pipelines uses more threads then current limit, will keep it. + /// It may happen if max_distributed_connections > max_threads + if (pipeline.max_threads > max_threads_limit) + max_threads_limit = pipeline.max_threads; } - if (!limit_max_threads) + if (!will_limit_max_threads) max_threads = 0; + else + max_threads = max_threads_limit; if (!extremes.empty()) { diff --git a/src/Processors/QueryPipeline.h b/src/Processors/QueryPipeline.h index adab155d201..3ea34302155 100644 --- a/src/Processors/QueryPipeline.h +++ b/src/Processors/QueryPipeline.h @@ -134,7 +134,7 @@ public: void enableQuotaForCurrentStreams(); - void unitePipelines(std::vector> pipelines, const Block & common_header); + void unitePipelines(std::vector> pipelines, const Block & common_header, size_t max_threads_limit = 0); PipelineExecutorPtr execute(); diff --git a/src/Processors/QueryPlan/UnionStep.cpp b/src/Processors/QueryPlan/UnionStep.cpp index bb19649f9c6..7b2510330f1 100644 --- a/src/Processors/QueryPlan/UnionStep.cpp +++ b/src/Processors/QueryPlan/UnionStep.cpp @@ -27,8 +27,7 @@ QueryPipelinePtr UnionStep::updatePipeline(QueryPipelines pipelines) return pipeline; } - pipeline->unitePipelines(std::move(pipelines), output_stream->header); - pipeline->limitMaxThreads(max_threads); + pipeline->unitePipelines(std::move(pipelines), output_stream->header ,max_threads); return pipeline; } From 6e599533a72fc6766fa1df7e262911ff5f5bf85f Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 3 Jul 2020 19:19:32 +0200 Subject: [PATCH 072/196] Add missing query context for system logs Needed to allow attaching materialized views with joins or with subqueries to system.logs. --- src/Interpreters/SystemLog.h | 7 ++- ...ized_view_with_join_on_query_log.reference | 10 ++++ ...terialized_view_with_join_on_query_log.sql | 52 +++++++++++++++++++ 3 files changed, 67 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference create mode 100644 tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 48cff8bf061..1b5bcbacc6d 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -62,7 +63,6 @@ namespace ErrorCodes #define DBMS_SYSTEM_LOG_QUEUE_SIZE 1048576 -class Context; class QueryLog; class QueryThreadLog; class PartLog; @@ -161,6 +161,7 @@ protected: private: /* Saving thread data */ Context & context; + Context insert_context; const StorageID table_id; const String storage_def; StoragePtr table; @@ -207,11 +208,13 @@ SystemLog::SystemLog(Context & context_, const String & storage_def_, size_t flush_interval_milliseconds_) : context(context_) + , insert_context(Context(context_)) , table_id(database_name_, table_name_) , storage_def(storage_def_) , flush_interval_milliseconds(flush_interval_milliseconds_) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); + insert_context.makeQueryContext(); // we need query context to do inserts to target table with MV containing subqueries or joins log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); } @@ -425,7 +428,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, insert->table_id = table_id; ASTPtr query_ptr(insert.release()); - InterpreterInsertQuery interpreter(query_ptr, context); + InterpreterInsertQuery interpreter(query_ptr, insert_context); BlockIO io = interpreter.execute(); io.out->writePrefix(); diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference new file mode 100644 index 00000000000..dfc14d94653 --- /dev/null +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference @@ -0,0 +1,10 @@ +1 +1 +1 +1 +=== system.query_log === +main_dashboard_bottom_query 2 +main_dashboard_top_query 2 +=== slowlog === +main_dashboard_bottom_query 1 +main_dashboard_top_query 1 diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql new file mode 100644 index 00000000000..3d1b464f164 --- /dev/null +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql @@ -0,0 +1,52 @@ +DROP TABLE IF EXISTS slow_log; +DROP TABLE IF EXISTS expected_times; + +CREATE TABLE expected_times (QUERY_GROUP_ID String, max_query_duration_ms UInt64) Engine=Memory; +INSERT INTO expected_times VALUES('main_dashboard_top_query', 100), ('main_dashboard_bottom_query', 100); + +CREATE MATERIALIZED VIEW slow_log Engine=Memory AS +( + SELECT * FROM + ( + SELECT + extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID, + * + FROM system.query_log + WHERE type<>1 and event_date >= yesterday() and event_time > now() - 120 + ) as ql + INNER JOIN expected_times USING (QUERY_GROUP_ID) + WHERE query_duration_ms > max_query_duration_ms +); + +SET log_queries=1; + +SELECT 1 /* QUERY_GROUP_ID:main_dashboard_top_query */; +SELECT 1 /* QUERY_GROUP_ID:main_dashboard_bottom_query */; + +SELECT 1 WHERE not ignore(sleep(0.105)) /* QUERY_GROUP_ID:main_dashboard_top_query */; +SELECT 1 WHERE not ignore(sleep(0.105)) /* QUERY_GROUP_ID:main_dashboard_bottom_query */; + +SET log_queries=0; +SYSTEM FLUSH LOGS; + +SELECT '=== system.query_log ==='; + +SELECT + extract(query,'/\\*\\s*QUERY_GROUP_ID:(.*?)\\s*\\*/') as QUERY_GROUP_ID, + count() +FROM system.query_log +WHERE type<>1 and event_date >= yesterday() and event_time > now() - 20 and QUERY_GROUP_ID<>'' +GROUP BY QUERY_GROUP_ID +ORDER BY QUERY_GROUP_ID; + +SELECT '=== slowlog ==='; + +SELECT + QUERY_GROUP_ID, + count() +FROM slow_log +GROUP BY QUERY_GROUP_ID +ORDER BY QUERY_GROUP_ID; + +DROP TABLE slow_log; +DROP TABLE expected_times; From a3f0ba17ea98a6c1889643b82f6ae35936b57c72 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 5 Jul 2020 19:26:57 +0300 Subject: [PATCH 073/196] Update SortDescription.h --- src/Core/SortDescription.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index f51ee0ba135..d433d369d0b 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -72,6 +72,8 @@ struct SortColumnDescription using SortDescription = std::vector; class Block; + +/// Outputs user-readable description into `out`. void dumpSortDescription(const SortDescription & description, const Block & header, WriteBuffer & out); } From 6a013b966c61cb0bbf2342d58e38e842057935ad Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 5 Jul 2020 19:27:54 +0300 Subject: [PATCH 074/196] Update AggregateDescription.cpp --- src/Interpreters/AggregateDescription.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp index f6f2e9287a7..4cf50cc1f35 100644 --- a/src/Interpreters/AggregateDescription.cpp +++ b/src/Interpreters/AggregateDescription.cpp @@ -27,6 +27,7 @@ void AggregateDescription::explain(WriteBuffer & out, size_t ident) const if (function) { + /// Double whitespace is intentional. out << prefix << " Function: " << function->getName(); const auto & params = function->getParameters(); From 37ac45643947e2ecf085e0c840341e526a3b44f1 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 6 Jul 2020 09:02:02 +0800 Subject: [PATCH 075/196] Support KILL QUERY [connection_id] for MySQL --- src/Server/MySQLHandler.cpp | 30 ++++++++++++++++++++++++++++-- src/Server/MySQLHandler.h | 1 + 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index a97182f15fc..6892ebbd31a 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -103,7 +103,8 @@ void MySQLHandler::run() { if (!handshake_response.database.empty()) connection_context.setCurrentDatabase(handshake_response.database); - connection_context.setCurrentQueryId(""); + connection_context.setCurrentQueryId(Poco::format("mysql:%lu", connection_id)); + } catch (const Exception & exc) { @@ -295,6 +296,12 @@ void MySQLHandler::comQuery(ReadBuffer & payload) replacement_query = boost::replace_all_copy(query, "SHOW TABLE STATUS LIKE ", show_table_status_replacement_query); } + if (0 == strncasecmp("KILL QUERY", query.c_str(), 10)) + { + should_replace = true; + replacement_query = kill_connection_id_replacement_query(query); + } + if (0 == strncasecmp("SHOW VARIABLES", query.c_str(), 13)) { should_replace = true; @@ -379,7 +386,7 @@ const String MySQLHandler::show_table_status_replacement_query("SELECT" " 'Dynamic' AS Row_format," " 0 AS Rows," " 0 AS Avg_row_length," - " 0 AS Data_length," + " 0 AS Data_length," " 0 AS Max_data_length," " 0 AS Index_length," " 0 AS Data_free," @@ -394,4 +401,23 @@ const String MySQLHandler::show_table_status_replacement_query("SELECT" " FROM system.tables" " WHERE name LIKE "); +String MySQLHandler::kill_connection_id_replacement_query(const String & query) +{ + const String s = "KILL QUERY "; + + if (query.size() > s.size()) + { + String process_id = query.data() + s.length(); + + static const std::regex expr{"^[0-9]"}; + if (std::regex_match(process_id, expr)) + { + String replacement = Poco::format("KILL QUERY WHERE query_id = 'mysql:%s'", process_id); + return replacement; + } + } + return query; } + +} + diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 5f506089493..41d4cc9a483 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -73,6 +73,7 @@ protected: private: static const String show_table_status_replacement_query; + String kill_connection_id_replacement_query(const String & query); }; #if USE_SSL From 95a7a09c37844e9c9537933d795d6fdfbcb46626 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 6 Jul 2020 04:50:45 +0300 Subject: [PATCH 076/196] Fix transform query for external databases in presense of aliases #12032 --- .../JoinToSubqueryTransformVisitor.cpp | 2 +- src/Interpreters/PredicateRewriteVisitor.cpp | 2 +- ..._transform_query_for_external_database.cpp | 12 ++++++++ .../transformQueryForExternalDatabase.cpp | 28 +++++++++++++++++++ 4 files changed, 42 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index a34276b5519..5f38f410e04 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -257,7 +257,7 @@ struct ColumnAliasesMatcher if (!last_table) { IdentifierSemantic::coverName(node, alias); - node.setAlias(""); + node.setAlias({}); } } else if (node.compound()) diff --git a/src/Interpreters/PredicateRewriteVisitor.cpp b/src/Interpreters/PredicateRewriteVisitor.cpp index 7fc45044a88..2a4bd4c1fd2 100644 --- a/src/Interpreters/PredicateRewriteVisitor.cpp +++ b/src/Interpreters/PredicateRewriteVisitor.cpp @@ -76,7 +76,7 @@ static void cleanAliasAndCollectIdentifiers(ASTPtr & predicate, std::vectortryGetAlias(); !alias.empty()) - predicate->setAlias(""); + predicate->setAlias({}); if (ASTIdentifier * identifier = predicate->as()) identifiers.emplace_back(identifier); diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index 318d667d9b0..bd7d7d5d1b8 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -27,6 +28,8 @@ struct State {"apply_type", std::make_shared()}, {"apply_status", std::make_shared()}, {"create_time", std::make_shared()}, + {"field", std::make_shared()}, + {"value", std::make_shared()}, }; static const State & instance() @@ -117,3 +120,12 @@ TEST(TransformQueryForExternalDatabase, Issue7245) R"(SELECT "apply_id", "apply_type", "apply_status", "create_time" FROM "test"."table" WHERE ("apply_type" = 2) AND ("create_time" > '2018-12-25 01:02:03') AND ("apply_status" IN (3, 4)))", state.context, state.columns); } + +TEST(TransformQueryForExternalDatabase, Aliases) +{ + const State & state = State::instance(); + + check("SELECT field AS value, field AS display WHERE field NOT IN ('') AND display LIKE '%test%'", + R"(SELECT "field" FROM "test"."table" WHERE ("field" NOT IN ('')) AND ("field" LIKE '%test%'))", + state.context, state.columns); +} diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 11b98b782e0..0d34f0b3068 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -71,6 +71,24 @@ public: } }; +class DropAliasesMatcher +{ +public: + struct Data {}; + Data data; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) + { + return true; + } + + static void visit(ASTPtr & node, Data) + { + if (!node->tryGetAlias().empty()) + node->setAlias({}); + } +}; + void replaceConstantExpressions(ASTPtr & node, const Context & context, const NamesAndTypesList & all_columns) { auto syntax_result = SyntaxAnalyzer(context).analyze(node, all_columns); @@ -80,6 +98,13 @@ void replaceConstantExpressions(ASTPtr & node, const Context & context, const Na visitor.visit(node); } +void dropAliases(ASTPtr & node) +{ + DropAliasesMatcher::Data data; + InDepthNodeVisitor visitor(data); + visitor.visit(node); +} + bool isCompatible(const IAST & node) { @@ -192,6 +217,9 @@ String transformQueryForExternalDatabase( } } + ASTPtr select_ptr = select; + dropAliases(select_ptr); + std::stringstream out; IAST::FormatSettings settings(out, true); settings.identifier_quoting_style = identifier_quoting_style; From cb04c503d7a9acbe88154034ab20cb445f1d6938 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 6 Jul 2020 10:07:38 +0800 Subject: [PATCH 077/196] Add MySQL to ClickHouse query replacement mapping table --- src/Server/MySQLHandler.cpp | 106 +++++++++++++++++++++--------------- src/Server/MySQLHandler.h | 5 +- 2 files changed, 65 insertions(+), 46 deletions(-) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 6892ebbd31a..9e42f5ebc05 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -45,6 +45,10 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } +static String select_empty_replacement_query(const String & query); +static String show_table_status_replacement_query(const String & query); +static String kill_connection_id_replacement_query(const String & query); + MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_) : Poco::Net::TCPServerConnection(socket_) @@ -57,6 +61,10 @@ MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & so server_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF; if (ssl_enabled) server_capability_flags |= CLIENT_SSL; + + replacements.emplace("KILL QUERY", kill_connection_id_replacement_query); + replacements.emplace("SHOW TABLE STATUS LIKE", show_table_status_replacement_query); + replacements.emplace("SHOW VARIABLES", select_empty_replacement_query); } void MySQLHandler::run() @@ -285,26 +293,18 @@ void MySQLHandler::comQuery(ReadBuffer & payload) } else { - String replacement_query = "SELECT ''"; + String replacement_query; bool should_replace = false; bool with_output = false; - // This is a workaround in order to support adding ClickHouse to MySQL using federated server. - if (0 == strncasecmp("SHOW TABLE STATUS LIKE", query.c_str(), 22)) + for (auto const & x : replacements) { - should_replace = true; - replacement_query = boost::replace_all_copy(query, "SHOW TABLE STATUS LIKE ", show_table_status_replacement_query); - } - - if (0 == strncasecmp("KILL QUERY", query.c_str(), 10)) - { - should_replace = true; - replacement_query = kill_connection_id_replacement_query(query); - } - - if (0 == strncasecmp("SHOW VARIABLES", query.c_str(), 13)) - { - should_replace = true; + if (0 == strncasecmp(x.first.c_str(), query.c_str(), x.first.size())) + { + should_replace = true; + replacement_query = x.second(query); + break; + } } ReadBufferFromString replacement(replacement_query); @@ -379,40 +379,58 @@ static bool isFederatedServerSetupSetCommand(const String & query) return 1 == std::regex_match(query, expr); } -const String MySQLHandler::show_table_status_replacement_query("SELECT" - " name AS Name," - " engine AS Engine," - " '10' AS Version," - " 'Dynamic' AS Row_format," - " 0 AS Rows," - " 0 AS Avg_row_length," - " 0 AS Data_length," - " 0 AS Max_data_length," - " 0 AS Index_length," - " 0 AS Data_free," - " 'NULL' AS Auto_increment," - " metadata_modification_time AS Create_time," - " metadata_modification_time AS Update_time," - " metadata_modification_time AS Check_time," - " 'utf8_bin' AS Collation," - " 'NULL' AS Checksum," - " '' AS Create_options," - " '' AS Comment" - " FROM system.tables" - " WHERE name LIKE "); - -String MySQLHandler::kill_connection_id_replacement_query(const String & query) +/// Replace "[query(such as SHOW VARIABLES...)]" into "". +static String select_empty_replacement_query(const String & query) { - const String s = "KILL QUERY "; + std::ignore = query; + return "select ''"; +} - if (query.size() > s.size()) +/// Replace "SHOW TABLE STATUS LIKE 'xx'" into "SELECT ... FROM system.tables WHERE name LIKE 'xx'". +static String show_table_status_replacement_query(const String & query) +{ + const String prefix = "SHOW TABLE STATUS LIKE "; + if (query.size() > prefix.size()) { - String process_id = query.data() + s.length(); + String suffix = query.data() + prefix.length(); + return ( + "SELECT" + " name AS Name," + " engine AS Engine," + " '10' AS Version," + " 'Dynamic' AS Row_format," + " 0 AS Rows," + " 0 AS Avg_row_length," + " 0 AS Data_length," + " 0 AS Max_data_length," + " 0 AS Index_length," + " 0 AS Data_free," + " 'NULL' AS Auto_increment," + " metadata_modification_time AS Create_time," + " metadata_modification_time AS Update_time," + " metadata_modification_time AS Check_time," + " 'utf8_bin' AS Collation," + " 'NULL' AS Checksum," + " '' AS Create_options," + " '' AS Comment" + " FROM system.tables" + " WHERE name LIKE " + + suffix); + } + return query; +} +/// Replace "KILL QUERY [connection_id]" into "KILL QUERY WHERE query_id = 'mysql:[connection_id]'". +static String kill_connection_id_replacement_query(const String & query) +{ + const String prefix = "KILL QUERY "; + if (query.size() > prefix.size()) + { + String suffix = query.data() + prefix.length(); static const std::regex expr{"^[0-9]"}; - if (std::regex_match(process_id, expr)) + if (std::regex_match(suffix, expr)) { - String replacement = Poco::format("KILL QUERY WHERE query_id = 'mysql:%s'", process_id); + String replacement = Poco::format("KILL QUERY WHERE query_id = 'mysql:%s'", suffix); return replacement; } } diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 41d4cc9a483..f7596850a8b 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -72,8 +72,9 @@ protected: bool secure_connection = false; private: - static const String show_table_status_replacement_query; - String kill_connection_id_replacement_query(const String & query); + using ReplacementFn = std::function; + using Replacements = std::unordered_map; + Replacements replacements; }; #if USE_SSL From 18c48ce12c9d86b1626c168a113993b32b3f131e Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Mon, 6 Jul 2020 16:26:06 +0800 Subject: [PATCH 078/196] Add integration test for mysql replacement query --- tests/integration/test_mysql_protocol/test.py | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 4ab225aee20..507445537b8 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -138,6 +138,34 @@ def test_mysql_client(mysql_client, server_address): assert stdout == '\n'.join(['column', '0', '0', '1', '1', '5', '5', 'tmp_column', '0', '1', '']) + # Show table status. + code, (stdout, stderr) = mysql_client.exec_run(''' + mysql --protocol tcp -h {host} -P {port} default -u default + --password=123 -e "show table status like 'xx';" + '''.format(host=server_address, port=server_port), demux=True) + assert code == 0 + + # show variables. + code, (stdout, stderr) = mysql_client.exec_run(''' + mysql --protocol tcp -h {host} -P {port} default -u default + --password=123 -e "show variables;" + '''.format(host=server_address, port=server_port), demux=True) + assert code == 0 + + # Kill query. + code, (stdout, stderr) = mysql_client.exec_run(''' + mysql --protocol tcp -h {host} -P {port} default -u default + --password=123 -e "kill query 0;" + '''.format(host=server_address, port=server_port), demux=True) + assert code == 0 + + code, (stdout, stderr) = mysql_client.exec_run(''' + mysql --protocol tcp -h {host} -P {port} default -u default + --password=123 -e "kill query where query_id='mysql:0';" + '''.format(host=server_address, port=server_port), demux=True) + assert code == 0 + + def test_mysql_federated(mysql_server, server_address): # For some reason it occasionally fails without retries. retries = 100 From c1fa01cabf396b6985025100cab4df9d1d0cc478 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 6 Jul 2020 13:58:47 +0300 Subject: [PATCH 079/196] Update questdb_sum_double.xml --- tests/performance/questdb_sum_double.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/questdb_sum_double.xml b/tests/performance/questdb_sum_double.xml index c65f5f9e464..e8b4b817b80 100644 --- a/tests/performance/questdb_sum_double.xml +++ b/tests/performance/questdb_sum_double.xml @@ -29,7 +29,7 @@
    CREATE TABLE `zz_{type}_{engine}` (x {type}) ENGINE {engine} - INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(1000000000) + INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(100000000) SELECT sum(x) FROM `zz_{type}_{engine}` From 0c7a261bc6e49c5c52f2c64bfdddd2652d4eecb8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 6 Jul 2020 16:45:54 +0300 Subject: [PATCH 080/196] Place common docker compose files to integration docker container --- docker/test/integration/README.md | 4 +- docker/test/integration/runner/Dockerfile | 1 + .../compose/docker_compose_cassandra.yml | 0 .../compose/docker_compose_hdfs.yml | 0 .../compose/docker_compose_kafka.yml | 0 .../compose/docker_compose_minio.yml | 0 .../compose/docker_compose_mongo.yml | 0 .../compose/docker_compose_mysql.yml | 0 .../compose/docker_compose_net.yml | 0 .../compose/docker_compose_postgres.yml | 0 .../compose/docker_compose_rabbitmq.yml | 0 .../compose/docker_compose_redis.yml | 0 .../compose/docker_compose_zookeeper.yml | 0 tests/integration/README.md | 3 +- tests/integration/helpers/cluster.py | 67 +++++++++++-------- 15 files changed, 45 insertions(+), 30 deletions(-) rename docker/test/integration/{ => runner}/compose/docker_compose_cassandra.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_hdfs.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_kafka.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_minio.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_mongo.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_mysql.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_net.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_postgres.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_rabbitmq.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_redis.yml (100%) rename docker/test/integration/{ => runner}/compose/docker_compose_zookeeper.yml (100%) diff --git a/docker/test/integration/README.md b/docker/test/integration/README.md index 4aa10d6db80..a11cf059655 100644 --- a/docker/test/integration/README.md +++ b/docker/test/integration/README.md @@ -1,6 +1,6 @@ ## Docker containers for integration tests - `base` container with required packages - `runner` container with that runs integration tests in docker -- `compose` contains docker_compose YaML files that are used in tests +- `runnner/compose` contains docker\_compose YaML files that are used in tests -How to run integration tests is described in tests/integration/README.md \ No newline at end of file +How to run integration tests is described in tests/integration/README.md diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 423ecb06122..7608666ecc2 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -63,6 +63,7 @@ RUN set -eux; \ COPY modprobe.sh /usr/local/bin/modprobe COPY dockerd-entrypoint.sh /usr/local/bin/ +COPY compose/ /compose/ RUN set -x \ && addgroup --system dockremap \ diff --git a/docker/test/integration/compose/docker_compose_cassandra.yml b/docker/test/integration/runner/compose/docker_compose_cassandra.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_cassandra.yml rename to docker/test/integration/runner/compose/docker_compose_cassandra.yml diff --git a/docker/test/integration/compose/docker_compose_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_hdfs.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_hdfs.yml rename to docker/test/integration/runner/compose/docker_compose_hdfs.yml diff --git a/docker/test/integration/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_kafka.yml rename to docker/test/integration/runner/compose/docker_compose_kafka.yml diff --git a/docker/test/integration/compose/docker_compose_minio.yml b/docker/test/integration/runner/compose/docker_compose_minio.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_minio.yml rename to docker/test/integration/runner/compose/docker_compose_minio.yml diff --git a/docker/test/integration/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_mongo.yml rename to docker/test/integration/runner/compose/docker_compose_mongo.yml diff --git a/docker/test/integration/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_mysql.yml rename to docker/test/integration/runner/compose/docker_compose_mysql.yml diff --git a/docker/test/integration/compose/docker_compose_net.yml b/docker/test/integration/runner/compose/docker_compose_net.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_net.yml rename to docker/test/integration/runner/compose/docker_compose_net.yml diff --git a/docker/test/integration/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_postgres.yml rename to docker/test/integration/runner/compose/docker_compose_postgres.yml diff --git a/docker/test/integration/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_rabbitmq.yml rename to docker/test/integration/runner/compose/docker_compose_rabbitmq.yml diff --git a/docker/test/integration/compose/docker_compose_redis.yml b/docker/test/integration/runner/compose/docker_compose_redis.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_redis.yml rename to docker/test/integration/runner/compose/docker_compose_redis.yml diff --git a/docker/test/integration/compose/docker_compose_zookeeper.yml b/docker/test/integration/runner/compose/docker_compose_zookeeper.yml similarity index 100% rename from docker/test/integration/compose/docker_compose_zookeeper.yml rename to docker/test/integration/runner/compose/docker_compose_zookeeper.yml diff --git a/tests/integration/README.md b/tests/integration/README.md index c72c009a0d6..a3eb577d609 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -25,12 +25,13 @@ To check, that you have access to Docker, run `docker ps`. Run the tests with the `pytest` command. To select which tests to run, use: `pytest -k ` By default tests are run with system-wide client binary, server binary and base configs. To change that, -set the following environment variables: +set the following environment variables:` * `CLICKHOUSE_TESTS_SERVER_BIN_PATH` to choose the server binary. * `CLICKHOUSE_TESTS_CLIENT_BIN_PATH` to choose the client binary. * `CLICKHOUSE_TESTS_BASE_CONFIG_DIR` to choose the directory from which base configs (`config.xml` and `users.xml`) are taken. +For tests that use common docker compose files you may need to set up their path with environment variable: `DOCKER_COMPOSE_DIR=$HOME/ClickHouse/docker/test/integration/runner/compose` ### Running with runner script diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 861f178b433..f3e5dc1fab5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -31,7 +31,7 @@ from .hdfs_api import HDFSApi HELPERS_DIR = p.dirname(__file__) CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..") -DOCKER_COMPOSE_DIR = p.join(CLICKHOUSE_ROOT_DIR, "docker/test/integration/compose/") +LOCAL_DOCKER_COMPOSE_DIR = p.join(CLICKHOUSE_ROOT_DIR, "docker/test/integration/runner/compose/") DEFAULT_ENV_NAME = 'env_file' SANITIZER_SIGN = "==================" @@ -52,7 +52,7 @@ def subprocess_check_call(args): def subprocess_call(args): - # Uncomment for debugging + # Uncomment for debugging..; # print('run:', ' ' . join(args)) subprocess.call(args) @@ -67,6 +67,17 @@ def get_odbc_bridge_path(): return '/usr/bin/clickhouse-odbc-bridge' return path +def get_docker_compose_path(): + compose_path = os.environ.get('DOCKER_COMPOSE_DIR') + if compose_path is not None: + return os.path.dirname(compose_path) + else: + if os.path.exists(os.path.dirname('/compose/')): + return os.path.dirname('/compose/') #default in docker runner container + else: + print("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR)) + return LOCAL_DOCKER_COMPOSE_DIR + class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. @@ -181,17 +192,19 @@ class ClickHouseCluster: ipv6_address=ipv6_address, with_installed_binary=with_installed_binary, tmpfs=tmpfs or []) + docker_compose_yml_dir = get_docker_compose_path() + self.instances[name] = instance if ipv4_address is not None or ipv6_address is not None: self.with_net_trics = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_net.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_net.yml')]) self.base_cmd.extend(['--file', instance.docker_compose_path]) cmds = [] if with_zookeeper and not self.with_zookeeper: if not zookeeper_docker_compose_path: - zookeeper_docker_compose_path = p.join(DOCKER_COMPOSE_DIR, 'docker_compose_zookeeper.yml') + zookeeper_docker_compose_path = p.join(docker_compose_yml_dir, 'docker_compose_zookeeper.yml') self.with_zookeeper = True self.zookeeper_use_tmpfs = zookeeper_use_tmpfs @@ -202,86 +215,86 @@ class ClickHouseCluster: if with_mysql and not self.with_mysql: self.with_mysql = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) if with_postgres and not self.with_postgres: self.with_postgres = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True if not self.with_mysql: self.with_mysql = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mysql.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) if not self.with_postgres: self.with_postgres = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', self.project_name, '--file', - p.join(DOCKER_COMPOSE_DIR, 'docker_compose_postgres.yml')] + p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_kafka and not self.with_kafka: self.with_kafka = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_kafka.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) self.base_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_kafka.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) if with_rabbitmq and not self.with_rabbitmq: self.with_rabbitmq = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_rabbitmq.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) self.base_rabbitmq_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_rabbitmq.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] cmds.append(self.base_rabbitmq_cmd) if with_hdfs and not self.with_hdfs: self.with_hdfs = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_hdfs.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) self.base_hdfs_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_hdfs.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] cmds.append(self.base_hdfs_cmd) if with_mongo and not self.with_mongo: self.with_mongo = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mongo.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) self.base_mongo_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_mongo.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] cmds.append(self.base_mongo_cmd) if self.with_net_trics: for cmd in cmds: - cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_net.yml')]) + cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_net.yml')]) if with_redis and not self.with_redis: self.with_redis = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_redis.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]) self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_redis.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')] if with_minio and not self.with_minio: self.with_minio = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_minio.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) self.base_minio_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_minio.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')] cmds.append(self.base_minio_cmd) if with_cassandra and not self.with_cassandra: self.with_cassandra = True - self.base_cmd.extend(['--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_cassandra.yml')]) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]) self.base_cassandra_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(DOCKER_COMPOSE_DIR, 'docker_compose_cassandra.yml')] + self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] return instance From d31ed58f01f33e3cd37d2230ff5703e9d5d0d673 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Jul 2020 17:33:31 +0300 Subject: [PATCH 081/196] done --- programs/copier/Internals.cpp | 11 --------- programs/copier/Internals.h | 3 +-- programs/copier/TaskTableAndShard.h | 2 +- src/DataTypes/NestedUtils.cpp | 11 +++++++++ src/DataTypes/NestedUtils.h | 2 ++ .../Algorithms/SummingSortedAlgorithm.cpp | 16 +++++++++---- .../Algorithms/SummingSortedAlgorithm.h | 2 ++ .../Merges/SummingSortedTransform.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 23 ++++++++++++++++--- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +++- ...merge_tree_exclude_partition_key.reference | 13 +++++++++++ ...mming_merge_tree_exclude_partition_key.sql | 13 +++++++++++ ...tree_explicit_columns_definition.reference | 0 ...merge_tree_explicit_columns_definition.sql | 7 ++++++ 16 files changed, 91 insertions(+), 24 deletions(-) create mode 100644 tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference create mode 100644 tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql create mode 100644 tests/queries/0_stateless/01373_summing_merge_tree_explicit_columns_definition.reference create mode 100644 tests/queries/0_stateless/01373_summing_merge_tree_explicit_columns_definition.sql diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 518395e3b7d..12da07a772a 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -169,17 +169,6 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast) } -String createCommaSeparatedStringFrom(const Names & names) -{ - std::ostringstream ss; - if (!names.empty()) - { - std::copy(names.begin(), std::prev(names.end()), std::ostream_iterator(ss, ", ")); - ss << names.back(); - } - return ss.str(); -} - Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) { const auto sorting_key_ast = extractOrderBy(storage_ast); diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index b1a94e1a5ca..b61b6d59629 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -40,6 +40,7 @@ #include #include #include +#include #include #include #include @@ -197,8 +198,6 @@ ASTPtr extractPrimaryKey(const ASTPtr & storage_ast); ASTPtr extractOrderBy(const ASTPtr & storage_ast); -String createCommaSeparatedStringFrom(const Names & names); - Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast); diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 32841e93a14..0ac533d9209 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -268,7 +268,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf ParserStorage parser_storage; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); - primary_key_comma_separated = createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); + primary_key_comma_separated = Nested::createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast); } diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 5f8b9f877bf..e365c73a845 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -70,6 +70,17 @@ std::pair splitName(const std::string & name) return {{ begin, first_end }, { second_begin, end }}; } +std::string createCommaSeparatedStringFrom(const Names & names) +{ + std::ostringstream ss; + if (!names.empty()) + { + std::copy(names.begin(), std::prev(names.end()), std::ostream_iterator(ss, ", ")); + ss << names.back(); + } + return ss.str(); +} + std::string extractTableName(const std::string & nested_name) { diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index 3039fd7f118..4be3e69edfd 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -13,6 +13,8 @@ namespace Nested std::pair splitName(const std::string & name); + std::string createCommaSeparatedStringFrom(const Names & names); + /// Returns the prefix of the name to the first '.'. Or the name is unchanged if there is no dot. std::string extractTableName(const std::string & nested_name); diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 3833e3288fd..bc0c0fef6dc 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -94,6 +94,12 @@ static bool isInPrimaryKey(const SortDescription & description, const std::strin return false; } +static bool isInPartitionKey(const std::string & column_name, const Names & partition_key_columns) +{ + auto is_in_partition_key = std::find(partition_key_columns.begin(), partition_key_columns.end(), column_name); + return is_in_partition_key != partition_key_columns.end(); +} + /// Returns true if merge result is not empty static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row & row, const ColumnRawPtrs & raw_columns, size_t row_number) @@ -181,7 +187,8 @@ static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, static SummingSortedAlgorithm::ColumnsDefinition defineColumns( const Block & header, const SortDescription & description, - const Names & column_names_to_sum) + const Names & column_names_to_sum, + const Names & partition_key_columns) { size_t num_columns = header.columns(); SummingSortedAlgorithm::ColumnsDefinition def; @@ -223,8 +230,8 @@ static SummingSortedAlgorithm::ColumnsDefinition defineColumns( continue; } - /// Are they inside the PK? - if (isInPrimaryKey(description, column.name, i)) + /// Are they inside the primary key or partiton key? + if (isInPrimaryKey(description, column.name, i) || isInPartitionKey(column.name, partition_key_columns)) { def.column_numbers_not_to_aggregate.push_back(i); continue; @@ -617,9 +624,10 @@ SummingSortedAlgorithm::SummingSortedAlgorithm( const Block & header, size_t num_inputs, SortDescription description_, const Names & column_names_to_sum, + const Names & partition_key_columns, size_t max_block_size) : IMergingAlgorithmWithDelayedChunk(num_inputs, std::move(description_)) - , columns_definition(defineColumns(header, description, column_names_to_sum)) + , columns_definition(defineColumns(header, description, column_names_to_sum, partition_key_columns)) , merged_data(getMergedDataColumns(header, columns_definition), max_block_size, columns_definition) { } diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index a188a5fb538..2a455ad4cea 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -20,6 +20,8 @@ public: SortDescription description_, /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, + /// List of partition key columns. They have to be excluded. + const Names & partition_key_columns, size_t max_block_size); void initialize(Inputs inputs) override; diff --git a/src/Processors/Merges/SummingSortedTransform.h b/src/Processors/Merges/SummingSortedTransform.h index 6fc22681132..22361bb1a44 100644 --- a/src/Processors/Merges/SummingSortedTransform.h +++ b/src/Processors/Merges/SummingSortedTransform.h @@ -16,6 +16,7 @@ public: SortDescription description_, /// List of columns to be summed. If empty, all numeric columns that are not in the description are taken. const Names & column_names_to_sum, + const Names & partition_key_columns, size_t max_block_size) : IMergingTransform( num_inputs, header, header, true, @@ -23,6 +24,7 @@ public: num_inputs, std::move(description_), column_names_to_sum, + partition_key_columns, max_block_size) { } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 1adb245d9e1..6c1277596bd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -109,6 +109,7 @@ namespace ErrorCodes extern const int UNKNOWN_DISK; extern const int NOT_ENOUGH_SPACE; extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; + extern const int BAD_ARGUMENTS; } @@ -169,7 +170,7 @@ MergeTreeData::MergeTreeData( const auto settings = getSettings(); /// NOTE: using the same columns list as is read when performing actual merges. - merging_params.check(metadata_.getColumns().getAllPhysical()); + merging_params.check(metadata_); if (metadata_.sampling_key.definition_ast != nullptr) { @@ -521,8 +522,10 @@ void MergeTreeData::checkStoragePolicy(const StoragePolicyPtr & new_storage_poli } -void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) const +void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadata) const { + const auto columns = metadata.getColumns().getAllPhysical(); + if (!sign_column.empty() && mode != MergingParams::Collapsing && mode != MergingParams::VersionedCollapsing) throw Exception("Sign column for MergeTree cannot be specified in modes except Collapsing or VersionedCollapsing.", ErrorCodes::LOGICAL_ERROR); @@ -607,6 +610,21 @@ void MergeTreeData::MergingParams::check(const NamesAndTypesList & columns) cons throw Exception( "Column " + column_to_sum + " listed in columns to sum does not exist in table declaration.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); } + + /// Check that summing columns are not in partition key. + if (metadata.isPartitionKeyDefined()) + { + auto partition_key_columns = metadata.getPartitionKey().column_names; + + Names names_intersection; + std::set_intersection(columns_to_sum.begin(), columns_to_sum.end(), + partition_key_columns.begin(), partition_key_columns.end(), + std::back_inserter(names_intersection)); + + if (!names_intersection.empty()) + throw Exception("Colums: " + Nested::createCommaSeparatedStringFrom(names_intersection) + + " listed both in colums to sum and in partition key. That is not allowed.", ErrorCodes::BAD_ARGUMENTS); + } } if (mode == MergingParams::Replacing) @@ -638,7 +656,6 @@ String MergeTreeData::MergingParams::getModeName() const __builtin_unreachable(); } - Int64 MergeTreeData::getMaxBlockNumber() const { auto lock = lockParts(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8fcb879b3ff..0038d2306a4 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -301,7 +301,7 @@ public: Graphite::Params graphite_params; /// Check that needed columns are present and have correct types. - void check(const NamesAndTypesList & columns) const; + void check(const StorageInMemoryMetadata & metadata) const; String getModeName() const; }; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 00830dd78c2..52aa92d9f48 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -739,6 +739,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); + Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; + Block header = pipes.at(0).getHeader(); for (size_t i = 0; i < sort_columns_size; ++i) sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); @@ -767,7 +769,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor case MergeTreeData::MergingParams::Summing: merged_transform = std::make_unique( - header, pipes.size(), sort_description, data.merging_params.columns_to_sum, merge_block_size); + header, pipes.size(), sort_description, data.merging_params.columns_to_sum, partition_key_columns, merge_block_size); break; case MergeTreeData::MergingParams::Aggregating: diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 69e819a3cf5..4eee1f1fccb 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1160,6 +1160,8 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( size_t sort_columns_size = sort_columns.size(); sort_description.reserve(sort_columns_size); + Names partition_key_columns = metadata_snapshot->getPartitionKey().column_names; + Block header = pipes.at(0).getHeader(); for (size_t i = 0; i < sort_columns_size; ++i) sort_description.emplace_back(header.getPositionByName(sort_columns[i]), 1, 1); @@ -1180,7 +1182,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( case MergeTreeData::MergingParams::Summing: return std::make_shared(header, pipes.size(), - sort_description, data.merging_params.columns_to_sum, max_block_size); + sort_description, data.merging_params.columns_to_sum, partition_key_columns, max_block_size); case MergeTreeData::MergingParams::Aggregating: return std::make_shared(header, pipes.size(), diff --git a/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference new file mode 100644 index 00000000000..cc0a7875159 --- /dev/null +++ b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference @@ -0,0 +1,13 @@ +0 17 +1 17 +2 17 +3 17 +4 17 +5 17 +6 17 +7 17 +8 17 +9 17 +10 17 +11 17 +12 17 diff --git a/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql new file mode 100644 index 00000000000..563d969ee36 --- /dev/null +++ b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS tt_01373; + +CREATE TABLE tt_01373 +(a Int64, d Int64, val Int64) +ENGINE = SummingMergeTree PARTITION BY (a) ORDER BY (d); + +INSERT INTO tt_01373 SELECT number%13, number%17, 1 from numbers(1000000); + +OPTIMIZE TABLE tt_01373 FINAL; + +SELECT a, count() FROM tt_01373 GROUP BY a ORDER BY a; + +DROP TABLE IF EXISTS tt_01373; diff --git a/tests/queries/0_stateless/01373_summing_merge_tree_explicit_columns_definition.reference b/tests/queries/0_stateless/01373_summing_merge_tree_explicit_columns_definition.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01373_summing_merge_tree_explicit_columns_definition.sql b/tests/queries/0_stateless/01373_summing_merge_tree_explicit_columns_definition.sql new file mode 100644 index 00000000000..cc456b3a257 --- /dev/null +++ b/tests/queries/0_stateless/01373_summing_merge_tree_explicit_columns_definition.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS tt_error_1373; + +CREATE TABLE tt_error_1373 +( a Int64, d Int64, val Int64 ) +ENGINE = SummingMergeTree((a, val)) PARTITION BY (a) ORDER BY (d); -- { serverError 36 } + +DROP TABLE IF EXISTS tt_error_1373; \ No newline at end of file From 8038383f06c03a1cc6a81d0f8f0715ead1f636ba Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 6 Jul 2020 19:24:33 +0200 Subject: [PATCH 082/196] Fix #10437, CR fixes --- src/Interpreters/SystemLog.h | 7 ++-- src/Storages/StorageBuffer.cpp | 5 ++- ...terialized_view_with_join_on_query_log.sql | 6 ++- ...ble_flush_with_materialized_view.reference | 3 ++ ...fer_table_flush_with_materialized_view.sql | 37 +++++++++++++++++++ 5 files changed, 52 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.reference create mode 100644 tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.sql diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 1b5bcbacc6d..7d533a3bab7 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -161,7 +161,6 @@ protected: private: /* Saving thread data */ Context & context; - Context insert_context; const StorageID table_id; const String storage_def; StoragePtr table; @@ -208,13 +207,11 @@ SystemLog::SystemLog(Context & context_, const String & storage_def_, size_t flush_interval_milliseconds_) : context(context_) - , insert_context(Context(context_)) , table_id(database_name_, table_name_) , storage_def(storage_def_) , flush_interval_milliseconds(flush_interval_milliseconds_) { assert(database_name_ == DatabaseCatalog::SYSTEM_DATABASE); - insert_context.makeQueryContext(); // we need query context to do inserts to target table with MV containing subqueries or joins log = &Poco::Logger::get("SystemLog (" + database_name_ + "." + table_name_ + ")"); } @@ -428,6 +425,10 @@ void SystemLog::flushImpl(const std::vector & to_flush, insert->table_id = table_id; ASTPtr query_ptr(insert.release()); + // we need query context to do inserts to target table with MV containing subqueries or joins + auto insert_context = Context(context); + insert_context.makeQueryContext(); + InterpreterInsertQuery interpreter(query_ptr, insert_context); BlockIO io = interpreter.execute(); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 85b61dd34f9..3bc88f5a289 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -713,7 +713,10 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl for (const auto & column : block_to_write) list_of_columns->children.push_back(std::make_shared(column.name)); - InterpreterInsertQuery interpreter{insert, global_context, allow_materialized}; + auto insert_context = Context(global_context); + insert_context.makeQueryContext(); + + InterpreterInsertQuery interpreter{insert, insert_context, allow_materialized}; auto block_io = interpreter.execute(); block_io.out->writePrefix(); diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql index 3d1b464f164..4f216ae647f 100644 --- a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql @@ -4,6 +4,10 @@ DROP TABLE IF EXISTS expected_times; CREATE TABLE expected_times (QUERY_GROUP_ID String, max_query_duration_ms UInt64) Engine=Memory; INSERT INTO expected_times VALUES('main_dashboard_top_query', 100), ('main_dashboard_bottom_query', 100); +SET log_queries=1; +SELECT 1; +SYSTEM FLUSH LOGS; + CREATE MATERIALIZED VIEW slow_log Engine=Memory AS ( SELECT * FROM @@ -18,8 +22,6 @@ CREATE MATERIALIZED VIEW slow_log Engine=Memory AS WHERE query_duration_ms > max_query_duration_ms ); -SET log_queries=1; - SELECT 1 /* QUERY_GROUP_ID:main_dashboard_top_query */; SELECT 1 /* QUERY_GROUP_ID:main_dashboard_bottom_query */; diff --git a/tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.reference b/tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.reference new file mode 100644 index 00000000000..083edaac248 --- /dev/null +++ b/tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.reference @@ -0,0 +1,3 @@ +2 +2 +2 diff --git a/tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.sql b/tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.sql new file mode 100644 index 00000000000..424c38d5590 --- /dev/null +++ b/tests/queries/0_stateless/01361_buffer_table_flush_with_materialized_view.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS t1_01361; +DROP TABLE IF EXISTS t2_01361; +DROP TABLE IF EXISTS mv1_01361; +DROP TABLE IF EXISTS b1_01361; + +CREATE TABLE t1_01361 ( + i UInt32, + time DateTime +) ENGINE = MergeTree() +PARTITION BY time +ORDER BY time; + +CREATE TABLE t2_01361 ( + i UInt32, + time DateTime +) ENGINE = MergeTree() +PARTITION BY time +ORDER BY time; + +CREATE MATERIALIZED VIEW mv1_01361 +TO t2_01361 +AS SELECT * FROM (SELECT * FROM t1_01361); + +CREATE TABLE b1_01361 AS t1_01361 +ENGINE = Buffer(currentDatabase(), t1_01361, 1, 0, 0, 1, 1, 1, 1); + +INSERT INTO b1_01361 VALUES (1, now()); +INSERT INTO b1_01361 VALUES (2, now()); + +SELECT count() FROM b1_01361; +SELECT count() FROM t1_01361; +SELECT count() FROM t2_01361; + +DROP TABLE IF EXISTS t1_01361; +DROP TABLE IF EXISTS t2_01361; +DROP TABLE IF EXISTS mv1_01361; +DROP TABLE IF EXISTS b1_01361; From 2b539e6ace2e09d5b6b04564073c4e9e3a7544ec Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Jul 2020 20:26:15 +0300 Subject: [PATCH 083/196] fix style --- src/Storages/MergeTree/MergeTreeData.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 6c1277596bd..79b3a9aff35 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -109,7 +109,6 @@ namespace ErrorCodes extern const int UNKNOWN_DISK; extern const int NOT_ENOUGH_SPACE; extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; - extern const int BAD_ARGUMENTS; } From 1b93e21db1e45b1b503573913ceef97107daf92a Mon Sep 17 00:00:00 2001 From: Nikita Mikhailov Date: Mon, 6 Jul 2020 21:38:58 +0300 Subject: [PATCH 084/196] add select final to test --- ...ing_merge_tree_exclude_partition_key.reference | 7 +++++++ ...3_summing_merge_tree_exclude_partition_key.sql | 15 ++++++++++++++- 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference index cc0a7875159..3732e9403ce 100644 --- a/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference +++ b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.reference @@ -1,3 +1,10 @@ +--- +1000000 +--- +17 +--- +0 17 +--- 0 17 1 17 2 17 diff --git a/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql index 563d969ee36..60c988a2e2f 100644 --- a/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql +++ b/tests/queries/0_stateless/01373_summing_merge_tree_exclude_partition_key.sql @@ -4,10 +4,23 @@ CREATE TABLE tt_01373 (a Int64, d Int64, val Int64) ENGINE = SummingMergeTree PARTITION BY (a) ORDER BY (d); +SYSTEM STOP MERGES; + INSERT INTO tt_01373 SELECT number%13, number%17, 1 from numbers(1000000); -OPTIMIZE TABLE tt_01373 FINAL; +SELECT '---'; +SELECT count(*) FROM tt_01373; +SELECT '---'; +SELECT count(*) FROM tt_01373 FINAL; + +SELECT '---'; +SELECT a, count() FROM tt_01373 FINAL GROUP BY a ORDER BY a; + +SYSTEM START MERGES; + +OPTIMIZE TABLE tt_01373 FINAL; +SELECT '---'; SELECT a, count() FROM tt_01373 GROUP BY a ORDER BY a; DROP TABLE IF EXISTS tt_01373; From 93517b4e827ca6b059e0940eb1a84421916b3b78 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 6 Jul 2020 20:48:33 +0200 Subject: [PATCH 085/196] Same change for Kafka - just in case, and to make it conform. --- src/Storages/Kafka/StorageKafka.cpp | 8 ++++---- src/Storages/Kafka/StorageKafka.h | 1 - 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 6499941a68d..f1d7650c785 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -125,7 +125,6 @@ StorageKafka::StorageKafka( std::unique_ptr kafka_settings_) : IStorage(table_id_) , global_context(context_.getGlobalContext()) - , kafka_context(std::make_shared(global_context)) , kafka_settings(std::move(kafka_settings_)) , topics(parseTopics(global_context.getMacros()->expand(kafka_settings->kafka_topic_list.value))) , brokers(global_context.getMacros()->expand(kafka_settings->kafka_broker_list.value)) @@ -145,9 +144,6 @@ StorageKafka::StorageKafka( setInMemoryMetadata(storage_metadata); task = global_context.getSchedulePool().createTask(log->name(), [this]{ threadFunc(); }); task->deactivate(); - - kafka_context->makeQueryContext(); - kafka_context->applySettingsChanges(settings_adjustments); } SettingsChanges StorageKafka::createSettingsAdjustments() @@ -530,6 +526,10 @@ bool StorageKafka::streamToViews() size_t block_size = getMaxBlockSize(); + auto kafka_context = std::make_shared(global_context); + kafka_context->makeQueryContext(); + kafka_context->applySettingsChanges(settings_adjustments); + // Create a stream for each consumer and join them in a union stream // Only insert into dependent views and expect that input blocks contain virtual columns InterpreterInsertQuery interpreter(insert, *kafka_context, false, true, true); diff --git a/src/Storages/Kafka/StorageKafka.h b/src/Storages/Kafka/StorageKafka.h index 6f479ba2089..b7e6ea2a7e0 100644 --- a/src/Storages/Kafka/StorageKafka.h +++ b/src/Storages/Kafka/StorageKafka.h @@ -70,7 +70,6 @@ protected: private: // Configuration and state Context & global_context; - std::shared_ptr kafka_context; std::unique_ptr kafka_settings; const Names topics; const String brokers; From 6310e490320788cc1471f01f66568155946c15f9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 22:34:00 +0300 Subject: [PATCH 086/196] Fix dictGet with bad arguments during GROUP BY injective functions elimination --- src/Interpreters/SyntaxAnalyzer.cpp | 14 +++++++++++--- ...ive_elimination_dictGet_BAD_ARGUMENTS.reference | 0 ...injective_elimination_dictGet_BAD_ARGUMENTS.sql | 1 + 3 files changed, 12 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.reference create mode 100644 tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index 9bc7ae055d2..e90b6da3dba 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -327,10 +327,18 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum continue; } - const auto & dict_name = function->arguments->children[0]->as().value.safeGet(); - const auto & dict_ptr = context.getExternalDictionariesLoader().getDictionary(dict_name); - const auto & attr_name = function->arguments->children[1]->as().value.safeGet(); + const auto * dict_name_ast = function->arguments->children[0]->as(); + const auto * attr_name_ast = function->arguments->children[1]->as(); + if (!dict_name_ast || !attr_name_ast) + { + ++i; + continue; + } + const auto & dict_name = dict_name_ast->value.safeGet(); + const auto & attr_name = attr_name_ast->value.safeGet(); + + const auto & dict_ptr = context.getExternalDictionariesLoader().getDictionary(dict_name); if (!dict_ptr->isInjective(attr_name)) { ++i; diff --git a/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.reference b/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql b/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql new file mode 100644 index 00000000000..88a2b25c2db --- /dev/null +++ b/tests/queries/0_stateless/01375_GROUP_BY_injective_elimination_dictGet_BAD_ARGUMENTS.sql @@ -0,0 +1 @@ +SELECT dictGetString(concat('default', '.countryId'), 'country', toUInt64(number)) AS country FROM numbers(2) GROUP BY country; -- { serverError 36; } From 128dd4fa8a0ce5c741caaffc7eb18dda2c2e9034 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 22:36:16 +0300 Subject: [PATCH 087/196] Fix dictGet arguments check during GROUP BY injective functions elimination This patch changes the place where the dictionary will be loaded (during syntax analysis), but I guess this is fine, it will be loaded anyway. Fixes: #10342 --- src/Interpreters/SyntaxAnalyzer.cpp | 1 + ...BY_injective_elimination_dictGet.reference | 1 + ...GROUP_BY_injective_elimination_dictGet.sql | 31 +++++++++++++++++++ 3 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference create mode 100644 tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index e90b6da3dba..2e270222c37 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -248,6 +248,7 @@ void executeScalarSubqueries(ASTPtr & query, const Context & context, size_t sub const std::unordered_set possibly_injective_function_names { + "dictGet", "dictGetString", "dictGetUInt8", "dictGetUInt16", diff --git a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference new file mode 100644 index 00000000000..9459d4ba2a0 --- /dev/null +++ b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.reference @@ -0,0 +1 @@ +1.1 diff --git a/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql new file mode 100644 index 00000000000..1c7a4d16f05 --- /dev/null +++ b/tests/queries/0_stateless/01376_GROUP_BY_injective_elimination_dictGet.sql @@ -0,0 +1,31 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/11469 +SELECT dictGet('default.countryId', 'country', toUInt64(number)) AS country FROM numbers(2) GROUP BY country; -- { serverError 36; } + + +-- with real dictionary +DROP TABLE IF EXISTS dictdb_01376.table_for_dict; +DROP DICTIONARY IF EXISTS dictdb_01376.dict_exists; +DROP DATABASE IF EXISTS dictdb_01376; + +CREATE DATABASE dictdb_01376 ENGINE = Ordinary; + +CREATE TABLE dictdb_01376.table_for_dict +( + key_column UInt64, + value Float64 +) +ENGINE = Memory(); + +INSERT INTO dictdb_01376.table_for_dict VALUES (1, 1.1); + +CREATE DICTIONARY IF NOT EXISTS dictdb_01376.dict_exists +( + key_column UInt64, + value Float64 DEFAULT 77.77 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'dictdb_01376')) +LIFETIME(1) +LAYOUT(FLAT()); + +SELECT dictGet('dictdb_01376.dict_exists', 'value', toUInt64(1)) as val FROM numbers(2) GROUP BY val; From 2d2a34d6cd72edc132cacbce53d37d20e561bab2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 21:37:05 +0300 Subject: [PATCH 088/196] Use ENABLE_LIBRARIES option for AMQP-CPP This is tiny fix, there are more problems that just this small little bit. --- cmake/find/amqpcpp.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/amqpcpp.cmake b/cmake/find/amqpcpp.cmake index 147824ff395..0868e76c28a 100644 --- a/cmake/find/amqpcpp.cmake +++ b/cmake/find/amqpcpp.cmake @@ -1,4 +1,4 @@ -SET(ENABLE_AMQPCPP 1) +SET(ENABLE_AMQPCPP ${ENABLE_LIBRARIES}) if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/AMQP-CPP/CMakeLists.txt") message (WARNING "submodule contrib/AMQP-CPP is missing. to fix try run: \n git submodule update --init --recursive") set (ENABLE_AMQPCPP 0) From a0f257b2376c3088ecc7e1a666aee8b1f5fa1916 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 21:37:59 +0300 Subject: [PATCH 089/196] Set GOOGLETEST_VERSION for googletest Otherwise cmake reports: CMake Warning at contrib/googletest/googletest/CMakeLists.txt:54 (project): VERSION keyword not followed by a value or was followed by a value that expanded to nothing. (since GOOGLETEST_VERSION is set in contrib/googletest/CMakeLists.txt) --- contrib/CMakeLists.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index e0cb8ad760a..012ae667b60 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -222,6 +222,7 @@ if (USE_INTERNAL_AVRO_LIBRARY) endif() if(USE_INTERNAL_GTEST_LIBRARY) + set(GOOGLETEST_VERSION 1.10.0) # master # Google Test from sources add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) # avoid problems with From b1733854319306568e10e41f36e437a36fae6892 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 21:39:45 +0300 Subject: [PATCH 090/196] Bump googletest to master (to fix gcc10 builds) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit gcc10 reports: FAILED: src/CMakeFiles/unit_tests_dbms.dir/Columns/tests/gtest_column_unique.cpp.o ../contrib/googletest/googletest/include/gtest/gtest-printers.h:287:7: error: use of deleted function ‘std::basic_ostream& std::operator<<(std::basic_ostream&, char8_t) [with _Traits = std::char_traits]’ 287 | *os << value; | ~~~~^~~~~~~~ In file included from ../base/common/../common/StringRef.h:6, from ../src/Columns/IColumn.h:7, from ../src/Columns/IColumnUnique.h:2, from ../src/Columns/ColumnUnique.h:2, from ../src/Columns/tests/gtest_column_unique.cpp:1: /usr/include/c++/10.1.0/ostream:544:5: note: declared here 544 | operator<<(basic_ostream&, char8_t) = delete; | --- contrib/googletest | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/googletest b/contrib/googletest index 703bd9caab5..356f2d264a4 160000 --- a/contrib/googletest +++ b/contrib/googletest @@ -1 +1 @@ -Subproject commit 703bd9caab50b139428cea1aaff9974ebee5742e +Subproject commit 356f2d264a485db2fcc50ec1c672e0d37b6cb39b From a276d0da4fd60191fdecae3655114e3d222088a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 21:46:16 +0300 Subject: [PATCH 091/196] gtest_compressionCodec: is_trivial+is_standard_layout over deprecated is_pod --- src/Compression/tests/gtest_compressionCodec.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index f9b4f829fff..d7c9464dcf2 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -63,6 +63,10 @@ std::vector operator+(std::vector && left, std::vector && right) namespace { +template using is_pod = std::is_trivial>; +template inline constexpr bool is_pod_v = is_pod::value; + + template struct AsHexStringHelper { @@ -84,7 +88,7 @@ std::ostream & operator << (std::ostream & ostr, const AsHexStringHelper & he template AsHexStringHelper AsHexString(const T & container) { - static_assert (sizeof(container[0]) == 1 && std::is_pod>::value, "Only works on containers of byte-size PODs."); + static_assert (sizeof(container[0]) == 1 && is_pod_v>, "Only works on containers of byte-size PODs."); return AsHexStringHelper{container}; } @@ -162,7 +166,7 @@ public: data_end(container.data() + container.size()), current_value(T{}) { - static_assert(sizeof(container[0]) == 1 && std::is_pod>::value, "Only works on containers of byte-size PODs."); + static_assert(sizeof(container[0]) == 1 && is_pod_v>, "Only works on containers of byte-size PODs."); read(); } From dab5b5ad7bad89234564b69b2ccc10d2a9be14b4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 21:49:59 +0300 Subject: [PATCH 092/196] gtest_compressionCodec: use fmt over boost::format MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit boost::format is not compiled under gcc10: from ../src/Compression/tests/gtest_compressionCodec.cpp:14: /usr/include/boost/format/alt_sstream_impl.hpp: In instantiation of ‘boost::io::basic_altstringbuf::int_type boost::io::basic_altstringbuf::overflow(boost::io::basic_altstringbuf::int_type) [with Ch = char; Tr = std::char_traits; Alloc = std::allocator; boost::io::basic_altstringbuf::int_type = int]’: /usr/include/boost/format/alt_sstream_impl.hpp:227:9: required from here /usr/include/boost/format/alt_sstream_impl.hpp:261:45: error: no matching function for call to ‘std::allocator::allocate(std::size_t&, char*)’ 261 | newptr = alloc_.allocate(new_size, is_allocated_? oldptr : 0); (although this is system-wide boost, it is pretty recent - 1.72) --- src/Compression/tests/gtest_compressionCodec.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index d7c9464dcf2..5ff1c7c01c1 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -11,7 +11,7 @@ #include #include -#include +#include #include #include @@ -390,7 +390,7 @@ CodecTestSequence makeSeq(Args && ... args) } return CodecTestSequence{ - (boost::format("%1% values of %2%") % std::size(vals) % type_name()).str(), + (fmt::format("{} values of {}", std::size(vals), type_name())), std::move(data), makeDataType() }; @@ -412,7 +412,7 @@ CodecTestSequence generateSeq(Generator gen, const char* gen_name, B Begin = 0, } return CodecTestSequence{ - (boost::format("%1% values of %2% from %3%") % (End - Begin) % type_name() % gen_name).str(), + (fmt::format("{} values of {} from {}", (End - Begin), type_name(), gen_name)), std::move(data), makeDataType() }; From 0a24d9eabf6cc368f33b3d0cbcf63bcdf8a66325 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 21:57:52 +0300 Subject: [PATCH 093/196] gtest_compressionCodec: fix lack of operator<< for char8_t --- src/Compression/tests/gtest_compressionCodec.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index 5ff1c7c01c1..e1899d448aa 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -256,8 +256,8 @@ template if (++mismatching_items <= MAX_MISMATCHING_ITEMS) { result << "\nmismatching " << sizeof(T) << "-byte item #" << i - << "\nexpected: " << bin(left_value) << " (0x" << std::hex << left_value << ")" - << "\ngot : " << bin(right_value) << " (0x" << std::hex << right_value << ")"; + << "\nexpected: " << bin(left_value) << " (0x" << std::hex << size_t(left_value) << ")" + << "\ngot : " << bin(right_value) << " (0x" << std::hex << size_t(right_value) << ")"; if (mismatching_items == MAX_MISMATCHING_ITEMS) { result << "\n..." << std::endl; From c59907c1d675d02ab748edee560405c5a33a4b43 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 21:59:05 +0300 Subject: [PATCH 094/196] gtest_weak_hash_32: fix lack of operator<< for char8_t --- src/Columns/tests/gtest_weak_hash_32.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/tests/gtest_weak_hash_32.cpp b/src/Columns/tests/gtest_weak_hash_32.cpp index 5d87d2aa2e1..fa49cacb4f0 100644 --- a/src/Columns/tests/gtest_weak_hash_32.cpp +++ b/src/Columns/tests/gtest_weak_hash_32.cpp @@ -56,7 +56,7 @@ void checkColumn( { if (it->second != hash[i]) { - std::cout << "Different hashes for the same equivalent class (" << val << "):\n"; + std::cout << "Different hashes for the same equivalent class (" << size_t(val) << "):\n"; std::cout << print_for_row(it->first) << '\n'; std::cout << print_for_row(i) << std::endl; } From 3b75856fcab281047d64342ddb3b1654f4e52d7d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 22:09:31 +0300 Subject: [PATCH 095/196] Force CMP0022 for googletest (to avoid using LINK_INTERFACE_LIBRARIES(_)?) Othewise cmake reports: -- Configuring done CMake Warning (dev) in contrib/googletest/googletest/CMakeLists.txt: Policy CMP0022 is not set: INTERFACE_LINK_LIBRARIES defines the link interface. Run "cmake --help-policy CMP0022" for policy details. Use the cmake_policy command to set the policy and suppress this warning. Target "gtest" has an INTERFACE_LINK_LIBRARIES property which differs from its LINK_INTERFACE_LIBRARIES_DEBUG properties. INTERFACE_LINK_LIBRARIES: global-group;Threads::Threads LINK_INTERFACE_LIBRARIES_DEBUG: Threads::Threads This warning is for project developers. Use -Wno-dev to suppress it. -- Generating done -- Build files have been written to: /src/ch/clickhouse/.cmake-tmp --- contrib/CMakeLists.txt | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 012ae667b60..ec23876bb88 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -222,11 +222,19 @@ if (USE_INTERNAL_AVRO_LIBRARY) endif() if(USE_INTERNAL_GTEST_LIBRARY) - set(GOOGLETEST_VERSION 1.10.0) # master - # Google Test from sources - add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) - # avoid problems with - target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0) + # Wrap into function because of CMAKE_POLICY_DEFAULT_CMP0022 + function(googletest_support) + set(GOOGLETEST_VERSION 1.10.0) # master + # Google Test from sources uses too old cmake, 2.6.x, and CMP0022 should + # set, to avoid using deprecated LINK_INTERFACE_LIBRARIES(_)? over + # INTERFACE_LINK_LIBRARIES. + set(CMAKE_POLICY_DEFAULT_CMP0022 NEW) + # Google Test from sources + add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) + # avoid problems with + target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0) + endfunction() + googletest_support() elseif(GTEST_SRC_DIR) add_subdirectory(${GTEST_SRC_DIR}/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) target_compile_definitions(gtest INTERFACE GTEST_HAS_POSIX_RE=0) From 0a3c1ef4500a8a454d2d2feb8974cf2c9d9268b8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Jul 2020 23:17:16 +0300 Subject: [PATCH 096/196] Set CMP0077 for re2 cmake reports: CMake Warning (dev) at contrib/re2/CMakeLists.txt:15 (option): Policy CMP0077 is not set: option() honors normal variables. Run "cmake --help-policy CMP0077" for policy details. Use the cmake_policy command to set the policy and suppress this warning. For compatibility with older versions of CMake, option is clearing the normal variable 'BUILD_SHARED_LIBS'. This warning is for project developers. Use -Wno-dev to suppress it. --- contrib/CMakeLists.txt | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index ec23876bb88..74c036fbd89 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -44,8 +44,13 @@ endif () if (USE_INTERNAL_RE2_LIBRARY) set(RE2_BUILD_TESTING 0 CACHE INTERNAL "") - add_subdirectory (re2) - add_subdirectory (re2_st) + function(re2_support) + # make option() honor normal variables for BUILD_SHARED_LIBS + set(CMAKE_POLICY_DEFAULT_CMP0077 NEW) + add_subdirectory (re2) + add_subdirectory (re2_st) + endfunction() + re2_support() endif () if (USE_INTERNAL_DOUBLE_CONVERSION_LIBRARY) From 4733504b517a89f5157ddc5dc756546349b8069e Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 6 Jul 2020 15:23:36 +0300 Subject: [PATCH 097/196] Don't split dictionary source's table name into schema and table name itself if ODBC driver doesn't support schema. --- programs/odbc-bridge/CMakeLists.txt | 1 + programs/odbc-bridge/HandlerFactory.cpp | 6 ++ programs/odbc-bridge/HandlerFactory.h | 3 +- programs/odbc-bridge/SchemaAllowedHandler.cpp | 76 +++++++++++++++++++ programs/odbc-bridge/SchemaAllowedHandler.h | 31 ++++++++ src/Common/XDBCBridgeHelper.h | 24 ++++++ .../CassandraDictionarySource.cpp | 2 +- .../ClickHouseDictionarySource.cpp | 4 +- src/Dictionaries/ExternalQueryBuilder.cpp | 16 +--- src/Dictionaries/ExternalQueryBuilder.h | 3 +- src/Dictionaries/MySQLDictionarySource.cpp | 4 +- src/Dictionaries/XDBCDictionarySource.cpp | 39 +++++++++- src/Dictionaries/XDBCDictionarySource.h | 1 + .../test_dictionaries_dependency/test.py | 1 - 14 files changed, 188 insertions(+), 23 deletions(-) create mode 100644 programs/odbc-bridge/SchemaAllowedHandler.cpp create mode 100644 programs/odbc-bridge/SchemaAllowedHandler.h diff --git a/programs/odbc-bridge/CMakeLists.txt b/programs/odbc-bridge/CMakeLists.txt index 4b63ed2596d..8cfa110adad 100644 --- a/programs/odbc-bridge/CMakeLists.txt +++ b/programs/odbc-bridge/CMakeLists.txt @@ -8,6 +8,7 @@ set (CLICKHOUSE_ODBC_BRIDGE_SOURCES ODBCBlockOutputStream.cpp ODBCBridge.cpp PingHandler.cpp + SchemaAllowedHandler.cpp validateODBCConnectionString.cpp ) set (CLICKHOUSE_ODBC_BRIDGE_LINK diff --git a/programs/odbc-bridge/HandlerFactory.cpp b/programs/odbc-bridge/HandlerFactory.cpp index a02fcadea52..0cc40480b87 100644 --- a/programs/odbc-bridge/HandlerFactory.cpp +++ b/programs/odbc-bridge/HandlerFactory.cpp @@ -29,6 +29,12 @@ Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco: return new IdentifierQuoteHandler(keep_alive_timeout, context); #else return nullptr; +#endif + else if (uri.getPath() == "/schema_allowed") +#if USE_ODBC + return new SchemaAllowedHandler(keep_alive_timeout, context); +#else + return nullptr; #endif else if (uri.getPath() == "/write") return new ODBCHandler(pool_map, keep_alive_timeout, context, "write"); diff --git a/programs/odbc-bridge/HandlerFactory.h b/programs/odbc-bridge/HandlerFactory.h index 35835de5dad..1d4edfc9dd1 100644 --- a/programs/odbc-bridge/HandlerFactory.h +++ b/programs/odbc-bridge/HandlerFactory.h @@ -6,6 +6,7 @@ #include "MainHandler.h" #include "ColumnInfoHandler.h" #include "IdentifierQuoteHandler.h" +#include "SchemaAllowedHandler.h" #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" @@ -15,7 +16,7 @@ namespace DB { -/** Factory for '/ping', '/', '/columns_info', '/identifier_quote' handlers. +/** Factory for '/ping', '/', '/columns_info', '/identifier_quote', '/schema_allowed' handlers. * Also stores Session pools for ODBC connections */ class HandlerFactory : public Poco::Net::HTTPRequestHandlerFactory diff --git a/programs/odbc-bridge/SchemaAllowedHandler.cpp b/programs/odbc-bridge/SchemaAllowedHandler.cpp new file mode 100644 index 00000000000..5aaba57399e --- /dev/null +++ b/programs/odbc-bridge/SchemaAllowedHandler.cpp @@ -0,0 +1,76 @@ +#include "SchemaAllowedHandler.h" + +#if USE_ODBC + +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include "validateODBCConnectionString.h" + +# define POCO_SQL_ODBC_CLASS Poco::Data::ODBC + +namespace DB +{ +namespace +{ + bool isSchemaAllowed(SQLHDBC hdbc) + { + std::string identifier; + + SQLSMALLINT t; + SQLRETURN r = POCO_SQL_ODBC_CLASS::SQLGetInfo(hdbc, SQL_SCHEMA_USAGE, nullptr, 0, &t); + + if (POCO_SQL_ODBC_CLASS::Utility::isError(r)) + throw POCO_SQL_ODBC_CLASS::ConnectionException(hdbc); + + return t != 0; + } +} + + +void SchemaAllowedHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) +{ + Poco::Net::HTMLForm params(request, request.stream()); + LOG_TRACE(log, "Request URI: {}", request.getURI()); + + auto process_error = [&response, this](const std::string & message) + { + response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR); + if (!response.sent()) + response.send() << message << std::endl; + LOG_WARNING(log, message); + }; + + if (!params.has("connection_string")) + { + process_error("No 'connection_string' in request URL"); + return; + } + + try + { + std::string connection_string = params.get("connection_string"); + POCO_SQL_ODBC_CLASS::SessionImpl session(validateODBCConnectionString(connection_string), DBMS_DEFAULT_CONNECT_TIMEOUT_SEC); + SQLHDBC hdbc = session.dbc().handle(); + + bool result = isSchemaAllowed(hdbc); + + WriteBufferFromHTTPServerResponse out(request, response, keep_alive_timeout); + writeBoolText(result, out); + } + catch (...) + { + process_error("Error getting schema usage from ODBC '" + getCurrentExceptionMessage(false) + "'"); + tryLogCurrentException(log); + } +} + +} + +#endif diff --git a/programs/odbc-bridge/SchemaAllowedHandler.h b/programs/odbc-bridge/SchemaAllowedHandler.h new file mode 100644 index 00000000000..76aa23b903c --- /dev/null +++ b/programs/odbc-bridge/SchemaAllowedHandler.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include + +#if USE_ODBC + +namespace DB +{ +class Context; + + +/// This handler establishes connection to database, and retrieve whether schema is allowed. +class SchemaAllowedHandler : public Poco::Net::HTTPRequestHandler +{ +public: + SchemaAllowedHandler(size_t keep_alive_timeout_, Context &) + : log(&Poco::Logger::get("SchemaAllowedHandler")), keep_alive_timeout(keep_alive_timeout_) + { + } + + void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; + +private: + Poco::Logger * log; + size_t keep_alive_timeout; +}; + +} + +#endif diff --git a/src/Common/XDBCBridgeHelper.h b/src/Common/XDBCBridgeHelper.h index 9f20c75182d..a425cd36b11 100644 --- a/src/Common/XDBCBridgeHelper.h +++ b/src/Common/XDBCBridgeHelper.h @@ -41,6 +41,7 @@ public: virtual Poco::URI getMainURI() const = 0; virtual Poco::URI getColumnsInfoURI() const = 0; virtual IdentifierQuotingStyle getIdentifierQuotingStyle() = 0; + virtual bool isSchemaAllowed() = 0; virtual String getName() const = 0; virtual ~IXDBCBridgeHelper() = default; @@ -61,6 +62,7 @@ private: Poco::Logger * log = &Poco::Logger::get(BridgeHelperMixin::getName() + "BridgeHelper"); std::optional quote_style; + std::optional is_schema_allowed; protected: auto getConnectionString() const @@ -80,6 +82,7 @@ public: static constexpr inline auto MAIN_HANDLER = "/"; static constexpr inline auto COL_INFO_HANDLER = "/columns_info"; static constexpr inline auto IDENTIFIER_QUOTE_HANDLER = "/identifier_quote"; + static constexpr inline auto SCHEMA_ALLOWED_HANDLER = "/schema_allowed"; static constexpr inline auto PING_OK_ANSWER = "Ok."; XDBCBridgeHelper(const Context & global_context_, const Poco::Timespan & http_timeout_, const std::string & connection_string_) @@ -128,6 +131,27 @@ public: return *quote_style; } + bool isSchemaAllowed() override + { + if (!is_schema_allowed.has_value()) + { + startBridgeSync(); + + auto uri = createBaseURI(); + uri.setPath(SCHEMA_ALLOWED_HANDLER); + uri.addQueryParameter("connection_string", getConnectionString()); + + ReadWriteBufferFromHTTP buf( + uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context)); + + bool res; + readBoolText(res, buf); + is_schema_allowed = res; + } + + return *is_schema_allowed; + } + /** * @todo leaky abstraction - used by external API's */ diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index c41f528db91..5c7fd4f50fd 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -102,7 +102,7 @@ CassandraDictionarySource::CassandraDictionarySource( , dict_struct(dict_struct_) , settings(settings_) , sample_block(sample_block_) - , query_builder(dict_struct, settings.db, settings.table, settings.where, IdentifierQuotingStyle::DoubleQuotes) + , query_builder(dict_struct, settings.db, "", settings.table, settings.where, IdentifierQuotingStyle::DoubleQuotes) { cassandraCheck(cass_cluster_set_contact_points(cluster, settings.host.c_str())); if (settings.port) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 9d3f6063a21..180750d143a 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -66,7 +66,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource( , where{config.getString(config_prefix + ".where", "")} , update_field{config.getString(config_prefix + ".update_field", "")} , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} - , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} , sample_block{sample_block_} , context(context_) , is_local{isLocalAddress({host, port}, secure ? context.getTCPPortSecure().value_or(0) : context.getTCPPort())} @@ -97,7 +97,7 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionar , update_field{other.update_field} , invalidate_query{other.invalidate_query} , invalidate_query_response{other.invalidate_query_response} - , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} , sample_block{other.sample_block} , context(other.context) , is_local{other.is_local} diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index e64f04d28f2..0cf7e28eb29 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -19,22 +19,12 @@ namespace ErrorCodes ExternalQueryBuilder::ExternalQueryBuilder( const DictionaryStructure & dict_struct_, const std::string & db_, + const std::string & schema_, const std::string & table_, const std::string & where_, IdentifierQuotingStyle quoting_style_) - : dict_struct(dict_struct_), db(db_), where(where_), quoting_style(quoting_style_) -{ - if (auto pos = table_.find('.'); pos != std::string::npos) - { - schema = table_.substr(0, pos); - table = table_.substr(pos + 1); - } - else - { - schema = ""; - table = table_; - } -} + : dict_struct(dict_struct_), db(db_), schema(schema_), table(table_), where(where_), quoting_style(quoting_style_) +{} void ExternalQueryBuilder::writeQuoted(const std::string & s, WriteBuffer & out) const diff --git a/src/Dictionaries/ExternalQueryBuilder.h b/src/Dictionaries/ExternalQueryBuilder.h index 3011efbc895..4c0e876b5db 100644 --- a/src/Dictionaries/ExternalQueryBuilder.h +++ b/src/Dictionaries/ExternalQueryBuilder.h @@ -18,8 +18,8 @@ struct ExternalQueryBuilder { const DictionaryStructure & dict_struct; std::string db; - std::string table; std::string schema; + std::string table; const std::string & where; IdentifierQuotingStyle quoting_style; @@ -28,6 +28,7 @@ struct ExternalQueryBuilder ExternalQueryBuilder( const DictionaryStructure & dict_struct_, const std::string & db_, + const std::string & schema_, const std::string & table_, const std::string & where_, IdentifierQuotingStyle quoting_style_); diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 505ce7b0c12..f016f2bf537 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -68,7 +68,7 @@ MySQLDictionarySource::MySQLDictionarySource( , dont_check_update_time{config.getBool(config_prefix + ".dont_check_update_time", false)} , sample_block{sample_block_} , pool{mysqlxx::PoolFactory::instance().get(config, config_prefix)} - , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} , load_all_query{query_builder.composeLoadAllQuery()} , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} , close_connection{config.getBool(config_prefix + ".close_connection", false) || config.getBool(config_prefix + ".share_connection", false)} @@ -87,7 +87,7 @@ MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other , dont_check_update_time{other.dont_check_update_time} , sample_block{other.sample_block} , pool{other.pool} - , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks} + , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} , load_all_query{other.load_all_query} , last_modification{other.last_modification} , invalidate_query{other.invalidate_query} diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 92af20e646b..b3393d55e5d 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -27,6 +27,7 @@ namespace DB namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace @@ -60,6 +61,39 @@ namespace std::unique_ptr read_buf; BlockInputStreamPtr reader; }; + + + ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct_, + const std::string & db_, + const std::string & schema_, + const std::string & table_, + const std::string & where_, + IXDBCBridgeHelper & bridge_) + { + std::string schema = schema_; + std::string table = table_; + + if (bridge_.isSchemaAllowed()) + { + if (schema.empty()) + { + if (auto pos = table.find('.'); pos != std::string::npos) + { + schema = table.substr(0, pos); + table = table.substr(pos + 1); + } + } + } + else + { + if (!schema.empty()) + throw Exception{"Dictionary source of type " + bridge_.getName() + " specifies a schema but schema is not supported by " + + bridge_.getName() + "-driver", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + return {dict_struct_, db_, schema, table, where_, bridge_.getIdentifierQuotingStyle()}; + } } static const UInt64 max_block_size = 8192; @@ -76,11 +110,12 @@ XDBCDictionarySource::XDBCDictionarySource( , update_time{std::chrono::system_clock::from_time_t(0)} , dict_struct{dict_struct_} , db{config_.getString(config_prefix_ + ".db", "")} + , schema{config_.getString(config_prefix_ + ".schema", "")} , table{config_.getString(config_prefix_ + ".table")} , where{config_.getString(config_prefix_ + ".where", "")} , update_field{config_.getString(config_prefix_ + ".update_field", "")} , sample_block{sample_block_} - , query_builder{dict_struct, db, table, where, bridge_->getIdentifierQuotingStyle()} + , query_builder{makeExternalQueryBuilder(dict_struct, db, schema, table, where, *bridge_)} , load_all_query{query_builder.composeLoadAllQuery()} , invalidate_query{config_.getString(config_prefix_ + ".invalidate_query", "")} , bridge_helper{bridge_} @@ -104,7 +139,7 @@ XDBCDictionarySource::XDBCDictionarySource(const XDBCDictionarySource & other) , where{other.where} , update_field{other.update_field} , sample_block{other.sample_block} - , query_builder{dict_struct, db, table, where, other.bridge_helper->getIdentifierQuotingStyle()} + , query_builder{other.query_builder} , load_all_query{other.load_all_query} , invalidate_query{other.invalidate_query} , invalidate_query_response{other.invalidate_query_response} diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 253f802d8fd..87bc42c76ab 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -69,6 +69,7 @@ private: std::chrono::time_point update_time; const DictionaryStructure dict_struct; const std::string db; + const std::string schema; const std::string table; const std::string where; const std::string update_field; diff --git a/tests/integration/test_dictionaries_dependency/test.py b/tests/integration/test_dictionaries_dependency/test.py index 31c5a6c549a..4505bf73a7c 100644 --- a/tests/integration/test_dictionaries_dependency/test.py +++ b/tests/integration/test_dictionaries_dependency/test.py @@ -87,7 +87,6 @@ def test_dependency_via_explicit_table(node): check() -@pytest.mark.skip(reason="TODO: should be fixed") @pytest.mark.parametrize("node", nodes) def test_dependency_via_dictionary_database(node): node.query("CREATE DATABASE dict_db ENGINE=Dictionary") From c78d44cb555f62f6212f944a77db8a32a7c59184 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 6 Jul 2020 23:24:11 +0200 Subject: [PATCH 098/196] Sync reference file with changes in sql file --- .../01360_materialized_view_with_join_on_query_log.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference index dfc14d94653..f6349a0b9b4 100644 --- a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.reference @@ -2,6 +2,7 @@ 1 1 1 +1 === system.query_log === main_dashboard_bottom_query 2 main_dashboard_top_query 2 From d68ecdc84c121a628a18945dac4e3b35f2b643dd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 Jul 2020 01:43:39 +0300 Subject: [PATCH 099/196] Cap max_memory_usage* limits to the process resident memory MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit There are still some issues with memory tracking, but now with per-user tracking: executeQuery: Code: 241, e.displayText() = DB::Exception: Memory limit (for user) exceeded: would use 437.72 GiB (attempt to allocate chunk of 4200926 bytes), maximum: 437.72 GiB (version 20.6.1.1) (from 10.7.140.7:31318) Although the server is mostly idle: SELECT formatReadableSize(memory_usage) FROM system.processes ┌─formatReadableSize(memory_usage)─┐ │ 289.28 MiB │ │ 155.75 MiB │ │ 0.00 B │ └──────────────────────────────────┘ Refs: https://github.com/ClickHouse/ClickHouse/pull/10496/files#r450206865 Cc: @alexey-milovidov --- src/Common/MemoryTracker.cpp | 11 +++++++++++ src/Interpreters/AsynchronousMetrics.cpp | 1 - 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 03bd8be94f3..b8ac09f9449 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -77,6 +77,17 @@ void MemoryTracker::alloc(Int64 size) Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed); Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed); + /// Cap the limit to the total_memory_tracker, since it may include some drift. + /// + /// And since total_memory_tracker is reseted to the process resident + /// memory peridically (in AsynchronousMetrics::update()), any limit can be + /// capped to it, to avoid possible drift. + if (unlikely(current_hard_limit && will_be > current_hard_limit)) + { + set(total_memory_tracker.amount); + will_be = size + amount.fetch_add(size, std::memory_order_relaxed); + } + std::bernoulli_distribution fault(fault_probability); if (unlikely(fault_probability && fault(thread_local_rng))) { diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index ac71a88dc00..5c734f4da2a 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -208,7 +208,6 @@ void AsynchronousMetrics::update() /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. /// See https://github.com/ClickHouse/ClickHouse/issues/10293 total_memory_tracker.set(data.resident); - CurrentMetrics::set(CurrentMetrics::MemoryTracking, data.resident); } #endif From f3ab0aa081f48d1ded13942c5f50853786a4cb36 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 Jul 2020 02:00:08 +0300 Subject: [PATCH 100/196] gtest_compressionCodec: suppress non instantiated gtest warning gtest reports: [ RUN ] GoogleTestVerification.UninstantiatedParameterizedTestSuite ../src/Compression/tests/gtest_compressionCodec.cpp:590: Failure Parameterized test suite CodecTestPerformance is defined via TEST_P, but never instantiated. None of the test cases will run. Either no INSTANTIATE_TEST_SUITE_P is provided or the only ones provided expand to nothing. Ideally, TEST_P definitions should only ever be included as part of binaries that intend to use them. (As opposed to, for example, being placed in a library that may be linked in to get other utilities.) To suppress this error for this test suite, insert the following line (in a non-header) in the namespace it is defined in: GTEST_ALLOW_UNINSTANTIATED_PARAMETERIZED_TEST(CodecTestPerformance); [ FAILED ] GoogleTestVerification.UninstantiatedParameterizedTestSuite (0 ms) --- src/Compression/tests/gtest_compressionCodec.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Compression/tests/gtest_compressionCodec.cpp b/src/Compression/tests/gtest_compressionCodec.cpp index e1899d448aa..3609e9f9d36 100644 --- a/src/Compression/tests/gtest_compressionCodec.cpp +++ b/src/Compression/tests/gtest_compressionCodec.cpp @@ -656,6 +656,7 @@ TEST_P(CodecTestPerformance, TranscodingWithDataType) std::cerr << std::endl; } +GTEST_ALLOW_UNINSTANTIATED_PARAMETERIZED_TEST(CodecTestPerformance); /////////////////////////////////////////////////////////////////////////////////////////////////// // Here we use generators to produce test payload for codecs. From e9d1aea74595805b27ada2b80c874edb0fd7ccb8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Jul 2020 09:19:03 +0300 Subject: [PATCH 101/196] Fix tests. --- src/Processors/QueryPipeline.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPipeline.cpp b/src/Processors/QueryPipeline.cpp index 6b121490be7..a7c585598ac 100644 --- a/src/Processors/QueryPipeline.cpp +++ b/src/Processors/QueryPipeline.cpp @@ -642,7 +642,7 @@ void QueryPipeline::unitePipelines( if (!will_limit_max_threads) max_threads = 0; else - max_threads = max_threads_limit; + limitMaxThreads(max_threads_limit); if (!extremes.empty()) { From 62b83b412f1e68181a16d427d6a131b1658480f4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 11:01:50 +0300 Subject: [PATCH 102/196] split --- ...sum_double.xml => questdb_sum_float32.xml} | 6 +--- tests/performance/questdb_sum_float64.xml | 33 +++++++++++++++++++ tests/performance/questdb_sum_int32.xml | 33 +++++++++++++++++++ 3 files changed, 67 insertions(+), 5 deletions(-) rename tests/performance/{questdb_sum_double.xml => questdb_sum_float32.xml} (84%) create mode 100644 tests/performance/questdb_sum_float64.xml create mode 100644 tests/performance/questdb_sum_int32.xml diff --git a/tests/performance/questdb_sum_double.xml b/tests/performance/questdb_sum_float32.xml similarity index 84% rename from tests/performance/questdb_sum_double.xml rename to tests/performance/questdb_sum_float32.xml index e8b4b817b80..e4057527e1b 100644 --- a/tests/performance/questdb_sum_double.xml +++ b/tests/performance/questdb_sum_float32.xml @@ -18,18 +18,14 @@ type - Float64 - Float64 NULL Float32 Float32 NULL - Int32 - Int32 NULL CREATE TABLE `zz_{type}_{engine}` (x {type}) ENGINE {engine} - INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(100000000) + INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(1000000000) SELECT sum(x) FROM `zz_{type}_{engine}` diff --git a/tests/performance/questdb_sum_float64.xml b/tests/performance/questdb_sum_float64.xml new file mode 100644 index 00000000000..16cef264ffa --- /dev/null +++ b/tests/performance/questdb_sum_float64.xml @@ -0,0 +1,33 @@ + + + 4 + 20G + 1 + 2000000000 + 10G + + + + + engine + + Memory + MergeTree ORDER BY tuple() + + + + type + + Float64 + Float64 NULL + + + + + CREATE TABLE `zz_{type}_{engine}` (x {type}) ENGINE {engine} + INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(1000000000) + + SELECT sum(x) FROM `zz_{type}_{engine}` + + DROP TABLE IF EXISTS `zz_{type}_{engine}` + diff --git a/tests/performance/questdb_sum_int32.xml b/tests/performance/questdb_sum_int32.xml new file mode 100644 index 00000000000..ae13210107e --- /dev/null +++ b/tests/performance/questdb_sum_int32.xml @@ -0,0 +1,33 @@ + + + 4 + 20G + 1 + 2000000000 + 10G + + + + + engine + + Memory + MergeTree ORDER BY tuple() + + + + type + + Int32 + Int32 NULL + + + + + CREATE TABLE `zz_{type}_{engine}` (x {type}) ENGINE {engine} + INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(1000000000) + + SELECT sum(x) FROM `zz_{type}_{engine}` + + DROP TABLE IF EXISTS `zz_{type}_{engine}` + From 68ca3b7aec5dc5323f428794c18f3a0f5cf739f7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 Jul 2020 11:21:00 +0300 Subject: [PATCH 103/196] Do not try to adjust memory tracker amount if it is not larger then in total --- src/Common/MemoryTracker.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index b8ac09f9449..ca21cec0212 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -84,8 +84,12 @@ void MemoryTracker::alloc(Int64 size) /// capped to it, to avoid possible drift. if (unlikely(current_hard_limit && will_be > current_hard_limit)) { - set(total_memory_tracker.amount); - will_be = size + amount.fetch_add(size, std::memory_order_relaxed); + Int64 total_amount = total_memory_tracker.get(); + if (amount > total_amount) + { + set(total_amount); + will_be = size + total_amount; + } } std::bernoulli_distribution fault(fault_probability); From a449f3e9ff950e88de9269ae01c61e0bde99e6b2 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Tue, 7 Jul 2020 12:03:37 +0300 Subject: [PATCH 104/196] A test for UInt8 as bool --- .../00552_logical_functions_uint8_as_bool.reference | 8 ++++++++ .../00552_logical_functions_uint8_as_bool.sql | 11 +++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference create mode 100644 tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql diff --git a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference new file mode 100644 index 00000000000..9a65a2603dc --- /dev/null +++ b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference @@ -0,0 +1,8 @@ +0 0 0 0 0 +0 0 1 0 1 +0 2 0 0 1 +0 2 1 0 1 +4 0 0 0 1 +4 0 1 0 1 +4 2 0 0 1 +4 2 1 1 1 diff --git a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql new file mode 100644 index 00000000000..46a5fa39d47 --- /dev/null +++ b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql @@ -0,0 +1,11 @@ + +-- Test that UInt8 type is processed correctly as bool + +SELECT + toUInt8(bitAnd(number, 4)) AS a, + toUInt8(bitAnd(number, 2)) AS b, + toUInt8(bitAnd(number, 1)) AS c, + a AND b AND c AS AND, + a OR b OR c AS OR +FROM numbers(8) +; \ No newline at end of file From 66128acd113bbcaf8c43bac9de13b39093f6775b Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Tue, 7 Jul 2020 12:17:35 +0300 Subject: [PATCH 105/196] Included const uint8 values in test --- .../00552_logical_functions_uint8_as_bool.reference | 1 + .../00552_logical_functions_uint8_as_bool.sql | 11 ++++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference index 9a65a2603dc..2a1151dc692 100644 --- a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference +++ b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.reference @@ -1,3 +1,4 @@ +1 1 1 1 1 1 0 0 0 0 0 0 0 1 0 1 0 2 0 0 1 diff --git a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql index 46a5fa39d47..feee33add1c 100644 --- a/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql +++ b/tests/queries/0_stateless/00552_logical_functions_uint8_as_bool.sql @@ -1,6 +1,15 @@ -- Test that UInt8 type is processed correctly as bool +SELECT + 1 AND 2, + 2 AND 4, + 1 AND 2 AND 4, + 1 OR 2, + 2 OR 4, + 1 OR 2 OR 4 +; + SELECT toUInt8(bitAnd(number, 4)) AS a, toUInt8(bitAnd(number, 2)) AS b, @@ -8,4 +17,4 @@ SELECT a AND b AND c AS AND, a OR b OR c AS OR FROM numbers(8) -; \ No newline at end of file +; From 7dba89eaa8ed460b237d6663fa32a687f613dbbd Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 12:49:14 +0300 Subject: [PATCH 106/196] changelog fixes --- CHANGELOG.md | 2 ++ utils/simple-backport/backport.sh | 14 ++++++++------ utils/simple-backport/changelog.sh | 6 +++--- 3 files changed, 13 insertions(+), 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5211da4203b..816aa556e81 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,9 +20,11 @@ * Support writes in ODBC Table function [#10554](https://github.com/ClickHouse/ClickHouse/pull/10554) ([ageraab](https://github.com/ageraab)). [#10901](https://github.com/ClickHouse/ClickHouse/pull/10901) ([tavplubix](https://github.com/tavplubix)). * Add query performance metrics based on Linux `perf_events` (these metrics are calculated with hardware CPU counters and OS counters). It is optional and requires `CAP_SYS_ADMIN` to be set on clickhouse binary. [#9545](https://github.com/ClickHouse/ClickHouse/pull/9545) [Andrey Skobtsov](https://github.com/And42). [#11226](https://github.com/ClickHouse/ClickHouse/pull/11226) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Now support `NULL` and `NOT NULL` modifiers for data types in `CREATE` query. [#11057](https://github.com/ClickHouse/ClickHouse/pull/11057) ([Павел Потемкин](https://github.com/Potya)). +* Add `ArrowStream` input and output format. [#11088](https://github.com/ClickHouse/ClickHouse/pull/11088) ([hcz](https://github.com/hczhcz)). * Added a new layout `direct` which loads all the data directly from the source for each query, without storing or caching data. [#10622](https://github.com/ClickHouse/ClickHouse/pull/10622) ([Artem Streltsov](https://github.com/kekekekule)). * Added new `complex_key_direct` layout to dictionaries, that does not store anything locally during query execution. [#10850](https://github.com/ClickHouse/ClickHouse/pull/10850) ([Artem Streltsov](https://github.com/kekekekule)). * Added support for MySQL style global variables syntax (stub). This is needed for compatibility of MySQL protocol. [#11832](https://github.com/ClickHouse/ClickHouse/pull/11832) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added syntax highligting to `clickhouse-client` using `replxx`. [#11422](https://github.com/ClickHouse/ClickHouse/pull/11422) ([Tagir Kuskarov](https://github.com/kuskarov)). * `minMap` and `maxMap` functions were added. [#11603](https://github.com/ClickHouse/ClickHouse/pull/11603) ([Ildus Kurbangaliev](https://github.com/ildus)). * Add the `system.asynchronous_metric_log` table that logs historical metrics from `system.asynchronous_metrics`. [#11588](https://github.com/ClickHouse/ClickHouse/pull/11588) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Add functions `extractAllGroupsHorizontal(haystack, re)` and `extractAllGroupsVertical(haystack, re)`. [#11554](https://github.com/ClickHouse/ClickHouse/pull/11554) ([Vasily Nemkov](https://github.com/Enmk)). diff --git a/utils/simple-backport/backport.sh b/utils/simple-backport/backport.sh index 80a5d82d6f0..dcab4106136 100755 --- a/utils/simple-backport/backport.sh +++ b/utils/simple-backport/backport.sh @@ -26,16 +26,18 @@ then echo Some commits will be missed, review these manually. fi -# NOTE keep in sync with ./changelog.sh. +# NOTE keep in sync with ./backport.sh. # Search for PR numbers in commit messages. First variant is normal merge, and second # variant is squashed. Next are some backport message variants. -find_prs=(sed -n "s/^.*Merge pull request #\([[:digit:]]\+\).*$/\1/p; +find_prs=(sed -n "s/^.*merge[d]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*(#\([[:digit:]]\+\))$/\1/p; - s/^.*back[- ]*port[ed of]*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*cherry[- ]*pick[ed of]*#\([[:digit:]]\+\).*$/\1/Ip") + s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; + s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") -"${find_prs[@]}" master-log.txt | sort -rn > master-prs.txt -"${find_prs[@]}" "$branch-log.txt" | sort -rn > "$branch-prs.txt" +# awk is to filter out small task numbers from different task tracker, which are +# referenced by documentation commits like '* DOCSUP-824: query log (#115)'. +"${find_prs[@]}" master-log.txt | sort -rn | uniq | awk '$0 > 1000 { print $0 }' > master-prs.txt +"${find_prs[@]}" "$branch-log.txt" | sort -rn | uniq | awk '$0 > 1000 { print $0 }' > "$branch-prs.txt" # Find all master PRs that are not in branch by calculating differences of two PR lists. grep -f "$branch-prs.txt" -F -x -v master-prs.txt > "$branch-diff-prs.txt" diff --git a/utils/simple-backport/changelog.sh b/utils/simple-backport/changelog.sh index b9cbbf0d5a2..4d768226e03 100755 --- a/utils/simple-backport/changelog.sh +++ b/utils/simple-backport/changelog.sh @@ -21,10 +21,10 @@ fi # NOTE keep in sync with ./backport.sh. # Search for PR numbers in commit messages. First variant is normal merge, and second # variant is squashed. Next are some backport message variants. -find_prs=(sed -n "s/^.*Merge pull request #\([[:digit:]]\+\).*$/\1/p; +find_prs=(sed -n "s/^.*merge[d]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*(#\([[:digit:]]\+\))$/\1/p; - s/^.*back[- ]*port[ed of]*#\([[:digit:]]\+\).*$/\1/Ip; - s/^.*cherry[- ]*pick[ed of]*#\([[:digit:]]\+\).*$/\1/Ip") + s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; + s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") # awk is to filter out small task numbers from different task tracker, which are # referenced by documentation commits like '* DOCSUP-824: query log (#115)'. From 416d9648c257c574fa505735379fb270978a6a74 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Jul 2020 12:58:14 +0300 Subject: [PATCH 107/196] remove questionable functionality --- .../MergeTree/MergeTreeBlockOutputStream.cpp | 12 +---------- src/Storages/MergeTree/MergeTreeData.cpp | 21 ------------------- src/Storages/MergeTree/MergeTreeData.h | 19 ----------------- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++-- src/Storages/StorageMergeTree.cpp | 6 +----- 5 files changed, 4 insertions(+), 58 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 744faa0767e..4269abe2655 100644 --- a/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -27,17 +27,7 @@ void MergeTreeBlockOutputStream::write(const Block & block) PartLog::addNewPart(storage.global_context, part, watch.elapsed()); - if (auto part_in_memory = asInMemoryPart(part)) - { - storage.in_memory_merges_throttler.add(part_in_memory->block.bytes(), part_in_memory->rows_count); - - if (storage.merging_mutating_task_handle && !storage.in_memory_merges_throttler.needDelayMerge()) - { - storage.in_memory_merges_throttler.reset(); - storage.merging_mutating_task_handle->signalReadyToRun(); - } - } - else if (storage.merging_mutating_task_handle) + if (storage.merging_mutating_task_handle) { /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. storage.merging_mutating_task_handle->signalReadyToRun(); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 893c481ef82..e19fea0916c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -139,7 +139,6 @@ MergeTreeData::MergeTreeData( , data_parts_by_info(data_parts_indexes.get()) , data_parts_by_state_and_info(data_parts_indexes.get()) , parts_mover(this) - , in_memory_merges_throttler(storage_settings.get()->min_bytes_for_compact_part, storage_settings.get()->min_rows_for_compact_part) { if (relative_data_path.empty()) throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); @@ -3606,24 +3605,4 @@ NamesAndTypesList MergeTreeData::getVirtuals() const }; } -bool MergeTreeData::MergesThrottler::needDelayMerge() const -{ - std::lock_guard lock(mutex); - return (!max_bytes || have_bytes < max_bytes) && (!max_rows || have_rows < max_rows); -} - -void MergeTreeData::MergesThrottler::add(size_t bytes, size_t rows) -{ - std::lock_guard lock(mutex); - have_bytes += bytes; - have_rows += rows; -} - -void MergeTreeData::MergesThrottler::reset() -{ - std::lock_guard lock(mutex); - have_bytes = 0; - have_rows = 0; -} - } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5201c9e26ee..1851f624e8a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -857,25 +857,6 @@ protected: bool areBackgroundMovesNeeded() const; - struct MergesThrottler - { - mutable std::mutex mutex; - size_t have_bytes = 0; - size_t have_rows = 0; - - size_t max_bytes; - size_t max_rows; - - MergesThrottler(size_t max_bytes_, size_t max_rows_) - : max_bytes(max_bytes_), max_rows(max_rows_) {} - - bool needDelayMerge() const; - void add(size_t bytes, size_t rows); - void reset(); - }; - - MergesThrottler in_memory_merges_throttler; - private: /// RAII Wrapper for atomic work with currently moving parts /// Acquire them in constructor and remove them in destructor diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 98a9e24db2b..b2a309e1397 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -31,8 +31,8 @@ struct MergeTreeSettings : public SettingsCollection /** Data storing format settings. */ \ M(SettingUInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(SettingUInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ - M(SettingUInt64, min_bytes_for_compact_part, 0, "Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ - M(SettingUInt64, min_rows_for_compact_part, 0, "Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ + M(SettingUInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ + M(SettingUInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(SettingBool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(SettingUInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ \ diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c93b88ecacd..13db64f7758 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -888,11 +888,7 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() ///TODO: read deduplicate option from table config if (merge(false /*aggressive*/, {} /*partition_id*/, false /*final*/, false /*deduplicate*/)) - { - return in_memory_merges_throttler.needDelayMerge() - ? BackgroundProcessingPoolTaskResult::NOTHING_TO_DO - : BackgroundProcessingPoolTaskResult::SUCCESS; - } + return BackgroundProcessingPoolTaskResult::SUCCESS; if (tryMutatePart()) return BackgroundProcessingPoolTaskResult::SUCCESS; From c8f51bdc8a805f476c29dd27da69fe2499cdd229 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 7 Jul 2020 12:00:39 +0200 Subject: [PATCH 108/196] Tests for fixed issues #10846 and #7347 --- .../0_stateless/01323_if_with_nulls.reference | 4 ++++ .../queries/0_stateless/01323_if_with_nulls.sql | 17 +++++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/tests/queries/0_stateless/01323_if_with_nulls.reference b/tests/queries/0_stateless/01323_if_with_nulls.reference index 6bf2d206e0b..80697f97f5c 100644 --- a/tests/queries/0_stateless/01323_if_with_nulls.reference +++ b/tests/queries/0_stateless/01323_if_with_nulls.reference @@ -18,3 +18,7 @@ Nullable(UInt8) \N 1 ok ok ok Nullable(UInt8) \N 1 ok ok ok Nullable(UInt8) \N 1 ok ok ok \N 1 Nullable(Int8) \N ok +\N Nullable(Float64) 0 +\N Nullable(Float64) 0 +1 +1 diff --git a/tests/queries/0_stateless/01323_if_with_nulls.sql b/tests/queries/0_stateless/01323_if_with_nulls.sql index f2cd943988f..6a4df79d765 100644 --- a/tests/queries/0_stateless/01323_if_with_nulls.sql +++ b/tests/queries/0_stateless/01323_if_with_nulls.sql @@ -36,3 +36,20 @@ SELECT b_num, isNull(b_num), toTypeName(b_num), b_num = 0, if(b_num = 0, 'fail', FROM (SELECT 1 k, toInt8(1) a_num) AS x LEFT JOIN (SELECT 2 k, toInt8(1) b_num) AS y USING (k); + +-- test case from https://github.com/ClickHouse/ClickHouse/issues/7347 +DROP TABLE IF EXISTS test_nullable_float_issue7347; +CREATE TABLE test_nullable_float_issue7347 (ne UInt64,test Nullable(Float64)) ENGINE = MergeTree() PRIMARY KEY (ne) ORDER BY (ne); +INSERT INTO test_nullable_float_issue7347 VALUES (1,NULL); + +SELECT test, toTypeName(test), IF(test = 0, 1, 0) FROM test_nullable_float_issue7347; + +WITH materialize(CAST(NULL, 'Nullable(Float64)')) AS test SELECT test, toTypeName(test), IF(test = 0, 1, 0); + +DROP TABLE test_nullable_float_issue7347; + +-- test case from https://github.com/ClickHouse/ClickHouse/issues/10846 + +SELECT if(isFinite(toUInt64OrZero(toNullable('123'))), 1, 0); + +SELECT if(materialize(isFinite(toUInt64OrZero(toNullable('123')))), 1, 0); From 2906ae37f50bfb5504ad80ec56585d245caa6f54 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Tue, 7 Jul 2020 13:26:11 +0300 Subject: [PATCH 109/196] Simple (and fast) inplace fix for UInt8 -> bool --- src/Functions/FunctionsLogical.cpp | 14 ++++++++------ src/Functions/FunctionsLogical.h | 9 ++++++--- 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 07351f6f2c9..5443df33d79 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -71,7 +71,7 @@ bool tryConvertColumnToBool(const IColumn * column, UInt8Container & res) std::transform( col->getData().cbegin(), col->getData().cend(), res.begin(), - [](const auto x) { return x != 0; }); + [](const auto x) { return !!x; }); return true; } @@ -145,6 +145,7 @@ inline bool extractConstColumnsAsTernary(ColumnRawPtrs & in, UInt8 & res_3v) } +/// N.B. This class calculates result only for non-nullable types template class AssociativeApplierImpl { @@ -158,7 +159,7 @@ public: /// Returns a combination of values in the i-th row of all columns stored in the constructor. inline ResultValueType apply(const size_t i) const { - const auto & a = vec[i]; + const auto a = !!vec[i]; if constexpr (Op::isSaturable()) return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i)); else @@ -179,7 +180,7 @@ public: explicit AssociativeApplierImpl(const UInt8ColumnPtrs & in) : vec(in[in.size() - 1]->getData()) {} - inline ResultValueType apply(const size_t i) const { return vec[i]; } + inline ResultValueType apply(const size_t i) const { return !!vec[i]; } private: const UInt8Container & vec; @@ -247,7 +248,7 @@ public: { const auto a = val_getter(i); if constexpr (Op::isSaturable()) - return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i)); + return Op::isSaturatedValueTernary(a) ? a : Op::apply(a, next.apply(i)); else return Op::apply(a, next.apply(i)); } @@ -332,7 +333,7 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn { /// Combine all constant columns into a single constant value. UInt8 const_3v_value = 0; - const bool has_consts = extractConstColumnsTernary(arguments, const_3v_value); + const bool has_consts = extractConstColumnsAsTernary(arguments, const_3v_value); /// If the constant value uniquely determines the result, return it. if (has_consts && (arguments.empty() || Op::isSaturatedValue(const_3v_value))) @@ -402,12 +403,13 @@ struct TypedExecutorInvoker }; +/// Types of all of the arguments are guaranteed to be non-nullable here template static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & result_info, size_t input_rows_count) { /// Combine all constant columns into a single constant value. UInt8 const_val = 0; - bool has_consts = extractConstColumns(arguments, const_val); + bool has_consts = extractConstColumnsAsBool(arguments, const_val); /// If the constant value uniquely determines the result, return it. if (has_consts && (arguments.empty() || Op::apply(const_val, 0) == Op::apply(const_val, 1))) diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 520a2418890..4751f0b7548 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -61,7 +61,8 @@ struct AndImpl using ResultType = UInt8; static inline constexpr bool isSaturable() { return true; } - static inline constexpr bool isSaturatedValue(UInt8 a) { return a == Ternary::False; } + static inline constexpr bool isSaturatedValue(bool a) { return !a; } + static inline constexpr bool isSaturatedValueTernary(UInt8 a) { return a == Ternary::False; } static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return a & b; } static inline constexpr bool specialImplementationForNulls() { return true; } }; @@ -71,7 +72,8 @@ struct OrImpl using ResultType = UInt8; static inline constexpr bool isSaturable() { return true; } - static inline constexpr bool isSaturatedValue(UInt8 a) { return a == Ternary::True; } + static inline constexpr bool isSaturatedValue(bool a) { return a; } + static inline constexpr bool isSaturatedValueTernary(UInt8 a) { return a == Ternary::True; } static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return a | b; } static inline constexpr bool specialImplementationForNulls() { return true; } }; @@ -82,7 +84,8 @@ struct XorImpl static inline constexpr bool isSaturable() { return false; } static inline constexpr bool isSaturatedValue(bool) { return false; } - static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return !!a != !!b; } + static inline constexpr bool isSaturatedValueTernary(UInt8) { return false; } + static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return a != b; } static inline constexpr bool specialImplementationForNulls() { return false; } #if USE_EMBEDDED_COMPILER From 688f185f9ed990b676430057f61773646f2f0f96 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 13:28:45 +0300 Subject: [PATCH 110/196] style fix for #12152 --- src/Server/MySQLHandler.cpp | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 9e42f5ebc05..7d91b18983d 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -45,9 +45,9 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } -static String select_empty_replacement_query(const String & query); -static String show_table_status_replacement_query(const String & query); -static String kill_connection_id_replacement_query(const String & query); +static String selectEmptyReplacementQuery(const String & query); +static String showTableStatusReplacementQuery(const String & query); +static String killConnectionIdReplacementQuery(const String & query); MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool ssl_enabled, size_t connection_id_) @@ -62,9 +62,9 @@ MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & so if (ssl_enabled) server_capability_flags |= CLIENT_SSL; - replacements.emplace("KILL QUERY", kill_connection_id_replacement_query); - replacements.emplace("SHOW TABLE STATUS LIKE", show_table_status_replacement_query); - replacements.emplace("SHOW VARIABLES", select_empty_replacement_query); + replacements.emplace("KILL QUERY", killConnectionIdReplacementQuery); + replacements.emplace("SHOW TABLE STATUS LIKE", showTableStatusReplacementQuery); + replacements.emplace("SHOW VARIABLES", selectEmptyReplacementQuery); } void MySQLHandler::run() @@ -380,14 +380,14 @@ static bool isFederatedServerSetupSetCommand(const String & query) } /// Replace "[query(such as SHOW VARIABLES...)]" into "". -static String select_empty_replacement_query(const String & query) +static String selectEmptyReplacementQuery(const String & query) { std::ignore = query; return "select ''"; } /// Replace "SHOW TABLE STATUS LIKE 'xx'" into "SELECT ... FROM system.tables WHERE name LIKE 'xx'". -static String show_table_status_replacement_query(const String & query) +static String showTableStatusReplacementQuery(const String & query) { const String prefix = "SHOW TABLE STATUS LIKE "; if (query.size() > prefix.size()) @@ -421,7 +421,7 @@ static String show_table_status_replacement_query(const String & query) } /// Replace "KILL QUERY [connection_id]" into "KILL QUERY WHERE query_id = 'mysql:[connection_id]'". -static String kill_connection_id_replacement_query(const String & query) +static String killConnectionIdReplacementQuery(const String & query) { const String prefix = "KILL QUERY "; if (query.size() > prefix.size()) From 6170bfd668c856ee2037961d1c07de3dca7e730f Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 7 Jul 2020 12:33:43 +0200 Subject: [PATCH 111/196] Added test for #3767 --- .../queries/0_stateless/01375_null_issue_3767.reference | 0 tests/queries/0_stateless/01375_null_issue_3767.sql | 9 +++++++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/01375_null_issue_3767.reference create mode 100644 tests/queries/0_stateless/01375_null_issue_3767.sql diff --git a/tests/queries/0_stateless/01375_null_issue_3767.reference b/tests/queries/0_stateless/01375_null_issue_3767.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01375_null_issue_3767.sql b/tests/queries/0_stateless/01375_null_issue_3767.sql new file mode 100644 index 00000000000..88b18e001f9 --- /dev/null +++ b/tests/queries/0_stateless/01375_null_issue_3767.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS null_issue_3767; + +CREATE TABLE null_issue_3767 (value Nullable(String)) ENGINE=Memory; + +INSERT INTO null_issue_3767 (value) VALUES ('A String'), (NULL); + +SELECT value FROM null_issue_3767 WHERE value NOT IN ('A String'); + +DROP TABLE null_issue_3767; From dc85f590b7485a801a15c70b68c1b990894bae0a Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 7 Jul 2020 18:35:16 +0800 Subject: [PATCH 112/196] Update zh kafka.md title (#12192) --- docs/zh/engines/table-engines/integrations/kafka.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/engines/table-engines/integrations/kafka.md b/docs/zh/engines/table-engines/integrations/kafka.md index c96b3bc22ed..557ab6a7845 100644 --- a/docs/zh/engines/table-engines/integrations/kafka.md +++ b/docs/zh/engines/table-engines/integrations/kafka.md @@ -1,4 +1,4 @@ -# 卡夫卡 {#kafka} +# Kafka {#kafka} 此引擎与 [Apache Kafka](http://kafka.apache.org/) 结合使用。 From 241b897e4bd40aa845b509c411d418ac8e338267 Mon Sep 17 00:00:00 2001 From: BohuTANG Date: Tue, 7 Jul 2020 18:35:39 +0800 Subject: [PATCH 113/196] Update index.md (#12191) Fix merge link broken --- docs/zh/engines/table-engines/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/engines/table-engines/index.md b/docs/zh/engines/table-engines/index.md index b025eb1f6c9..f31fa257135 100644 --- a/docs/zh/engines/table-engines/index.md +++ b/docs/zh/engines/table-engines/index.md @@ -52,7 +52,7 @@ - [Distributed](special/distributed.md#distributed) - [MaterializedView](special/materializedview.md#materializedview) - [Dictionary](special/dictionary.md#dictionary) -- [Merge](special/merge.md#merge +- [Merge](special/merge.md#merge) - [File](special/file.md#file) - [Null](special/null.md#null) - [Set](special/set.md#set) From f8d584d5090a53d2306628d6a56afaca770d7773 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 13:49:18 +0300 Subject: [PATCH 114/196] changelog fixes --- CHANGELOG.md | 7 +++++++ utils/simple-backport/backport.sh | 2 +- utils/simple-backport/changelog.sh | 2 +- 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 816aa556e81..54f574cc347 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,6 +21,7 @@ * Add query performance metrics based on Linux `perf_events` (these metrics are calculated with hardware CPU counters and OS counters). It is optional and requires `CAP_SYS_ADMIN` to be set on clickhouse binary. [#9545](https://github.com/ClickHouse/ClickHouse/pull/9545) [Andrey Skobtsov](https://github.com/And42). [#11226](https://github.com/ClickHouse/ClickHouse/pull/11226) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Now support `NULL` and `NOT NULL` modifiers for data types in `CREATE` query. [#11057](https://github.com/ClickHouse/ClickHouse/pull/11057) ([Павел Потемкин](https://github.com/Potya)). * Add `ArrowStream` input and output format. [#11088](https://github.com/ClickHouse/ClickHouse/pull/11088) ([hcz](https://github.com/hczhcz)). +* Support Cassandra as external dictionary source. [#4978](https://github.com/ClickHouse/ClickHouse/pull/4978) ([favstovol](https://github.com/favstovol)). * Added a new layout `direct` which loads all the data directly from the source for each query, without storing or caching data. [#10622](https://github.com/ClickHouse/ClickHouse/pull/10622) ([Artem Streltsov](https://github.com/kekekekule)). * Added new `complex_key_direct` layout to dictionaries, that does not store anything locally during query execution. [#10850](https://github.com/ClickHouse/ClickHouse/pull/10850) ([Artem Streltsov](https://github.com/kekekekule)). * Added support for MySQL style global variables syntax (stub). This is needed for compatibility of MySQL protocol. [#11832](https://github.com/ClickHouse/ClickHouse/pull/11832) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -32,6 +33,7 @@ * Add `netloc` function for extracting network location, similar to `urlparse(url)`, `netloc` in python. [#11356](https://github.com/ClickHouse/ClickHouse/pull/11356) ([Guillaume Tassery](https://github.com/YiuRULE)). * Add 2 more virtual columns for engine=Kafka to access message headers. [#11283](https://github.com/ClickHouse/ClickHouse/pull/11283) ([filimonov](https://github.com/filimonov)). * Add `_timestamp_ms` virtual column for Kafka engine (type is `Nullable(DateTime64(3))`). [#11260](https://github.com/ClickHouse/ClickHouse/pull/11260) ([filimonov](https://github.com/filimonov)). +* Add function `randomFixedString`. [#10866](https://github.com/ClickHouse/ClickHouse/pull/10866) ([Andrei Nekrashevich](https://github.com/xolm)). * Add function `fuzzBits` that randomly flips bits in a string with given probability. [#11237](https://github.com/ClickHouse/ClickHouse/pull/11237) ([Andrei Nekrashevich](https://github.com/xolm)). * Allow comparison of numbers with constant string in comparison operators, IN and VALUES sections. [#11647](https://github.com/ClickHouse/ClickHouse/pull/11647) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add `round_robin` load_balancing mode. [#11645](https://github.com/ClickHouse/ClickHouse/pull/11645) ([Azat Khuzhin](https://github.com/azat)). @@ -140,6 +142,7 @@ * Fixed parseDateTime64BestEffort argument resolution bugs. [#10925](https://github.com/ClickHouse/ClickHouse/issues/10925). [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). * Now it's possible to `ADD/DROP` and `RENAME` the same one column in a single `ALTER` query. Exception message for simultaneous `MODIFY` and `RENAME` became more clear. Partially fixes [#10669](https://github.com/ClickHouse/ClickHouse/issues/10669). [#11037](https://github.com/ClickHouse/ClickHouse/pull/11037) ([alesapin](https://github.com/alesapin)). * Fixed parsing of S3 URLs. [#11036](https://github.com/ClickHouse/ClickHouse/pull/11036) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix memory tracking for two-level `GROUP BY` when there is a `LIMIT`. [#11022](https://github.com/ClickHouse/ClickHouse/pull/11022) ([Azat Khuzhin](https://github.com/azat)). * Fix very rare potential use-after-free error in MergeTree if table was not created successfully. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix metadata (relative path for rename) and data (relative path for symlink) handling for Atomic database. [#10980](https://github.com/ClickHouse/ClickHouse/pull/10980) ([Azat Khuzhin](https://github.com/azat)). * Fix server crash on concurrent `ALTER` and `DROP DATABASE` queries with `Atomic` database engine. [#10968](https://github.com/ClickHouse/ClickHouse/pull/10968) ([tavplubix](https://github.com/tavplubix)). @@ -208,6 +211,7 @@ * Default user and database creation on docker image starting. [#10637](https://github.com/ClickHouse/ClickHouse/pull/10637) ([Paramtamtam](https://github.com/tarampampam)). * When multiline query is printed to server log, the lines are joined. Make it to work correct in case of multiline string literals, identifiers and single-line comments. This fixes [#3853](https://github.com/ClickHouse/ClickHouse/issues/3853). [#11686](https://github.com/ClickHouse/ClickHouse/pull/11686) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Multiple names are now allowed in commands: CREATE USER, CREATE ROLE, ALTER USER, SHOW CREATE USER, SHOW GRANTS and so on. [#11670](https://github.com/ClickHouse/ClickHouse/pull/11670) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add support for distributed DDL (`UPDATE/DELETE/DROP PARTITION`) on cross replication clusters. [#11508](https://github.com/ClickHouse/ClickHouse/pull/11508) ([frank lee](https://github.com/etah000)). * Clear password from command line in `clickhouse-client` and `clickhouse-benchmark` if the user has specified it with explicit value. This prevents password exposure by `ps` and similar tools. [#11665](https://github.com/ClickHouse/ClickHouse/pull/11665) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Don't use debug info from ELF file if it doesn't correspond to the running binary. It is needed to avoid printing wrong function names and source locations in stack traces. This fixes [#7514](https://github.com/ClickHouse/ClickHouse/issues/7514). [#11657](https://github.com/ClickHouse/ClickHouse/pull/11657) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Return NULL/zero when value is not parsed completely in parseDateTimeBestEffortOrNull/Zero functions. This fixes [#7876](https://github.com/ClickHouse/ClickHouse/issues/7876). [#11653](https://github.com/ClickHouse/ClickHouse/pull/11653) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -223,6 +227,7 @@ * Automatically update DNS cache, which is used to check if user is allowed to connect from an address. [#11487](https://github.com/ClickHouse/ClickHouse/pull/11487) ([tavplubix](https://github.com/tavplubix)). * OPTIMIZE FINAL will force merge even if concurrent merges are performed. This closes [#11309](https://github.com/ClickHouse/ClickHouse/issues/11309) and closes [#11322](https://github.com/ClickHouse/ClickHouse/issues/11322). [#11346](https://github.com/ClickHouse/ClickHouse/pull/11346) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Suppress output of cancelled queries in clickhouse-client. In previous versions result may continue to print in terminal even after you press Ctrl+C to cancel query. This closes [#9473](https://github.com/ClickHouse/ClickHouse/issues/9473). [#11342](https://github.com/ClickHouse/ClickHouse/pull/11342) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now history file is updated after each query and there is no race condition if multiple clients use one history file. This fixes [#9897](https://github.com/ClickHouse/ClickHouse/issues/9897). [#11453](https://github.com/ClickHouse/ClickHouse/pull/11453) ([Tagir Kuskarov](https://github.com/kuskarov)). * Better log messages in while reloading configuration. [#11341](https://github.com/ClickHouse/ClickHouse/pull/11341) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Remove trailing whitespaces from formatted queries in `clickhouse-client` or `clickhouse-format` in some cases. [#11325](https://github.com/ClickHouse/ClickHouse/pull/11325) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add setting "output_format_pretty_max_value_width". If value is longer, it will be cut to avoid output of too large values in terminal. This closes [#11140](https://github.com/ClickHouse/ClickHouse/issues/11140). [#11324](https://github.com/ClickHouse/ClickHouse/pull/11324) ([alexey-milovidov](https://github.com/alexey-milovidov)). @@ -231,6 +236,7 @@ * Support kafka_client_id parameter for Kafka tables. It also changes the default `client.id` used by ClickHouse when communicating with Kafka to be more verbose and usable. [#11252](https://github.com/ClickHouse/ClickHouse/pull/11252) ([filimonov](https://github.com/filimonov)). * Keep the value of `DistributedFilesToInsert` metric on exceptions. In previous versions, the value was set when we are going to send some files, but it is zero, if there was an exception and some files are still pending. Now it corresponds to the number of pending files in filesystem. [#11220](https://github.com/ClickHouse/ClickHouse/pull/11220) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Add support for multi-word data type names (such as `DOUBLE PRECISION` and `CHAR VARYING`) for better SQL compatibility. [#11214](https://github.com/ClickHouse/ClickHouse/pull/11214) ([Павел Потемкин](https://github.com/Potya)). +* Provide synonyms for some data types. [#10856](https://github.com/ClickHouse/ClickHouse/pull/10856) ([Павел Потемкин](https://github.com/Potya)). * The query log is now enabled by default. [#11184](https://github.com/ClickHouse/ClickHouse/pull/11184) ([Ivan Blinkov](https://github.com/blinkov)). * Show authentication type in table system.users and while executing SHOW CREATE USER query. [#11080](https://github.com/ClickHouse/ClickHouse/pull/11080) ([Vitaly Baranov](https://github.com/vitlibar)). * Remove data on explicit `DROP DATABASE` for `Memory` database engine. Fixes [#10557](https://github.com/ClickHouse/ClickHouse/issues/10557). [#11021](https://github.com/ClickHouse/ClickHouse/pull/11021) ([tavplubix](https://github.com/tavplubix)). @@ -257,6 +263,7 @@ * New optimization that takes all operations out of `any` function, enabled with `optimize_move_functions_out_of_any` [#11529](https://github.com/ClickHouse/ClickHouse/pull/11529) ([Ruslan](https://github.com/kamalov-ruslan)). * Improve performance of `clickhouse-client` in interactive mode when Pretty formats are used. In previous versions, significant amount of time can be spent calculating visible width of UTF-8 string. This closes [#11323](https://github.com/ClickHouse/ClickHouse/issues/11323). [#11323](https://github.com/ClickHouse/ClickHouse/pull/11323) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Improved performance for queries with `ORDER BY` and small `LIMIT` (less, then `max_block_size`). [#11171](https://github.com/ClickHouse/ClickHouse/pull/11171) ([Albert Kidrachev](https://github.com/Provet)). +* Add runtime CPU detection to select and dispatch the best function implementation. Add support for codegeneration for multiple targets. This closes [#1017](https://github.com/ClickHouse/ClickHouse/issues/1017). [#10058](https://github.com/ClickHouse/ClickHouse/pull/10058) ([DimasKovas](https://github.com/DimasKovas)). * Enable `mlock` of clickhouse binary by default. It will prevent clickhouse executable from being paged out under high IO load. [#11139](https://github.com/ClickHouse/ClickHouse/pull/11139) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Make queries with `sum` aggregate function and without GROUP BY keys to run multiple times faster. [#10992](https://github.com/ClickHouse/ClickHouse/pull/10992) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Improving radix sort (used in `ORDER BY` with simple keys) by removing some redundant data moves. [#10981](https://github.com/ClickHouse/ClickHouse/pull/10981) ([Arslan Gumerov](https://github.com/g-arslan)). diff --git a/utils/simple-backport/backport.sh b/utils/simple-backport/backport.sh index dcab4106136..71920304d56 100755 --- a/utils/simple-backport/backport.sh +++ b/utils/simple-backport/backport.sh @@ -29,7 +29,7 @@ fi # NOTE keep in sync with ./backport.sh. # Search for PR numbers in commit messages. First variant is normal merge, and second # variant is squashed. Next are some backport message variants. -find_prs=(sed -n "s/^.*merge[d]*.*#\([[:digit:]]\+\).*$/\1/Ip; +find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*(#\([[:digit:]]\+\))$/\1/p; s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") diff --git a/utils/simple-backport/changelog.sh b/utils/simple-backport/changelog.sh index 4d768226e03..33908414235 100755 --- a/utils/simple-backport/changelog.sh +++ b/utils/simple-backport/changelog.sh @@ -21,7 +21,7 @@ fi # NOTE keep in sync with ./backport.sh. # Search for PR numbers in commit messages. First variant is normal merge, and second # variant is squashed. Next are some backport message variants. -find_prs=(sed -n "s/^.*merge[d]*.*#\([[:digit:]]\+\).*$/\1/Ip; +find_prs=(sed -n "s/^.*merg[eding]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*(#\([[:digit:]]\+\))$/\1/p; s/^.*back[- ]*port[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip; s/^.*cherry[- ]*pick[ed of]*.*#\([[:digit:]]\+\).*$/\1/Ip") From 270551e599fbd3ff9e21cc09f0d9f44853f3eb07 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 7 Jul 2020 14:45:20 +0300 Subject: [PATCH 115/196] done --- src/Common/ErrorCodes.cpp | 1 + src/IO/WriteBufferFromFileDescriptor.cpp | 13 ++++++++ src/IO/WriteBufferFromFileDescriptor.h | 2 ++ .../Formats/Impl/CSVRowOutputFormat.cpp | 2 +- .../Formats/Impl/CSVRowOutputFormat.h | 3 +- src/Storages/StorageFile.cpp | 18 +++++++---- .../01375_storage_file_write_prefix.reference | 30 +++++++++++++++++++ .../01375_storage_file_write_prefix.sql | 14 +++++++++ 8 files changed, 76 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01375_storage_file_write_prefix.reference create mode 100644 tests/queries/0_stateless/01375_storage_file_write_prefix.sql diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 0c305075c66..7fcd7572039 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -497,6 +497,7 @@ namespace ErrorCodes extern const int CASSANDRA_INTERNAL_ERROR = 528; extern const int NOT_A_LEADER = 529; extern const int CANNOT_CONNECT_RABBITMQ = 530; + extern const int CANNOT_FSTAT = 531; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/src/IO/WriteBufferFromFileDescriptor.cpp b/src/IO/WriteBufferFromFileDescriptor.cpp index 9710c5a979b..a59ae20c588 100644 --- a/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/src/IO/WriteBufferFromFileDescriptor.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include @@ -33,6 +35,7 @@ namespace ErrorCodes extern const int CANNOT_FSYNC; extern const int CANNOT_SEEK_THROUGH_FILE; extern const int CANNOT_TRUNCATE_FILE; + extern const int CANNOT_FSTAT; } @@ -130,4 +133,14 @@ void WriteBufferFromFileDescriptor::truncate(off_t length) throwFromErrnoWithPath("Cannot truncate file " + getFileName(), getFileName(), ErrorCodes::CANNOT_TRUNCATE_FILE); } + +off_t WriteBufferFromFileDescriptor::size() +{ + struct stat buf; + int res = fstat(fd, &buf); + if (-1 == res) + throwFromErrnoWithPath("Cannot execute fstat " + getFileName(), getFileName(), ErrorCodes::CANNOT_FSTAT); + return buf.st_size; +} + } diff --git a/src/IO/WriteBufferFromFileDescriptor.h b/src/IO/WriteBufferFromFileDescriptor.h index d7df04ee940..18c0ac64f63 100644 --- a/src/IO/WriteBufferFromFileDescriptor.h +++ b/src/IO/WriteBufferFromFileDescriptor.h @@ -44,6 +44,8 @@ public: off_t seek(off_t offset, int whence); void truncate(off_t length); + + off_t size(); }; } diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index 27d022250f2..2ff8defd3b0 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -19,7 +19,7 @@ CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header_ } -void CSVRowOutputFormat::writePrefix() +void CSVRowOutputFormat::doWritePrefix() { const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/src/Processors/Formats/Impl/CSVRowOutputFormat.h index 28bd8edf31a..05b097f7514 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.h @@ -27,10 +27,11 @@ public: void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; void writeFieldDelimiter() override; void writeRowEndDelimiter() override; - void writePrefix() override; void writeBeforeTotals() override; void writeBeforeExtremes() override; + void doWritePrefix() override; + /// https://www.iana.org/assignments/media-types/text/csv String getContentType() const override { diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index c7671fd8759..221e9db9428 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -435,6 +435,7 @@ public: , metadata_snapshot(metadata_snapshot_) , lock(storage.rwlock) { + std::unique_ptr naked_buffer = nullptr; if (storage.use_table_fd) { /** NOTE: Using real file binded to FD may be misleading: @@ -442,17 +443,21 @@ public: * INSERT data; SELECT *; last SELECT returns only insert_data */ storage.table_fd_was_used = true; - write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(storage.table_fd), compression_method, 3); + naked_buffer = std::make_unique(storage.table_fd); } else { if (storage.paths.size() != 1) throw Exception("Table '" + storage.getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); - write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), - compression_method, 3); + naked_buffer = std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); } + /// In case of CSVWithNames we have already written prefix. + if (naked_buffer->size()) + prefix_written = true; + + write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); + writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, metadata_snapshot->getSampleBlock(), context); } @@ -465,7 +470,9 @@ public: void writePrefix() override { - writer->writePrefix(); + if (!prefix_written) + writer->writePrefix(); + prefix_written = true; } void writeSuffix() override @@ -484,6 +491,7 @@ private: std::unique_lock lock; std::unique_ptr write_buf; BlockOutputStreamPtr writer; + bool prefix_written{false}; }; BlockOutputStreamPtr StorageFile::write( diff --git a/tests/queries/0_stateless/01375_storage_file_write_prefix.reference b/tests/queries/0_stateless/01375_storage_file_write_prefix.reference new file mode 100644 index 00000000000..ed9a18b9346 --- /dev/null +++ b/tests/queries/0_stateless/01375_storage_file_write_prefix.reference @@ -0,0 +1,30 @@ +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 diff --git a/tests/queries/0_stateless/01375_storage_file_write_prefix.sql b/tests/queries/0_stateless/01375_storage_file_write_prefix.sql new file mode 100644 index 00000000000..a1f90e6a7a2 --- /dev/null +++ b/tests/queries/0_stateless/01375_storage_file_write_prefix.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS tmp_01375; +DROP TABLE IF EXISTS table_csv_01375; + +CREATE TABLE tmp_01375 (n UInt32, s String) ENGINE = Memory; +CREATE TABLE table_csv_01375 AS tmp ENGINE = File(CSVWithNames); + +INSERT INTO table_csv_01375 SELECT number as n, toString(n) as s FROM numbers(10); +INSERT INTO table_csv_01375 SELECT number as n, toString(n) as s FROM numbers(10); +INSERT INTO table_csv_01375 SELECT number as n, toString(n) as s FROM numbers(10); + +SELECT * FROM table_csv_01375; + +DROP TABLE IF EXISTS tmp_01375; +DROP TABLE IF EXISTS table_csv_01375; From d5c3cadcfc0e450ae024c41ba86843ab50724f69 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 7 Jul 2020 14:53:27 +0300 Subject: [PATCH 116/196] DOCSUP-1348 Russian translation for new functions (#133) (#12194) * Russian translation for new functions * Apply suggestions from code review Co-authored-by: BayoNet * Minor updates to russian text. Co-authored-by: Olga Revyakina Co-authored-by: BayoNet Co-authored-by: Sergei Shtykov Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> Co-authored-by: Olga Revyakina --- docs/en/sql-reference/functions/geo.md | 85 ++++--- docs/ru/sql-reference/functions/geo.md | 293 ++++++++++++++++++++++++- 2 files changed, 342 insertions(+), 36 deletions(-) diff --git a/docs/en/sql-reference/functions/geo.md b/docs/en/sql-reference/functions/geo.md index 65925f8a64b..0e8deb7a2c7 100644 --- a/docs/en/sql-reference/functions/geo.md +++ b/docs/en/sql-reference/functions/geo.md @@ -267,7 +267,7 @@ SELECT geohashesInBox(24.48, 40.56, 24.785, 40.81, 4) AS thasos ## h3GetBaseCell {#h3getbasecell} -Returns the base cell number of the index. +Returns the base cell number of the H3 index. **Syntax** @@ -275,20 +275,22 @@ Returns the base cell number of the index. h3GetBaseCell(index) ``` -**Parameters** +**Parameter** - `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -**Returned values** +**Returned value** -- Hexagon base cell number. Type: [UInt8](../../sql-reference/data-types/int-uint.md). +- Hexagon base cell number. + +Type: [UInt8](../../sql-reference/data-types/int-uint.md). **Example** Query: ``` sql -SELECT h3GetBaseCell(612916788725809151) as basecell +SELECT h3GetBaseCell(612916788725809151) as basecell; ``` Result: @@ -301,7 +303,7 @@ Result: ## h3HexAreaM2 {#h3hexaream2} -Average hexagon area in square meters at the given resolution. +Returns average hexagon area in square meters at the given resolution. **Syntax** @@ -309,20 +311,22 @@ Average hexagon area in square meters at the given resolution. h3HexAreaM2(resolution) ``` -**Parameters** +**Parameter** - `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). -**Returned values** +**Returned value** -- Area in m². Type: [Float64](../../sql-reference/data-types/float.md). +- Area in square meters. + +Type: [Float64](../../sql-reference/data-types/float.md). **Example** Query: ``` sql -SELECT h3HexAreaM2(13) as area +SELECT h3HexAreaM2(13) as area; ``` Result: @@ -335,7 +339,7 @@ Result: ## h3IndexesAreNeighbors {#h3indexesareneighbors} -Returns whether or not the provided H3Indexes are neighbors. +Returns whether or not the provided H3 indexes are neighbors. **Syntax** @@ -348,16 +352,19 @@ h3IndexesAreNeighbors(index1, index2) - `index1` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - `index2` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -**Returned values** +**Returned value** -- Returns `1` if the indexes are neighbors, `0` otherwise. Type: [UInt8](../../sql-reference/data-types/int-uint.md). +- `1` — Indexes are neighbours. +- `0` — Indexes are not neighbours. + +Type: [UInt8](../../sql-reference/data-types/int-uint.md). **Example** Query: ``` sql -SELECT h3IndexesAreNeighbors(617420388351344639, 617420388352655359) AS n +SELECT h3IndexesAreNeighbors(617420388351344639, 617420388352655359) AS n; ``` Result: @@ -370,7 +377,7 @@ Result: ## h3ToChildren {#h3tochildren} -Returns an array with the child indexes of the given index. +Returns an array of child indexes for the given H3 index. **Syntax** @@ -385,14 +392,16 @@ h3ToChildren(index, resolution) **Returned values** -- Array with the child H3 indexes. Array of type: [UInt64](../../sql-reference/data-types/int-uint.md). +- Array of the child H3-indexes. + +Type: [Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md)). **Example** Query: ``` sql -SELECT h3ToChildren(599405990164561919, 6) AS children +SELECT h3ToChildren(599405990164561919, 6) AS children; ``` Result: @@ -405,7 +414,7 @@ Result: ## h3ToParent {#h3toparent} -Returns the parent (coarser) index containing the given index. +Returns the parent (coarser) index containing the given H3 index. **Syntax** @@ -418,16 +427,18 @@ h3ToParent(index, resolution) - `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). - `resolution` — Index resolution. Range: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). -**Returned values** +**Returned value** -- Parent H3 index. Type: [UInt64](../../sql-reference/data-types/int-uint.md). +- Parent H3 index. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Example** Query: ``` sql -SELECT h3ToParent(599405990164561919, 3) as parent +SELECT h3ToParent(599405990164561919, 3) as parent; ``` Result: @@ -440,26 +451,28 @@ Result: ## h3ToString {#h3tostring} -Converts the H3Index representation of the index to the string representation. +Converts the `H3Index` representation of the index to the string representation. ``` sql h3ToString(index) ``` -**Parameters** +**Parameter** - `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -**Returned values** +**Returned value** -- String representation of the H3 index. Type: [String](../../sql-reference/data-types/string.md). +- String representation of the H3 index. + +Type: [String](../../sql-reference/data-types/string.md). **Example** Query: ``` sql -SELECT h3ToString(617420388352917503) as h3_string +SELECT h3ToString(617420388352917503) as h3_string; ``` Result: @@ -472,17 +485,19 @@ Result: ## stringToH3 {#stringtoh3} -Converts the string representation to H3Index (UInt64) representation. +Converts the string representation to the `H3Index` (UInt64) representation. + +**Syntax** ``` sql stringToH3(index_str) ``` -**Parameters** +**Parameter** - `index_str` — String representation of the H3 index. Type: [String](../../sql-reference/data-types/string.md). -**Returned values** +**Returned value** - Hexagon index number. Returns 0 on error. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -491,7 +506,7 @@ stringToH3(index_str) Query: ``` sql -SELECT stringToH3('89184926cc3ffff') as index +SELECT stringToH3('89184926cc3ffff') as index; ``` Result: @@ -504,7 +519,7 @@ Result: ## h3GetResolution {#h3getresolution} -Returns the resolution of the index. +Returns the resolution of the H3 index. **Syntax** @@ -512,11 +527,11 @@ Returns the resolution of the index. h3GetResolution(index) ``` -**Parameters** +**Parameter** - `index` — Hexagon index number. Type: [UInt64](../../sql-reference/data-types/int-uint.md). -**Returned values** +**Returned value** - Index resolution. Range: `[0, 15]`. Type: [UInt8](../../sql-reference/data-types/int-uint.md). @@ -525,7 +540,7 @@ h3GetResolution(index) Query: ``` sql -SELECT h3GetResolution(617420388352917503) as res +SELECT h3GetResolution(617420388352917503) as res; ``` Result: @@ -536,4 +551,4 @@ Result: └─────┘ ``` -[Original article](https://clickhouse.tech/docs/en/query_language/functions/geo/) +[Original article](https://clickhouse.tech/docs/en/sql-reference/functions/geo/) diff --git a/docs/ru/sql-reference/functions/geo.md b/docs/ru/sql-reference/functions/geo.md index 45c30b3c2cd..bf3f90ed47e 100644 --- a/docs/ru/sql-reference/functions/geo.md +++ b/docs/ru/sql-reference/functions/geo.md @@ -382,4 +382,295 @@ SELECT arrayJoin(h3kRing(644325529233966508, 1)) AS h3index └────────────────────┘ ``` -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/geo/) +## h3GetBaseCell {#h3getbasecell} + +Определяет номер базовой (верхнеуровневой) шестиугольной H3-ячейки для указанной ячейки. + +**Синтаксис** + +``` sql +h3GetBaseCell(index) +``` + +**Параметр** + +- `index` — индекс шестиугольной ячейки. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Индекс базовой шестиугольной ячейки. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3GetBaseCell(612916788725809151) as basecell; +``` + +Результат: + +``` text +┌─basecell─┐ +│ 12 │ +└──────────┘ +``` + +## h3HexAreaM2 {#h3hexaream2} + +Определяет среднюю площадь шестиугольной H3-ячейки заданного разрешения в квадратных метрах. + +**Синтаксис** + +``` sql +h3HexAreaM2(resolution) +``` + +**Параметр** + +- `resolution` — разрешение. Диапазон: `[0, 15]`. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Площадь в квадратных метрах. Тип: [Float64](../../sql-reference/data-types/float.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3HexAreaM2(13) as area; +``` + +Результат: + +``` text +┌─area─┐ +│ 43.9 │ +└──────┘ +``` + +## h3IndexesAreNeighbors {#h3indexesareneighbors} + +Определяет, являются ли H3-ячейки соседями. + +**Синтаксис** + +``` sql +h3IndexesAreNeighbors(index1, index2) +``` + +**Параметры** + +- `index1` — индекс шестиугольной ячейки. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). +- `index2` — индекс шестиугольной ячейки. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- `1` — ячейки являются соседями. +- `0` — ячейки не являются соседями. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3IndexesAreNeighbors(617420388351344639, 617420388352655359) AS n; +``` + +Результат: + +``` text +┌─n─┐ +│ 1 │ +└───┘ +``` + +## h3ToChildren {#h3tochildren} + +Формирует массив дочерних (вложенных) H3-ячеек для указанной ячейки. + +**Синтаксис** + +``` sql +h3ToChildren(index, resolution) +``` + +**Параметры** + +- `index` — индекс шестиугольной ячейки. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). +- `resolution` — разрешение. Диапазон: `[0, 15]`. Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Массив дочерних H3-ячеек. + +Тип: [Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md)). + +**Пример** + +Запрос: + +``` sql +SELECT h3ToChildren(599405990164561919, 6) AS children; +``` + +Результат: + +``` text +┌─children───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ [603909588852408319,603909588986626047,603909589120843775,603909589255061503,603909589389279231,603909589523496959,603909589657714687] │ +└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` + +## h3ToParent {#h3toparent} + +Определяет родительскую (более крупную) H3-ячейку, содержащую указанную ячейку. + +**Синтаксис** + +``` sql +h3ToParent(index, resolution) +``` + +**Параметры** + +- `index` — индекс шестиугольной ячейки. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). +- `resolution` — разрешение. Диапазон: `[0, 15]`. Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Индекс родительской H3-ячейки. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3ToParent(599405990164561919, 3) as parent; +``` + +Результат: + +``` text +┌─────────────parent─┐ +│ 590398848891879423 │ +└────────────────────┘ +``` + +## h3ToString {#h3tostring} + +Преобразует H3-индекс из числового представления `H3Index` в строковое. + +``` sql +h3ToString(index) +``` + +**Параметр** + +- `index` — индекс шестиугольной ячейки. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Строковое представление H3-индекса. + +Тип: [String](../../sql-reference/data-types/string.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3ToString(617420388352917503) as h3_string; +``` + +Результат: + +``` text +┌─h3_string───────┐ +│ 89184926cdbffff │ +└─────────────────┘ +``` + +## stringToH3 {#stringtoh3} + +Преобразует H3-индекс из строкового представления в числовое представление `H3Index`. + +**Синтаксис** + +``` sql +stringToH3(index_str) +``` + +**Параметр** + +- `index_str` — строковое представление H3-индекса. Тип: [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- Числовое представление индекса шестиугольной ячейки. +- `0`, если при преобразовании возникла ошибка. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT stringToH3('89184926cc3ffff') as index; +``` + +Результат: + +``` text +┌──────────────index─┐ +│ 617420388351344639 │ +└────────────────────┘ +``` + +## h3GetResolution {#h3getresolution} + +Определяет разрешение H3-ячейки. + +**Синтаксис** + +``` sql +h3GetResolution(index) +``` + +**Параметр** + +- `index` — индекс шестиугольной ячейки. Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +- Разрешение ячейки. Диапазон: `[0, 15]`. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +``` sql +SELECT h3GetResolution(617420388352917503) as res; +``` + +Результат: + +``` text +┌─res─┐ +│ 9 │ +└─────┘ +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/functions/geo/) From f5a7d8a5c73254d2457b04f87b4f1fc80effc28f Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 7 Jul 2020 15:35:06 +0300 Subject: [PATCH 117/196] Add runner for testflows --- docker/test/testflows/runner/Dockerfile | 76 +++++++++++ .../testflows/runner/dockerd-entrypoint.sh | 26 ++++ docker/test/testflows/runner/modprobe.sh | 20 +++ tests/testflows/runner | 121 ++++++++++++++++++ 4 files changed, 243 insertions(+) create mode 100644 docker/test/testflows/runner/Dockerfile create mode 100755 docker/test/testflows/runner/dockerd-entrypoint.sh create mode 100755 docker/test/testflows/runner/modprobe.sh create mode 100755 tests/testflows/runner diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile new file mode 100644 index 00000000000..7dd73b0e0bb --- /dev/null +++ b/docker/test/testflows/runner/Dockerfile @@ -0,0 +1,76 @@ +# docker build -t yandex/clickhouse-testflows-runner . +FROM ubuntu:20.04 + +RUN apt-get update \ + && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ + ca-certificates \ + bash \ + btrfs-progs \ + e2fsprogs \ + iptables \ + xfsprogs \ + tar \ + pigz \ + wget \ + git \ + iproute2 \ + cgroupfs-mount \ + python3-pip \ + tzdata \ + libreadline-dev \ + libicu-dev \ + bsdutils \ + curl \ + liblua5.1-dev \ + luajit \ + libssl-dev \ + libcurl4-openssl-dev \ + gdb \ + && rm -rf \ + /var/lib/apt/lists/* \ + /var/cache/debconf \ + /tmp/* \ + && apt-get clean + +ENV TZ=Europe/Moscow +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +RUN pip3 install urllib3 testflows==1.6.24 docker-compose docker dicttoxml kazoo tzlocal + +ENV DOCKER_CHANNEL stable +ENV DOCKER_VERSION 17.09.1-ce + +RUN set -eux; \ + \ +# this "case" statement is generated via "update.sh" + \ + if ! wget -O docker.tgz "https://download.docker.com/linux/static/${DOCKER_CHANNEL}/x86_64/docker-${DOCKER_VERSION}.tgz"; then \ + echo >&2 "error: failed to download 'docker-${DOCKER_VERSION}' from '${DOCKER_CHANNEL}' for '${x86_64}'"; \ + exit 1; \ + fi; \ + \ + tar --extract \ + --file docker.tgz \ + --strip-components 1 \ + --directory /usr/local/bin/ \ + ; \ + rm docker.tgz; \ + \ + dockerd --version; \ + docker --version + +COPY modprobe.sh /usr/local/bin/modprobe +COPY dockerd-entrypoint.sh /usr/local/bin/ + +RUN set -x \ + && addgroup --system dockremap \ + && adduser --system dockremap \ + && adduser dockremap dockremap \ + && echo 'dockremap:165536:65536' >> /etc/subuid \ + && echo 'dockremap:165536:65536' >> /etc/subgid + +VOLUME /var/lib/docker +EXPOSE 2375 +ENTRYPOINT ["dockerd-entrypoint.sh"] +CMD ["sh", "-c", "python3 regression.py --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS} && cat test.log | tfs report results --format json > results.json"] + diff --git a/docker/test/testflows/runner/dockerd-entrypoint.sh b/docker/test/testflows/runner/dockerd-entrypoint.sh new file mode 100755 index 00000000000..b10deaded08 --- /dev/null +++ b/docker/test/testflows/runner/dockerd-entrypoint.sh @@ -0,0 +1,26 @@ +#!/bin/bash +set -e + +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile & + +set +e +reties=0 +while true; do + docker info &>/dev/null && break + reties=$[$reties+1] + if [[ $reties -ge 100 ]]; then # 10 sec max + echo "Can't start docker daemon, timeout exceeded." >&2 + exit 1; + fi + sleep 0.1 +done +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_ODBC_BRIDGE_BINARY_PATH=/clickhouse-odbc-bridge + +cd /ClickHouse/tests/testflows +exec "$@" diff --git a/docker/test/testflows/runner/modprobe.sh b/docker/test/testflows/runner/modprobe.sh new file mode 100755 index 00000000000..cb6a527736b --- /dev/null +++ b/docker/test/testflows/runner/modprobe.sh @@ -0,0 +1,20 @@ +#!/bin/sh +set -eu + +# "modprobe" without modprobe +# https://twitter.com/lucabruno/status/902934379835662336 + +# this isn't 100% fool-proof, but it'll have a much higher success rate than simply using the "real" modprobe + +# Docker often uses "modprobe -va foo bar baz" +# so we ignore modules that start with "-" +for module; do + if [ "${module#-}" = "$module" ]; then + ip link show "$module" || true + lsmod | grep "$module" || true + fi +done + +# remove /usr/local/... from PATH so we can exec the real modprobe as a last resort +export PATH='/usr/sbin:/usr/bin:/sbin:/bin' +exec modprobe "$@" diff --git a/tests/testflows/runner b/tests/testflows/runner new file mode 100755 index 00000000000..eef51b3c20f --- /dev/null +++ b/tests/testflows/runner @@ -0,0 +1,121 @@ +#!/usr/bin/env python +#-*- coding: utf-8 -*- +import subprocess +import os +import getpass +import argparse +import logging +import signal +import subprocess +import sys + +CUR_FILE_DIR = os.path.dirname(os.path.realpath(__file__)) +DEFAULT_CLICKHOUSE_ROOT = os.path.abspath(os.path.join(CUR_FILE_DIR, "../../")) +CURRENT_WORK_DIR = os.getcwd() +CONTAINER_NAME = "clickhouse_testflows_tests" + +DIND_TESTFLOWS_TESTS_IMAGE_NAME = "yandex/clickhouse-testflows-runner" + +def check_args_and_update_paths(args): + if not os.path.isabs(args.binary): + args.binary = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.binary)) + + if not args.bridge_binary: + args.bridge_binary = os.path.join(os.path.dirname(args.binary), 'clickhouse-odbc-bridge') + elif not os.path.isabs(args.bridge_binary): + args.bridge_binary = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.bridge_binary)) + + if not os.path.isabs(args.configs_dir): + args.configs_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.configs_dir)) + + if not os.path.isabs(args.clickhouse_root): + args.clickhouse_root = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.clickhouse_root)) + + for path in [args.binary, args.configs_dir, args.clickhouse_root]: + if not os.path.exists(path): + raise Exception("Path {} doesn't exists".format(path)) + +def docker_kill_handler_handler(signum, frame): + subprocess.check_call('docker kill $(docker ps -a -q --filter name={name} --format="{{{{.ID}}}}")'.format(name=CONTAINER_NAME), shell=True) + raise KeyboardInterrupt("Killed by Ctrl+C") + +signal.signal(signal.SIGINT, docker_kill_handler_handler) + +if __name__ == "__main__": + logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') + parser = argparse.ArgumentParser(description="ClickHouse testflows runner") + + parser.add_argument( + "--binary", + default=os.environ.get("CLICKHOUSE_TESTS_SERVER_BIN_PATH", os.environ.get("CLICKHOUSE_TESTS_CLIENT_BIN_PATH", "/usr/bin/clickhouse")), + help="Path to clickhouse binary") + + parser.add_argument( + "--bridge-binary", + default=os.environ.get("CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH", ""), + help="Path to clickhouse-odbc-bridge binary. Defaults to clickhouse-odbc-bridge in the same dir as clickhouse.") + + parser.add_argument( + "--configs-dir", + default=os.environ.get("CLICKHOUSE_TESTS_BASE_CONFIG_DIR", os.path.join(DEFAULT_CLICKHOUSE_ROOT, "programs/server")), + help="Path to clickhouse configs directory") + + parser.add_argument( + "--clickhouse-root", + default=DEFAULT_CLICKHOUSE_ROOT, + help="Path to repository root folder") + + parser.add_argument( + "--command", + default='', + help="Set it to run some other command in container (for example bash)") + + parser.add_argument( + "--disable-net-host", + action='store_true', + default=False, + help="Don't use net host in parent docker container") + + parser.add_argument( + "--docker-image-version", + default="latest", + help="Version of docker image which runner will use to run tests") + + + parser.add_argument('testflows_args', nargs='*', help="args for testflows command") + + args = parser.parse_args() + + check_args_and_update_paths(args) + + net = "" + if not args.disable_net_host: + net = "--net=host" + + # create named volume which will be used inside to store images and other docker related files, + # to avoid redownloading it every time + # + # should be removed manually when not needed + subprocess.check_call('docker volume create {name}_volume'.format(name=CONTAINER_NAME), shell=True) + + # enable tty mode & interactive for docker if we have real tty + tty = "" + if sys.stdout.isatty() and sys.stdin.isatty(): + tty = "-it" + + cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ + --volume={cfg}:/clickhouse-config --volume={pth}:/ClickHouse --volume={name}_volume:/var/lib/docker -e TESTFLOWS_OPTS='{opts}' {img} {command}".format( + net=net, + tty=tty, + bin=args.binary, + bridge_bin=args.bridge_binary, + cfg=args.configs_dir, + pth=args.clickhouse_root, + opts=' '.join(args.testflows_args), + img=DIND_TESTFLOWS_TESTS_IMAGE_NAME + ":" + args.docker_image_version, + name=CONTAINER_NAME, + command=args.command + ) + + print("Running testflows container as: '" + cmd + "'.") + subprocess.check_call(cmd, shell=True) From 0c1985990b4eff91809c19cfd3f00cd754676ba2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 15:35:53 +0300 Subject: [PATCH 118/196] fixes --- programs/client/Client.cpp | 87 ++++++++++++++++++++++++--------- programs/client/QueryFuzzer.cpp | 4 +- 2 files changed, 67 insertions(+), 24 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 2a500ea277f..17a54a52b16 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -784,43 +784,71 @@ private: return; } - /// If 'query' parameter is not set, read a query from stdin. - /// The query is read entirely into memory (streaming is disabled). + // Try to stream the queries from stdin, without reading all of them + // into memory. The interface of the parser does not support streaming, + // in particular, it can't distinguish the end of partial input buffer + // and the final end of input file. This means we have to try to split + // the input into separate queries here. Two patterns of input are + // especially interesing: + // 1) multiline query: + // select 1 + // from system.numbers; + // + // 2) csv insert with in-place data: + // insert into t format CSV 1;2 + // + // (1) means we can't split on new line, and (2) means we can't split on + // semicolon. Solution: split on ';\n'. This sequence is frequent enough + // in the SQL tests which are our principal input for fuzzing. Now we + // have another interesting case: + // 3) escaped semicolon followed by newline, e.g. + // select '; + // ' + // + // To handle (3), parse until we can, and read more data if the parser + // complains. Hopefully this should be enough... ReadBufferFromFileDescriptor in(STDIN_FILENO); std::string text; while (!in.eof()) { - // Read until ';' + // Read until separator. while (!in.eof()) { - char * next_semicolon = find_first_symbols<';'>(in.position(), + char * next_separator = find_first_symbols<';'>(in.position(), in.buffer().end()); - if (next_semicolon < in.buffer().end()) + if (next_separator < in.buffer().end()) { - // Found the semicolon, append it as well. - next_semicolon++; - text.append(in.position(), next_semicolon - in.position()); - in.position() = next_semicolon; - break; + next_separator++; + if (next_separator < in.buffer().end() + && *next_separator == '\n') + { + // Found ';\n', append it to the query text and try to + // parse. + next_separator++; + text.append(in.position(), next_separator - in.position()); + in.position() = next_separator; + break; + } } // Didn't find the semicolon and reached the end of buffer. - text.append(in.position(), next_semicolon - in.position()); - in.position() = next_semicolon; + text.append(in.position(), next_separator - in.position()); + in.position() = next_separator; if (text.size() > 1024 * 1024) { - // We've read a lot of text and still haven't seen a semicolon. + // We've read a lot of text and still haven't seen a separator. // Likely some pathological input, just fall through to prevent // too long loops. break; } } + // Parse and execute what we've read. fprintf(stderr, "will now parse '%s'\n", text.c_str()); - const auto new_end = processWithFuzzing(text); + const auto * new_end = processWithFuzzing(text); if (new_end > &text[0]) { @@ -840,17 +868,19 @@ private: { // Uh-oh... std::cerr << "Lost connection to the server." << std::endl; - last_exception_received_from_server.reset(new Exception(210, "~")); + last_exception_received_from_server + = std::make_unique(210, "~"); return; } if (text.size() > 4 * 1024) { - // Some pathological situation where the text is larger than 1MB + // Some pathological situation where the text is larger than 4kB // and we still cannot parse a single query in it. Abort. std::cerr << "Read too much text and still can't parse a query." " Aborting." << std::endl; - last_exception_received_from_server.reset(new Exception(1, "~")); + last_exception_received_from_server + = std::make_unique(1, "~"); // return; exit(1); } @@ -990,7 +1020,7 @@ private: ++begin; } - const auto this_query_begin = begin; + const auto * this_query_begin = begin; ASTPtr orig_ast = parseQuery(begin, end, true); if (!orig_ast) @@ -999,7 +1029,7 @@ private: return begin; } - auto as_insert = orig_ast->as(); + auto * as_insert = orig_ast->as(); if (as_insert && as_insert->data) { // INSERT data is ended by newline @@ -1019,9 +1049,13 @@ private: ASTPtr ast_to_process; try { + std::stringstream dump_before_fuzz; + fuzz_base->dumpTree(dump_before_fuzz); auto base_before_fuzz = fuzz_base->formatForErrorMessage(); + ast_to_process = fuzz_base->clone(); fuzzer.fuzzMain(ast_to_process); + auto base_after_fuzz = fuzz_base->formatForErrorMessage(); // Debug AST cloning errors. @@ -1030,6 +1064,10 @@ private: fprintf(stderr, "base before fuzz: %s\n" "base after fuzz: %s\n", base_before_fuzz.c_str(), base_after_fuzz.c_str()); + fprintf(stderr, "dump before fuzz:\n%s\n", + dump_before_fuzz.str().c_str()); + fprintf(stderr, "dump after fuzz:\n"); + fuzz_base->dumpTree(std::cerr); assert(false); } @@ -1052,10 +1090,14 @@ private: std::cerr << "Error on processing query: " << ast_to_process->formatForErrorMessage() << std::endl << last_exception_received_from_server->message(); } - if (received_exception_from_server && !ignore_error) + if (received_exception_from_server) { - // fuzz again - fprintf(stderr, "got error, will fuzz again\n"); + // Query completed with error, ignore it and fuzz again. + fprintf(stderr, "Got error, will fuzz again\n"); + + received_exception_from_server = false; + last_exception_received_from_server.reset(); + continue; } else if (ast_to_process->formatForErrorMessage().size() > 500) @@ -1111,6 +1153,7 @@ private: void processParsedSingleQuery() { resetOutput(); + last_exception_received_from_server.reset(); received_exception_from_server = false; if (echo_queries) diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index 72e7fb1aa9a..0617a0e9fdf 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -35,7 +35,7 @@ namespace ErrorCodes Field QueryFuzzer::getRandomField(int type) { - switch(type) + switch (type) { case 0: { @@ -458,4 +458,4 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) std::cout << std::endl << std::endl; } -} // namespace DB +} From e89ccfd5bf99e5c98abe18b0290990d47cd350f6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 7 Jul 2020 16:11:19 +0300 Subject: [PATCH 119/196] No color --- docker/test/testflows/runner/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile index 7dd73b0e0bb..71dca40bca6 100644 --- a/docker/test/testflows/runner/Dockerfile +++ b/docker/test/testflows/runner/Dockerfile @@ -72,5 +72,5 @@ RUN set -x \ VOLUME /var/lib/docker EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] -CMD ["sh", "-c", "python3 regression.py --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS} && cat test.log | tfs report results --format json > results.json"] +CMD ["sh", "-c", "python3 regression.py --no-color --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS} && cat test.log | tfs report results --format json > results.json"] From 78ba9c986ff635b88d2bd6cdeb4e3b7591d9937a Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Tue, 7 Jul 2020 16:20:48 +0300 Subject: [PATCH 120/196] throw exception on redirect limit in S3 request --- src/IO/S3/PocoHTTPClient.cpp | 9 ++++++++- .../test_storage_s3/s3_mock/mock_s3.py | 9 ++++++++- tests/integration/test_storage_s3/test.py | 16 ++++++++++++++++ 3 files changed, 32 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 56632b22071..f2315861bf5 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -14,6 +14,11 @@ #include #include +namespace DB::ErrorCodes +{ + extern const int TOO_MANY_REDIRECTS; +} + namespace DB::S3 { PocoHTTPClient::PocoHTTPClient(const Aws::Client::ClientConfiguration & clientConfiguration) @@ -153,8 +158,10 @@ void PocoHTTPClient::MakeRequestInternal( else response->GetResponseStream().SetUnderlyingStream(std::make_shared(session, response_body_stream)); - break; + return; } + throw Exception(String("Too many redirects while trying to access ") + request.GetUri().GetURIString(), + ErrorCodes::TOO_MANY_REDIRECTS); } catch (...) { diff --git a/tests/integration/test_storage_s3/s3_mock/mock_s3.py b/tests/integration/test_storage_s3/s3_mock/mock_s3.py index 35b477d6b10..5b422f6a73a 100644 --- a/tests/integration/test_storage_s3/s3_mock/mock_s3.py +++ b/tests/integration/test_storage_s3/s3_mock/mock_s3.py @@ -1,4 +1,11 @@ -from bottle import abort, route, run, request +from bottle import abort, route, run, request, response + + +@route('/redirected/<_path>') +def infinite_redirect(_path): + response.set_header("Location", request.url) + response.status = 307 + return 'Redirected' @route('/<_bucket>/<_path>') diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index b25e5907e62..56e500fed13 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -336,3 +336,19 @@ def test_get_csv_default(cluster): instance = cluster.instances["dummy"] # type: ClickHouseInstance result = run_query(instance, get_query) assert result == '1\t2\t3\n' + + +def test_infinite_redirect(cluster): + bucket = "redirected" + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + filename = "test.csv" + get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( + bucket="redirected", + file=filename, + table_format=table_format) + instance = cluster.instances["dummy"] # type: ClickHouseInstance + try: + result = run_query(instance, get_query) + except Exception as e: + assert str(e).find("Too many redirects while trying to access") != -1 + From ffad7eb966e27f454eab9e524fbe446bf8cc52d4 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 7 Jul 2020 17:07:56 +0300 Subject: [PATCH 121/196] fix test --- tests/queries/0_stateless/01375_storage_file_write_prefix.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01375_storage_file_write_prefix.sql b/tests/queries/0_stateless/01375_storage_file_write_prefix.sql index a1f90e6a7a2..b335db03f6c 100644 --- a/tests/queries/0_stateless/01375_storage_file_write_prefix.sql +++ b/tests/queries/0_stateless/01375_storage_file_write_prefix.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS tmp_01375; DROP TABLE IF EXISTS table_csv_01375; CREATE TABLE tmp_01375 (n UInt32, s String) ENGINE = Memory; -CREATE TABLE table_csv_01375 AS tmp ENGINE = File(CSVWithNames); +CREATE TABLE table_csv_01375 AS tmp_01375 ENGINE = File(CSVWithNames); INSERT INTO table_csv_01375 SELECT number as n, toString(n) as s FROM numbers(10); INSERT INTO table_csv_01375 SELECT number as n, toString(n) as s FROM numbers(10); From 13f601c9c09b2eff892dcf81d0e4e4c90c8d3e85 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 7 Jul 2020 17:17:22 +0300 Subject: [PATCH 122/196] [docs] add intrdocution for commercial page (#12187) --- docs/en/commercial/index.md | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/docs/en/commercial/index.md b/docs/en/commercial/index.md index f9065c7cd50..a8358f48b97 100644 --- a/docs/en/commercial/index.md +++ b/docs/en/commercial/index.md @@ -1,7 +1,18 @@ --- toc_folder_title: Commercial toc_priority: 70 -toc_title: Commercial +toc_title: Introduction --- +# ClickHouse Commercial Services +This section is a directory of commercial service providers specializing in ClickHouse. They are independent companies not necessarily affiliated with Yandex. + +Service categories: + +- [Cloud](cloud.md) +- [Support](support.md) + + +!!! note "For service providers" + If you happen to represent one of them, feel free to open a pull request adding your company to the respective section (or even adding a new section if the service doesn't fit into existing categories). The easiest way to open a pull-request for documentation page is by using a “pencil” edit button in the top-right corner. If your service available in some local market, make sure to mention it in a localized documentation page as well (or at least point it out in a pull-request description). From ff4505123189d229bd4a93eb28045880924b4c72 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Tue, 7 Jul 2020 17:25:44 +0300 Subject: [PATCH 123/196] DOCS-647: toStartOfSecond (#12190) * DOCSUP-1120 Documentation for the toStartOfSecond function (#131) * Doc toStartOfSecond function * Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: BayoNet * Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: BayoNet * Minor update for english text, russian translation added. Co-authored-by: Olga Revyakina Co-authored-by: BayoNet * CLICKHOUSEDOCS-647: Minor text edits. * Update docs/en/sql-reference/functions/date-time-functions.md * Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> Co-authored-by: Olga Revyakina Co-authored-by: Sergei Shtykov Co-authored-by: Ivan Blinkov --- .../functions/date-time-functions.md | 57 +++++++++++++++++++ .../functions/date-time-functions.md | 56 ++++++++++++++++++ 2 files changed, 113 insertions(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3cbc7c73543..4eb316e0455 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -149,6 +149,63 @@ Rounds down a date with time to the start of the hour. Rounds down a date with time to the start of the minute. +## toStartOfSecond {#tostartofsecond} + +Truncates sub-seconds. + +**Syntax** + +``` sql +toStartOfSecond(value[, timezone]) +``` + +**Parameters** + +- `value` — Date and time. [DateTime64](../data-types/datetime64.md). +- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../data-types/string.md). + +**Returned value** + +- Input value without sub-seconds. + +Type: [DateTime64](../data-types/datetime64.md). + +**Examples** + +Query without timezone: + +``` sql +WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 +SELECT toStartOfSecond(dt64); +``` + +Result: + +``` text +┌───toStartOfSecond(dt64)─┐ +│ 2020-01-01 10:20:30.000 │ +└─────────────────────────┘ +``` + +Query with timezone: + +``` sql +WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 +SELECT toStartOfSecond(dt64, 'Europe/Moscow'); +``` + +Result: + +``` text +┌─toStartOfSecond(dt64, 'Europe/Moscow')─┐ +│ 2020-01-01 13:20:30.000 │ +└────────────────────────────────────────┘ +``` + +**See also** + +- [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) server configuration parameter. + ## toStartOfFiveMinute {#tostartoffiveminute} Rounds down a date with time to the start of the five-minute interval. diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index ecd9c760fbc..c4bc42c6ece 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -121,6 +121,62 @@ Result: Округляет дату-с-временем вниз до начала минуты. +## toStartOfSecond {#tostartofsecond} + +Отсекает доли секунды. + +**Синтаксис** + +``` sql +toStartOfSecond(value[, timezone]) +``` + +**Параметры** + +- `value` — Дата и время. [DateTime64](../data-types/datetime64.md). +- `timezone` — [Часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) для возвращаемого значения (необязательно). Если параметр не задан, используется часовой пояс параметра `value`. [String](../data-types/string.md). + +**Возвращаемое значение** + +- Входное значение с отсеченными долями секунды. + +Тип: [DateTime64](../data-types/datetime64.md). + +**Примеры** + +Пример без часового пояса: + +``` sql +WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(dt64); +``` + +Результат: + +``` text +┌───toStartOfSecond(dt64)─┐ +│ 2020-01-01 10:20:30.000 │ +└─────────────────────────┘ +``` + +Пример с часовым поясом: + +``` sql +WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(dt64, 'Europe/Moscow'); +``` + +Результат: + +``` text +┌─toStartOfSecond(dt64, 'Europe/Moscow')─┐ +│ 2020-01-01 13:20:30.000 │ +└────────────────────────────────────────┘ +``` + +**См. также** + +- Часовая зона сервера, конфигурационный параметр [timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). + + ## toStartOfFiveMinute {#tostartoffiveminute} Округляет дату-с-временем вниз до начала пятиминутного интервала. From eefb498517a16c346061c07b5cc5280b21c0c68b Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 7 Jul 2020 17:45:04 +0300 Subject: [PATCH 124/196] [docs] refactor Domains overview (#12186) --- .../sql-reference/data-types/domains/index.md | 27 +++++++++++++++- .../data-types/domains/overview.md | 30 ----------------- .../sql-reference/data-types/domains/index.md | 27 +++++++++++++++- .../data-types/domains/overview.md | 32 ------------------- .../sql-reference/data-types/domains/index.md | 27 +++++++++++++++- .../data-types/domains/overview.md | 32 ------------------- .../sql-reference/data-types/domains/index.md | 25 +++++++++++++++ .../data-types/domains/overview.md | 32 ------------------- .../sql-reference/data-types/domains/index.md | 25 +++++++++++++++ .../data-types/domains/overview.md | 32 ------------------- docs/redirects.txt | 1 + .../sql-reference/data-types/domains/index.md | 29 ++++++++++++++++- .../data-types/domains/overview.md | 26 --------------- .../sql-reference/data-types/domains/index.md | 26 +++++++++++++++ .../data-types/domains/overview.md | 32 ------------------- .../sql-reference/data-types/domains/index.md | 30 +++++++++++++++-- .../data-types/domains/overview.md | 26 --------------- 17 files changed, 211 insertions(+), 248 deletions(-) delete mode 100644 docs/en/sql-reference/data-types/domains/overview.md delete mode 100644 docs/es/sql-reference/data-types/domains/overview.md delete mode 100644 docs/fa/sql-reference/data-types/domains/overview.md delete mode 100644 docs/fr/sql-reference/data-types/domains/overview.md delete mode 100644 docs/ja/sql-reference/data-types/domains/overview.md delete mode 100644 docs/ru/sql-reference/data-types/domains/overview.md delete mode 100644 docs/tr/sql-reference/data-types/domains/overview.md delete mode 100644 docs/zh/sql-reference/data-types/domains/overview.md diff --git a/docs/en/sql-reference/data-types/domains/index.md b/docs/en/sql-reference/data-types/domains/index.md index ddcb5b21d82..30aca1eb059 100644 --- a/docs/en/sql-reference/data-types/domains/index.md +++ b/docs/en/sql-reference/data-types/domains/index.md @@ -1,6 +1,31 @@ --- -toc_folder_title: Domains toc_priority: 56 +toc_folder_title: Domains +toc_title: Overview --- +# Domains {#domains} +Domains are special-purpose types that add some extra features atop of existing base type, but leaving on-wire and on-disc format of the underlying data type intact. At the moment, ClickHouse does not support user-defined domains. + +You can use domains anywhere corresponding base type can be used, for example: + +- Create a column of a domain type +- Read/write values from/to domain column +- Use it as an index if a base type can be used as an index +- Call functions with values of domain column + +### Extra Features of Domains {#extra-features-of-domains} + +- Explicit column type name in `SHOW CREATE TABLE` or `DESCRIBE TABLE` +- Input from human-friendly format with `INSERT INTO domain_table(domain_column) VALUES(...)` +- Output to human-friendly format for `SELECT domain_column FROM domain_table` +- Loading data from an external source in the human-friendly format: `INSERT INTO domain_table FORMAT CSV ...` + +### Limitations {#limitations} + +- Can’t convert index column of base type to domain type via `ALTER TABLE`. +- Can’t implicitly convert string values into domain values when inserting data from another column or table. +- Domain adds no constrains on stored values. + +[Original article](https://clickhouse.tech/docs/en/data_types/domains/) diff --git a/docs/en/sql-reference/data-types/domains/overview.md b/docs/en/sql-reference/data-types/domains/overview.md deleted file mode 100644 index aea7307d048..00000000000 --- a/docs/en/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,30 +0,0 @@ ---- -toc_priority: 58 -toc_title: Overview ---- - -# Domains {#domains} - -Domains are special-purpose types that add some extra features atop of existing base type, but leaving on-wire and on-disc format of the underlying data type intact. At the moment, ClickHouse does not support user-defined domains. - -You can use domains anywhere corresponding base type can be used, for example: - -- Create a column of a domain type -- Read/write values from/to domain column -- Use it as an index if a base type can be used as an index -- Call functions with values of domain column - -### Extra Features of Domains {#extra-features-of-domains} - -- Explicit column type name in `SHOW CREATE TABLE` or `DESCRIBE TABLE` -- Input from human-friendly format with `INSERT INTO domain_table(domain_column) VALUES(...)` -- Output to human-friendly format for `SELECT domain_column FROM domain_table` -- Loading data from an external source in the human-friendly format: `INSERT INTO domain_table FORMAT CSV ...` - -### Limitations {#limitations} - -- Can’t convert index column of base type to domain type via `ALTER TABLE`. -- Can’t implicitly convert string values into domain values when inserting data from another column or table. -- Domain adds no constrains on stored values. - -[Original article](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/es/sql-reference/data-types/domains/index.md b/docs/es/sql-reference/data-types/domains/index.md index f4bfc581b91..136058e35c8 100644 --- a/docs/es/sql-reference/data-types/domains/index.md +++ b/docs/es/sql-reference/data-types/domains/index.md @@ -1,8 +1,33 @@ --- machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: Dominio toc_priority: 56 +toc_folder_title: Dominio +toc_title: "Descripci\xF3n" --- +# Dominio {#domains} +Los dominios son tipos de propósito especial que agregan algunas características adicionales encima del tipo base existente, pero dejando intacto el formato en cable y en disco del tipo de datos subyacente. Por el momento, ClickHouse no admite dominios definidos por el usuario. + +Puede usar dominios en cualquier lugar que se pueda usar el tipo base correspondiente, por ejemplo: + +- Crear una columna de un tipo de dominio +- Leer/escribir valores desde/a la columna de dominio +- Úselo como un índice si un tipo base se puede usar como un índice +- Funciones de llamada con valores de la columna de dominio + +### Características adicionales de los dominios {#extra-features-of-domains} + +- Nombre de tipo de columna explícito en `SHOW CREATE TABLE` o `DESCRIBE TABLE` +- Entrada del formato humano-amistoso con `INSERT INTO domain_table(domain_column) VALUES(...)` +- Salida al formato humano-amistoso para `SELECT domain_column FROM domain_table` +- Carga de datos desde una fuente externa en el formato de uso humano: `INSERT INTO domain_table FORMAT CSV ...` + +### Limitacion {#limitations} + +- No se puede convertir la columna de índice del tipo base al tipo de dominio a través de `ALTER TABLE`. +- No se pueden convertir implícitamente valores de cadena en valores de dominio al insertar datos de otra columna o tabla. +- Domain no agrega restricciones en los valores almacenados. + +[Artículo Original](https://clickhouse.tech/docs/en/data_types/domains/) diff --git a/docs/es/sql-reference/data-types/domains/overview.md b/docs/es/sql-reference/data-types/domains/overview.md deleted file mode 100644 index c6fc9057c81..00000000000 --- a/docs/es/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: "Descripci\xF3n" ---- - -# Dominio {#domains} - -Los dominios son tipos de propósito especial que agregan algunas características adicionales encima del tipo base existente, pero dejando intacto el formato en cable y en disco del tipo de datos subyacente. Por el momento, ClickHouse no admite dominios definidos por el usuario. - -Puede usar dominios en cualquier lugar que se pueda usar el tipo base correspondiente, por ejemplo: - -- Crear una columna de un tipo de dominio -- Leer/escribir valores desde/a la columna de dominio -- Úselo como un índice si un tipo base se puede usar como un índice -- Funciones de llamada con valores de la columna de dominio - -### Características adicionales de los dominios {#extra-features-of-domains} - -- Nombre de tipo de columna explícito en `SHOW CREATE TABLE` o `DESCRIBE TABLE` -- Entrada del formato humano-amistoso con `INSERT INTO domain_table(domain_column) VALUES(...)` -- Salida al formato humano-amistoso para `SELECT domain_column FROM domain_table` -- Carga de datos desde una fuente externa en el formato de uso humano: `INSERT INTO domain_table FORMAT CSV ...` - -### Limitacion {#limitations} - -- No se puede convertir la columna de índice del tipo base al tipo de dominio a través de `ALTER TABLE`. -- No se pueden convertir implícitamente valores de cadena en valores de dominio al insertar datos de otra columna o tabla. -- Domain no agrega restricciones en los valores almacenados. - -[Artículo Original](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/fa/sql-reference/data-types/domains/index.md b/docs/fa/sql-reference/data-types/domains/index.md index 089e1c43eed..a05eea1b59c 100644 --- a/docs/fa/sql-reference/data-types/domains/index.md +++ b/docs/fa/sql-reference/data-types/domains/index.md @@ -1,8 +1,33 @@ --- machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "\u062F\u0627\u0645\u0646\u0647" toc_priority: 56 +toc_folder_title: "\u062F\u0627\u0645\u0646\u0647" +toc_title: "\u0628\u0631\u0631\u0633\u06CC \u0627\u062C\u0645\u0627\u0644\u06CC" --- +# دامنه {#domains} +دامنه انواع خاصی است که اضافه کردن برخی از ویژگی های اضافی در بالای نوع پایه موجود, اما ترک بر روی سیم و بر روی دیسک فرمت از نوع داده اساسی دست نخورده. درحال حاضر, تاتر می کند دامنه تعریف شده توسط کاربر را پشتیبانی نمی کند. + +شما می توانید دامنه در هر نقطه نوع پایه مربوطه استفاده می شود, مثلا: + +- ایجاد یک ستون از یک نوع دامنه +- خواندن / نوشتن مقادیر از / به ستون دامنه +- اگر یک نوع پایه می تواند به عنوان یک شاخص استفاده می شود به عنوان شاخص استفاده می شود +- توابع تماس با مقادیر ستون دامنه + +### ویژگی های اضافی از دامنه {#extra-features-of-domains} + +- صریح نام نوع ستون در `SHOW CREATE TABLE` یا `DESCRIBE TABLE` +- ورودی از فرمت انسان دوستانه با `INSERT INTO domain_table(domain_column) VALUES(...)` +- خروجی به فرمت انسان دوستانه برای `SELECT domain_column FROM domain_table` +- بارگیری داده ها از یک منبع خارجی در قالب انسان دوستانه: `INSERT INTO domain_table FORMAT CSV ...` + +### محدودیت ها {#limitations} + +- می توانید ستون شاخص از نوع پایه به نوع دامنه از طریق تبدیل کنید `ALTER TABLE`. +- نمی تواند به طور ضمنی تبدیل مقادیر رشته به ارزش دامنه در هنگام قرار دادن داده ها از ستون یا جدول دیگر. +- دامنه می افزاید: هیچ محدودیتی در مقادیر ذخیره شده. + +[مقاله اصلی](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/fa/sql-reference/data-types/domains/overview.md b/docs/fa/sql-reference/data-types/domains/overview.md deleted file mode 100644 index 4507ca850ef..00000000000 --- a/docs/fa/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: "\u0628\u0631\u0631\u0633\u06CC \u0627\u062C\u0645\u0627\u0644\u06CC" ---- - -# دامنه {#domains} - -دامنه انواع خاصی است که اضافه کردن برخی از ویژگی های اضافی در بالای نوع پایه موجود, اما ترک بر روی سیم و بر روی دیسک فرمت از نوع داده اساسی دست نخورده. درحال حاضر, تاتر می کند دامنه تعریف شده توسط کاربر را پشتیبانی نمی کند. - -شما می توانید دامنه در هر نقطه نوع پایه مربوطه استفاده می شود, مثلا: - -- ایجاد یک ستون از یک نوع دامنه -- خواندن / نوشتن مقادیر از / به ستون دامنه -- اگر یک نوع پایه می تواند به عنوان یک شاخص استفاده می شود به عنوان شاخص استفاده می شود -- توابع تماس با مقادیر ستون دامنه - -### ویژگی های اضافی از دامنه {#extra-features-of-domains} - -- صریح نام نوع ستون در `SHOW CREATE TABLE` یا `DESCRIBE TABLE` -- ورودی از فرمت انسان دوستانه با `INSERT INTO domain_table(domain_column) VALUES(...)` -- خروجی به فرمت انسان دوستانه برای `SELECT domain_column FROM domain_table` -- بارگیری داده ها از یک منبع خارجی در قالب انسان دوستانه: `INSERT INTO domain_table FORMAT CSV ...` - -### محدودیت ها {#limitations} - -- می توانید ستون شاخص از نوع پایه به نوع دامنه از طریق تبدیل کنید `ALTER TABLE`. -- نمی تواند به طور ضمنی تبدیل مقادیر رشته به ارزش دامنه در هنگام قرار دادن داده ها از ستون یا جدول دیگر. -- دامنه می افزاید: هیچ محدودیتی در مقادیر ذخیره شده. - -[مقاله اصلی](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/fr/sql-reference/data-types/domains/index.md b/docs/fr/sql-reference/data-types/domains/index.md index ffe64acb834..7e11f9a8a68 100644 --- a/docs/fr/sql-reference/data-types/domains/index.md +++ b/docs/fr/sql-reference/data-types/domains/index.md @@ -3,6 +3,31 @@ machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: Domaine toc_priority: 56 +toc_title: "Aper\xE7u" --- +# Domaine {#domains} +Les domaines sont des types spéciaux qui ajoutent des fonctionnalités supplémentaires au sommet du type de base existant, mais en laissant le format on-wire et on-disc du type de données sous-jacent intact. À l'heure actuelle, ClickHouse ne prend pas en charge les domaines définis par l'utilisateur. + +Vous pouvez utiliser des domaines partout type de base correspondant peut être utilisé, par exemple: + +- Créer une colonne d'un type de domaine +- Valeurs de lecture / écriture depuis / vers la colonne de domaine +- L'utiliser comme un indice si un type de base peut être utilisée comme un indice +- Fonctions d'appel avec des valeurs de colonne de domaine + +### Fonctionnalités supplémentaires des domaines {#extra-features-of-domains} + +- Nom de type de colonne explicite dans `SHOW CREATE TABLE` ou `DESCRIBE TABLE` +- Entrée du format convivial avec `INSERT INTO domain_table(domain_column) VALUES(...)` +- Sortie au format convivial pour `SELECT domain_column FROM domain_table` +- Chargement de données à partir d'une source externe dans un format convivial: `INSERT INTO domain_table FORMAT CSV ...` + +### Limitation {#limitations} + +- Impossible de convertir la colonne d'index du type de base en type de domaine via `ALTER TABLE`. +- Impossible de convertir implicitement des valeurs de chaîne en valeurs de domaine lors de l'insertion de données d'une autre colonne ou table. +- Le domaine n'ajoute aucune contrainte sur les valeurs stockées. + +[Article Original](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/fr/sql-reference/data-types/domains/overview.md b/docs/fr/sql-reference/data-types/domains/overview.md deleted file mode 100644 index 149ea84ba7b..00000000000 --- a/docs/fr/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: "Aper\xE7u" ---- - -# Domaine {#domains} - -Les domaines sont des types spéciaux qui ajoutent des fonctionnalités supplémentaires au sommet du type de base existant, mais en laissant le format on-wire et on-disc du type de données sous-jacent intact. À l'heure actuelle, ClickHouse ne prend pas en charge les domaines définis par l'utilisateur. - -Vous pouvez utiliser des domaines partout type de base correspondant peut être utilisé, par exemple: - -- Créer une colonne d'un type de domaine -- Valeurs de lecture / écriture depuis / vers la colonne de domaine -- L'utiliser comme un indice si un type de base peut être utilisée comme un indice -- Fonctions d'appel avec des valeurs de colonne de domaine - -### Fonctionnalités supplémentaires des domaines {#extra-features-of-domains} - -- Nom de type de colonne explicite dans `SHOW CREATE TABLE` ou `DESCRIBE TABLE` -- Entrée du format convivial avec `INSERT INTO domain_table(domain_column) VALUES(...)` -- Sortie au format convivial pour `SELECT domain_column FROM domain_table` -- Chargement de données à partir d'une source externe dans un format convivial: `INSERT INTO domain_table FORMAT CSV ...` - -### Limitation {#limitations} - -- Impossible de convertir la colonne d'index du type de base en type de domaine via `ALTER TABLE`. -- Impossible de convertir implicitement des valeurs de chaîne en valeurs de domaine lors de l'insertion de données d'une autre colonne ou table. -- Le domaine n'ajoute aucune contrainte sur les valeurs stockées. - -[Article Original](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/ja/sql-reference/data-types/domains/index.md b/docs/ja/sql-reference/data-types/domains/index.md index d476fcefb51..4f8c2b7add8 100644 --- a/docs/ja/sql-reference/data-types/domains/index.md +++ b/docs/ja/sql-reference/data-types/domains/index.md @@ -3,6 +3,31 @@ machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: "\u30C9\u30E1\u30A4\u30F3" toc_priority: 56 +toc_title: "\u6982\u8981" --- +# ドメイン {#domains} +ドメインは、既存の基本型の上にいくつかの余分な機能を追加する特殊な目的の型ですが、基になるデータ型のオンワイヤおよびオンディスク形式は 現時点では、ClickHouseはユーザー定義ドメインをサポートしていません。 + +たとえば、対応する基本タイプを使用できる任意の場所でドメインを使用できます: + +- ドメイン型の列を作成する +- ドメイン列から/への読み取り/書き込み値 +- 基本型をインデックスとして使用できる場合は、インデックスとして使用します +- ドメイン列の値を持つ関数の呼び出し + +### ドメインの追加機能 {#extra-features-of-domains} + +- 明示的な列タイプ名 `SHOW CREATE TABLE` または `DESCRIBE TABLE` +- 人間に優しいフォーマットからの入力 `INSERT INTO domain_table(domain_column) VALUES(...)` +- 人間に優しいフォーマットへの出力 `SELECT domain_column FROM domain_table` +- 人間に優しい形式で外部ソースからデータを読み込む: `INSERT INTO domain_table FORMAT CSV ...` + +### 制限 {#limitations} + +- 基本型のインデックス列をドメイン型に変換できません `ALTER TABLE`. +- 別の列または表からデータを挿入するときに、文字列値を暗黙的にドメイン値に変換できません。 +- ドメインは、格納された値に制約を追加しません。 + +[元の記事](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/ja/sql-reference/data-types/domains/overview.md b/docs/ja/sql-reference/data-types/domains/overview.md deleted file mode 100644 index 958fce5beb4..00000000000 --- a/docs/ja/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: "\u6982\u8981" ---- - -# ドメイン {#domains} - -ドメインは、既存の基本型の上にいくつかの余分な機能を追加する特殊な目的の型ですが、基になるデータ型のオンワイヤおよびオンディスク形式は 現時点では、ClickHouseはユーザー定義ドメインをサポートしていません。 - -たとえば、対応する基本タイプを使用できる任意の場所でドメインを使用できます: - -- ドメイン型の列を作成する -- ドメイン列から/への読み取り/書き込み値 -- 基本型をインデックスとして使用できる場合は、インデックスとして使用します -- ドメイン列の値を持つ関数の呼び出し - -### ドメインの追加機能 {#extra-features-of-domains} - -- 明示的な列タイプ名 `SHOW CREATE TABLE` または `DESCRIBE TABLE` -- 人間に優しいフォーマットからの入力 `INSERT INTO domain_table(domain_column) VALUES(...)` -- 人間に優しいフォーマットへの出力 `SELECT domain_column FROM domain_table` -- 人間に優しい形式で外部ソースからデータを読み込む: `INSERT INTO domain_table FORMAT CSV ...` - -### 制限 {#limitations} - -- 基本型のインデックス列をドメイン型に変換できません `ALTER TABLE`. -- 別の列または表からデータを挿入するときに、文字列値を暗黙的にドメイン値に変換できません。 -- ドメインは、格納された値に制約を追加しません。 - -[元の記事](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/redirects.txt b/docs/redirects.txt index 233492f09ac..b4c93ac4908 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -290,6 +290,7 @@ query_language/table_functions/remote.md sql-reference/table-functions/remote.md query_language/table_functions/url.md sql-reference/table-functions/url.md roadmap.md whats-new/roadmap.md security_changelog.md whats-new/security-changelog.md +sql-reference/data-types/domains/overview.md sql-reference/data-types/domains/index.md sql_reference/aggregate_functions/combinators.md sql-reference/aggregate-functions/combinators.md sql_reference/aggregate_functions/index.md sql-reference/aggregate-functions/index.md sql_reference/aggregate_functions/parametric_functions.md sql-reference/aggregate-functions/parametric-functions.md diff --git a/docs/ru/sql-reference/data-types/domains/index.md b/docs/ru/sql-reference/data-types/domains/index.md index ddcb5b21d82..fe5c7ab7349 100644 --- a/docs/ru/sql-reference/data-types/domains/index.md +++ b/docs/ru/sql-reference/data-types/domains/index.md @@ -1,6 +1,33 @@ --- -toc_folder_title: Domains +toc_folder_title: Домены +toc_title_title: Обзор toc_priority: 56 --- +# Домены {#domeny} + +Домены — это типы данных специального назначения, которые добавляют некоторые дополнительные функции поверх существующего базового типа. На данный момент ClickHouse не поддерживает пользовательские домены. + +Вы можете использовать домены везде, где можно использовать соответствующий базовый тип: + +- Создание столбца с доменным типом данных. +- Чтение/запись значений из/в столбец с доменным типом данных. +- Используйте его как индекс, если базовый тип можно использовать в качестве индекса. +- Вызов функций со значениями столбца, имеющего доменный тип данных. +- и так далее. + +### Дополнительные возможности доменов {#dopolnitelnye-vozmozhnosti-domenov} + +- Явное название типа данных столбца в запросах `SHOW CREATE TABLE` и `DESCRIBE TABLE` +- Ввод данных в удобном человеку формате `INSERT INTO domain_table(domain_column) VALUES(...)` +- Вывод данных в удобном человеку формате `SELECT domain_column FROM domain_table` +- Загрузка данных из внешнего источника в удобном для человека формате: `INSERT INTO domain_table FORMAT CSV ...` + +### Ограничения {#ogranicheniia} + +- Невозможно преобразовать базовый тип данных в доменный для индексного столбца с помощью `ALTER TABLE`. +- Невозможно неявно преобразовывать строковые значение в значения с доменным типом данных при вставке данных из другого столбца или таблицы. +- Домен не добавляет ограничения на хранимые значения. + +[Оригинальная статья](https://clickhouse.tech/docs/ru/data_types/domains/overview) diff --git a/docs/ru/sql-reference/data-types/domains/overview.md b/docs/ru/sql-reference/data-types/domains/overview.md deleted file mode 100644 index 6feac834e5e..00000000000 --- a/docs/ru/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,26 +0,0 @@ -# Домены {#domeny} - -Домены — это типы данных специального назначения, которые добавляют некоторые дополнительные функции поверх существующего базового типа. На данный момент ClickHouse не поддерживает пользовательские домены. - -Вы можете использовать домены везде, где можно использовать соответствующий базовый тип: - -- Создание столбца с доменным типом данных. -- Чтение/запись значений из/в столбец с доменным типом данных. -- Используйте его как индекс, если базовый тип можно использовать в качестве индекса. -- Вызов функций со значениями столбца, имеющего доменный тип данных. -- и так далее. - -### Дополнительные возможности доменов {#dopolnitelnye-vozmozhnosti-domenov} - -- Явное название типа данных столбца в запросах `SHOW CREATE TABLE` и `DESCRIBE TABLE` -- Ввод данных в удобном человеку формате `INSERT INTO domain_table(domain_column) VALUES(...)` -- Вывод данных в удобном человеку формате `SELECT domain_column FROM domain_table` -- Загрузка данных из внешнего источника в удобном для человека формате: `INSERT INTO domain_table FORMAT CSV ...` - -### Ограничения {#ogranicheniia} - -- Невозможно преобразовать базовый тип данных в доменный для индексного столбца с помощью `ALTER TABLE`. -- Невозможно неявно преобразовывать строковые значение в значения с доменным типом данных при вставке данных из другого столбца или таблицы. -- Домен не добавляет ограничения на хранимые значения. - -[Оригинальная статья](https://clickhouse.tech/docs/ru/data_types/domains/overview) diff --git a/docs/tr/sql-reference/data-types/domains/index.md b/docs/tr/sql-reference/data-types/domains/index.md index e439d110325..7ef688b3578 100644 --- a/docs/tr/sql-reference/data-types/domains/index.md +++ b/docs/tr/sql-reference/data-types/domains/index.md @@ -1,8 +1,34 @@ --- machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd +toc_priority: 58 +toc_title: "Genel bak\u0131\u015F" toc_folder_title: Etkiler toc_priority: 56 --- +# Etkiler {#domains} +Etki alanları, varolan temel türün üstüne bazı ek özellikler ekleyen, ancak temel veri türünün kablolu ve disk üstü biçimini sağlam bırakan özel amaçlı türlerdir. Şu anda, ClickHouse kullanıcı tanımlı etki alanlarını desteklemiyor. + +Örneğin, ilgili taban türünün kullanılabileceği her yerde etki alanlarını kullanabilirsiniz: + +- Etki alanı türünde bir sütun oluşturma +- Alan sütunundan/alanına değerleri okuma / yazma +- Bir temel türü bir dizin olarak kullanılabilir, bir dizin olarak kullanın +- Etki alanı sütun değerleri ile çağrı fonksiyonları + +### Alanların ekstra özellikleri {#extra-features-of-domains} + +- Açık sütun türü adı `SHOW CREATE TABLE` veya `DESCRIBE TABLE` +- İle insan dostu format inputtan giriş `INSERT INTO domain_table(domain_column) VALUES(...)` +- İçin insan dostu forma outputta çıktı `SELECT domain_column FROM domain_table` +- Harici bir kaynaktan insan dostu biçimde veri yükleme: `INSERT INTO domain_table FORMAT CSV ...` + +### Sınırlamalar {#limitations} + +- Temel türün dizin sütununu etki alanı türüne dönüştürülemiyor `ALTER TABLE`. +- Başka bir sütun veya tablodan veri eklerken dize değerlerini dolaylı olarak etki alanı değerlerine dönüştüremez. +- Etki alanı, depolanan değerler üzerinde hiçbir kısıtlama ekler. + +[Orijinal makale](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/tr/sql-reference/data-types/domains/overview.md b/docs/tr/sql-reference/data-types/domains/overview.md deleted file mode 100644 index cfab9f3701e..00000000000 --- a/docs/tr/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,32 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 58 -toc_title: "Genel bak\u0131\u015F" ---- - -# Etkiler {#domains} - -Etki alanları, varolan temel türün üstüne bazı ek özellikler ekleyen, ancak temel veri türünün kablolu ve disk üstü biçimini sağlam bırakan özel amaçlı türlerdir. Şu anda, ClickHouse kullanıcı tanımlı etki alanlarını desteklemiyor. - -Örneğin, ilgili taban türünün kullanılabileceği her yerde etki alanlarını kullanabilirsiniz: - -- Etki alanı türünde bir sütun oluşturma -- Alan sütunundan/alanına değerleri okuma / yazma -- Bir temel türü bir dizin olarak kullanılabilir, bir dizin olarak kullanın -- Etki alanı sütun değerleri ile çağrı fonksiyonları - -### Alanların ekstra özellikleri {#extra-features-of-domains} - -- Açık sütun türü adı `SHOW CREATE TABLE` veya `DESCRIBE TABLE` -- İle insan dostu format inputtan giriş `INSERT INTO domain_table(domain_column) VALUES(...)` -- İçin insan dostu forma outputta çıktı `SELECT domain_column FROM domain_table` -- Harici bir kaynaktan insan dostu biçimde veri yükleme: `INSERT INTO domain_table FORMAT CSV ...` - -### Sınırlamalar {#limitations} - -- Temel türün dizin sütununu etki alanı türüne dönüştürülemiyor `ALTER TABLE`. -- Başka bir sütun veya tablodan veri eklerken dize değerlerini dolaylı olarak etki alanı değerlerine dönüştüremez. -- Etki alanı, depolanan değerler üzerinde hiçbir kısıtlama ekler. - -[Orijinal makale](https://clickhouse.tech/docs/en/data_types/domains/overview) diff --git a/docs/zh/sql-reference/data-types/domains/index.md b/docs/zh/sql-reference/data-types/domains/index.md index e05c61e0dbb..6ef788b0650 100644 --- a/docs/zh/sql-reference/data-types/domains/index.md +++ b/docs/zh/sql-reference/data-types/domains/index.md @@ -1,8 +1,34 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: "\u57DF" +toc_title: "域" toc_priority: 56 --- +# 域 {#domains} + +Domain类型是特定实现的类型,它总是与某个现存的基础类型保持二进制兼容的同时添加一些额外的特性,以能够在维持磁盘数据不变的情况下使用这些额外的特性。目前ClickHouse暂不支持自定义domain类型。 + +如果你可以在一个地方使用与Domain类型二进制兼容的基础类型,那么在相同的地方您也可以使用Domain类型,例如: + +- 使用Domain类型作为表中列的类型 +- 对Domain类型的列进行读/写数据 +- 如果与Domain二进制兼容的基础类型可以作为索引,那么Domain类型也可以作为索引 +- 将Domain类型作为参数传递给函数使用 +- 其他 + +### Domains的额外特性 {#domainsde-e-wai-te-xing} + +- 在执行SHOW CREATE TABLE 或 DESCRIBE TABLE时,其对应的列总是展示为Domain类型的名称 +- 在INSERT INTO domain\_table(domain\_column) VALUES(…)中输入数据总是以更人性化的格式进行输入 +- 在SELECT domain\_column FROM domain\_table中数据总是以更人性化的格式输出 +- 在INSERT INTO domain\_table FORMAT CSV …中,实现外部源数据以更人性化的格式载入 + +### Domains类型的限制 {#domainslei-xing-de-xian-zhi} + +- 无法通过`ALTER TABLE`将基础类型的索引转换为Domain类型的索引。 +- 当从其他列或表插入数据时,无法将string类型的值隐式地转换为Domain类型的值。 +- 无法对存储为Domain类型的值添加约束。 + +[来源文章](https://clickhouse.tech/docs/en/data_types/domains/overview) + diff --git a/docs/zh/sql-reference/data-types/domains/overview.md b/docs/zh/sql-reference/data-types/domains/overview.md deleted file mode 100644 index 689acb9cb60..00000000000 --- a/docs/zh/sql-reference/data-types/domains/overview.md +++ /dev/null @@ -1,26 +0,0 @@ -# 域 {#domains} - -Domain类型是特定实现的类型,它总是与某个现存的基础类型保持二进制兼容的同时添加一些额外的特性,以能够在维持磁盘数据不变的情况下使用这些额外的特性。目前ClickHouse暂不支持自定义domain类型。 - -如果你可以在一个地方使用与Domain类型二进制兼容的基础类型,那么在相同的地方您也可以使用Domain类型,例如: - -- 使用Domain类型作为表中列的类型 -- 对Domain类型的列进行读/写数据 -- 如果与Domain二进制兼容的基础类型可以作为索引,那么Domain类型也可以作为索引 -- 将Domain类型作为参数传递给函数使用 -- 其他 - -### Domains的额外特性 {#domainsde-e-wai-te-xing} - -- 在执行SHOW CREATE TABLE 或 DESCRIBE TABLE时,其对应的列总是展示为Domain类型的名称 -- 在INSERT INTO domain\_table(domain\_column) VALUES(…)中输入数据总是以更人性化的格式进行输入 -- 在SELECT domain\_column FROM domain\_table中数据总是以更人性化的格式输出 -- 在INSERT INTO domain\_table FORMAT CSV …中,实现外部源数据以更人性化的格式载入 - -### Domains类型的限制 {#domainslei-xing-de-xian-zhi} - -- 无法通过`ALTER TABLE`将基础类型的索引转换为Domain类型的索引。 -- 当从其他列或表插入数据时,无法将string类型的值隐式地转换为Domain类型的值。 -- 无法对存储为Domain类型的值添加约束。 - -[来源文章](https://clickhouse.tech/docs/en/data_types/domains/overview) From a7348ed2c8fdc6b4ec67b82ee071fea8086b50e6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 7 Jul 2020 18:48:47 +0300 Subject: [PATCH 125/196] Add to images.json --- docker/images.json | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/images.json b/docker/images.json index 23f8cc0d9fd..45209fc775f 100644 --- a/docker/images.json +++ b/docker/images.json @@ -79,5 +79,9 @@ "docker/test/integration/runner": { "name": "yandex/clickhouse-integration-tests-runner", "dependent": [] + }, + "docker/test/testflows/runner": { + "name": "yandex/clickhouse-testflows-runner", + "dependent": [] } } From a5cb0aed12b90fa4e23db81dbd5b8da7ef98d4fd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 Jul 2020 19:22:41 +0300 Subject: [PATCH 126/196] Sanitize LINK_LIBRARIES property for the directories (#12160) When you will try to link target with the directory (that exists), cmake will skip this without an error, only the following warning will be reported: target_link_libraries(main /tmp) WARNING: Target "main" requests linking to directory "/tmp". Targets may link only to libraries. CMake is dropping the item. And there is no cmake policy that controls this. (I guess the reason that it is allowed is because of FRAMEWORK for OSX). So to avoid error-prone cmake rules, this can be sanitized. There are the following ways: - overwrite target_link_libraries()/link_libraries() and check *before* calling real macro, but this requires duplicate all supported syntax -- too complex - overwrite target_link_libraries() and check LINK_LIBRARIES property, this works great -- but cannot be used with link_libraries() - use BUILDSYSTEM_TARGETS property to get list of all targets and sanitize -- this will work. I also tested it with the following patch: $ git di diff --git a/base/daemon/CMakeLists.txt b/base/daemon/CMakeLists.txt index 26d59a57e7..35e6ff6432 100644 --- a/base/daemon/CMakeLists.txt +++ b/base/daemon/CMakeLists.txt @@ -9,4 +9,5 @@ target_link_libraries (daemon PUBLIC loggers PRIVATE clickhouse_common_io clickh if (USE_SENTRY) target_link_libraries (daemon PRIVATE ${SENTRY_LIBRARY}) + target_link_libraries (daemon PRIVATE /tmp) endif () And it works: CMake Error at cmake/sanitize_target_link_libraries.cmake:48 (message): daemon requested to link with directory: /tmp Call Stack (most recent call first): cmake/sanitize_target_link_libraries.cmake:55 (sanitize_link_libraries) CMakeLists.txt:425 (include) Refs: #12041 --- CMakeLists.txt | 2 + cmake/sanitize_target_link_libraries.cmake | 56 ++++++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 cmake/sanitize_target_link_libraries.cmake diff --git a/CMakeLists.txt b/CMakeLists.txt index 8cde8f6fec3..d4d325818e8 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -421,3 +421,5 @@ add_subdirectory (tests) add_subdirectory (utils) include (cmake/print_include_directories.cmake) + +include (cmake/sanitize_target_link_libraries.cmake) diff --git a/cmake/sanitize_target_link_libraries.cmake b/cmake/sanitize_target_link_libraries.cmake new file mode 100644 index 00000000000..d66ea338a52 --- /dev/null +++ b/cmake/sanitize_target_link_libraries.cmake @@ -0,0 +1,56 @@ +# When you will try to link target with the directory (that exists), cmake will +# skip this without an error, only the following warning will be reported: +# +# target_link_libraries(main /tmp) +# +# WARNING: Target "main" requests linking to directory "/tmp". Targets may link only to libraries. CMake is dropping the item. +# +# And there is no cmake policy that controls this. +# (I guess the reason that it is allowed is because of FRAMEWORK for OSX). +# +# So to avoid error-prone cmake rules, this can be sanitized. +# There are the following ways: +# - overwrite target_link_libraries()/link_libraries() and check *before* +# calling real macro, but this requires duplicate all supported syntax +# -- too complex +# - overwrite target_link_libraries() and check LINK_LIBRARIES property, this +# works great +# -- but cannot be used with link_libraries() +# - use BUILDSYSTEM_TARGETS property to get list of all targets and sanitize +# -- this will work. + +# https://stackoverflow.com/a/62311397/328260 +function (get_all_targets var) + set (targets) + get_all_targets_recursive (targets ${CMAKE_CURRENT_SOURCE_DIR}) + set (${var} ${targets} PARENT_SCOPE) +endfunction() +macro (get_all_targets_recursive targets dir) + get_property (subdirectories DIRECTORY ${dir} PROPERTY SUBDIRECTORIES) + foreach (subdir ${subdirectories}) + get_all_targets_recursive (${targets} ${subdir}) + endforeach () + get_property (current_targets DIRECTORY ${dir} PROPERTY BUILDSYSTEM_TARGETS) + list (APPEND ${targets} ${current_targets}) +endmacro () + +macro (sanitize_link_libraries target) + get_target_property(target_type ${target} TYPE) + if (${target_type} STREQUAL "INTERFACE_LIBRARY") + get_property(linked_libraries TARGET ${target} PROPERTY INTERFACE_LINK_LIBRARIES) + else() + get_property(linked_libraries TARGET ${target} PROPERTY LINK_LIBRARIES) + endif() + foreach (linked_library ${linked_libraries}) + if (TARGET ${linked_library}) + # just in case, skip if TARGET + elseif (IS_DIRECTORY ${linked_library}) + message(FATAL_ERROR "${target} requested to link with directory: ${linked_library}") + endif() + endforeach() +endmacro() + +get_all_targets (all_targets) +foreach (target ${all_targets}) + sanitize_link_libraries(${target}) +endforeach() From 1d1b49340c06353a7fa62e72164ee29afcfe85c0 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 19:31:58 +0300 Subject: [PATCH 127/196] cleanup --- programs/client/QueryFuzzer.cpp | 52 ++++++--------------------------- programs/client/QueryFuzzer.h | 27 +++++++++++------ programs/ya.make | 1 + 3 files changed, 28 insertions(+), 52 deletions(-) diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index 0617a0e9fdf..d9147431ed3 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -240,10 +240,6 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(ASTPtr ast) fprintf(stderr, "no random col!\n"); } } - - /* - fuzz(impl->children); - */ } void QueryFuzzer::fuzz(ASTs & asts) @@ -259,8 +255,6 @@ void QueryFuzzer::fuzz(ASTPtr & ast) if (!ast) return; - //fprintf(stderr, "name: %s\n", demangle(typeid(*ast).name()).c_str()); - if (auto * with_union = typeid_cast(ast.get())) { fuzz(with_union->list_of_selects); @@ -321,55 +315,37 @@ void QueryFuzzer::fuzz(ASTPtr & ast) { fuzz(ast->children); } - - /* - if (auto * with_alias = dynamic_cast(ast.get())) - { - int dice = fuzz_rand() % 20; - if (dice == 0) - { - with_alias->alias = aliases[fuzz_rand() % aliases.size()]; - } - else if (dice < 5) - { - with_alias->alias = ""; - } - } - */ } +/* + * This functions collects various parts of query that we can then substitute + * to a query being fuzzed. + * + * TODO: we just stop remembering new parts after our corpus reaches certain size. + * This is boring, should implement a random replacement of existing parst with + * small probability. Do this after we add this fuzzer to CI and fix all the + * problems it can routinely find even in this boring version. + */ void QueryFuzzer::collectFuzzInfoMain(const ASTPtr ast) { collectFuzzInfoRecurse(ast); - /* - with_alias.clear(); - for (const auto & [name, value] : with_alias_map) - { - with_alias.push_back(value); - //fprintf(stderr, "alias %s\n", value->formatForErrorMessage().c_str()); - } - */ - aliases.clear(); for (const auto & alias : aliases_set) { aliases.push_back(alias); - //fprintf(stderr, "alias %s\n", alias.c_str()); } column_like.clear(); for (const auto & [name, value] : column_like_map) { column_like.push_back(value); - //fprintf(stderr, "column %s\n", name.c_str()); } table_like.clear(); for (const auto & [name, value] : table_like_map) { table_like.push_back(value); - //fprintf(stderr, "table %s\n", name.c_str()); } } @@ -440,19 +416,9 @@ void QueryFuzzer::collectFuzzInfoRecurse(const ASTPtr ast) void QueryFuzzer::fuzzMain(ASTPtr & ast) { - /* - std::cerr << "before: " << std::endl; - ast->dumpTree(std::cerr); - */ - collectFuzzInfoMain(ast); fuzz(ast); - /* - std::cerr << "after: " << std::endl; - ast->dumpTree(std::cerr); - */ - std::cout << std::endl; formatAST(*ast, std::cout); std::cout << std::endl << std::endl; diff --git a/programs/client/QueryFuzzer.h b/programs/client/QueryFuzzer.h index 4b8a713bc4c..58168089b1b 100644 --- a/programs/client/QueryFuzzer.h +++ b/programs/client/QueryFuzzer.h @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -11,17 +12,21 @@ namespace DB { +/* + * This is an AST-based query fuzzer that makes random modifications to query + * AST, changing numbers, list of columns, functions, etc. It remembers part of + * queries it fuzzed previously, and can substitute these parts to new fuzzed + * queries, so you want to feed it a lot of queries to get some interesting mix + * of them. Normally we feed SQL regression tests to it. + */ struct QueryFuzzer { - //pcg64 fuzz_rand{static_cast(rand())}; - pcg64 fuzz_rand{clock_gettime_ns()}; - - // Collection of asts with alias. - /* - std::unordered_map with_alias_map; - std::vector with_alias; - */ + pcg64 fuzz_rand{randomSeed()}; + // These arrays hold parts of queries that we can substitute into the query + // we are currently fuzzing. We add some part from each new query we are asked + // to fuzz, and keep this state between queries, so the fuzzing output becomes + // more interesting over time, as the queries mix. std::unordered_set aliases_set; std::vector aliases; @@ -31,6 +36,11 @@ struct QueryFuzzer std::unordered_map table_like_map; std::vector table_like; + // This is the only function you have to call -- it will modify the passed + // ASTPtr to point to new AST with some random changes. + void fuzzMain(ASTPtr & ast); + + // Variuos helper functions follow, normally you shouldn't have to call them. Field getRandomField(int type); Field fuzzField(Field field); ASTPtr getRandomColumnLike(); @@ -43,7 +53,6 @@ struct QueryFuzzer void addTableLike(const ASTPtr ast); void addColumnLike(const ASTPtr ast); void collectFuzzInfoRecurse(const ASTPtr ast); - void fuzzMain(ASTPtr & ast); }; } diff --git a/programs/ya.make b/programs/ya.make index f4a61850212..7c967470640 100644 --- a/programs/ya.make +++ b/programs/ya.make @@ -17,6 +17,7 @@ SRCS( main.cpp client/Client.cpp + client/QueryFuzzer.cpp client/ConnectionParameters.cpp client/Suggest.cpp extract-from-config/ExtractFromConfig.cpp From d26441cdc4d3f9a12ff40f41eb869ac74c64cf26 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 19:33:19 +0300 Subject: [PATCH 128/196] cleanup --- programs/client/Client.cpp | 45 -------------------------------------- 1 file changed, 45 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 17a54a52b16..e0f12e4bc1c 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -2323,48 +2323,6 @@ public: } -using signal_function = void(int, siginfo_t*, void*); - -/// Setup signal handlers. -static void add_signal_handler(const std::vector & signals, signal_function handler) -{ - struct sigaction sa; - memset(&sa, 0, sizeof(sa)); - sa.sa_sigaction = handler; - sa.sa_flags = SA_SIGINFO; - - { -#if defined(OS_DARWIN) - sigemptyset(&sa.sa_mask); - for (auto signal : signals) - sigaddset(&sa.sa_mask, signal); -#else - if (sigemptyset(&sa.sa_mask)) - throw Poco::Exception("Cannot set signal handler."); - - for (auto signal : signals) - if (sigaddset(&sa.sa_mask, signal)) - throw Poco::Exception("Cannot set signal handler."); -#endif - - for (auto signal : signals) - if (sigaction(signal, &sa, nullptr)) - throw Poco::Exception("Cannot set signal handler."); - } -}; - -/** Handler for "fault" or diagnostic signals. */ -static void signalHandler(int sig, siginfo_t * /*info*/, void * context) -{ - const ucontext_t signal_context = *reinterpret_cast(context); - const StackTrace stack_trace(signal_context); - std::cerr << fmt::format("Received signal {} at: {}", sig, - stack_trace.toString()) << std::endl; - - signal(sig, SIG_DFL); - raise(sig); -} - #pragma GCC diagnostic ignored "-Wunused-function" #pragma GCC diagnostic ignored "-Wmissing-declarations" @@ -2372,9 +2330,6 @@ int mainEntryClickHouseClient(int argc, char ** argv) { try { - add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, - SIGPIPE}, signalHandler); - DB::Client client; client.init(argc, argv); return client.run(); From f514d0da77453db5cf07be4cda79abf466cf0c0a Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Tue, 7 Jul 2020 19:59:42 +0300 Subject: [PATCH 129/196] Revert "Run perf tests with memory sampling (for allocations >1M)" --- .../config/users.d/perf-comparison-tweaks-users.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml index 8173934fa3e..6e3e3df5d39 100644 --- a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml +++ b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml @@ -6,8 +6,6 @@ 1 1 1 - 1 - 1048576 From d85391f3ac051803944b3720df2abadc02fae621 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 20:07:04 +0300 Subject: [PATCH 130/196] too slow --- tests/performance/questdb_sum_float64.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/questdb_sum_float64.xml b/tests/performance/questdb_sum_float64.xml index 16cef264ffa..fde475a1431 100644 --- a/tests/performance/questdb_sum_float64.xml +++ b/tests/performance/questdb_sum_float64.xml @@ -25,7 +25,7 @@ CREATE TABLE `zz_{type}_{engine}` (x {type}) ENGINE {engine} - INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(1000000000) + INSERT INTO `zz_{type}_{engine}` SELECT rand() FROM numbers(500000000) SELECT sum(x) FROM `zz_{type}_{engine}` From 51c6683391b24e3dc2179b607f359924d9b1a351 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 7 Jul 2020 20:10:55 +0300 Subject: [PATCH 131/196] style --- programs/client/QueryFuzzer.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index d9147431ed3..b24add1e733 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -27,12 +27,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - - Field QueryFuzzer::getRandomField(int type) { switch (type) From 9c829e3046cae0711ca822998d1a6b3e59b54edb Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 7 Jul 2020 20:18:38 +0300 Subject: [PATCH 132/196] [docs] add intrdocution for statements page (#12189) * [docs] add intrdocution for statements page * Update index.md --- docs/en/sql-reference/statements/index.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/sql-reference/statements/index.md b/docs/en/sql-reference/statements/index.md index 034c541a1c4..3a2c09d94cb 100644 --- a/docs/en/sql-reference/statements/index.md +++ b/docs/en/sql-reference/statements/index.md @@ -1,6 +1,19 @@ --- toc_folder_title: Statements toc_priority: 31 +toc_hidden: true --- +# ClickHouse SQL Statements +Statements represent various kinds of action you can perform using SQL queries. Each kind of statement has it's own syntax and usage details that are described separately: + +- [SELECT](select/index.md) +- [INSERT INTO](insert-into.md) +- [CREATE](create.md) +- [ALTER](alter.md) +- [SYSTEM](system.md) +- [SHOW](show.md) +- [GRANT](grant.md) +- [REVOKE](revoke.md) +- [Other](misc.md) From 391d31d7f6604b364c8b726d36c47246d4277777 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 7 Jul 2020 20:28:20 +0300 Subject: [PATCH 133/196] Update index.md --- docs/en/operations/optimizing-performance/index.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/operations/optimizing-performance/index.md b/docs/en/operations/optimizing-performance/index.md index 6e6cef109c1..ca7b96268d8 100644 --- a/docs/en/operations/optimizing-performance/index.md +++ b/docs/en/operations/optimizing-performance/index.md @@ -1,6 +1,9 @@ --- toc_folder_title: Optimizing Performance toc_priority: 52 +toc_hidden: true --- +# Optimizing Performance +- [Sampling query profiler](sampling-query-profiler.md) From b0ff1fede54e2df105bf8288ec01dc989e626aae Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 7 Jul 2020 21:17:43 +0300 Subject: [PATCH 134/196] Update README.md --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 46154ea3af8..c0de8deb945 100644 --- a/README.md +++ b/README.md @@ -16,4 +16,5 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse at Yandex Cloud Webinar (in Russian)](https://cloud.yandex.ru/events/144) on July 7, 2020. +* [ClickHouse for genetic data (in Russian)]https://cloud.yandex.ru/events/152) on July 14, 2020. +* [ClickHouse virtual office gours](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/271522978/) on July 15, 2020. From ba437c370f5ba030665bb108915e8431244161ed Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 7 Jul 2020 21:20:40 +0300 Subject: [PATCH 135/196] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index c0de8deb945..1f9f31b3f23 100644 --- a/README.md +++ b/README.md @@ -16,5 +16,5 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [ClickHouse for genetic data (in Russian)]https://cloud.yandex.ru/events/152) on July 14, 2020. +* [ClickHouse for genetic data (in Russian)](https://cloud.yandex.ru/events/152) on July 14, 2020. * [ClickHouse virtual office gours](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/271522978/) on July 15, 2020. From 5e062e851dcf7ae0e4134c6086d54f80c48a4bc8 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Tue, 7 Jul 2020 21:21:36 +0300 Subject: [PATCH 136/196] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 1f9f31b3f23..ef4e02c5434 100644 --- a/README.md +++ b/README.md @@ -17,4 +17,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events * [ClickHouse for genetic data (in Russian)](https://cloud.yandex.ru/events/152) on July 14, 2020. -* [ClickHouse virtual office gours](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/271522978/) on July 15, 2020. +* [ClickHouse virtual office hours](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/271522978/) on July 15, 2020. From 32227b9c38976fcc523b68ac62ed08ba36ea2370 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 Jul 2020 21:43:38 +0300 Subject: [PATCH 137/196] AMQP requires libuv Otherwise fails: FAILED: src/CMakeFiles/dbms.dir/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp.o ... In file included from ../src/Storages/RabbitMQ/RabbitMQHandler.h:9, from ../src/Storages/RabbitMQ/StorageRabbitMQ.h:11, from ../src/Storages/RabbitMQ/RabbitMQBlockInputStream.h:5, from ../src/Storages/RabbitMQ/RabbitMQBlockInputStream.cpp:3: ../contrib/AMQP-CPP/include/amqpcpp/libuv.h:22:10: fatal error: uv.h: No such file or directory --- contrib/CMakeLists.txt | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 74c036fbd89..72e48ffff19 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -304,12 +304,13 @@ if (USE_FASTOPS) add_subdirectory (fastops-cmake) endif() +if (USE_AMQPCPP OR USE_CASSANDRA) + add_subdirectory (libuv) +endif() if (USE_AMQPCPP) add_subdirectory (amqpcpp-cmake) endif() - if (USE_CASSANDRA) - add_subdirectory (libuv) add_subdirectory (cassandra) endif() From 6ed2908e2ad1bb956ca7dec599013491a6ccd25c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Jul 2020 22:51:32 +0300 Subject: [PATCH 138/196] Rename ident to indent. --- src/Interpreters/AggregateDescription.cpp | 4 ++-- src/Interpreters/AggregateDescription.h | 2 +- src/Interpreters/Aggregator.cpp | 6 +++--- src/Interpreters/Aggregator.h | 2 +- src/Processors/QueryPlan/IQueryPlanStep.cpp | 8 ++++---- src/Processors/QueryPlan/IQueryPlanStep.h | 4 ++-- src/Processors/QueryPlan/QueryPlan.cpp | 6 +++--- 7 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/AggregateDescription.cpp b/src/Interpreters/AggregateDescription.cpp index 4cf50cc1f35..e483eb1b7a1 100644 --- a/src/Interpreters/AggregateDescription.cpp +++ b/src/Interpreters/AggregateDescription.cpp @@ -5,9 +5,9 @@ namespace DB { -void AggregateDescription::explain(WriteBuffer & out, size_t ident) const +void AggregateDescription::explain(WriteBuffer & out, size_t indent) const { - String prefix(ident, ' '); + String prefix(indent, ' '); out << prefix << column_name << '\n'; diff --git a/src/Interpreters/AggregateDescription.h b/src/Interpreters/AggregateDescription.h index 6038b224821..396a62c446a 100644 --- a/src/Interpreters/AggregateDescription.h +++ b/src/Interpreters/AggregateDescription.h @@ -16,7 +16,7 @@ struct AggregateDescription Names argument_names; /// used if no `arguments` are specified. String column_name; /// What name to use for a column with aggregate function values - void explain(WriteBuffer & out, size_t ident) const; /// Get description for EXPLAIN query. + void explain(WriteBuffer & out, size_t indent) const; /// Get description for EXPLAIN query. }; using AggregateDescriptions = std::vector; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index fdbf0587f78..afd94e3b5b8 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -151,13 +151,13 @@ Block Aggregator::Params::getHeader( return materializeBlock(res); } -void Aggregator::Params::explain(WriteBuffer & out, size_t ident) const +void Aggregator::Params::explain(WriteBuffer & out, size_t indent) const { Strings res; const auto & header = src_header ? src_header : intermediate_header; - String prefix(ident, ' '); + String prefix(indent, ' '); { /// Dump keys. @@ -184,7 +184,7 @@ void Aggregator::Params::explain(WriteBuffer & out, size_t ident) const out << prefix << "Aggregates:\n"; for (const auto & aggregate : aggregates) - aggregate.explain(out, ident + 4); + aggregate.explain(out, indent + 4); } } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 1ae0898d9ca..87480301c69 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -925,7 +925,7 @@ public: } /// Returns keys and aggregated for EXPLAIN query - void explain(WriteBuffer & out, size_t ident) const; + void explain(WriteBuffer & out, size_t indent) const; }; Aggregator(const Params & params_); diff --git a/src/Processors/QueryPlan/IQueryPlanStep.cpp b/src/Processors/QueryPlan/IQueryPlanStep.cpp index ec75564e175..0be40019c58 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.cpp +++ b/src/Processors/QueryPlan/IQueryPlanStep.cpp @@ -20,7 +20,7 @@ const DataStream & IQueryPlanStep::getOutputStream() const static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep::FormatSettings & settings) { - String prefix(settings.offset, settings.ident_char); + String prefix(settings.offset, settings.indent_char); prefix += "Header"; if (count > 1) @@ -36,7 +36,7 @@ static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep: return; } - prefix.assign(prefix.size(), settings.ident_char); + prefix.assign(prefix.size(), settings.indent_char); bool first = true; for (const auto & elem : header) @@ -52,7 +52,7 @@ static void doDescribeHeader(const Block & header, size_t count, IQueryPlanStep: static void doDescribeProcessor(const IProcessor & processor, size_t count, IQueryPlanStep::FormatSettings & settings) { - settings.out << String(settings.offset, settings.ident_char) << processor.getName(); + settings.out << String(settings.offset, settings.indent_char) << processor.getName(); if (count > 1) settings.out << " × " << std::to_string(count); @@ -84,7 +84,7 @@ static void doDescribeProcessor(const IProcessor & processor, size_t count, IQue doDescribeHeader(*last_header, num_equal_headers, settings); } - settings.offset += settings.ident; + settings.offset += settings.indent; } void IQueryPlanStep::describePipeline(const Processors & processors, FormatSettings & settings) diff --git a/src/Processors/QueryPlan/IQueryPlanStep.h b/src/Processors/QueryPlan/IQueryPlanStep.h index a4e3f6cd5af..558b6a825cb 100644 --- a/src/Processors/QueryPlan/IQueryPlanStep.h +++ b/src/Processors/QueryPlan/IQueryPlanStep.h @@ -65,8 +65,8 @@ public: { WriteBuffer & out; size_t offset = 0; - const size_t ident = 2; - const char ident_char = ' '; + const size_t indent = 2; + const char indent_char = ' '; const bool write_header = false; }; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index eb77438f7eb..d3ec3edfd10 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -244,7 +244,7 @@ void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & opt if (!frame.is_description_printed) { - settings.offset = (stack.size() - 1) * settings.ident; + settings.offset = (stack.size() - 1) * settings.indent; explainStep(*frame.node->step, settings, options); frame.is_description_printed = true; } @@ -261,11 +261,11 @@ void QueryPlan::explainPlan(WriteBuffer & buffer, const ExplainPlanOptions & opt static void explainPipelineStep(IQueryPlanStep & step, IQueryPlanStep::FormatSettings & settings) { - settings.out << String(settings.offset, settings.ident_char) << "(" << step.getName() << ")\n"; + settings.out << String(settings.offset, settings.indent_char) << "(" << step.getName() << ")\n"; size_t current_offset = settings.offset; step.describePipeline(settings); if (current_offset == settings.offset) - settings.offset += settings.ident; + settings.offset += settings.indent; } void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptions & options) From fd5e08884a49a87dbd347d7847acde5e7bd7cbe1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 7 Jul 2020 00:36:10 +0300 Subject: [PATCH 139/196] Bump arrow to 0.17 (and flatbuffers to v1.12, required by arrow) MOTIVATION: - remove double-conversion external dependency - remove flatc (but flatbuffers is still required, arrow just shipped with generated files and that's it) CHANGED: - remove pre-generated headers, it is shipped with the arrow - remove flatc (see above) NOTES (see tests changes): - and snappy error is reported as unsupported compression. --- contrib/arrow | 2 +- contrib/arrow-cmake/CMakeLists.txt | 90 +- .../cpp/src/parquet/parquet_constants.cpp | 17 - .../cpp/src/parquet/parquet_constants.h | 24 - .../cpp/src/parquet/parquet_types.cpp | 6677 ----------------- .../cpp/src/parquet/parquet_types.h | 2602 ------- .../cpp/src/parquet/parquet_version.h | 7 + contrib/flatbuffers | 2 +- .../Formats/Impl/ArrowBufferedStreams.cpp | 28 +- .../Formats/Impl/ArrowBufferedStreams.h | 10 +- .../0_stateless/00900_parquet_load.reference | 2 +- 11 files changed, 55 insertions(+), 9406 deletions(-) delete mode 100644 contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp delete mode 100644 contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h delete mode 100644 contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp delete mode 100644 contrib/arrow-cmake/cpp/src/parquet/parquet_types.h diff --git a/contrib/arrow b/contrib/arrow index b789226ccb2..3cbcb7b62c2 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit b789226ccb2124285792107c758bb3b40b3d082a +Subproject commit 3cbcb7b62c2f2d02851bff837758637eb592a64b diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index afcdae68e77..be8cb9a9708 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -1,5 +1,3 @@ -include(ExternalProject) - set (CMAKE_CXX_STANDARD 17) # === thrift @@ -77,14 +75,9 @@ add_custom_command(OUTPUT orc_proto.pb.h orc_proto.pb.cc # === flatbuffers - -############################################################## -# fbs - Step 1: build flatbuffers lib and flatc compiler -############################################################## set(FLATBUFFERS_SRC_DIR ${ClickHouse_SOURCE_DIR}/contrib/flatbuffers) set(FLATBUFFERS_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/flatbuffers) set(FLATBUFFERS_INCLUDE_DIR ${FLATBUFFERS_SRC_DIR}/include) -set(FLATBUFFERS_COMPILER "$") # set flatbuffers CMake options if (${USE_STATIC_LIBRARIES}) @@ -94,57 +87,11 @@ else () set(FLATBUFFERS_BUILD_SHAREDLIB ON CACHE BOOL "Enable the build of the flatbuffers shared library") set(FLATBUFFERS_BUILD_FLATLIB OFF CACHE BOOL "Disable the build of the flatbuffers library") endif () -set(FLATBUFFERS_BUILD_FLATC ON CACHE BOOL "Build flatbuffers compiler") set(FLATBUFFERS_BUILD_TESTS OFF CACHE BOOL "Skip flatbuffers tests") add_subdirectory(${FLATBUFFERS_SRC_DIR} "${FLATBUFFERS_BINARY_DIR}") -################################### -# fbs - Step 2: compile *.fbs files -################################### -set(ARROW_IPC_SRC_DIR ${ARROW_SRC_DIR}/arrow/ipc) -set(ARROW_FORMAT_SRC_DIR ${ARROW_SRC_DIR}/../../format) - -set(ARROW_GENERATED_INCLUDE_DIR ${CMAKE_CURRENT_BINARY_DIR}/arrow_gen_headers) -set(FLATBUFFERS_COMPILED_OUT_DIR ${ARROW_GENERATED_INCLUDE_DIR}/arrow/ipc) - -set(FBS_OUTPUT_FILES - "${FLATBUFFERS_COMPILED_OUT_DIR}/File_generated.h" - "${FLATBUFFERS_COMPILED_OUT_DIR}/Message_generated.h" - "${FLATBUFFERS_COMPILED_OUT_DIR}/feather_generated.h" - "${FLATBUFFERS_COMPILED_OUT_DIR}/Schema_generated.h" - "${FLATBUFFERS_COMPILED_OUT_DIR}/SparseTensor_generated.h" - "${FLATBUFFERS_COMPILED_OUT_DIR}/Tensor_generated.h") - -set(FBS_SRC - ${ARROW_FORMAT_SRC_DIR}/Message.fbs - ${ARROW_FORMAT_SRC_DIR}/File.fbs - ${ARROW_FORMAT_SRC_DIR}/Schema.fbs - ${ARROW_FORMAT_SRC_DIR}/Tensor.fbs - ${ARROW_FORMAT_SRC_DIR}/SparseTensor.fbs - ${ARROW_IPC_SRC_DIR}/feather.fbs) - -foreach (FIL ${FBS_SRC}) - get_filename_component(ABS_FIL ${FIL} ABSOLUTE) - list(APPEND ABS_FBS_SRC ${ABS_FIL}) -endforeach () - -message(STATUS "FLATBUFFERS_LIBRARY: ${FLATBUFFERS_LIBRARY}, FLATBUFFERS_COMPILER: ${FLATBUFFERS_COMPILER}") -message(STATUS "FLATBUFFERS_COMPILED_OUT_DIR: ${FLATBUFFERS_COMPILED_OUT_DIR}") -message(STATUS "flatc: ${FLATBUFFERS_COMPILER} -c -o ${FLATBUFFERS_COMPILED_OUT_DIR}/ ${ABS_FBS_SRC}") - -add_custom_command(OUTPUT ${FBS_OUTPUT_FILES} - COMMAND ${FLATBUFFERS_COMPILER} - -c - -o - ${FLATBUFFERS_COMPILED_OUT_DIR}/ - ${ABS_FBS_SRC} - DEPENDS flatc ${ABS_FBS_SRC} - COMMENT "Running flatc compiler on ${ABS_FBS_SRC}" - VERBATIM) - -add_custom_target(metadata_fbs DEPENDS ${FBS_OUTPUT_FILES}) -add_dependencies(metadata_fbs flatc) +message(STATUS "FLATBUFFERS_LIBRARY: ${FLATBUFFERS_LIBRARY}") # arrow-cmake cmake file calling orc cmake subroutine which detects certain compiler features. # Apple Clang compiler failed to compile this code without specifying c++11 standard. @@ -203,6 +150,7 @@ configure_file("${LIBRARY_DIR}/util/config.h.cmake" "${CMAKE_CURRENT_SOURCE_DIR} set(ARROW_SRCS ${LIBRARY_DIR}/array.cc ${LIBRARY_DIR}/buffer.cc + ${LIBRARY_DIR}/device.cc ${LIBRARY_DIR}/builder.cc ${LIBRARY_DIR}/compare.cc ${LIBRARY_DIR}/extension_type.cc @@ -219,6 +167,11 @@ set(ARROW_SRCS ${LIBRARY_DIR}/type.cc ${LIBRARY_DIR}/visitor.cc + ${LIBRARY_DIR}/tensor/coo_converter.cc + ${LIBRARY_DIR}/tensor/csc_converter.cc + ${LIBRARY_DIR}/tensor/csf_converter.cc + ${LIBRARY_DIR}/tensor/csr_converter.cc + ${LIBRARY_DIR}/array/builder_adaptive.cc ${LIBRARY_DIR}/array/builder_base.cc ${LIBRARY_DIR}/array/builder_binary.cc @@ -230,6 +183,7 @@ set(ARROW_SRCS ${LIBRARY_DIR}/array/concatenate.cc ${LIBRARY_DIR}/array/dict_internal.cc ${LIBRARY_DIR}/array/diff.cc + ${LIBRARY_DIR}/array/validate.cc ${LIBRARY_DIR}/csv/converter.cc ${LIBRARY_DIR}/csv/chunker.cc @@ -237,6 +191,7 @@ set(ARROW_SRCS ${LIBRARY_DIR}/csv/options.cc ${LIBRARY_DIR}/csv/parser.cc ${LIBRARY_DIR}/csv/reader.cc + ${LIBRARY_DIR}/csv/column_decoder.cc ${LIBRARY_DIR}/ipc/dictionary.cc ${LIBRARY_DIR}/ipc/feather.cc @@ -251,7 +206,6 @@ set(ARROW_SRCS ${LIBRARY_DIR}/io/file.cc ${LIBRARY_DIR}/io/interfaces.cc ${LIBRARY_DIR}/io/memory.cc - ${LIBRARY_DIR}/io/readahead.cc ${LIBRARY_DIR}/io/slow.cc ${LIBRARY_DIR}/util/basic_decimal.cc @@ -274,6 +228,12 @@ set(ARROW_SRCS ${LIBRARY_DIR}/util/thread_pool.cc ${LIBRARY_DIR}/util/trie.cc ${LIBRARY_DIR}/util/utf8.cc + ${LIBRARY_DIR}/util/future.cc + ${LIBRARY_DIR}/util/formatting.cc + ${LIBRARY_DIR}/util/parsing.cc + ${LIBRARY_DIR}/util/time.cc + ${LIBRARY_DIR}/util/delimiting.cc + ${LIBRARY_DIR}/util/iterator.cc ${LIBRARY_DIR}/vendored/base64.cpp ${ORC_SRCS} @@ -321,7 +281,7 @@ endif () add_library(${ARROW_LIBRARY} ${ARROW_SRCS}) # Arrow dependencies -add_dependencies(${ARROW_LIBRARY} ${FLATBUFFERS_LIBRARY} metadata_fbs) +add_dependencies(${ARROW_LIBRARY} ${FLATBUFFERS_LIBRARY}) target_link_libraries(${ARROW_LIBRARY} PRIVATE ${FLATBUFFERS_LIBRARY} boost::filesystem) @@ -352,17 +312,18 @@ target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${ORC_BUILD_INCLUDE_D target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${ORC_ADDITION_SOURCE_DIR}) target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${ARROW_SRC_DIR}) target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${FLATBUFFERS_INCLUDE_DIR}) -target_include_directories(${ARROW_LIBRARY} PRIVATE SYSTEM ${ARROW_GENERATED_INCLUDE_DIR}) # === parquet set(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/parquet) +set(GEN_LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src/generated) # arrow/cpp/src/parquet/CMakeLists.txt set(PARQUET_SRCS ${LIBRARY_DIR}/arrow/reader.cc ${LIBRARY_DIR}/arrow/reader_internal.cc ${LIBRARY_DIR}/arrow/schema.cc ${LIBRARY_DIR}/arrow/writer.cc + ${LIBRARY_DIR}/arrow/path_internal.cc ${LIBRARY_DIR}/bloom_filter.cc ${LIBRARY_DIR}/column_reader.cc ${LIBRARY_DIR}/column_scanner.cc @@ -379,16 +340,19 @@ set(PARQUET_SRCS ${LIBRARY_DIR}/schema.cc ${LIBRARY_DIR}/statistics.cc ${LIBRARY_DIR}/types.cc + ${LIBRARY_DIR}/encryption.cc + ${LIBRARY_DIR}/encryption_internal.cc + ${LIBRARY_DIR}/internal_file_decryptor.cc + ${LIBRARY_DIR}/internal_file_encryptor.cc + + ${GEN_LIBRARY_DIR}/parquet_constants.cpp + ${GEN_LIBRARY_DIR}/parquet_types.cpp ) #list(TRANSFORM PARQUET_SRCS PREPEND ${LIBRARY_DIR}/) # cmake 3.12 -list(APPEND PARQUET_SRCS - ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src/parquet/parquet_constants.cpp - ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src/parquet/parquet_types.cpp - ) add_library(${PARQUET_LIBRARY} ${PARQUET_SRCS}) -target_include_directories(${PARQUET_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src) +target_include_directories(${PARQUET_LIBRARY} SYSTEM PUBLIC ${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/src ${CMAKE_CURRENT_SOURCE_DIR}/cpp/src PRIVATE ${OPENSSL_INCLUDE_DIR}) include(${ClickHouse_SOURCE_DIR}/contrib/thrift/build/cmake/ConfigureChecks.cmake) # makes config.h -target_link_libraries(${PARQUET_LIBRARY} PUBLIC ${ARROW_LIBRARY} PRIVATE ${THRIFT_LIBRARY} boost::headers_only boost::regex) +target_link_libraries(${PARQUET_LIBRARY} PUBLIC ${ARROW_LIBRARY} PRIVATE ${THRIFT_LIBRARY} boost::headers_only boost::regex ${OPENSSL_LIBRARIES}) if (SANITIZE STREQUAL "undefined") target_compile_options(${PARQUET_LIBRARY} PRIVATE -fno-sanitize=undefined) diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp b/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp deleted file mode 100644 index 8efba22eb0e..00000000000 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.cpp +++ /dev/null @@ -1,17 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.12.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#include "parquet_constants.h" - -namespace parquet { namespace format { - -const parquetConstants g_parquet_constants; - -parquetConstants::parquetConstants() { -} - -}} // namespace - diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h b/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h deleted file mode 100644 index a84198bfa44..00000000000 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_constants.h +++ /dev/null @@ -1,24 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.12.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#ifndef parquet_CONSTANTS_H -#define parquet_CONSTANTS_H - -#include "parquet_types.h" - -namespace parquet { namespace format { - -class parquetConstants { - public: - parquetConstants(); - -}; - -extern const parquetConstants g_parquet_constants; - -}} // namespace - -#endif diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp b/contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp deleted file mode 100644 index 9ad74689567..00000000000 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.cpp +++ /dev/null @@ -1,6677 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.12.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#include "parquet_types.h" - -#include -#include - -#include - -namespace parquet { namespace format { - -int _kTypeValues[] = { - Type::BOOLEAN, - Type::INT32, - Type::INT64, - Type::INT96, - Type::FLOAT, - Type::DOUBLE, - Type::BYTE_ARRAY, - Type::FIXED_LEN_BYTE_ARRAY -}; -const char* _kTypeNames[] = { - "BOOLEAN", - "INT32", - "INT64", - "INT96", - "FLOAT", - "DOUBLE", - "BYTE_ARRAY", - "FIXED_LEN_BYTE_ARRAY" -}; -const std::map _Type_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, _kTypeValues, _kTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const Type::type& val) { - std::map::const_iterator it = _Type_VALUES_TO_NAMES.find(val); - if (it != _Type_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kConvertedTypeValues[] = { - ConvertedType::UTF8, - ConvertedType::MAP, - ConvertedType::MAP_KEY_VALUE, - ConvertedType::LIST, - ConvertedType::ENUM, - ConvertedType::DECIMAL, - ConvertedType::DATE, - ConvertedType::TIME_MILLIS, - ConvertedType::TIME_MICROS, - ConvertedType::TIMESTAMP_MILLIS, - ConvertedType::TIMESTAMP_MICROS, - ConvertedType::UINT_8, - ConvertedType::UINT_16, - ConvertedType::UINT_32, - ConvertedType::UINT_64, - ConvertedType::INT_8, - ConvertedType::INT_16, - ConvertedType::INT_32, - ConvertedType::INT_64, - ConvertedType::JSON, - ConvertedType::BSON, - ConvertedType::INTERVAL -}; -const char* _kConvertedTypeNames[] = { - "UTF8", - "MAP", - "MAP_KEY_VALUE", - "LIST", - "ENUM", - "DECIMAL", - "DATE", - "TIME_MILLIS", - "TIME_MICROS", - "TIMESTAMP_MILLIS", - "TIMESTAMP_MICROS", - "UINT_8", - "UINT_16", - "UINT_32", - "UINT_64", - "INT_8", - "INT_16", - "INT_32", - "INT_64", - "JSON", - "BSON", - "INTERVAL" -}; -const std::map _ConvertedType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(22, _kConvertedTypeValues, _kConvertedTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const ConvertedType::type& val) { - std::map::const_iterator it = _ConvertedType_VALUES_TO_NAMES.find(val); - if (it != _ConvertedType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kFieldRepetitionTypeValues[] = { - FieldRepetitionType::REQUIRED, - FieldRepetitionType::OPTIONAL, - FieldRepetitionType::REPEATED -}; -const char* _kFieldRepetitionTypeNames[] = { - "REQUIRED", - "OPTIONAL", - "REPEATED" -}; -const std::map _FieldRepetitionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kFieldRepetitionTypeValues, _kFieldRepetitionTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const FieldRepetitionType::type& val) { - std::map::const_iterator it = _FieldRepetitionType_VALUES_TO_NAMES.find(val); - if (it != _FieldRepetitionType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kEncodingValues[] = { - Encoding::PLAIN, - Encoding::PLAIN_DICTIONARY, - Encoding::RLE, - Encoding::BIT_PACKED, - Encoding::DELTA_BINARY_PACKED, - Encoding::DELTA_LENGTH_BYTE_ARRAY, - Encoding::DELTA_BYTE_ARRAY, - Encoding::RLE_DICTIONARY -}; -const char* _kEncodingNames[] = { - "PLAIN", - "PLAIN_DICTIONARY", - "RLE", - "BIT_PACKED", - "DELTA_BINARY_PACKED", - "DELTA_LENGTH_BYTE_ARRAY", - "DELTA_BYTE_ARRAY", - "RLE_DICTIONARY" -}; -const std::map _Encoding_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(8, _kEncodingValues, _kEncodingNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const Encoding::type& val) { - std::map::const_iterator it = _Encoding_VALUES_TO_NAMES.find(val); - if (it != _Encoding_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kCompressionCodecValues[] = { - CompressionCodec::UNCOMPRESSED, - CompressionCodec::SNAPPY, - CompressionCodec::GZIP, - CompressionCodec::LZO, - CompressionCodec::BROTLI, - CompressionCodec::LZ4, - CompressionCodec::ZSTD -}; -const char* _kCompressionCodecNames[] = { - "UNCOMPRESSED", - "SNAPPY", - "GZIP", - "LZO", - "BROTLI", - "LZ4", - "ZSTD" -}; -const std::map _CompressionCodec_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(7, _kCompressionCodecValues, _kCompressionCodecNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const CompressionCodec::type& val) { - std::map::const_iterator it = _CompressionCodec_VALUES_TO_NAMES.find(val); - if (it != _CompressionCodec_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kPageTypeValues[] = { - PageType::DATA_PAGE, - PageType::INDEX_PAGE, - PageType::DICTIONARY_PAGE, - PageType::DATA_PAGE_V2 -}; -const char* _kPageTypeNames[] = { - "DATA_PAGE", - "INDEX_PAGE", - "DICTIONARY_PAGE", - "DATA_PAGE_V2" -}; -const std::map _PageType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(4, _kPageTypeValues, _kPageTypeNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const PageType::type& val) { - std::map::const_iterator it = _PageType_VALUES_TO_NAMES.find(val); - if (it != _PageType_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - -int _kBoundaryOrderValues[] = { - BoundaryOrder::UNORDERED, - BoundaryOrder::ASCENDING, - BoundaryOrder::DESCENDING -}; -const char* _kBoundaryOrderNames[] = { - "UNORDERED", - "ASCENDING", - "DESCENDING" -}; -const std::map _BoundaryOrder_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kBoundaryOrderValues, _kBoundaryOrderNames), ::apache::thrift::TEnumIterator(-1, NULL, NULL)); - -std::ostream& operator<<(std::ostream& out, const BoundaryOrder::type& val) { - std::map::const_iterator it = _BoundaryOrder_VALUES_TO_NAMES.find(val); - if (it != _BoundaryOrder_VALUES_TO_NAMES.end()) { - out << it->second; - } else { - out << static_cast(val); - } - return out; -} - - -Statistics::~Statistics() throw() { -} - - -void Statistics::__set_max(const std::string& val) { - this->max = val; -__isset.max = true; -} - -void Statistics::__set_min(const std::string& val) { - this->min = val; -__isset.min = true; -} - -void Statistics::__set_null_count(const int64_t val) { - this->null_count = val; -__isset.null_count = true; -} - -void Statistics::__set_distinct_count(const int64_t val) { - this->distinct_count = val; -__isset.distinct_count = true; -} - -void Statistics::__set_max_value(const std::string& val) { - this->max_value = val; -__isset.max_value = true; -} - -void Statistics::__set_min_value(const std::string& val) { - this->min_value = val; -__isset.min_value = true; -} -std::ostream& operator<<(std::ostream& out, const Statistics& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t Statistics::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->max); - this->__isset.max = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->min); - this->__isset.min = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->null_count); - this->__isset.null_count = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->distinct_count); - this->__isset.distinct_count = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->max_value); - this->__isset.max_value = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->min_value); - this->__isset.min_value = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t Statistics::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("Statistics"); - - if (this->__isset.max) { - xfer += oprot->writeFieldBegin("max", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->max); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.min) { - xfer += oprot->writeFieldBegin("min", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->min); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.null_count) { - xfer += oprot->writeFieldBegin("null_count", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->null_count); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.distinct_count) { - xfer += oprot->writeFieldBegin("distinct_count", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->distinct_count); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.max_value) { - xfer += oprot->writeFieldBegin("max_value", ::apache::thrift::protocol::T_STRING, 5); - xfer += oprot->writeBinary(this->max_value); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.min_value) { - xfer += oprot->writeFieldBegin("min_value", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeBinary(this->min_value); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(Statistics &a, Statistics &b) { - using ::std::swap; - swap(a.max, b.max); - swap(a.min, b.min); - swap(a.null_count, b.null_count); - swap(a.distinct_count, b.distinct_count); - swap(a.max_value, b.max_value); - swap(a.min_value, b.min_value); - swap(a.__isset, b.__isset); -} - -Statistics::Statistics(const Statistics& other0) { - max = other0.max; - min = other0.min; - null_count = other0.null_count; - distinct_count = other0.distinct_count; - max_value = other0.max_value; - min_value = other0.min_value; - __isset = other0.__isset; -} -Statistics& Statistics::operator=(const Statistics& other1) { - max = other1.max; - min = other1.min; - null_count = other1.null_count; - distinct_count = other1.distinct_count; - max_value = other1.max_value; - min_value = other1.min_value; - __isset = other1.__isset; - return *this; -} -void Statistics::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "Statistics("; - out << "max="; (__isset.max ? (out << to_string(max)) : (out << "")); - out << ", " << "min="; (__isset.min ? (out << to_string(min)) : (out << "")); - out << ", " << "null_count="; (__isset.null_count ? (out << to_string(null_count)) : (out << "")); - out << ", " << "distinct_count="; (__isset.distinct_count ? (out << to_string(distinct_count)) : (out << "")); - out << ", " << "max_value="; (__isset.max_value ? (out << to_string(max_value)) : (out << "")); - out << ", " << "min_value="; (__isset.min_value ? (out << to_string(min_value)) : (out << "")); - out << ")"; -} - - -StringType::~StringType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const StringType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t StringType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t StringType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("StringType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(StringType &a, StringType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -StringType::StringType(const StringType& other2) { - (void) other2; -} -StringType& StringType::operator=(const StringType& other3) { - (void) other3; - return *this; -} -void StringType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "StringType("; - out << ")"; -} - - -UUIDType::~UUIDType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const UUIDType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t UUIDType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t UUIDType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("UUIDType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(UUIDType &a, UUIDType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -UUIDType::UUIDType(const UUIDType& other4) { - (void) other4; -} -UUIDType& UUIDType::operator=(const UUIDType& other5) { - (void) other5; - return *this; -} -void UUIDType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "UUIDType("; - out << ")"; -} - - -MapType::~MapType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const MapType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t MapType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t MapType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("MapType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(MapType &a, MapType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -MapType::MapType(const MapType& other6) { - (void) other6; -} -MapType& MapType::operator=(const MapType& other7) { - (void) other7; - return *this; -} -void MapType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "MapType("; - out << ")"; -} - - -ListType::~ListType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const ListType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ListType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ListType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ListType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ListType &a, ListType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -ListType::ListType(const ListType& other8) { - (void) other8; -} -ListType& ListType::operator=(const ListType& other9) { - (void) other9; - return *this; -} -void ListType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ListType("; - out << ")"; -} - - -EnumType::~EnumType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const EnumType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t EnumType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t EnumType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("EnumType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(EnumType &a, EnumType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -EnumType::EnumType(const EnumType& other10) { - (void) other10; -} -EnumType& EnumType::operator=(const EnumType& other11) { - (void) other11; - return *this; -} -void EnumType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "EnumType("; - out << ")"; -} - - -DateType::~DateType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const DateType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DateType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t DateType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DateType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DateType &a, DateType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -DateType::DateType(const DateType& other12) { - (void) other12; -} -DateType& DateType::operator=(const DateType& other13) { - (void) other13; - return *this; -} -void DateType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DateType("; - out << ")"; -} - - -NullType::~NullType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const NullType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NullType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t NullType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NullType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NullType &a, NullType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -NullType::NullType(const NullType& other14) { - (void) other14; -} -NullType& NullType::operator=(const NullType& other15) { - (void) other15; - return *this; -} -void NullType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NullType("; - out << ")"; -} - - -DecimalType::~DecimalType() throw() { -} - - -void DecimalType::__set_scale(const int32_t val) { - this->scale = val; -} - -void DecimalType::__set_precision(const int32_t val) { - this->precision = val; -} -std::ostream& operator<<(std::ostream& out, const DecimalType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DecimalType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_scale = false; - bool isset_precision = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->scale); - isset_scale = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->precision); - isset_precision = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_scale) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_precision) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DecimalType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DecimalType"); - - xfer += oprot->writeFieldBegin("scale", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->scale); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("precision", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->precision); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DecimalType &a, DecimalType &b) { - using ::std::swap; - swap(a.scale, b.scale); - swap(a.precision, b.precision); -} - -DecimalType::DecimalType(const DecimalType& other16) { - scale = other16.scale; - precision = other16.precision; -} -DecimalType& DecimalType::operator=(const DecimalType& other17) { - scale = other17.scale; - precision = other17.precision; - return *this; -} -void DecimalType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DecimalType("; - out << "scale=" << to_string(scale); - out << ", " << "precision=" << to_string(precision); - out << ")"; -} - - -MilliSeconds::~MilliSeconds() throw() { -} - -std::ostream& operator<<(std::ostream& out, const MilliSeconds& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t MilliSeconds::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t MilliSeconds::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("MilliSeconds"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(MilliSeconds &a, MilliSeconds &b) { - using ::std::swap; - (void) a; - (void) b; -} - -MilliSeconds::MilliSeconds(const MilliSeconds& other18) { - (void) other18; -} -MilliSeconds& MilliSeconds::operator=(const MilliSeconds& other19) { - (void) other19; - return *this; -} -void MilliSeconds::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "MilliSeconds("; - out << ")"; -} - - -MicroSeconds::~MicroSeconds() throw() { -} - -std::ostream& operator<<(std::ostream& out, const MicroSeconds& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t MicroSeconds::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t MicroSeconds::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("MicroSeconds"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(MicroSeconds &a, MicroSeconds &b) { - using ::std::swap; - (void) a; - (void) b; -} - -MicroSeconds::MicroSeconds(const MicroSeconds& other20) { - (void) other20; -} -MicroSeconds& MicroSeconds::operator=(const MicroSeconds& other21) { - (void) other21; - return *this; -} -void MicroSeconds::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "MicroSeconds("; - out << ")"; -} - - -NanoSeconds::~NanoSeconds() throw() { -} - -std::ostream& operator<<(std::ostream& out, const NanoSeconds& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t NanoSeconds::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t NanoSeconds::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("NanoSeconds"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(NanoSeconds &a, NanoSeconds &b) { - using ::std::swap; - (void) a; - (void) b; -} - -NanoSeconds::NanoSeconds(const NanoSeconds& other22) { - (void) other22; -} -NanoSeconds& NanoSeconds::operator=(const NanoSeconds& other23) { - (void) other23; - return *this; -} -void NanoSeconds::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "NanoSeconds("; - out << ")"; -} - - -TimeUnit::~TimeUnit() throw() { -} - - -void TimeUnit::__set_MILLIS(const MilliSeconds& val) { - this->MILLIS = val; -__isset.MILLIS = true; -} - -void TimeUnit::__set_MICROS(const MicroSeconds& val) { - this->MICROS = val; -__isset.MICROS = true; -} - -void TimeUnit::__set_NANOS(const NanoSeconds& val) { - this->NANOS = val; -__isset.NANOS = true; -} -std::ostream& operator<<(std::ostream& out, const TimeUnit& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TimeUnit::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->MILLIS.read(iprot); - this->__isset.MILLIS = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->MICROS.read(iprot); - this->__isset.MICROS = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->NANOS.read(iprot); - this->__isset.NANOS = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t TimeUnit::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TimeUnit"); - - if (this->__isset.MILLIS) { - xfer += oprot->writeFieldBegin("MILLIS", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->MILLIS.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.MICROS) { - xfer += oprot->writeFieldBegin("MICROS", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->MICROS.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.NANOS) { - xfer += oprot->writeFieldBegin("NANOS", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->NANOS.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TimeUnit &a, TimeUnit &b) { - using ::std::swap; - swap(a.MILLIS, b.MILLIS); - swap(a.MICROS, b.MICROS); - swap(a.NANOS, b.NANOS); - swap(a.__isset, b.__isset); -} - -TimeUnit::TimeUnit(const TimeUnit& other24) { - MILLIS = other24.MILLIS; - MICROS = other24.MICROS; - NANOS = other24.NANOS; - __isset = other24.__isset; -} -TimeUnit& TimeUnit::operator=(const TimeUnit& other25) { - MILLIS = other25.MILLIS; - MICROS = other25.MICROS; - NANOS = other25.NANOS; - __isset = other25.__isset; - return *this; -} -void TimeUnit::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TimeUnit("; - out << "MILLIS="; (__isset.MILLIS ? (out << to_string(MILLIS)) : (out << "")); - out << ", " << "MICROS="; (__isset.MICROS ? (out << to_string(MICROS)) : (out << "")); - out << ", " << "NANOS="; (__isset.NANOS ? (out << to_string(NANOS)) : (out << "")); - out << ")"; -} - - -TimestampType::~TimestampType() throw() { -} - - -void TimestampType::__set_isAdjustedToUTC(const bool val) { - this->isAdjustedToUTC = val; -} - -void TimestampType::__set_unit(const TimeUnit& val) { - this->unit = val; -} -std::ostream& operator<<(std::ostream& out, const TimestampType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TimestampType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_isAdjustedToUTC = false; - bool isset_unit = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isAdjustedToUTC); - isset_isAdjustedToUTC = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->unit.read(iprot); - isset_unit = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_isAdjustedToUTC) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_unit) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t TimestampType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TimestampType"); - - xfer += oprot->writeFieldBegin("isAdjustedToUTC", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->isAdjustedToUTC); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("unit", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->unit.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TimestampType &a, TimestampType &b) { - using ::std::swap; - swap(a.isAdjustedToUTC, b.isAdjustedToUTC); - swap(a.unit, b.unit); -} - -TimestampType::TimestampType(const TimestampType& other26) { - isAdjustedToUTC = other26.isAdjustedToUTC; - unit = other26.unit; -} -TimestampType& TimestampType::operator=(const TimestampType& other27) { - isAdjustedToUTC = other27.isAdjustedToUTC; - unit = other27.unit; - return *this; -} -void TimestampType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TimestampType("; - out << "isAdjustedToUTC=" << to_string(isAdjustedToUTC); - out << ", " << "unit=" << to_string(unit); - out << ")"; -} - - -TimeType::~TimeType() throw() { -} - - -void TimeType::__set_isAdjustedToUTC(const bool val) { - this->isAdjustedToUTC = val; -} - -void TimeType::__set_unit(const TimeUnit& val) { - this->unit = val; -} -std::ostream& operator<<(std::ostream& out, const TimeType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TimeType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_isAdjustedToUTC = false; - bool isset_unit = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isAdjustedToUTC); - isset_isAdjustedToUTC = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->unit.read(iprot); - isset_unit = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_isAdjustedToUTC) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_unit) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t TimeType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TimeType"); - - xfer += oprot->writeFieldBegin("isAdjustedToUTC", ::apache::thrift::protocol::T_BOOL, 1); - xfer += oprot->writeBool(this->isAdjustedToUTC); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("unit", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->unit.write(oprot); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TimeType &a, TimeType &b) { - using ::std::swap; - swap(a.isAdjustedToUTC, b.isAdjustedToUTC); - swap(a.unit, b.unit); -} - -TimeType::TimeType(const TimeType& other28) { - isAdjustedToUTC = other28.isAdjustedToUTC; - unit = other28.unit; -} -TimeType& TimeType::operator=(const TimeType& other29) { - isAdjustedToUTC = other29.isAdjustedToUTC; - unit = other29.unit; - return *this; -} -void TimeType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TimeType("; - out << "isAdjustedToUTC=" << to_string(isAdjustedToUTC); - out << ", " << "unit=" << to_string(unit); - out << ")"; -} - - -IntType::~IntType() throw() { -} - - -void IntType::__set_bitWidth(const int8_t val) { - this->bitWidth = val; -} - -void IntType::__set_isSigned(const bool val) { - this->isSigned = val; -} -std::ostream& operator<<(std::ostream& out, const IntType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t IntType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_bitWidth = false; - bool isset_isSigned = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_BYTE) { - xfer += iprot->readByte(this->bitWidth); - isset_bitWidth = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->isSigned); - isset_isSigned = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_bitWidth) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_isSigned) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t IntType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("IntType"); - - xfer += oprot->writeFieldBegin("bitWidth", ::apache::thrift::protocol::T_BYTE, 1); - xfer += oprot->writeByte(this->bitWidth); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("isSigned", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->isSigned); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(IntType &a, IntType &b) { - using ::std::swap; - swap(a.bitWidth, b.bitWidth); - swap(a.isSigned, b.isSigned); -} - -IntType::IntType(const IntType& other30) { - bitWidth = other30.bitWidth; - isSigned = other30.isSigned; -} -IntType& IntType::operator=(const IntType& other31) { - bitWidth = other31.bitWidth; - isSigned = other31.isSigned; - return *this; -} -void IntType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "IntType("; - out << "bitWidth=" << to_string(bitWidth); - out << ", " << "isSigned=" << to_string(isSigned); - out << ")"; -} - - -JsonType::~JsonType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const JsonType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t JsonType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t JsonType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("JsonType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(JsonType &a, JsonType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -JsonType::JsonType(const JsonType& other32) { - (void) other32; -} -JsonType& JsonType::operator=(const JsonType& other33) { - (void) other33; - return *this; -} -void JsonType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "JsonType("; - out << ")"; -} - - -BsonType::~BsonType() throw() { -} - -std::ostream& operator<<(std::ostream& out, const BsonType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t BsonType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t BsonType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("BsonType"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(BsonType &a, BsonType &b) { - using ::std::swap; - (void) a; - (void) b; -} - -BsonType::BsonType(const BsonType& other34) { - (void) other34; -} -BsonType& BsonType::operator=(const BsonType& other35) { - (void) other35; - return *this; -} -void BsonType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "BsonType("; - out << ")"; -} - - -LogicalType::~LogicalType() throw() { -} - - -void LogicalType::__set_STRING(const StringType& val) { - this->STRING = val; -__isset.STRING = true; -} - -void LogicalType::__set_MAP(const MapType& val) { - this->MAP = val; -__isset.MAP = true; -} - -void LogicalType::__set_LIST(const ListType& val) { - this->LIST = val; -__isset.LIST = true; -} - -void LogicalType::__set_ENUM(const EnumType& val) { - this->ENUM = val; -__isset.ENUM = true; -} - -void LogicalType::__set_DECIMAL(const DecimalType& val) { - this->DECIMAL = val; -__isset.DECIMAL = true; -} - -void LogicalType::__set_DATE(const DateType& val) { - this->DATE = val; -__isset.DATE = true; -} - -void LogicalType::__set_TIME(const TimeType& val) { - this->TIME = val; -__isset.TIME = true; -} - -void LogicalType::__set_TIMESTAMP(const TimestampType& val) { - this->TIMESTAMP = val; -__isset.TIMESTAMP = true; -} - -void LogicalType::__set_INTEGER(const IntType& val) { - this->INTEGER = val; -__isset.INTEGER = true; -} - -void LogicalType::__set_UNKNOWN(const NullType& val) { - this->UNKNOWN = val; -__isset.UNKNOWN = true; -} - -void LogicalType::__set_JSON(const JsonType& val) { - this->JSON = val; -__isset.JSON = true; -} - -void LogicalType::__set_BSON(const BsonType& val) { - this->BSON = val; -__isset.BSON = true; -} - -void LogicalType::__set_UUID(const UUIDType& val) { - this->UUID = val; -__isset.UUID = true; -} -std::ostream& operator<<(std::ostream& out, const LogicalType& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t LogicalType::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->STRING.read(iprot); - this->__isset.STRING = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->MAP.read(iprot); - this->__isset.MAP = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->LIST.read(iprot); - this->__isset.LIST = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->ENUM.read(iprot); - this->__isset.ENUM = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->DECIMAL.read(iprot); - this->__isset.DECIMAL = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->DATE.read(iprot); - this->__isset.DATE = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->TIME.read(iprot); - this->__isset.TIME = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->TIMESTAMP.read(iprot); - this->__isset.TIMESTAMP = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->INTEGER.read(iprot); - this->__isset.INTEGER = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 11: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->UNKNOWN.read(iprot); - this->__isset.UNKNOWN = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 12: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->JSON.read(iprot); - this->__isset.JSON = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 13: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->BSON.read(iprot); - this->__isset.BSON = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 14: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->UUID.read(iprot); - this->__isset.UUID = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t LogicalType::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("LogicalType"); - - if (this->__isset.STRING) { - xfer += oprot->writeFieldBegin("STRING", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->STRING.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.MAP) { - xfer += oprot->writeFieldBegin("MAP", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->MAP.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.LIST) { - xfer += oprot->writeFieldBegin("LIST", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->LIST.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ENUM) { - xfer += oprot->writeFieldBegin("ENUM", ::apache::thrift::protocol::T_STRUCT, 4); - xfer += this->ENUM.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.DECIMAL) { - xfer += oprot->writeFieldBegin("DECIMAL", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->DECIMAL.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.DATE) { - xfer += oprot->writeFieldBegin("DATE", ::apache::thrift::protocol::T_STRUCT, 6); - xfer += this->DATE.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.TIME) { - xfer += oprot->writeFieldBegin("TIME", ::apache::thrift::protocol::T_STRUCT, 7); - xfer += this->TIME.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.TIMESTAMP) { - xfer += oprot->writeFieldBegin("TIMESTAMP", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->TIMESTAMP.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.INTEGER) { - xfer += oprot->writeFieldBegin("INTEGER", ::apache::thrift::protocol::T_STRUCT, 10); - xfer += this->INTEGER.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.UNKNOWN) { - xfer += oprot->writeFieldBegin("UNKNOWN", ::apache::thrift::protocol::T_STRUCT, 11); - xfer += this->UNKNOWN.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.JSON) { - xfer += oprot->writeFieldBegin("JSON", ::apache::thrift::protocol::T_STRUCT, 12); - xfer += this->JSON.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.BSON) { - xfer += oprot->writeFieldBegin("BSON", ::apache::thrift::protocol::T_STRUCT, 13); - xfer += this->BSON.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.UUID) { - xfer += oprot->writeFieldBegin("UUID", ::apache::thrift::protocol::T_STRUCT, 14); - xfer += this->UUID.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(LogicalType &a, LogicalType &b) { - using ::std::swap; - swap(a.STRING, b.STRING); - swap(a.MAP, b.MAP); - swap(a.LIST, b.LIST); - swap(a.ENUM, b.ENUM); - swap(a.DECIMAL, b.DECIMAL); - swap(a.DATE, b.DATE); - swap(a.TIME, b.TIME); - swap(a.TIMESTAMP, b.TIMESTAMP); - swap(a.INTEGER, b.INTEGER); - swap(a.UNKNOWN, b.UNKNOWN); - swap(a.JSON, b.JSON); - swap(a.BSON, b.BSON); - swap(a.UUID, b.UUID); - swap(a.__isset, b.__isset); -} - -LogicalType::LogicalType(const LogicalType& other36) { - STRING = other36.STRING; - MAP = other36.MAP; - LIST = other36.LIST; - ENUM = other36.ENUM; - DECIMAL = other36.DECIMAL; - DATE = other36.DATE; - TIME = other36.TIME; - TIMESTAMP = other36.TIMESTAMP; - INTEGER = other36.INTEGER; - UNKNOWN = other36.UNKNOWN; - JSON = other36.JSON; - BSON = other36.BSON; - UUID = other36.UUID; - __isset = other36.__isset; -} -LogicalType& LogicalType::operator=(const LogicalType& other37) { - STRING = other37.STRING; - MAP = other37.MAP; - LIST = other37.LIST; - ENUM = other37.ENUM; - DECIMAL = other37.DECIMAL; - DATE = other37.DATE; - TIME = other37.TIME; - TIMESTAMP = other37.TIMESTAMP; - INTEGER = other37.INTEGER; - UNKNOWN = other37.UNKNOWN; - JSON = other37.JSON; - BSON = other37.BSON; - UUID = other37.UUID; - __isset = other37.__isset; - return *this; -} -void LogicalType::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "LogicalType("; - out << "STRING="; (__isset.STRING ? (out << to_string(STRING)) : (out << "")); - out << ", " << "MAP="; (__isset.MAP ? (out << to_string(MAP)) : (out << "")); - out << ", " << "LIST="; (__isset.LIST ? (out << to_string(LIST)) : (out << "")); - out << ", " << "ENUM="; (__isset.ENUM ? (out << to_string(ENUM)) : (out << "")); - out << ", " << "DECIMAL="; (__isset.DECIMAL ? (out << to_string(DECIMAL)) : (out << "")); - out << ", " << "DATE="; (__isset.DATE ? (out << to_string(DATE)) : (out << "")); - out << ", " << "TIME="; (__isset.TIME ? (out << to_string(TIME)) : (out << "")); - out << ", " << "TIMESTAMP="; (__isset.TIMESTAMP ? (out << to_string(TIMESTAMP)) : (out << "")); - out << ", " << "INTEGER="; (__isset.INTEGER ? (out << to_string(INTEGER)) : (out << "")); - out << ", " << "UNKNOWN="; (__isset.UNKNOWN ? (out << to_string(UNKNOWN)) : (out << "")); - out << ", " << "JSON="; (__isset.JSON ? (out << to_string(JSON)) : (out << "")); - out << ", " << "BSON="; (__isset.BSON ? (out << to_string(BSON)) : (out << "")); - out << ", " << "UUID="; (__isset.UUID ? (out << to_string(UUID)) : (out << "")); - out << ")"; -} - - -SchemaElement::~SchemaElement() throw() { -} - - -void SchemaElement::__set_type(const Type::type val) { - this->type = val; -__isset.type = true; -} - -void SchemaElement::__set_type_length(const int32_t val) { - this->type_length = val; -__isset.type_length = true; -} - -void SchemaElement::__set_repetition_type(const FieldRepetitionType::type val) { - this->repetition_type = val; -__isset.repetition_type = true; -} - -void SchemaElement::__set_name(const std::string& val) { - this->name = val; -} - -void SchemaElement::__set_num_children(const int32_t val) { - this->num_children = val; -__isset.num_children = true; -} - -void SchemaElement::__set_converted_type(const ConvertedType::type val) { - this->converted_type = val; -__isset.converted_type = true; -} - -void SchemaElement::__set_scale(const int32_t val) { - this->scale = val; -__isset.scale = true; -} - -void SchemaElement::__set_precision(const int32_t val) { - this->precision = val; -__isset.precision = true; -} - -void SchemaElement::__set_field_id(const int32_t val) { - this->field_id = val; -__isset.field_id = true; -} - -void SchemaElement::__set_logicalType(const LogicalType& val) { - this->logicalType = val; -__isset.logicalType = true; -} -std::ostream& operator<<(std::ostream& out, const SchemaElement& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SchemaElement::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_name = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast38; - xfer += iprot->readI32(ecast38); - this->type = (Type::type)ecast38; - this->__isset.type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->type_length); - this->__isset.type_length = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast39; - xfer += iprot->readI32(ecast39); - this->repetition_type = (FieldRepetitionType::type)ecast39; - this->__isset.repetition_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->name); - isset_name = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->num_children); - this->__isset.num_children = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast40; - xfer += iprot->readI32(ecast40); - this->converted_type = (ConvertedType::type)ecast40; - this->__isset.converted_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->scale); - this->__isset.scale = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->precision); - this->__isset.precision = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->field_id); - this->__isset.field_id = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->logicalType.read(iprot); - this->__isset.logicalType = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_name) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t SchemaElement::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SchemaElement"); - - if (this->__isset.type) { - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.type_length) { - xfer += oprot->writeFieldBegin("type_length", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->type_length); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.repetition_type) { - xfer += oprot->writeFieldBegin("repetition_type", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->repetition_type); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("name", ::apache::thrift::protocol::T_STRING, 4); - xfer += oprot->writeString(this->name); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.num_children) { - xfer += oprot->writeFieldBegin("num_children", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->num_children); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.converted_type) { - xfer += oprot->writeFieldBegin("converted_type", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32((int32_t)this->converted_type); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.scale) { - xfer += oprot->writeFieldBegin("scale", ::apache::thrift::protocol::T_I32, 7); - xfer += oprot->writeI32(this->scale); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.precision) { - xfer += oprot->writeFieldBegin("precision", ::apache::thrift::protocol::T_I32, 8); - xfer += oprot->writeI32(this->precision); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.field_id) { - xfer += oprot->writeFieldBegin("field_id", ::apache::thrift::protocol::T_I32, 9); - xfer += oprot->writeI32(this->field_id); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.logicalType) { - xfer += oprot->writeFieldBegin("logicalType", ::apache::thrift::protocol::T_STRUCT, 10); - xfer += this->logicalType.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SchemaElement &a, SchemaElement &b) { - using ::std::swap; - swap(a.type, b.type); - swap(a.type_length, b.type_length); - swap(a.repetition_type, b.repetition_type); - swap(a.name, b.name); - swap(a.num_children, b.num_children); - swap(a.converted_type, b.converted_type); - swap(a.scale, b.scale); - swap(a.precision, b.precision); - swap(a.field_id, b.field_id); - swap(a.logicalType, b.logicalType); - swap(a.__isset, b.__isset); -} - -SchemaElement::SchemaElement(const SchemaElement& other41) { - type = other41.type; - type_length = other41.type_length; - repetition_type = other41.repetition_type; - name = other41.name; - num_children = other41.num_children; - converted_type = other41.converted_type; - scale = other41.scale; - precision = other41.precision; - field_id = other41.field_id; - logicalType = other41.logicalType; - __isset = other41.__isset; -} -SchemaElement& SchemaElement::operator=(const SchemaElement& other42) { - type = other42.type; - type_length = other42.type_length; - repetition_type = other42.repetition_type; - name = other42.name; - num_children = other42.num_children; - converted_type = other42.converted_type; - scale = other42.scale; - precision = other42.precision; - field_id = other42.field_id; - logicalType = other42.logicalType; - __isset = other42.__isset; - return *this; -} -void SchemaElement::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SchemaElement("; - out << "type="; (__isset.type ? (out << to_string(type)) : (out << "")); - out << ", " << "type_length="; (__isset.type_length ? (out << to_string(type_length)) : (out << "")); - out << ", " << "repetition_type="; (__isset.repetition_type ? (out << to_string(repetition_type)) : (out << "")); - out << ", " << "name=" << to_string(name); - out << ", " << "num_children="; (__isset.num_children ? (out << to_string(num_children)) : (out << "")); - out << ", " << "converted_type="; (__isset.converted_type ? (out << to_string(converted_type)) : (out << "")); - out << ", " << "scale="; (__isset.scale ? (out << to_string(scale)) : (out << "")); - out << ", " << "precision="; (__isset.precision ? (out << to_string(precision)) : (out << "")); - out << ", " << "field_id="; (__isset.field_id ? (out << to_string(field_id)) : (out << "")); - out << ", " << "logicalType="; (__isset.logicalType ? (out << to_string(logicalType)) : (out << "")); - out << ")"; -} - - -DataPageHeader::~DataPageHeader() throw() { -} - - -void DataPageHeader::__set_num_values(const int32_t val) { - this->num_values = val; -} - -void DataPageHeader::__set_encoding(const Encoding::type val) { - this->encoding = val; -} - -void DataPageHeader::__set_definition_level_encoding(const Encoding::type val) { - this->definition_level_encoding = val; -} - -void DataPageHeader::__set_repetition_level_encoding(const Encoding::type val) { - this->repetition_level_encoding = val; -} - -void DataPageHeader::__set_statistics(const Statistics& val) { - this->statistics = val; -__isset.statistics = true; -} -std::ostream& operator<<(std::ostream& out, const DataPageHeader& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DataPageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_num_values = false; - bool isset_encoding = false; - bool isset_definition_level_encoding = false; - bool isset_repetition_level_encoding = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->num_values); - isset_num_values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast43; - xfer += iprot->readI32(ecast43); - this->encoding = (Encoding::type)ecast43; - isset_encoding = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast44; - xfer += iprot->readI32(ecast44); - this->definition_level_encoding = (Encoding::type)ecast44; - isset_definition_level_encoding = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast45; - xfer += iprot->readI32(ecast45); - this->repetition_level_encoding = (Encoding::type)ecast45; - isset_repetition_level_encoding = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->statistics.read(iprot); - this->__isset.statistics = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_num_values) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_encoding) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_definition_level_encoding) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_repetition_level_encoding) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DataPageHeader::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DataPageHeader"); - - xfer += oprot->writeFieldBegin("num_values", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->num_values); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("encoding", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->encoding); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("definition_level_encoding", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32((int32_t)this->definition_level_encoding); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("repetition_level_encoding", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->repetition_level_encoding); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.statistics) { - xfer += oprot->writeFieldBegin("statistics", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->statistics.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DataPageHeader &a, DataPageHeader &b) { - using ::std::swap; - swap(a.num_values, b.num_values); - swap(a.encoding, b.encoding); - swap(a.definition_level_encoding, b.definition_level_encoding); - swap(a.repetition_level_encoding, b.repetition_level_encoding); - swap(a.statistics, b.statistics); - swap(a.__isset, b.__isset); -} - -DataPageHeader::DataPageHeader(const DataPageHeader& other46) { - num_values = other46.num_values; - encoding = other46.encoding; - definition_level_encoding = other46.definition_level_encoding; - repetition_level_encoding = other46.repetition_level_encoding; - statistics = other46.statistics; - __isset = other46.__isset; -} -DataPageHeader& DataPageHeader::operator=(const DataPageHeader& other47) { - num_values = other47.num_values; - encoding = other47.encoding; - definition_level_encoding = other47.definition_level_encoding; - repetition_level_encoding = other47.repetition_level_encoding; - statistics = other47.statistics; - __isset = other47.__isset; - return *this; -} -void DataPageHeader::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DataPageHeader("; - out << "num_values=" << to_string(num_values); - out << ", " << "encoding=" << to_string(encoding); - out << ", " << "definition_level_encoding=" << to_string(definition_level_encoding); - out << ", " << "repetition_level_encoding=" << to_string(repetition_level_encoding); - out << ", " << "statistics="; (__isset.statistics ? (out << to_string(statistics)) : (out << "")); - out << ")"; -} - - -IndexPageHeader::~IndexPageHeader() throw() { -} - -std::ostream& operator<<(std::ostream& out, const IndexPageHeader& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t IndexPageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t IndexPageHeader::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("IndexPageHeader"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(IndexPageHeader &a, IndexPageHeader &b) { - using ::std::swap; - (void) a; - (void) b; -} - -IndexPageHeader::IndexPageHeader(const IndexPageHeader& other48) { - (void) other48; -} -IndexPageHeader& IndexPageHeader::operator=(const IndexPageHeader& other49) { - (void) other49; - return *this; -} -void IndexPageHeader::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "IndexPageHeader("; - out << ")"; -} - - -DictionaryPageHeader::~DictionaryPageHeader() throw() { -} - - -void DictionaryPageHeader::__set_num_values(const int32_t val) { - this->num_values = val; -} - -void DictionaryPageHeader::__set_encoding(const Encoding::type val) { - this->encoding = val; -} - -void DictionaryPageHeader::__set_is_sorted(const bool val) { - this->is_sorted = val; -__isset.is_sorted = true; -} -std::ostream& operator<<(std::ostream& out, const DictionaryPageHeader& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DictionaryPageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_num_values = false; - bool isset_encoding = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->num_values); - isset_num_values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast50; - xfer += iprot->readI32(ecast50); - this->encoding = (Encoding::type)ecast50; - isset_encoding = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->is_sorted); - this->__isset.is_sorted = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_num_values) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_encoding) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DictionaryPageHeader::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DictionaryPageHeader"); - - xfer += oprot->writeFieldBegin("num_values", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->num_values); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("encoding", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->encoding); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.is_sorted) { - xfer += oprot->writeFieldBegin("is_sorted", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->is_sorted); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DictionaryPageHeader &a, DictionaryPageHeader &b) { - using ::std::swap; - swap(a.num_values, b.num_values); - swap(a.encoding, b.encoding); - swap(a.is_sorted, b.is_sorted); - swap(a.__isset, b.__isset); -} - -DictionaryPageHeader::DictionaryPageHeader(const DictionaryPageHeader& other51) { - num_values = other51.num_values; - encoding = other51.encoding; - is_sorted = other51.is_sorted; - __isset = other51.__isset; -} -DictionaryPageHeader& DictionaryPageHeader::operator=(const DictionaryPageHeader& other52) { - num_values = other52.num_values; - encoding = other52.encoding; - is_sorted = other52.is_sorted; - __isset = other52.__isset; - return *this; -} -void DictionaryPageHeader::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DictionaryPageHeader("; - out << "num_values=" << to_string(num_values); - out << ", " << "encoding=" << to_string(encoding); - out << ", " << "is_sorted="; (__isset.is_sorted ? (out << to_string(is_sorted)) : (out << "")); - out << ")"; -} - - -DataPageHeaderV2::~DataPageHeaderV2() throw() { -} - - -void DataPageHeaderV2::__set_num_values(const int32_t val) { - this->num_values = val; -} - -void DataPageHeaderV2::__set_num_nulls(const int32_t val) { - this->num_nulls = val; -} - -void DataPageHeaderV2::__set_num_rows(const int32_t val) { - this->num_rows = val; -} - -void DataPageHeaderV2::__set_encoding(const Encoding::type val) { - this->encoding = val; -} - -void DataPageHeaderV2::__set_definition_levels_byte_length(const int32_t val) { - this->definition_levels_byte_length = val; -} - -void DataPageHeaderV2::__set_repetition_levels_byte_length(const int32_t val) { - this->repetition_levels_byte_length = val; -} - -void DataPageHeaderV2::__set_is_compressed(const bool val) { - this->is_compressed = val; -__isset.is_compressed = true; -} - -void DataPageHeaderV2::__set_statistics(const Statistics& val) { - this->statistics = val; -__isset.statistics = true; -} -std::ostream& operator<<(std::ostream& out, const DataPageHeaderV2& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t DataPageHeaderV2::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_num_values = false; - bool isset_num_nulls = false; - bool isset_num_rows = false; - bool isset_encoding = false; - bool isset_definition_levels_byte_length = false; - bool isset_repetition_levels_byte_length = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->num_values); - isset_num_values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->num_nulls); - isset_num_nulls = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->num_rows); - isset_num_rows = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast53; - xfer += iprot->readI32(ecast53); - this->encoding = (Encoding::type)ecast53; - isset_encoding = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->definition_levels_byte_length); - isset_definition_levels_byte_length = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->repetition_levels_byte_length); - isset_repetition_levels_byte_length = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->is_compressed); - this->__isset.is_compressed = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->statistics.read(iprot); - this->__isset.statistics = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_num_values) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_num_nulls) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_num_rows) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_encoding) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_definition_levels_byte_length) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_repetition_levels_byte_length) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t DataPageHeaderV2::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("DataPageHeaderV2"); - - xfer += oprot->writeFieldBegin("num_values", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->num_values); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("num_nulls", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->num_nulls); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("num_rows", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32(this->num_rows); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("encoding", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->encoding); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("definition_levels_byte_length", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->definition_levels_byte_length); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("repetition_levels_byte_length", ::apache::thrift::protocol::T_I32, 6); - xfer += oprot->writeI32(this->repetition_levels_byte_length); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.is_compressed) { - xfer += oprot->writeFieldBegin("is_compressed", ::apache::thrift::protocol::T_BOOL, 7); - xfer += oprot->writeBool(this->is_compressed); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.statistics) { - xfer += oprot->writeFieldBegin("statistics", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->statistics.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(DataPageHeaderV2 &a, DataPageHeaderV2 &b) { - using ::std::swap; - swap(a.num_values, b.num_values); - swap(a.num_nulls, b.num_nulls); - swap(a.num_rows, b.num_rows); - swap(a.encoding, b.encoding); - swap(a.definition_levels_byte_length, b.definition_levels_byte_length); - swap(a.repetition_levels_byte_length, b.repetition_levels_byte_length); - swap(a.is_compressed, b.is_compressed); - swap(a.statistics, b.statistics); - swap(a.__isset, b.__isset); -} - -DataPageHeaderV2::DataPageHeaderV2(const DataPageHeaderV2& other54) { - num_values = other54.num_values; - num_nulls = other54.num_nulls; - num_rows = other54.num_rows; - encoding = other54.encoding; - definition_levels_byte_length = other54.definition_levels_byte_length; - repetition_levels_byte_length = other54.repetition_levels_byte_length; - is_compressed = other54.is_compressed; - statistics = other54.statistics; - __isset = other54.__isset; -} -DataPageHeaderV2& DataPageHeaderV2::operator=(const DataPageHeaderV2& other55) { - num_values = other55.num_values; - num_nulls = other55.num_nulls; - num_rows = other55.num_rows; - encoding = other55.encoding; - definition_levels_byte_length = other55.definition_levels_byte_length; - repetition_levels_byte_length = other55.repetition_levels_byte_length; - is_compressed = other55.is_compressed; - statistics = other55.statistics; - __isset = other55.__isset; - return *this; -} -void DataPageHeaderV2::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "DataPageHeaderV2("; - out << "num_values=" << to_string(num_values); - out << ", " << "num_nulls=" << to_string(num_nulls); - out << ", " << "num_rows=" << to_string(num_rows); - out << ", " << "encoding=" << to_string(encoding); - out << ", " << "definition_levels_byte_length=" << to_string(definition_levels_byte_length); - out << ", " << "repetition_levels_byte_length=" << to_string(repetition_levels_byte_length); - out << ", " << "is_compressed="; (__isset.is_compressed ? (out << to_string(is_compressed)) : (out << "")); - out << ", " << "statistics="; (__isset.statistics ? (out << to_string(statistics)) : (out << "")); - out << ")"; -} - - -PageHeader::~PageHeader() throw() { -} - - -void PageHeader::__set_type(const PageType::type val) { - this->type = val; -} - -void PageHeader::__set_uncompressed_page_size(const int32_t val) { - this->uncompressed_page_size = val; -} - -void PageHeader::__set_compressed_page_size(const int32_t val) { - this->compressed_page_size = val; -} - -void PageHeader::__set_crc(const int32_t val) { - this->crc = val; -__isset.crc = true; -} - -void PageHeader::__set_data_page_header(const DataPageHeader& val) { - this->data_page_header = val; -__isset.data_page_header = true; -} - -void PageHeader::__set_index_page_header(const IndexPageHeader& val) { - this->index_page_header = val; -__isset.index_page_header = true; -} - -void PageHeader::__set_dictionary_page_header(const DictionaryPageHeader& val) { - this->dictionary_page_header = val; -__isset.dictionary_page_header = true; -} - -void PageHeader::__set_data_page_header_v2(const DataPageHeaderV2& val) { - this->data_page_header_v2 = val; -__isset.data_page_header_v2 = true; -} -std::ostream& operator<<(std::ostream& out, const PageHeader& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PageHeader::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_type = false; - bool isset_uncompressed_page_size = false; - bool isset_compressed_page_size = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast56; - xfer += iprot->readI32(ecast56); - this->type = (PageType::type)ecast56; - isset_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->uncompressed_page_size); - isset_uncompressed_page_size = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->compressed_page_size); - isset_compressed_page_size = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->crc); - this->__isset.crc = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->data_page_header.read(iprot); - this->__isset.data_page_header = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->index_page_header.read(iprot); - this->__isset.index_page_header = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->dictionary_page_header.read(iprot); - this->__isset.dictionary_page_header = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->data_page_header_v2.read(iprot); - this->__isset.data_page_header_v2 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_uncompressed_page_size) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_compressed_page_size) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PageHeader::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PageHeader"); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("uncompressed_page_size", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->uncompressed_page_size); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("compressed_page_size", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32(this->compressed_page_size); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.crc) { - xfer += oprot->writeFieldBegin("crc", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32(this->crc); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.data_page_header) { - xfer += oprot->writeFieldBegin("data_page_header", ::apache::thrift::protocol::T_STRUCT, 5); - xfer += this->data_page_header.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.index_page_header) { - xfer += oprot->writeFieldBegin("index_page_header", ::apache::thrift::protocol::T_STRUCT, 6); - xfer += this->index_page_header.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.dictionary_page_header) { - xfer += oprot->writeFieldBegin("dictionary_page_header", ::apache::thrift::protocol::T_STRUCT, 7); - xfer += this->dictionary_page_header.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.data_page_header_v2) { - xfer += oprot->writeFieldBegin("data_page_header_v2", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->data_page_header_v2.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PageHeader &a, PageHeader &b) { - using ::std::swap; - swap(a.type, b.type); - swap(a.uncompressed_page_size, b.uncompressed_page_size); - swap(a.compressed_page_size, b.compressed_page_size); - swap(a.crc, b.crc); - swap(a.data_page_header, b.data_page_header); - swap(a.index_page_header, b.index_page_header); - swap(a.dictionary_page_header, b.dictionary_page_header); - swap(a.data_page_header_v2, b.data_page_header_v2); - swap(a.__isset, b.__isset); -} - -PageHeader::PageHeader(const PageHeader& other57) { - type = other57.type; - uncompressed_page_size = other57.uncompressed_page_size; - compressed_page_size = other57.compressed_page_size; - crc = other57.crc; - data_page_header = other57.data_page_header; - index_page_header = other57.index_page_header; - dictionary_page_header = other57.dictionary_page_header; - data_page_header_v2 = other57.data_page_header_v2; - __isset = other57.__isset; -} -PageHeader& PageHeader::operator=(const PageHeader& other58) { - type = other58.type; - uncompressed_page_size = other58.uncompressed_page_size; - compressed_page_size = other58.compressed_page_size; - crc = other58.crc; - data_page_header = other58.data_page_header; - index_page_header = other58.index_page_header; - dictionary_page_header = other58.dictionary_page_header; - data_page_header_v2 = other58.data_page_header_v2; - __isset = other58.__isset; - return *this; -} -void PageHeader::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PageHeader("; - out << "type=" << to_string(type); - out << ", " << "uncompressed_page_size=" << to_string(uncompressed_page_size); - out << ", " << "compressed_page_size=" << to_string(compressed_page_size); - out << ", " << "crc="; (__isset.crc ? (out << to_string(crc)) : (out << "")); - out << ", " << "data_page_header="; (__isset.data_page_header ? (out << to_string(data_page_header)) : (out << "")); - out << ", " << "index_page_header="; (__isset.index_page_header ? (out << to_string(index_page_header)) : (out << "")); - out << ", " << "dictionary_page_header="; (__isset.dictionary_page_header ? (out << to_string(dictionary_page_header)) : (out << "")); - out << ", " << "data_page_header_v2="; (__isset.data_page_header_v2 ? (out << to_string(data_page_header_v2)) : (out << "")); - out << ")"; -} - - -KeyValue::~KeyValue() throw() { -} - - -void KeyValue::__set_key(const std::string& val) { - this->key = val; -} - -void KeyValue::__set_value(const std::string& val) { - this->value = val; -__isset.value = true; -} -std::ostream& operator<<(std::ostream& out, const KeyValue& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t KeyValue::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_key = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->key); - isset_key = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->value); - this->__isset.value = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_key) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t KeyValue::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("KeyValue"); - - xfer += oprot->writeFieldBegin("key", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->key); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.value) { - xfer += oprot->writeFieldBegin("value", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeString(this->value); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(KeyValue &a, KeyValue &b) { - using ::std::swap; - swap(a.key, b.key); - swap(a.value, b.value); - swap(a.__isset, b.__isset); -} - -KeyValue::KeyValue(const KeyValue& other59) { - key = other59.key; - value = other59.value; - __isset = other59.__isset; -} -KeyValue& KeyValue::operator=(const KeyValue& other60) { - key = other60.key; - value = other60.value; - __isset = other60.__isset; - return *this; -} -void KeyValue::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "KeyValue("; - out << "key=" << to_string(key); - out << ", " << "value="; (__isset.value ? (out << to_string(value)) : (out << "")); - out << ")"; -} - - -SortingColumn::~SortingColumn() throw() { -} - - -void SortingColumn::__set_column_idx(const int32_t val) { - this->column_idx = val; -} - -void SortingColumn::__set_descending(const bool val) { - this->descending = val; -} - -void SortingColumn::__set_nulls_first(const bool val) { - this->nulls_first = val; -} -std::ostream& operator<<(std::ostream& out, const SortingColumn& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t SortingColumn::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_column_idx = false; - bool isset_descending = false; - bool isset_nulls_first = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->column_idx); - isset_column_idx = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->descending); - isset_descending = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->nulls_first); - isset_nulls_first = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_column_idx) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_descending) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_nulls_first) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t SortingColumn::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("SortingColumn"); - - xfer += oprot->writeFieldBegin("column_idx", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->column_idx); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("descending", ::apache::thrift::protocol::T_BOOL, 2); - xfer += oprot->writeBool(this->descending); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("nulls_first", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->nulls_first); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(SortingColumn &a, SortingColumn &b) { - using ::std::swap; - swap(a.column_idx, b.column_idx); - swap(a.descending, b.descending); - swap(a.nulls_first, b.nulls_first); -} - -SortingColumn::SortingColumn(const SortingColumn& other61) { - column_idx = other61.column_idx; - descending = other61.descending; - nulls_first = other61.nulls_first; -} -SortingColumn& SortingColumn::operator=(const SortingColumn& other62) { - column_idx = other62.column_idx; - descending = other62.descending; - nulls_first = other62.nulls_first; - return *this; -} -void SortingColumn::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "SortingColumn("; - out << "column_idx=" << to_string(column_idx); - out << ", " << "descending=" << to_string(descending); - out << ", " << "nulls_first=" << to_string(nulls_first); - out << ")"; -} - - -PageEncodingStats::~PageEncodingStats() throw() { -} - - -void PageEncodingStats::__set_page_type(const PageType::type val) { - this->page_type = val; -} - -void PageEncodingStats::__set_encoding(const Encoding::type val) { - this->encoding = val; -} - -void PageEncodingStats::__set_count(const int32_t val) { - this->count = val; -} -std::ostream& operator<<(std::ostream& out, const PageEncodingStats& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PageEncodingStats::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_page_type = false; - bool isset_encoding = false; - bool isset_count = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast63; - xfer += iprot->readI32(ecast63); - this->page_type = (PageType::type)ecast63; - isset_page_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast64; - xfer += iprot->readI32(ecast64); - this->encoding = (Encoding::type)ecast64; - isset_encoding = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->count); - isset_count = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_page_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_encoding) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_count) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PageEncodingStats::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PageEncodingStats"); - - xfer += oprot->writeFieldBegin("page_type", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->page_type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("encoding", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32((int32_t)this->encoding); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("count", ::apache::thrift::protocol::T_I32, 3); - xfer += oprot->writeI32(this->count); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PageEncodingStats &a, PageEncodingStats &b) { - using ::std::swap; - swap(a.page_type, b.page_type); - swap(a.encoding, b.encoding); - swap(a.count, b.count); -} - -PageEncodingStats::PageEncodingStats(const PageEncodingStats& other65) { - page_type = other65.page_type; - encoding = other65.encoding; - count = other65.count; -} -PageEncodingStats& PageEncodingStats::operator=(const PageEncodingStats& other66) { - page_type = other66.page_type; - encoding = other66.encoding; - count = other66.count; - return *this; -} -void PageEncodingStats::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PageEncodingStats("; - out << "page_type=" << to_string(page_type); - out << ", " << "encoding=" << to_string(encoding); - out << ", " << "count=" << to_string(count); - out << ")"; -} - - -ColumnMetaData::~ColumnMetaData() throw() { -} - - -void ColumnMetaData::__set_type(const Type::type val) { - this->type = val; -} - -void ColumnMetaData::__set_encodings(const std::vector & val) { - this->encodings = val; -} - -void ColumnMetaData::__set_path_in_schema(const std::vector & val) { - this->path_in_schema = val; -} - -void ColumnMetaData::__set_codec(const CompressionCodec::type val) { - this->codec = val; -} - -void ColumnMetaData::__set_num_values(const int64_t val) { - this->num_values = val; -} - -void ColumnMetaData::__set_total_uncompressed_size(const int64_t val) { - this->total_uncompressed_size = val; -} - -void ColumnMetaData::__set_total_compressed_size(const int64_t val) { - this->total_compressed_size = val; -} - -void ColumnMetaData::__set_key_value_metadata(const std::vector & val) { - this->key_value_metadata = val; -__isset.key_value_metadata = true; -} - -void ColumnMetaData::__set_data_page_offset(const int64_t val) { - this->data_page_offset = val; -} - -void ColumnMetaData::__set_index_page_offset(const int64_t val) { - this->index_page_offset = val; -__isset.index_page_offset = true; -} - -void ColumnMetaData::__set_dictionary_page_offset(const int64_t val) { - this->dictionary_page_offset = val; -__isset.dictionary_page_offset = true; -} - -void ColumnMetaData::__set_statistics(const Statistics& val) { - this->statistics = val; -__isset.statistics = true; -} - -void ColumnMetaData::__set_encoding_stats(const std::vector & val) { - this->encoding_stats = val; -__isset.encoding_stats = true; -} -std::ostream& operator<<(std::ostream& out, const ColumnMetaData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_type = false; - bool isset_encodings = false; - bool isset_path_in_schema = false; - bool isset_codec = false; - bool isset_num_values = false; - bool isset_total_uncompressed_size = false; - bool isset_total_compressed_size = false; - bool isset_data_page_offset = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast67; - xfer += iprot->readI32(ecast67); - this->type = (Type::type)ecast67; - isset_type = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->encodings.clear(); - uint32_t _size68; - ::apache::thrift::protocol::TType _etype71; - xfer += iprot->readListBegin(_etype71, _size68); - this->encodings.resize(_size68); - uint32_t _i72; - for (_i72 = 0; _i72 < _size68; ++_i72) - { - int32_t ecast73; - xfer += iprot->readI32(ecast73); - this->encodings[_i72] = (Encoding::type)ecast73; - } - xfer += iprot->readListEnd(); - } - isset_encodings = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->path_in_schema.clear(); - uint32_t _size74; - ::apache::thrift::protocol::TType _etype77; - xfer += iprot->readListBegin(_etype77, _size74); - this->path_in_schema.resize(_size74); - uint32_t _i78; - for (_i78 = 0; _i78 < _size74; ++_i78) - { - xfer += iprot->readString(this->path_in_schema[_i78]); - } - xfer += iprot->readListEnd(); - } - isset_path_in_schema = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast79; - xfer += iprot->readI32(ecast79); - this->codec = (CompressionCodec::type)ecast79; - isset_codec = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->num_values); - isset_num_values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->total_uncompressed_size); - isset_total_uncompressed_size = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->total_compressed_size); - isset_total_compressed_size = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->key_value_metadata.clear(); - uint32_t _size80; - ::apache::thrift::protocol::TType _etype83; - xfer += iprot->readListBegin(_etype83, _size80); - this->key_value_metadata.resize(_size80); - uint32_t _i84; - for (_i84 = 0; _i84 < _size80; ++_i84) - { - xfer += this->key_value_metadata[_i84].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.key_value_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->data_page_offset); - isset_data_page_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 10: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->index_page_offset); - this->__isset.index_page_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 11: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->dictionary_page_offset); - this->__isset.dictionary_page_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 12: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->statistics.read(iprot); - this->__isset.statistics = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 13: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->encoding_stats.clear(); - uint32_t _size85; - ::apache::thrift::protocol::TType _etype88; - xfer += iprot->readListBegin(_etype88, _size85); - this->encoding_stats.resize(_size85); - uint32_t _i89; - for (_i89 = 0; _i89 < _size85; ++_i89) - { - xfer += this->encoding_stats[_i89].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.encoding_stats = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_type) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_encodings) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_path_in_schema) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_codec) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_num_values) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_total_uncompressed_size) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_total_compressed_size) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_data_page_offset) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ColumnMetaData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnMetaData"); - - xfer += oprot->writeFieldBegin("type", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32((int32_t)this->type); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("encodings", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I32, static_cast(this->encodings.size())); - std::vector ::const_iterator _iter90; - for (_iter90 = this->encodings.begin(); _iter90 != this->encodings.end(); ++_iter90) - { - xfer += oprot->writeI32((int32_t)(*_iter90)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("path_in_schema", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->path_in_schema.size())); - std::vector ::const_iterator _iter91; - for (_iter91 = this->path_in_schema.begin(); _iter91 != this->path_in_schema.end(); ++_iter91) - { - xfer += oprot->writeString((*_iter91)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("codec", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->codec); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("num_values", ::apache::thrift::protocol::T_I64, 5); - xfer += oprot->writeI64(this->num_values); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("total_uncompressed_size", ::apache::thrift::protocol::T_I64, 6); - xfer += oprot->writeI64(this->total_uncompressed_size); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("total_compressed_size", ::apache::thrift::protocol::T_I64, 7); - xfer += oprot->writeI64(this->total_compressed_size); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.key_value_metadata) { - xfer += oprot->writeFieldBegin("key_value_metadata", ::apache::thrift::protocol::T_LIST, 8); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->key_value_metadata.size())); - std::vector ::const_iterator _iter92; - for (_iter92 = this->key_value_metadata.begin(); _iter92 != this->key_value_metadata.end(); ++_iter92) - { - xfer += (*_iter92).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("data_page_offset", ::apache::thrift::protocol::T_I64, 9); - xfer += oprot->writeI64(this->data_page_offset); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.index_page_offset) { - xfer += oprot->writeFieldBegin("index_page_offset", ::apache::thrift::protocol::T_I64, 10); - xfer += oprot->writeI64(this->index_page_offset); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.dictionary_page_offset) { - xfer += oprot->writeFieldBegin("dictionary_page_offset", ::apache::thrift::protocol::T_I64, 11); - xfer += oprot->writeI64(this->dictionary_page_offset); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.statistics) { - xfer += oprot->writeFieldBegin("statistics", ::apache::thrift::protocol::T_STRUCT, 12); - xfer += this->statistics.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.encoding_stats) { - xfer += oprot->writeFieldBegin("encoding_stats", ::apache::thrift::protocol::T_LIST, 13); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->encoding_stats.size())); - std::vector ::const_iterator _iter93; - for (_iter93 = this->encoding_stats.begin(); _iter93 != this->encoding_stats.end(); ++_iter93) - { - xfer += (*_iter93).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnMetaData &a, ColumnMetaData &b) { - using ::std::swap; - swap(a.type, b.type); - swap(a.encodings, b.encodings); - swap(a.path_in_schema, b.path_in_schema); - swap(a.codec, b.codec); - swap(a.num_values, b.num_values); - swap(a.total_uncompressed_size, b.total_uncompressed_size); - swap(a.total_compressed_size, b.total_compressed_size); - swap(a.key_value_metadata, b.key_value_metadata); - swap(a.data_page_offset, b.data_page_offset); - swap(a.index_page_offset, b.index_page_offset); - swap(a.dictionary_page_offset, b.dictionary_page_offset); - swap(a.statistics, b.statistics); - swap(a.encoding_stats, b.encoding_stats); - swap(a.__isset, b.__isset); -} - -ColumnMetaData::ColumnMetaData(const ColumnMetaData& other94) { - type = other94.type; - encodings = other94.encodings; - path_in_schema = other94.path_in_schema; - codec = other94.codec; - num_values = other94.num_values; - total_uncompressed_size = other94.total_uncompressed_size; - total_compressed_size = other94.total_compressed_size; - key_value_metadata = other94.key_value_metadata; - data_page_offset = other94.data_page_offset; - index_page_offset = other94.index_page_offset; - dictionary_page_offset = other94.dictionary_page_offset; - statistics = other94.statistics; - encoding_stats = other94.encoding_stats; - __isset = other94.__isset; -} -ColumnMetaData& ColumnMetaData::operator=(const ColumnMetaData& other95) { - type = other95.type; - encodings = other95.encodings; - path_in_schema = other95.path_in_schema; - codec = other95.codec; - num_values = other95.num_values; - total_uncompressed_size = other95.total_uncompressed_size; - total_compressed_size = other95.total_compressed_size; - key_value_metadata = other95.key_value_metadata; - data_page_offset = other95.data_page_offset; - index_page_offset = other95.index_page_offset; - dictionary_page_offset = other95.dictionary_page_offset; - statistics = other95.statistics; - encoding_stats = other95.encoding_stats; - __isset = other95.__isset; - return *this; -} -void ColumnMetaData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnMetaData("; - out << "type=" << to_string(type); - out << ", " << "encodings=" << to_string(encodings); - out << ", " << "path_in_schema=" << to_string(path_in_schema); - out << ", " << "codec=" << to_string(codec); - out << ", " << "num_values=" << to_string(num_values); - out << ", " << "total_uncompressed_size=" << to_string(total_uncompressed_size); - out << ", " << "total_compressed_size=" << to_string(total_compressed_size); - out << ", " << "key_value_metadata="; (__isset.key_value_metadata ? (out << to_string(key_value_metadata)) : (out << "")); - out << ", " << "data_page_offset=" << to_string(data_page_offset); - out << ", " << "index_page_offset="; (__isset.index_page_offset ? (out << to_string(index_page_offset)) : (out << "")); - out << ", " << "dictionary_page_offset="; (__isset.dictionary_page_offset ? (out << to_string(dictionary_page_offset)) : (out << "")); - out << ", " << "statistics="; (__isset.statistics ? (out << to_string(statistics)) : (out << "")); - out << ", " << "encoding_stats="; (__isset.encoding_stats ? (out << to_string(encoding_stats)) : (out << "")); - out << ")"; -} - - -EncryptionWithFooterKey::~EncryptionWithFooterKey() throw() { -} - -std::ostream& operator<<(std::ostream& out, const EncryptionWithFooterKey& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t EncryptionWithFooterKey::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t EncryptionWithFooterKey::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("EncryptionWithFooterKey"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(EncryptionWithFooterKey &a, EncryptionWithFooterKey &b) { - using ::std::swap; - (void) a; - (void) b; -} - -EncryptionWithFooterKey::EncryptionWithFooterKey(const EncryptionWithFooterKey& other96) { - (void) other96; -} -EncryptionWithFooterKey& EncryptionWithFooterKey::operator=(const EncryptionWithFooterKey& other97) { - (void) other97; - return *this; -} -void EncryptionWithFooterKey::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "EncryptionWithFooterKey("; - out << ")"; -} - - -EncryptionWithColumnKey::~EncryptionWithColumnKey() throw() { -} - - -void EncryptionWithColumnKey::__set_path_in_schema(const std::vector & val) { - this->path_in_schema = val; -} - -void EncryptionWithColumnKey::__set_key_metadata(const std::string& val) { - this->key_metadata = val; -__isset.key_metadata = true; -} -std::ostream& operator<<(std::ostream& out, const EncryptionWithColumnKey& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t EncryptionWithColumnKey::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_path_in_schema = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->path_in_schema.clear(); - uint32_t _size98; - ::apache::thrift::protocol::TType _etype101; - xfer += iprot->readListBegin(_etype101, _size98); - this->path_in_schema.resize(_size98); - uint32_t _i102; - for (_i102 = 0; _i102 < _size98; ++_i102) - { - xfer += iprot->readString(this->path_in_schema[_i102]); - } - xfer += iprot->readListEnd(); - } - isset_path_in_schema = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->key_metadata); - this->__isset.key_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_path_in_schema) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t EncryptionWithColumnKey::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("EncryptionWithColumnKey"); - - xfer += oprot->writeFieldBegin("path_in_schema", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->path_in_schema.size())); - std::vector ::const_iterator _iter103; - for (_iter103 = this->path_in_schema.begin(); _iter103 != this->path_in_schema.end(); ++_iter103) - { - xfer += oprot->writeString((*_iter103)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.key_metadata) { - xfer += oprot->writeFieldBegin("key_metadata", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->key_metadata); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(EncryptionWithColumnKey &a, EncryptionWithColumnKey &b) { - using ::std::swap; - swap(a.path_in_schema, b.path_in_schema); - swap(a.key_metadata, b.key_metadata); - swap(a.__isset, b.__isset); -} - -EncryptionWithColumnKey::EncryptionWithColumnKey(const EncryptionWithColumnKey& other104) { - path_in_schema = other104.path_in_schema; - key_metadata = other104.key_metadata; - __isset = other104.__isset; -} -EncryptionWithColumnKey& EncryptionWithColumnKey::operator=(const EncryptionWithColumnKey& other105) { - path_in_schema = other105.path_in_schema; - key_metadata = other105.key_metadata; - __isset = other105.__isset; - return *this; -} -void EncryptionWithColumnKey::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "EncryptionWithColumnKey("; - out << "path_in_schema=" << to_string(path_in_schema); - out << ", " << "key_metadata="; (__isset.key_metadata ? (out << to_string(key_metadata)) : (out << "")); - out << ")"; -} - - -ColumnCryptoMetaData::~ColumnCryptoMetaData() throw() { -} - - -void ColumnCryptoMetaData::__set_ENCRYPTION_WITH_FOOTER_KEY(const EncryptionWithFooterKey& val) { - this->ENCRYPTION_WITH_FOOTER_KEY = val; -__isset.ENCRYPTION_WITH_FOOTER_KEY = true; -} - -void ColumnCryptoMetaData::__set_ENCRYPTION_WITH_COLUMN_KEY(const EncryptionWithColumnKey& val) { - this->ENCRYPTION_WITH_COLUMN_KEY = val; -__isset.ENCRYPTION_WITH_COLUMN_KEY = true; -} -std::ostream& operator<<(std::ostream& out, const ColumnCryptoMetaData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnCryptoMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->ENCRYPTION_WITH_FOOTER_KEY.read(iprot); - this->__isset.ENCRYPTION_WITH_FOOTER_KEY = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->ENCRYPTION_WITH_COLUMN_KEY.read(iprot); - this->__isset.ENCRYPTION_WITH_COLUMN_KEY = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ColumnCryptoMetaData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnCryptoMetaData"); - - if (this->__isset.ENCRYPTION_WITH_FOOTER_KEY) { - xfer += oprot->writeFieldBegin("ENCRYPTION_WITH_FOOTER_KEY", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->ENCRYPTION_WITH_FOOTER_KEY.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ENCRYPTION_WITH_COLUMN_KEY) { - xfer += oprot->writeFieldBegin("ENCRYPTION_WITH_COLUMN_KEY", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->ENCRYPTION_WITH_COLUMN_KEY.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnCryptoMetaData &a, ColumnCryptoMetaData &b) { - using ::std::swap; - swap(a.ENCRYPTION_WITH_FOOTER_KEY, b.ENCRYPTION_WITH_FOOTER_KEY); - swap(a.ENCRYPTION_WITH_COLUMN_KEY, b.ENCRYPTION_WITH_COLUMN_KEY); - swap(a.__isset, b.__isset); -} - -ColumnCryptoMetaData::ColumnCryptoMetaData(const ColumnCryptoMetaData& other106) { - ENCRYPTION_WITH_FOOTER_KEY = other106.ENCRYPTION_WITH_FOOTER_KEY; - ENCRYPTION_WITH_COLUMN_KEY = other106.ENCRYPTION_WITH_COLUMN_KEY; - __isset = other106.__isset; -} -ColumnCryptoMetaData& ColumnCryptoMetaData::operator=(const ColumnCryptoMetaData& other107) { - ENCRYPTION_WITH_FOOTER_KEY = other107.ENCRYPTION_WITH_FOOTER_KEY; - ENCRYPTION_WITH_COLUMN_KEY = other107.ENCRYPTION_WITH_COLUMN_KEY; - __isset = other107.__isset; - return *this; -} -void ColumnCryptoMetaData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnCryptoMetaData("; - out << "ENCRYPTION_WITH_FOOTER_KEY="; (__isset.ENCRYPTION_WITH_FOOTER_KEY ? (out << to_string(ENCRYPTION_WITH_FOOTER_KEY)) : (out << "")); - out << ", " << "ENCRYPTION_WITH_COLUMN_KEY="; (__isset.ENCRYPTION_WITH_COLUMN_KEY ? (out << to_string(ENCRYPTION_WITH_COLUMN_KEY)) : (out << "")); - out << ")"; -} - - -ColumnChunk::~ColumnChunk() throw() { -} - - -void ColumnChunk::__set_file_path(const std::string& val) { - this->file_path = val; -__isset.file_path = true; -} - -void ColumnChunk::__set_file_offset(const int64_t val) { - this->file_offset = val; -} - -void ColumnChunk::__set_meta_data(const ColumnMetaData& val) { - this->meta_data = val; -__isset.meta_data = true; -} - -void ColumnChunk::__set_offset_index_offset(const int64_t val) { - this->offset_index_offset = val; -__isset.offset_index_offset = true; -} - -void ColumnChunk::__set_offset_index_length(const int32_t val) { - this->offset_index_length = val; -__isset.offset_index_length = true; -} - -void ColumnChunk::__set_column_index_offset(const int64_t val) { - this->column_index_offset = val; -__isset.column_index_offset = true; -} - -void ColumnChunk::__set_column_index_length(const int32_t val) { - this->column_index_length = val; -__isset.column_index_length = true; -} - -void ColumnChunk::__set_crypto_metadata(const ColumnCryptoMetaData& val) { - this->crypto_metadata = val; -__isset.crypto_metadata = true; -} - -void ColumnChunk::__set_encrypted_column_metadata(const std::string& val) { - this->encrypted_column_metadata = val; -__isset.encrypted_column_metadata = true; -} -std::ostream& operator<<(std::ostream& out, const ColumnChunk& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnChunk::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_file_offset = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->file_path); - this->__isset.file_path = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->file_offset); - isset_file_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->meta_data.read(iprot); - this->__isset.meta_data = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->offset_index_offset); - this->__isset.offset_index_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->offset_index_length); - this->__isset.offset_index_length = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->column_index_offset); - this->__isset.column_index_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->column_index_length); - this->__isset.column_index_length = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->crypto_metadata.read(iprot); - this->__isset.crypto_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->encrypted_column_metadata); - this->__isset.encrypted_column_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_file_offset) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ColumnChunk::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnChunk"); - - if (this->__isset.file_path) { - xfer += oprot->writeFieldBegin("file_path", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeString(this->file_path); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldBegin("file_offset", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->file_offset); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.meta_data) { - xfer += oprot->writeFieldBegin("meta_data", ::apache::thrift::protocol::T_STRUCT, 3); - xfer += this->meta_data.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.offset_index_offset) { - xfer += oprot->writeFieldBegin("offset_index_offset", ::apache::thrift::protocol::T_I64, 4); - xfer += oprot->writeI64(this->offset_index_offset); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.offset_index_length) { - xfer += oprot->writeFieldBegin("offset_index_length", ::apache::thrift::protocol::T_I32, 5); - xfer += oprot->writeI32(this->offset_index_length); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.column_index_offset) { - xfer += oprot->writeFieldBegin("column_index_offset", ::apache::thrift::protocol::T_I64, 6); - xfer += oprot->writeI64(this->column_index_offset); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.column_index_length) { - xfer += oprot->writeFieldBegin("column_index_length", ::apache::thrift::protocol::T_I32, 7); - xfer += oprot->writeI32(this->column_index_length); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.crypto_metadata) { - xfer += oprot->writeFieldBegin("crypto_metadata", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->crypto_metadata.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.encrypted_column_metadata) { - xfer += oprot->writeFieldBegin("encrypted_column_metadata", ::apache::thrift::protocol::T_STRING, 9); - xfer += oprot->writeBinary(this->encrypted_column_metadata); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnChunk &a, ColumnChunk &b) { - using ::std::swap; - swap(a.file_path, b.file_path); - swap(a.file_offset, b.file_offset); - swap(a.meta_data, b.meta_data); - swap(a.offset_index_offset, b.offset_index_offset); - swap(a.offset_index_length, b.offset_index_length); - swap(a.column_index_offset, b.column_index_offset); - swap(a.column_index_length, b.column_index_length); - swap(a.crypto_metadata, b.crypto_metadata); - swap(a.encrypted_column_metadata, b.encrypted_column_metadata); - swap(a.__isset, b.__isset); -} - -ColumnChunk::ColumnChunk(const ColumnChunk& other108) { - file_path = other108.file_path; - file_offset = other108.file_offset; - meta_data = other108.meta_data; - offset_index_offset = other108.offset_index_offset; - offset_index_length = other108.offset_index_length; - column_index_offset = other108.column_index_offset; - column_index_length = other108.column_index_length; - crypto_metadata = other108.crypto_metadata; - encrypted_column_metadata = other108.encrypted_column_metadata; - __isset = other108.__isset; -} -ColumnChunk& ColumnChunk::operator=(const ColumnChunk& other109) { - file_path = other109.file_path; - file_offset = other109.file_offset; - meta_data = other109.meta_data; - offset_index_offset = other109.offset_index_offset; - offset_index_length = other109.offset_index_length; - column_index_offset = other109.column_index_offset; - column_index_length = other109.column_index_length; - crypto_metadata = other109.crypto_metadata; - encrypted_column_metadata = other109.encrypted_column_metadata; - __isset = other109.__isset; - return *this; -} -void ColumnChunk::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnChunk("; - out << "file_path="; (__isset.file_path ? (out << to_string(file_path)) : (out << "")); - out << ", " << "file_offset=" << to_string(file_offset); - out << ", " << "meta_data="; (__isset.meta_data ? (out << to_string(meta_data)) : (out << "")); - out << ", " << "offset_index_offset="; (__isset.offset_index_offset ? (out << to_string(offset_index_offset)) : (out << "")); - out << ", " << "offset_index_length="; (__isset.offset_index_length ? (out << to_string(offset_index_length)) : (out << "")); - out << ", " << "column_index_offset="; (__isset.column_index_offset ? (out << to_string(column_index_offset)) : (out << "")); - out << ", " << "column_index_length="; (__isset.column_index_length ? (out << to_string(column_index_length)) : (out << "")); - out << ", " << "crypto_metadata="; (__isset.crypto_metadata ? (out << to_string(crypto_metadata)) : (out << "")); - out << ", " << "encrypted_column_metadata="; (__isset.encrypted_column_metadata ? (out << to_string(encrypted_column_metadata)) : (out << "")); - out << ")"; -} - - -RowGroup::~RowGroup() throw() { -} - - -void RowGroup::__set_columns(const std::vector & val) { - this->columns = val; -} - -void RowGroup::__set_total_byte_size(const int64_t val) { - this->total_byte_size = val; -} - -void RowGroup::__set_num_rows(const int64_t val) { - this->num_rows = val; -} - -void RowGroup::__set_sorting_columns(const std::vector & val) { - this->sorting_columns = val; -__isset.sorting_columns = true; -} - -void RowGroup::__set_file_offset(const int64_t val) { - this->file_offset = val; -__isset.file_offset = true; -} - -void RowGroup::__set_total_compressed_size(const int64_t val) { - this->total_compressed_size = val; -__isset.total_compressed_size = true; -} - -void RowGroup::__set_ordinal(const int16_t val) { - this->ordinal = val; -__isset.ordinal = true; -} -std::ostream& operator<<(std::ostream& out, const RowGroup& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t RowGroup::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_columns = false; - bool isset_total_byte_size = false; - bool isset_num_rows = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->columns.clear(); - uint32_t _size110; - ::apache::thrift::protocol::TType _etype113; - xfer += iprot->readListBegin(_etype113, _size110); - this->columns.resize(_size110); - uint32_t _i114; - for (_i114 = 0; _i114 < _size110; ++_i114) - { - xfer += this->columns[_i114].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_columns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->total_byte_size); - isset_total_byte_size = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->num_rows); - isset_num_rows = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->sorting_columns.clear(); - uint32_t _size115; - ::apache::thrift::protocol::TType _etype118; - xfer += iprot->readListBegin(_etype118, _size115); - this->sorting_columns.resize(_size115); - uint32_t _i119; - for (_i119 = 0; _i119 < _size115; ++_i119) - { - xfer += this->sorting_columns[_i119].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.sorting_columns = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->file_offset); - this->__isset.file_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->total_compressed_size); - this->__isset.total_compressed_size = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_I16) { - xfer += iprot->readI16(this->ordinal); - this->__isset.ordinal = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_columns) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_total_byte_size) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_num_rows) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t RowGroup::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("RowGroup"); - - xfer += oprot->writeFieldBegin("columns", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->columns.size())); - std::vector ::const_iterator _iter120; - for (_iter120 = this->columns.begin(); _iter120 != this->columns.end(); ++_iter120) - { - xfer += (*_iter120).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("total_byte_size", ::apache::thrift::protocol::T_I64, 2); - xfer += oprot->writeI64(this->total_byte_size); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("num_rows", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->num_rows); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.sorting_columns) { - xfer += oprot->writeFieldBegin("sorting_columns", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->sorting_columns.size())); - std::vector ::const_iterator _iter121; - for (_iter121 = this->sorting_columns.begin(); _iter121 != this->sorting_columns.end(); ++_iter121) - { - xfer += (*_iter121).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.file_offset) { - xfer += oprot->writeFieldBegin("file_offset", ::apache::thrift::protocol::T_I64, 5); - xfer += oprot->writeI64(this->file_offset); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.total_compressed_size) { - xfer += oprot->writeFieldBegin("total_compressed_size", ::apache::thrift::protocol::T_I64, 6); - xfer += oprot->writeI64(this->total_compressed_size); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.ordinal) { - xfer += oprot->writeFieldBegin("ordinal", ::apache::thrift::protocol::T_I16, 7); - xfer += oprot->writeI16(this->ordinal); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(RowGroup &a, RowGroup &b) { - using ::std::swap; - swap(a.columns, b.columns); - swap(a.total_byte_size, b.total_byte_size); - swap(a.num_rows, b.num_rows); - swap(a.sorting_columns, b.sorting_columns); - swap(a.file_offset, b.file_offset); - swap(a.total_compressed_size, b.total_compressed_size); - swap(a.ordinal, b.ordinal); - swap(a.__isset, b.__isset); -} - -RowGroup::RowGroup(const RowGroup& other122) { - columns = other122.columns; - total_byte_size = other122.total_byte_size; - num_rows = other122.num_rows; - sorting_columns = other122.sorting_columns; - file_offset = other122.file_offset; - total_compressed_size = other122.total_compressed_size; - ordinal = other122.ordinal; - __isset = other122.__isset; -} -RowGroup& RowGroup::operator=(const RowGroup& other123) { - columns = other123.columns; - total_byte_size = other123.total_byte_size; - num_rows = other123.num_rows; - sorting_columns = other123.sorting_columns; - file_offset = other123.file_offset; - total_compressed_size = other123.total_compressed_size; - ordinal = other123.ordinal; - __isset = other123.__isset; - return *this; -} -void RowGroup::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "RowGroup("; - out << "columns=" << to_string(columns); - out << ", " << "total_byte_size=" << to_string(total_byte_size); - out << ", " << "num_rows=" << to_string(num_rows); - out << ", " << "sorting_columns="; (__isset.sorting_columns ? (out << to_string(sorting_columns)) : (out << "")); - out << ", " << "file_offset="; (__isset.file_offset ? (out << to_string(file_offset)) : (out << "")); - out << ", " << "total_compressed_size="; (__isset.total_compressed_size ? (out << to_string(total_compressed_size)) : (out << "")); - out << ", " << "ordinal="; (__isset.ordinal ? (out << to_string(ordinal)) : (out << "")); - out << ")"; -} - - -TypeDefinedOrder::~TypeDefinedOrder() throw() { -} - -std::ostream& operator<<(std::ostream& out, const TypeDefinedOrder& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t TypeDefinedOrder::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - xfer += iprot->skip(ftype); - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t TypeDefinedOrder::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("TypeDefinedOrder"); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(TypeDefinedOrder &a, TypeDefinedOrder &b) { - using ::std::swap; - (void) a; - (void) b; -} - -TypeDefinedOrder::TypeDefinedOrder(const TypeDefinedOrder& other124) { - (void) other124; -} -TypeDefinedOrder& TypeDefinedOrder::operator=(const TypeDefinedOrder& other125) { - (void) other125; - return *this; -} -void TypeDefinedOrder::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "TypeDefinedOrder("; - out << ")"; -} - - -ColumnOrder::~ColumnOrder() throw() { -} - - -void ColumnOrder::__set_TYPE_ORDER(const TypeDefinedOrder& val) { - this->TYPE_ORDER = val; -__isset.TYPE_ORDER = true; -} -std::ostream& operator<<(std::ostream& out, const ColumnOrder& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnOrder::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->TYPE_ORDER.read(iprot); - this->__isset.TYPE_ORDER = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t ColumnOrder::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnOrder"); - - if (this->__isset.TYPE_ORDER) { - xfer += oprot->writeFieldBegin("TYPE_ORDER", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->TYPE_ORDER.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnOrder &a, ColumnOrder &b) { - using ::std::swap; - swap(a.TYPE_ORDER, b.TYPE_ORDER); - swap(a.__isset, b.__isset); -} - -ColumnOrder::ColumnOrder(const ColumnOrder& other126) { - TYPE_ORDER = other126.TYPE_ORDER; - __isset = other126.__isset; -} -ColumnOrder& ColumnOrder::operator=(const ColumnOrder& other127) { - TYPE_ORDER = other127.TYPE_ORDER; - __isset = other127.__isset; - return *this; -} -void ColumnOrder::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnOrder("; - out << "TYPE_ORDER="; (__isset.TYPE_ORDER ? (out << to_string(TYPE_ORDER)) : (out << "")); - out << ")"; -} - - -PageLocation::~PageLocation() throw() { -} - - -void PageLocation::__set_offset(const int64_t val) { - this->offset = val; -} - -void PageLocation::__set_compressed_page_size(const int32_t val) { - this->compressed_page_size = val; -} - -void PageLocation::__set_first_row_index(const int64_t val) { - this->first_row_index = val; -} -std::ostream& operator<<(std::ostream& out, const PageLocation& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t PageLocation::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_offset = false; - bool isset_compressed_page_size = false; - bool isset_first_row_index = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->offset); - isset_offset = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->compressed_page_size); - isset_compressed_page_size = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->first_row_index); - isset_first_row_index = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_offset) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_compressed_page_size) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_first_row_index) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t PageLocation::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("PageLocation"); - - xfer += oprot->writeFieldBegin("offset", ::apache::thrift::protocol::T_I64, 1); - xfer += oprot->writeI64(this->offset); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("compressed_page_size", ::apache::thrift::protocol::T_I32, 2); - xfer += oprot->writeI32(this->compressed_page_size); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("first_row_index", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->first_row_index); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(PageLocation &a, PageLocation &b) { - using ::std::swap; - swap(a.offset, b.offset); - swap(a.compressed_page_size, b.compressed_page_size); - swap(a.first_row_index, b.first_row_index); -} - -PageLocation::PageLocation(const PageLocation& other128) { - offset = other128.offset; - compressed_page_size = other128.compressed_page_size; - first_row_index = other128.first_row_index; -} -PageLocation& PageLocation::operator=(const PageLocation& other129) { - offset = other129.offset; - compressed_page_size = other129.compressed_page_size; - first_row_index = other129.first_row_index; - return *this; -} -void PageLocation::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "PageLocation("; - out << "offset=" << to_string(offset); - out << ", " << "compressed_page_size=" << to_string(compressed_page_size); - out << ", " << "first_row_index=" << to_string(first_row_index); - out << ")"; -} - - -OffsetIndex::~OffsetIndex() throw() { -} - - -void OffsetIndex::__set_page_locations(const std::vector & val) { - this->page_locations = val; -} -std::ostream& operator<<(std::ostream& out, const OffsetIndex& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t OffsetIndex::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_page_locations = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->page_locations.clear(); - uint32_t _size130; - ::apache::thrift::protocol::TType _etype133; - xfer += iprot->readListBegin(_etype133, _size130); - this->page_locations.resize(_size130); - uint32_t _i134; - for (_i134 = 0; _i134 < _size130; ++_i134) - { - xfer += this->page_locations[_i134].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_page_locations = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_page_locations) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t OffsetIndex::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("OffsetIndex"); - - xfer += oprot->writeFieldBegin("page_locations", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->page_locations.size())); - std::vector ::const_iterator _iter135; - for (_iter135 = this->page_locations.begin(); _iter135 != this->page_locations.end(); ++_iter135) - { - xfer += (*_iter135).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(OffsetIndex &a, OffsetIndex &b) { - using ::std::swap; - swap(a.page_locations, b.page_locations); -} - -OffsetIndex::OffsetIndex(const OffsetIndex& other136) { - page_locations = other136.page_locations; -} -OffsetIndex& OffsetIndex::operator=(const OffsetIndex& other137) { - page_locations = other137.page_locations; - return *this; -} -void OffsetIndex::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "OffsetIndex("; - out << "page_locations=" << to_string(page_locations); - out << ")"; -} - - -ColumnIndex::~ColumnIndex() throw() { -} - - -void ColumnIndex::__set_null_pages(const std::vector & val) { - this->null_pages = val; -} - -void ColumnIndex::__set_min_values(const std::vector & val) { - this->min_values = val; -} - -void ColumnIndex::__set_max_values(const std::vector & val) { - this->max_values = val; -} - -void ColumnIndex::__set_boundary_order(const BoundaryOrder::type val) { - this->boundary_order = val; -} - -void ColumnIndex::__set_null_counts(const std::vector & val) { - this->null_counts = val; -__isset.null_counts = true; -} -std::ostream& operator<<(std::ostream& out, const ColumnIndex& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t ColumnIndex::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_null_pages = false; - bool isset_min_values = false; - bool isset_max_values = false; - bool isset_boundary_order = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->null_pages.clear(); - uint32_t _size138; - ::apache::thrift::protocol::TType _etype141; - xfer += iprot->readListBegin(_etype141, _size138); - this->null_pages.resize(_size138); - uint32_t _i142; - for (_i142 = 0; _i142 < _size138; ++_i142) - { - xfer += iprot->readBool(this->null_pages[_i142]); - } - xfer += iprot->readListEnd(); - } - isset_null_pages = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->min_values.clear(); - uint32_t _size143; - ::apache::thrift::protocol::TType _etype146; - xfer += iprot->readListBegin(_etype146, _size143); - this->min_values.resize(_size143); - uint32_t _i147; - for (_i147 = 0; _i147 < _size143; ++_i147) - { - xfer += iprot->readBinary(this->min_values[_i147]); - } - xfer += iprot->readListEnd(); - } - isset_min_values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->max_values.clear(); - uint32_t _size148; - ::apache::thrift::protocol::TType _etype151; - xfer += iprot->readListBegin(_etype151, _size148); - this->max_values.resize(_size148); - uint32_t _i152; - for (_i152 = 0; _i152 < _size148; ++_i152) - { - xfer += iprot->readBinary(this->max_values[_i152]); - } - xfer += iprot->readListEnd(); - } - isset_max_values = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_I32) { - int32_t ecast153; - xfer += iprot->readI32(ecast153); - this->boundary_order = (BoundaryOrder::type)ecast153; - isset_boundary_order = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->null_counts.clear(); - uint32_t _size154; - ::apache::thrift::protocol::TType _etype157; - xfer += iprot->readListBegin(_etype157, _size154); - this->null_counts.resize(_size154); - uint32_t _i158; - for (_i158 = 0; _i158 < _size154; ++_i158) - { - xfer += iprot->readI64(this->null_counts[_i158]); - } - xfer += iprot->readListEnd(); - } - this->__isset.null_counts = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_null_pages) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_min_values) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_max_values) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_boundary_order) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t ColumnIndex::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("ColumnIndex"); - - xfer += oprot->writeFieldBegin("null_pages", ::apache::thrift::protocol::T_LIST, 1); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_BOOL, static_cast(this->null_pages.size())); - std::vector ::const_iterator _iter159; - for (_iter159 = this->null_pages.begin(); _iter159 != this->null_pages.end(); ++_iter159) - { - xfer += oprot->writeBool((*_iter159)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("min_values", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->min_values.size())); - std::vector ::const_iterator _iter160; - for (_iter160 = this->min_values.begin(); _iter160 != this->min_values.end(); ++_iter160) - { - xfer += oprot->writeBinary((*_iter160)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("max_values", ::apache::thrift::protocol::T_LIST, 3); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRING, static_cast(this->max_values.size())); - std::vector ::const_iterator _iter161; - for (_iter161 = this->max_values.begin(); _iter161 != this->max_values.end(); ++_iter161) - { - xfer += oprot->writeBinary((*_iter161)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("boundary_order", ::apache::thrift::protocol::T_I32, 4); - xfer += oprot->writeI32((int32_t)this->boundary_order); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.null_counts) { - xfer += oprot->writeFieldBegin("null_counts", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_I64, static_cast(this->null_counts.size())); - std::vector ::const_iterator _iter162; - for (_iter162 = this->null_counts.begin(); _iter162 != this->null_counts.end(); ++_iter162) - { - xfer += oprot->writeI64((*_iter162)); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(ColumnIndex &a, ColumnIndex &b) { - using ::std::swap; - swap(a.null_pages, b.null_pages); - swap(a.min_values, b.min_values); - swap(a.max_values, b.max_values); - swap(a.boundary_order, b.boundary_order); - swap(a.null_counts, b.null_counts); - swap(a.__isset, b.__isset); -} - -ColumnIndex::ColumnIndex(const ColumnIndex& other163) { - null_pages = other163.null_pages; - min_values = other163.min_values; - max_values = other163.max_values; - boundary_order = other163.boundary_order; - null_counts = other163.null_counts; - __isset = other163.__isset; -} -ColumnIndex& ColumnIndex::operator=(const ColumnIndex& other164) { - null_pages = other164.null_pages; - min_values = other164.min_values; - max_values = other164.max_values; - boundary_order = other164.boundary_order; - null_counts = other164.null_counts; - __isset = other164.__isset; - return *this; -} -void ColumnIndex::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "ColumnIndex("; - out << "null_pages=" << to_string(null_pages); - out << ", " << "min_values=" << to_string(min_values); - out << ", " << "max_values=" << to_string(max_values); - out << ", " << "boundary_order=" << to_string(boundary_order); - out << ", " << "null_counts="; (__isset.null_counts ? (out << to_string(null_counts)) : (out << "")); - out << ")"; -} - - -AesGcmV1::~AesGcmV1() throw() { -} - - -void AesGcmV1::__set_aad_prefix(const std::string& val) { - this->aad_prefix = val; -__isset.aad_prefix = true; -} - -void AesGcmV1::__set_aad_file_unique(const std::string& val) { - this->aad_file_unique = val; -__isset.aad_file_unique = true; -} - -void AesGcmV1::__set_supply_aad_prefix(const bool val) { - this->supply_aad_prefix = val; -__isset.supply_aad_prefix = true; -} -std::ostream& operator<<(std::ostream& out, const AesGcmV1& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AesGcmV1::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->aad_prefix); - this->__isset.aad_prefix = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->aad_file_unique); - this->__isset.aad_file_unique = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->supply_aad_prefix); - this->__isset.supply_aad_prefix = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t AesGcmV1::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AesGcmV1"); - - if (this->__isset.aad_prefix) { - xfer += oprot->writeFieldBegin("aad_prefix", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->aad_prefix); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.aad_file_unique) { - xfer += oprot->writeFieldBegin("aad_file_unique", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->aad_file_unique); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.supply_aad_prefix) { - xfer += oprot->writeFieldBegin("supply_aad_prefix", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->supply_aad_prefix); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AesGcmV1 &a, AesGcmV1 &b) { - using ::std::swap; - swap(a.aad_prefix, b.aad_prefix); - swap(a.aad_file_unique, b.aad_file_unique); - swap(a.supply_aad_prefix, b.supply_aad_prefix); - swap(a.__isset, b.__isset); -} - -AesGcmV1::AesGcmV1(const AesGcmV1& other165) { - aad_prefix = other165.aad_prefix; - aad_file_unique = other165.aad_file_unique; - supply_aad_prefix = other165.supply_aad_prefix; - __isset = other165.__isset; -} -AesGcmV1& AesGcmV1::operator=(const AesGcmV1& other166) { - aad_prefix = other166.aad_prefix; - aad_file_unique = other166.aad_file_unique; - supply_aad_prefix = other166.supply_aad_prefix; - __isset = other166.__isset; - return *this; -} -void AesGcmV1::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AesGcmV1("; - out << "aad_prefix="; (__isset.aad_prefix ? (out << to_string(aad_prefix)) : (out << "")); - out << ", " << "aad_file_unique="; (__isset.aad_file_unique ? (out << to_string(aad_file_unique)) : (out << "")); - out << ", " << "supply_aad_prefix="; (__isset.supply_aad_prefix ? (out << to_string(supply_aad_prefix)) : (out << "")); - out << ")"; -} - - -AesGcmCtrV1::~AesGcmCtrV1() throw() { -} - - -void AesGcmCtrV1::__set_aad_prefix(const std::string& val) { - this->aad_prefix = val; -__isset.aad_prefix = true; -} - -void AesGcmCtrV1::__set_aad_file_unique(const std::string& val) { - this->aad_file_unique = val; -__isset.aad_file_unique = true; -} - -void AesGcmCtrV1::__set_supply_aad_prefix(const bool val) { - this->supply_aad_prefix = val; -__isset.supply_aad_prefix = true; -} -std::ostream& operator<<(std::ostream& out, const AesGcmCtrV1& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t AesGcmCtrV1::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->aad_prefix); - this->__isset.aad_prefix = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->aad_file_unique); - this->__isset.aad_file_unique = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_BOOL) { - xfer += iprot->readBool(this->supply_aad_prefix); - this->__isset.supply_aad_prefix = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t AesGcmCtrV1::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("AesGcmCtrV1"); - - if (this->__isset.aad_prefix) { - xfer += oprot->writeFieldBegin("aad_prefix", ::apache::thrift::protocol::T_STRING, 1); - xfer += oprot->writeBinary(this->aad_prefix); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.aad_file_unique) { - xfer += oprot->writeFieldBegin("aad_file_unique", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->aad_file_unique); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.supply_aad_prefix) { - xfer += oprot->writeFieldBegin("supply_aad_prefix", ::apache::thrift::protocol::T_BOOL, 3); - xfer += oprot->writeBool(this->supply_aad_prefix); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(AesGcmCtrV1 &a, AesGcmCtrV1 &b) { - using ::std::swap; - swap(a.aad_prefix, b.aad_prefix); - swap(a.aad_file_unique, b.aad_file_unique); - swap(a.supply_aad_prefix, b.supply_aad_prefix); - swap(a.__isset, b.__isset); -} - -AesGcmCtrV1::AesGcmCtrV1(const AesGcmCtrV1& other167) { - aad_prefix = other167.aad_prefix; - aad_file_unique = other167.aad_file_unique; - supply_aad_prefix = other167.supply_aad_prefix; - __isset = other167.__isset; -} -AesGcmCtrV1& AesGcmCtrV1::operator=(const AesGcmCtrV1& other168) { - aad_prefix = other168.aad_prefix; - aad_file_unique = other168.aad_file_unique; - supply_aad_prefix = other168.supply_aad_prefix; - __isset = other168.__isset; - return *this; -} -void AesGcmCtrV1::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "AesGcmCtrV1("; - out << "aad_prefix="; (__isset.aad_prefix ? (out << to_string(aad_prefix)) : (out << "")); - out << ", " << "aad_file_unique="; (__isset.aad_file_unique ? (out << to_string(aad_file_unique)) : (out << "")); - out << ", " << "supply_aad_prefix="; (__isset.supply_aad_prefix ? (out << to_string(supply_aad_prefix)) : (out << "")); - out << ")"; -} - - -EncryptionAlgorithm::~EncryptionAlgorithm() throw() { -} - - -void EncryptionAlgorithm::__set_AES_GCM_V1(const AesGcmV1& val) { - this->AES_GCM_V1 = val; -__isset.AES_GCM_V1 = true; -} - -void EncryptionAlgorithm::__set_AES_GCM_CTR_V1(const AesGcmCtrV1& val) { - this->AES_GCM_CTR_V1 = val; -__isset.AES_GCM_CTR_V1 = true; -} -std::ostream& operator<<(std::ostream& out, const EncryptionAlgorithm& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t EncryptionAlgorithm::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->AES_GCM_V1.read(iprot); - this->__isset.AES_GCM_V1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->AES_GCM_CTR_V1.read(iprot); - this->__isset.AES_GCM_CTR_V1 = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - return xfer; -} - -uint32_t EncryptionAlgorithm::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("EncryptionAlgorithm"); - - if (this->__isset.AES_GCM_V1) { - xfer += oprot->writeFieldBegin("AES_GCM_V1", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->AES_GCM_V1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.AES_GCM_CTR_V1) { - xfer += oprot->writeFieldBegin("AES_GCM_CTR_V1", ::apache::thrift::protocol::T_STRUCT, 2); - xfer += this->AES_GCM_CTR_V1.write(oprot); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(EncryptionAlgorithm &a, EncryptionAlgorithm &b) { - using ::std::swap; - swap(a.AES_GCM_V1, b.AES_GCM_V1); - swap(a.AES_GCM_CTR_V1, b.AES_GCM_CTR_V1); - swap(a.__isset, b.__isset); -} - -EncryptionAlgorithm::EncryptionAlgorithm(const EncryptionAlgorithm& other169) { - AES_GCM_V1 = other169.AES_GCM_V1; - AES_GCM_CTR_V1 = other169.AES_GCM_CTR_V1; - __isset = other169.__isset; -} -EncryptionAlgorithm& EncryptionAlgorithm::operator=(const EncryptionAlgorithm& other170) { - AES_GCM_V1 = other170.AES_GCM_V1; - AES_GCM_CTR_V1 = other170.AES_GCM_CTR_V1; - __isset = other170.__isset; - return *this; -} -void EncryptionAlgorithm::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "EncryptionAlgorithm("; - out << "AES_GCM_V1="; (__isset.AES_GCM_V1 ? (out << to_string(AES_GCM_V1)) : (out << "")); - out << ", " << "AES_GCM_CTR_V1="; (__isset.AES_GCM_CTR_V1 ? (out << to_string(AES_GCM_CTR_V1)) : (out << "")); - out << ")"; -} - - -FileMetaData::~FileMetaData() throw() { -} - - -void FileMetaData::__set_version(const int32_t val) { - this->version = val; -} - -void FileMetaData::__set_schema(const std::vector & val) { - this->schema = val; -} - -void FileMetaData::__set_num_rows(const int64_t val) { - this->num_rows = val; -} - -void FileMetaData::__set_row_groups(const std::vector & val) { - this->row_groups = val; -} - -void FileMetaData::__set_key_value_metadata(const std::vector & val) { - this->key_value_metadata = val; -__isset.key_value_metadata = true; -} - -void FileMetaData::__set_created_by(const std::string& val) { - this->created_by = val; -__isset.created_by = true; -} - -void FileMetaData::__set_column_orders(const std::vector & val) { - this->column_orders = val; -__isset.column_orders = true; -} - -void FileMetaData::__set_encryption_algorithm(const EncryptionAlgorithm& val) { - this->encryption_algorithm = val; -__isset.encryption_algorithm = true; -} - -void FileMetaData::__set_footer_signing_key_metadata(const std::string& val) { - this->footer_signing_key_metadata = val; -__isset.footer_signing_key_metadata = true; -} -std::ostream& operator<<(std::ostream& out, const FileMetaData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t FileMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_version = false; - bool isset_schema = false; - bool isset_num_rows = false; - bool isset_row_groups = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_I32) { - xfer += iprot->readI32(this->version); - isset_version = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->schema.clear(); - uint32_t _size171; - ::apache::thrift::protocol::TType _etype174; - xfer += iprot->readListBegin(_etype174, _size171); - this->schema.resize(_size171); - uint32_t _i175; - for (_i175 = 0; _i175 < _size171; ++_i175) - { - xfer += this->schema[_i175].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_schema = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 3: - if (ftype == ::apache::thrift::protocol::T_I64) { - xfer += iprot->readI64(this->num_rows); - isset_num_rows = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 4: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->row_groups.clear(); - uint32_t _size176; - ::apache::thrift::protocol::TType _etype179; - xfer += iprot->readListBegin(_etype179, _size176); - this->row_groups.resize(_size176); - uint32_t _i180; - for (_i180 = 0; _i180 < _size176; ++_i180) - { - xfer += this->row_groups[_i180].read(iprot); - } - xfer += iprot->readListEnd(); - } - isset_row_groups = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 5: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->key_value_metadata.clear(); - uint32_t _size181; - ::apache::thrift::protocol::TType _etype184; - xfer += iprot->readListBegin(_etype184, _size181); - this->key_value_metadata.resize(_size181); - uint32_t _i185; - for (_i185 = 0; _i185 < _size181; ++_i185) - { - xfer += this->key_value_metadata[_i185].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.key_value_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 6: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readString(this->created_by); - this->__isset.created_by = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 7: - if (ftype == ::apache::thrift::protocol::T_LIST) { - { - this->column_orders.clear(); - uint32_t _size186; - ::apache::thrift::protocol::TType _etype189; - xfer += iprot->readListBegin(_etype189, _size186); - this->column_orders.resize(_size186); - uint32_t _i190; - for (_i190 = 0; _i190 < _size186; ++_i190) - { - xfer += this->column_orders[_i190].read(iprot); - } - xfer += iprot->readListEnd(); - } - this->__isset.column_orders = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 8: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->encryption_algorithm.read(iprot); - this->__isset.encryption_algorithm = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 9: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->footer_signing_key_metadata); - this->__isset.footer_signing_key_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_version) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_schema) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_num_rows) - throw TProtocolException(TProtocolException::INVALID_DATA); - if (!isset_row_groups) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t FileMetaData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("FileMetaData"); - - xfer += oprot->writeFieldBegin("version", ::apache::thrift::protocol::T_I32, 1); - xfer += oprot->writeI32(this->version); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("schema", ::apache::thrift::protocol::T_LIST, 2); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->schema.size())); - std::vector ::const_iterator _iter191; - for (_iter191 = this->schema.begin(); _iter191 != this->schema.end(); ++_iter191) - { - xfer += (*_iter191).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("num_rows", ::apache::thrift::protocol::T_I64, 3); - xfer += oprot->writeI64(this->num_rows); - xfer += oprot->writeFieldEnd(); - - xfer += oprot->writeFieldBegin("row_groups", ::apache::thrift::protocol::T_LIST, 4); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->row_groups.size())); - std::vector ::const_iterator _iter192; - for (_iter192 = this->row_groups.begin(); _iter192 != this->row_groups.end(); ++_iter192) - { - xfer += (*_iter192).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - - if (this->__isset.key_value_metadata) { - xfer += oprot->writeFieldBegin("key_value_metadata", ::apache::thrift::protocol::T_LIST, 5); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->key_value_metadata.size())); - std::vector ::const_iterator _iter193; - for (_iter193 = this->key_value_metadata.begin(); _iter193 != this->key_value_metadata.end(); ++_iter193) - { - xfer += (*_iter193).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.created_by) { - xfer += oprot->writeFieldBegin("created_by", ::apache::thrift::protocol::T_STRING, 6); - xfer += oprot->writeString(this->created_by); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.column_orders) { - xfer += oprot->writeFieldBegin("column_orders", ::apache::thrift::protocol::T_LIST, 7); - { - xfer += oprot->writeListBegin(::apache::thrift::protocol::T_STRUCT, static_cast(this->column_orders.size())); - std::vector ::const_iterator _iter194; - for (_iter194 = this->column_orders.begin(); _iter194 != this->column_orders.end(); ++_iter194) - { - xfer += (*_iter194).write(oprot); - } - xfer += oprot->writeListEnd(); - } - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.encryption_algorithm) { - xfer += oprot->writeFieldBegin("encryption_algorithm", ::apache::thrift::protocol::T_STRUCT, 8); - xfer += this->encryption_algorithm.write(oprot); - xfer += oprot->writeFieldEnd(); - } - if (this->__isset.footer_signing_key_metadata) { - xfer += oprot->writeFieldBegin("footer_signing_key_metadata", ::apache::thrift::protocol::T_STRING, 9); - xfer += oprot->writeBinary(this->footer_signing_key_metadata); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(FileMetaData &a, FileMetaData &b) { - using ::std::swap; - swap(a.version, b.version); - swap(a.schema, b.schema); - swap(a.num_rows, b.num_rows); - swap(a.row_groups, b.row_groups); - swap(a.key_value_metadata, b.key_value_metadata); - swap(a.created_by, b.created_by); - swap(a.column_orders, b.column_orders); - swap(a.encryption_algorithm, b.encryption_algorithm); - swap(a.footer_signing_key_metadata, b.footer_signing_key_metadata); - swap(a.__isset, b.__isset); -} - -FileMetaData::FileMetaData(const FileMetaData& other195) { - version = other195.version; - schema = other195.schema; - num_rows = other195.num_rows; - row_groups = other195.row_groups; - key_value_metadata = other195.key_value_metadata; - created_by = other195.created_by; - column_orders = other195.column_orders; - encryption_algorithm = other195.encryption_algorithm; - footer_signing_key_metadata = other195.footer_signing_key_metadata; - __isset = other195.__isset; -} -FileMetaData& FileMetaData::operator=(const FileMetaData& other196) { - version = other196.version; - schema = other196.schema; - num_rows = other196.num_rows; - row_groups = other196.row_groups; - key_value_metadata = other196.key_value_metadata; - created_by = other196.created_by; - column_orders = other196.column_orders; - encryption_algorithm = other196.encryption_algorithm; - footer_signing_key_metadata = other196.footer_signing_key_metadata; - __isset = other196.__isset; - return *this; -} -void FileMetaData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "FileMetaData("; - out << "version=" << to_string(version); - out << ", " << "schema=" << to_string(schema); - out << ", " << "num_rows=" << to_string(num_rows); - out << ", " << "row_groups=" << to_string(row_groups); - out << ", " << "key_value_metadata="; (__isset.key_value_metadata ? (out << to_string(key_value_metadata)) : (out << "")); - out << ", " << "created_by="; (__isset.created_by ? (out << to_string(created_by)) : (out << "")); - out << ", " << "column_orders="; (__isset.column_orders ? (out << to_string(column_orders)) : (out << "")); - out << ", " << "encryption_algorithm="; (__isset.encryption_algorithm ? (out << to_string(encryption_algorithm)) : (out << "")); - out << ", " << "footer_signing_key_metadata="; (__isset.footer_signing_key_metadata ? (out << to_string(footer_signing_key_metadata)) : (out << "")); - out << ")"; -} - - -FileCryptoMetaData::~FileCryptoMetaData() throw() { -} - - -void FileCryptoMetaData::__set_encryption_algorithm(const EncryptionAlgorithm& val) { - this->encryption_algorithm = val; -} - -void FileCryptoMetaData::__set_key_metadata(const std::string& val) { - this->key_metadata = val; -__isset.key_metadata = true; -} -std::ostream& operator<<(std::ostream& out, const FileCryptoMetaData& obj) -{ - obj.printTo(out); - return out; -} - - -uint32_t FileCryptoMetaData::read(::apache::thrift::protocol::TProtocol* iprot) { - - ::apache::thrift::protocol::TInputRecursionTracker tracker(*iprot); - uint32_t xfer = 0; - std::string fname; - ::apache::thrift::protocol::TType ftype; - int16_t fid; - - xfer += iprot->readStructBegin(fname); - - using ::apache::thrift::protocol::TProtocolException; - - bool isset_encryption_algorithm = false; - - while (true) - { - xfer += iprot->readFieldBegin(fname, ftype, fid); - if (ftype == ::apache::thrift::protocol::T_STOP) { - break; - } - switch (fid) - { - case 1: - if (ftype == ::apache::thrift::protocol::T_STRUCT) { - xfer += this->encryption_algorithm.read(iprot); - isset_encryption_algorithm = true; - } else { - xfer += iprot->skip(ftype); - } - break; - case 2: - if (ftype == ::apache::thrift::protocol::T_STRING) { - xfer += iprot->readBinary(this->key_metadata); - this->__isset.key_metadata = true; - } else { - xfer += iprot->skip(ftype); - } - break; - default: - xfer += iprot->skip(ftype); - break; - } - xfer += iprot->readFieldEnd(); - } - - xfer += iprot->readStructEnd(); - - if (!isset_encryption_algorithm) - throw TProtocolException(TProtocolException::INVALID_DATA); - return xfer; -} - -uint32_t FileCryptoMetaData::write(::apache::thrift::protocol::TProtocol* oprot) const { - uint32_t xfer = 0; - ::apache::thrift::protocol::TOutputRecursionTracker tracker(*oprot); - xfer += oprot->writeStructBegin("FileCryptoMetaData"); - - xfer += oprot->writeFieldBegin("encryption_algorithm", ::apache::thrift::protocol::T_STRUCT, 1); - xfer += this->encryption_algorithm.write(oprot); - xfer += oprot->writeFieldEnd(); - - if (this->__isset.key_metadata) { - xfer += oprot->writeFieldBegin("key_metadata", ::apache::thrift::protocol::T_STRING, 2); - xfer += oprot->writeBinary(this->key_metadata); - xfer += oprot->writeFieldEnd(); - } - xfer += oprot->writeFieldStop(); - xfer += oprot->writeStructEnd(); - return xfer; -} - -void swap(FileCryptoMetaData &a, FileCryptoMetaData &b) { - using ::std::swap; - swap(a.encryption_algorithm, b.encryption_algorithm); - swap(a.key_metadata, b.key_metadata); - swap(a.__isset, b.__isset); -} - -FileCryptoMetaData::FileCryptoMetaData(const FileCryptoMetaData& other197) { - encryption_algorithm = other197.encryption_algorithm; - key_metadata = other197.key_metadata; - __isset = other197.__isset; -} -FileCryptoMetaData& FileCryptoMetaData::operator=(const FileCryptoMetaData& other198) { - encryption_algorithm = other198.encryption_algorithm; - key_metadata = other198.key_metadata; - __isset = other198.__isset; - return *this; -} -void FileCryptoMetaData::printTo(std::ostream& out) const { - using ::apache::thrift::to_string; - out << "FileCryptoMetaData("; - out << "encryption_algorithm=" << to_string(encryption_algorithm); - out << ", " << "key_metadata="; (__isset.key_metadata ? (out << to_string(key_metadata)) : (out << "")); - out << ")"; -} - -}} // namespace diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.h b/contrib/arrow-cmake/cpp/src/parquet/parquet_types.h deleted file mode 100644 index 65e9e8226bb..00000000000 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_types.h +++ /dev/null @@ -1,2602 +0,0 @@ -/** - * Autogenerated by Thrift Compiler (0.12.0) - * - * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING - * @generated - */ -#ifndef parquet_TYPES_H -#define parquet_TYPES_H - -#include - -#include -#include -#include -#include -#include - -#include - -#include "parquet/windows_compatibility.h" - -namespace parquet { namespace format { - -struct Type { - enum type { - BOOLEAN = 0, - INT32 = 1, - INT64 = 2, - INT96 = 3, - FLOAT = 4, - DOUBLE = 5, - BYTE_ARRAY = 6, - FIXED_LEN_BYTE_ARRAY = 7 - }; -}; - -extern const std::map _Type_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const Type::type& val); - -struct ConvertedType { - enum type { - UTF8 = 0, - MAP = 1, - MAP_KEY_VALUE = 2, - LIST = 3, - ENUM = 4, - DECIMAL = 5, - DATE = 6, - TIME_MILLIS = 7, - TIME_MICROS = 8, - TIMESTAMP_MILLIS = 9, - TIMESTAMP_MICROS = 10, - UINT_8 = 11, - UINT_16 = 12, - UINT_32 = 13, - UINT_64 = 14, - INT_8 = 15, - INT_16 = 16, - INT_32 = 17, - INT_64 = 18, - JSON = 19, - BSON = 20, - INTERVAL = 21 - }; -}; - -extern const std::map _ConvertedType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const ConvertedType::type& val); - -struct FieldRepetitionType { - enum type { - REQUIRED = 0, - OPTIONAL = 1, - REPEATED = 2 - }; -}; - -extern const std::map _FieldRepetitionType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const FieldRepetitionType::type& val); - -struct Encoding { - enum type { - PLAIN = 0, - PLAIN_DICTIONARY = 2, - RLE = 3, - BIT_PACKED = 4, - DELTA_BINARY_PACKED = 5, - DELTA_LENGTH_BYTE_ARRAY = 6, - DELTA_BYTE_ARRAY = 7, - RLE_DICTIONARY = 8 - }; -}; - -extern const std::map _Encoding_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const Encoding::type& val); - -struct CompressionCodec { - enum type { - UNCOMPRESSED = 0, - SNAPPY = 1, - GZIP = 2, - LZO = 3, - BROTLI = 4, - LZ4 = 5, - ZSTD = 6 - }; -}; - -extern const std::map _CompressionCodec_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const CompressionCodec::type& val); - -struct PageType { - enum type { - DATA_PAGE = 0, - INDEX_PAGE = 1, - DICTIONARY_PAGE = 2, - DATA_PAGE_V2 = 3 - }; -}; - -extern const std::map _PageType_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const PageType::type& val); - -struct BoundaryOrder { - enum type { - UNORDERED = 0, - ASCENDING = 1, - DESCENDING = 2 - }; -}; - -extern const std::map _BoundaryOrder_VALUES_TO_NAMES; - -std::ostream& operator<<(std::ostream& out, const BoundaryOrder::type& val); - -class Statistics; - -class StringType; - -class UUIDType; - -class MapType; - -class ListType; - -class EnumType; - -class DateType; - -class NullType; - -class DecimalType; - -class MilliSeconds; - -class MicroSeconds; - -class NanoSeconds; - -class TimeUnit; - -class TimestampType; - -class TimeType; - -class IntType; - -class JsonType; - -class BsonType; - -class LogicalType; - -class SchemaElement; - -class DataPageHeader; - -class IndexPageHeader; - -class DictionaryPageHeader; - -class DataPageHeaderV2; - -class PageHeader; - -class KeyValue; - -class SortingColumn; - -class PageEncodingStats; - -class ColumnMetaData; - -class EncryptionWithFooterKey; - -class EncryptionWithColumnKey; - -class ColumnCryptoMetaData; - -class ColumnChunk; - -class RowGroup; - -class TypeDefinedOrder; - -class ColumnOrder; - -class PageLocation; - -class OffsetIndex; - -class ColumnIndex; - -class AesGcmV1; - -class AesGcmCtrV1; - -class EncryptionAlgorithm; - -class FileMetaData; - -class FileCryptoMetaData; - -typedef struct _Statistics__isset { - _Statistics__isset() : max(false), min(false), null_count(false), distinct_count(false), max_value(false), min_value(false) {} - bool max :1; - bool min :1; - bool null_count :1; - bool distinct_count :1; - bool max_value :1; - bool min_value :1; -} _Statistics__isset; - -class Statistics : public virtual ::apache::thrift::TBase { - public: - - Statistics(const Statistics&); - Statistics& operator=(const Statistics&); - Statistics() : max(), min(), null_count(0), distinct_count(0), max_value(), min_value() { - } - - virtual ~Statistics() throw(); - std::string max; - std::string min; - int64_t null_count; - int64_t distinct_count; - std::string max_value; - std::string min_value; - - _Statistics__isset __isset; - - void __set_max(const std::string& val); - - void __set_min(const std::string& val); - - void __set_null_count(const int64_t val); - - void __set_distinct_count(const int64_t val); - - void __set_max_value(const std::string& val); - - void __set_min_value(const std::string& val); - - bool operator == (const Statistics & rhs) const - { - if (__isset.max != rhs.__isset.max) - return false; - else if (__isset.max && !(max == rhs.max)) - return false; - if (__isset.min != rhs.__isset.min) - return false; - else if (__isset.min && !(min == rhs.min)) - return false; - if (__isset.null_count != rhs.__isset.null_count) - return false; - else if (__isset.null_count && !(null_count == rhs.null_count)) - return false; - if (__isset.distinct_count != rhs.__isset.distinct_count) - return false; - else if (__isset.distinct_count && !(distinct_count == rhs.distinct_count)) - return false; - if (__isset.max_value != rhs.__isset.max_value) - return false; - else if (__isset.max_value && !(max_value == rhs.max_value)) - return false; - if (__isset.min_value != rhs.__isset.min_value) - return false; - else if (__isset.min_value && !(min_value == rhs.min_value)) - return false; - return true; - } - bool operator != (const Statistics &rhs) const { - return !(*this == rhs); - } - - bool operator < (const Statistics & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(Statistics &a, Statistics &b); - -std::ostream& operator<<(std::ostream& out, const Statistics& obj); - - -class StringType : public virtual ::apache::thrift::TBase { - public: - - StringType(const StringType&); - StringType& operator=(const StringType&); - StringType() { - } - - virtual ~StringType() throw(); - - bool operator == (const StringType & /* rhs */) const - { - return true; - } - bool operator != (const StringType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const StringType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(StringType &a, StringType &b); - -std::ostream& operator<<(std::ostream& out, const StringType& obj); - - -class UUIDType : public virtual ::apache::thrift::TBase { - public: - - UUIDType(const UUIDType&); - UUIDType& operator=(const UUIDType&); - UUIDType() { - } - - virtual ~UUIDType() throw(); - - bool operator == (const UUIDType & /* rhs */) const - { - return true; - } - bool operator != (const UUIDType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const UUIDType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(UUIDType &a, UUIDType &b); - -std::ostream& operator<<(std::ostream& out, const UUIDType& obj); - - -class MapType : public virtual ::apache::thrift::TBase { - public: - - MapType(const MapType&); - MapType& operator=(const MapType&); - MapType() { - } - - virtual ~MapType() throw(); - - bool operator == (const MapType & /* rhs */) const - { - return true; - } - bool operator != (const MapType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const MapType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(MapType &a, MapType &b); - -std::ostream& operator<<(std::ostream& out, const MapType& obj); - - -class ListType : public virtual ::apache::thrift::TBase { - public: - - ListType(const ListType&); - ListType& operator=(const ListType&); - ListType() { - } - - virtual ~ListType() throw(); - - bool operator == (const ListType & /* rhs */) const - { - return true; - } - bool operator != (const ListType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ListType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(ListType &a, ListType &b); - -std::ostream& operator<<(std::ostream& out, const ListType& obj); - - -class EnumType : public virtual ::apache::thrift::TBase { - public: - - EnumType(const EnumType&); - EnumType& operator=(const EnumType&); - EnumType() { - } - - virtual ~EnumType() throw(); - - bool operator == (const EnumType & /* rhs */) const - { - return true; - } - bool operator != (const EnumType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const EnumType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(EnumType &a, EnumType &b); - -std::ostream& operator<<(std::ostream& out, const EnumType& obj); - - -class DateType : public virtual ::apache::thrift::TBase { - public: - - DateType(const DateType&); - DateType& operator=(const DateType&); - DateType() { - } - - virtual ~DateType() throw(); - - bool operator == (const DateType & /* rhs */) const - { - return true; - } - bool operator != (const DateType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DateType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(DateType &a, DateType &b); - -std::ostream& operator<<(std::ostream& out, const DateType& obj); - - -class NullType : public virtual ::apache::thrift::TBase { - public: - - NullType(const NullType&); - NullType& operator=(const NullType&); - NullType() { - } - - virtual ~NullType() throw(); - - bool operator == (const NullType & /* rhs */) const - { - return true; - } - bool operator != (const NullType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NullType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(NullType &a, NullType &b); - -std::ostream& operator<<(std::ostream& out, const NullType& obj); - - -class DecimalType : public virtual ::apache::thrift::TBase { - public: - - DecimalType(const DecimalType&); - DecimalType& operator=(const DecimalType&); - DecimalType() : scale(0), precision(0) { - } - - virtual ~DecimalType() throw(); - int32_t scale; - int32_t precision; - - void __set_scale(const int32_t val); - - void __set_precision(const int32_t val); - - bool operator == (const DecimalType & rhs) const - { - if (!(scale == rhs.scale)) - return false; - if (!(precision == rhs.precision)) - return false; - return true; - } - bool operator != (const DecimalType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DecimalType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(DecimalType &a, DecimalType &b); - -std::ostream& operator<<(std::ostream& out, const DecimalType& obj); - - -class MilliSeconds : public virtual ::apache::thrift::TBase { - public: - - MilliSeconds(const MilliSeconds&); - MilliSeconds& operator=(const MilliSeconds&); - MilliSeconds() { - } - - virtual ~MilliSeconds() throw(); - - bool operator == (const MilliSeconds & /* rhs */) const - { - return true; - } - bool operator != (const MilliSeconds &rhs) const { - return !(*this == rhs); - } - - bool operator < (const MilliSeconds & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(MilliSeconds &a, MilliSeconds &b); - -std::ostream& operator<<(std::ostream& out, const MilliSeconds& obj); - - -class MicroSeconds : public virtual ::apache::thrift::TBase { - public: - - MicroSeconds(const MicroSeconds&); - MicroSeconds& operator=(const MicroSeconds&); - MicroSeconds() { - } - - virtual ~MicroSeconds() throw(); - - bool operator == (const MicroSeconds & /* rhs */) const - { - return true; - } - bool operator != (const MicroSeconds &rhs) const { - return !(*this == rhs); - } - - bool operator < (const MicroSeconds & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(MicroSeconds &a, MicroSeconds &b); - -std::ostream& operator<<(std::ostream& out, const MicroSeconds& obj); - - -class NanoSeconds : public virtual ::apache::thrift::TBase { - public: - - NanoSeconds(const NanoSeconds&); - NanoSeconds& operator=(const NanoSeconds&); - NanoSeconds() { - } - - virtual ~NanoSeconds() throw(); - - bool operator == (const NanoSeconds & /* rhs */) const - { - return true; - } - bool operator != (const NanoSeconds &rhs) const { - return !(*this == rhs); - } - - bool operator < (const NanoSeconds & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(NanoSeconds &a, NanoSeconds &b); - -std::ostream& operator<<(std::ostream& out, const NanoSeconds& obj); - -typedef struct _TimeUnit__isset { - _TimeUnit__isset() : MILLIS(false), MICROS(false), NANOS(false) {} - bool MILLIS :1; - bool MICROS :1; - bool NANOS :1; -} _TimeUnit__isset; - -class TimeUnit : public virtual ::apache::thrift::TBase { - public: - - TimeUnit(const TimeUnit&); - TimeUnit& operator=(const TimeUnit&); - TimeUnit() { - } - - virtual ~TimeUnit() throw(); - MilliSeconds MILLIS; - MicroSeconds MICROS; - NanoSeconds NANOS; - - _TimeUnit__isset __isset; - - void __set_MILLIS(const MilliSeconds& val); - - void __set_MICROS(const MicroSeconds& val); - - void __set_NANOS(const NanoSeconds& val); - - bool operator == (const TimeUnit & rhs) const - { - if (__isset.MILLIS != rhs.__isset.MILLIS) - return false; - else if (__isset.MILLIS && !(MILLIS == rhs.MILLIS)) - return false; - if (__isset.MICROS != rhs.__isset.MICROS) - return false; - else if (__isset.MICROS && !(MICROS == rhs.MICROS)) - return false; - if (__isset.NANOS != rhs.__isset.NANOS) - return false; - else if (__isset.NANOS && !(NANOS == rhs.NANOS)) - return false; - return true; - } - bool operator != (const TimeUnit &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TimeUnit & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(TimeUnit &a, TimeUnit &b); - -std::ostream& operator<<(std::ostream& out, const TimeUnit& obj); - - -class TimestampType : public virtual ::apache::thrift::TBase { - public: - - TimestampType(const TimestampType&); - TimestampType& operator=(const TimestampType&); - TimestampType() : isAdjustedToUTC(0) { - } - - virtual ~TimestampType() throw(); - bool isAdjustedToUTC; - TimeUnit unit; - - void __set_isAdjustedToUTC(const bool val); - - void __set_unit(const TimeUnit& val); - - bool operator == (const TimestampType & rhs) const - { - if (!(isAdjustedToUTC == rhs.isAdjustedToUTC)) - return false; - if (!(unit == rhs.unit)) - return false; - return true; - } - bool operator != (const TimestampType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TimestampType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(TimestampType &a, TimestampType &b); - -std::ostream& operator<<(std::ostream& out, const TimestampType& obj); - - -class TimeType : public virtual ::apache::thrift::TBase { - public: - - TimeType(const TimeType&); - TimeType& operator=(const TimeType&); - TimeType() : isAdjustedToUTC(0) { - } - - virtual ~TimeType() throw(); - bool isAdjustedToUTC; - TimeUnit unit; - - void __set_isAdjustedToUTC(const bool val); - - void __set_unit(const TimeUnit& val); - - bool operator == (const TimeType & rhs) const - { - if (!(isAdjustedToUTC == rhs.isAdjustedToUTC)) - return false; - if (!(unit == rhs.unit)) - return false; - return true; - } - bool operator != (const TimeType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TimeType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(TimeType &a, TimeType &b); - -std::ostream& operator<<(std::ostream& out, const TimeType& obj); - - -class IntType : public virtual ::apache::thrift::TBase { - public: - - IntType(const IntType&); - IntType& operator=(const IntType&); - IntType() : bitWidth(0), isSigned(0) { - } - - virtual ~IntType() throw(); - int8_t bitWidth; - bool isSigned; - - void __set_bitWidth(const int8_t val); - - void __set_isSigned(const bool val); - - bool operator == (const IntType & rhs) const - { - if (!(bitWidth == rhs.bitWidth)) - return false; - if (!(isSigned == rhs.isSigned)) - return false; - return true; - } - bool operator != (const IntType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const IntType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(IntType &a, IntType &b); - -std::ostream& operator<<(std::ostream& out, const IntType& obj); - - -class JsonType : public virtual ::apache::thrift::TBase { - public: - - JsonType(const JsonType&); - JsonType& operator=(const JsonType&); - JsonType() { - } - - virtual ~JsonType() throw(); - - bool operator == (const JsonType & /* rhs */) const - { - return true; - } - bool operator != (const JsonType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const JsonType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(JsonType &a, JsonType &b); - -std::ostream& operator<<(std::ostream& out, const JsonType& obj); - - -class BsonType : public virtual ::apache::thrift::TBase { - public: - - BsonType(const BsonType&); - BsonType& operator=(const BsonType&); - BsonType() { - } - - virtual ~BsonType() throw(); - - bool operator == (const BsonType & /* rhs */) const - { - return true; - } - bool operator != (const BsonType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const BsonType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(BsonType &a, BsonType &b); - -std::ostream& operator<<(std::ostream& out, const BsonType& obj); - -typedef struct _LogicalType__isset { - _LogicalType__isset() : STRING(false), MAP(false), LIST(false), ENUM(false), DECIMAL(false), DATE(false), TIME(false), TIMESTAMP(false), INTEGER(false), UNKNOWN(false), JSON(false), BSON(false), UUID(false) {} - bool STRING :1; - bool MAP :1; - bool LIST :1; - bool ENUM :1; - bool DECIMAL :1; - bool DATE :1; - bool TIME :1; - bool TIMESTAMP :1; - bool INTEGER :1; - bool UNKNOWN :1; - bool JSON :1; - bool BSON :1; - bool UUID :1; -} _LogicalType__isset; - -class LogicalType : public virtual ::apache::thrift::TBase { - public: - - LogicalType(const LogicalType&); - LogicalType& operator=(const LogicalType&); - LogicalType() { - } - - virtual ~LogicalType() throw(); - StringType STRING; - MapType MAP; - ListType LIST; - EnumType ENUM; - DecimalType DECIMAL; - DateType DATE; - TimeType TIME; - TimestampType TIMESTAMP; - IntType INTEGER; - NullType UNKNOWN; - JsonType JSON; - BsonType BSON; - UUIDType UUID; - - _LogicalType__isset __isset; - - void __set_STRING(const StringType& val); - - void __set_MAP(const MapType& val); - - void __set_LIST(const ListType& val); - - void __set_ENUM(const EnumType& val); - - void __set_DECIMAL(const DecimalType& val); - - void __set_DATE(const DateType& val); - - void __set_TIME(const TimeType& val); - - void __set_TIMESTAMP(const TimestampType& val); - - void __set_INTEGER(const IntType& val); - - void __set_UNKNOWN(const NullType& val); - - void __set_JSON(const JsonType& val); - - void __set_BSON(const BsonType& val); - - void __set_UUID(const UUIDType& val); - - bool operator == (const LogicalType & rhs) const - { - if (__isset.STRING != rhs.__isset.STRING) - return false; - else if (__isset.STRING && !(STRING == rhs.STRING)) - return false; - if (__isset.MAP != rhs.__isset.MAP) - return false; - else if (__isset.MAP && !(MAP == rhs.MAP)) - return false; - if (__isset.LIST != rhs.__isset.LIST) - return false; - else if (__isset.LIST && !(LIST == rhs.LIST)) - return false; - if (__isset.ENUM != rhs.__isset.ENUM) - return false; - else if (__isset.ENUM && !(ENUM == rhs.ENUM)) - return false; - if (__isset.DECIMAL != rhs.__isset.DECIMAL) - return false; - else if (__isset.DECIMAL && !(DECIMAL == rhs.DECIMAL)) - return false; - if (__isset.DATE != rhs.__isset.DATE) - return false; - else if (__isset.DATE && !(DATE == rhs.DATE)) - return false; - if (__isset.TIME != rhs.__isset.TIME) - return false; - else if (__isset.TIME && !(TIME == rhs.TIME)) - return false; - if (__isset.TIMESTAMP != rhs.__isset.TIMESTAMP) - return false; - else if (__isset.TIMESTAMP && !(TIMESTAMP == rhs.TIMESTAMP)) - return false; - if (__isset.INTEGER != rhs.__isset.INTEGER) - return false; - else if (__isset.INTEGER && !(INTEGER == rhs.INTEGER)) - return false; - if (__isset.UNKNOWN != rhs.__isset.UNKNOWN) - return false; - else if (__isset.UNKNOWN && !(UNKNOWN == rhs.UNKNOWN)) - return false; - if (__isset.JSON != rhs.__isset.JSON) - return false; - else if (__isset.JSON && !(JSON == rhs.JSON)) - return false; - if (__isset.BSON != rhs.__isset.BSON) - return false; - else if (__isset.BSON && !(BSON == rhs.BSON)) - return false; - if (__isset.UUID != rhs.__isset.UUID) - return false; - else if (__isset.UUID && !(UUID == rhs.UUID)) - return false; - return true; - } - bool operator != (const LogicalType &rhs) const { - return !(*this == rhs); - } - - bool operator < (const LogicalType & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(LogicalType &a, LogicalType &b); - -std::ostream& operator<<(std::ostream& out, const LogicalType& obj); - -typedef struct _SchemaElement__isset { - _SchemaElement__isset() : type(false), type_length(false), repetition_type(false), num_children(false), converted_type(false), scale(false), precision(false), field_id(false), logicalType(false) {} - bool type :1; - bool type_length :1; - bool repetition_type :1; - bool num_children :1; - bool converted_type :1; - bool scale :1; - bool precision :1; - bool field_id :1; - bool logicalType :1; -} _SchemaElement__isset; - -class SchemaElement : public virtual ::apache::thrift::TBase { - public: - - SchemaElement(const SchemaElement&); - SchemaElement& operator=(const SchemaElement&); - SchemaElement() : type((Type::type)0), type_length(0), repetition_type((FieldRepetitionType::type)0), name(), num_children(0), converted_type((ConvertedType::type)0), scale(0), precision(0), field_id(0) { - } - - virtual ~SchemaElement() throw(); - Type::type type; - int32_t type_length; - FieldRepetitionType::type repetition_type; - std::string name; - int32_t num_children; - ConvertedType::type converted_type; - int32_t scale; - int32_t precision; - int32_t field_id; - LogicalType logicalType; - - _SchemaElement__isset __isset; - - void __set_type(const Type::type val); - - void __set_type_length(const int32_t val); - - void __set_repetition_type(const FieldRepetitionType::type val); - - void __set_name(const std::string& val); - - void __set_num_children(const int32_t val); - - void __set_converted_type(const ConvertedType::type val); - - void __set_scale(const int32_t val); - - void __set_precision(const int32_t val); - - void __set_field_id(const int32_t val); - - void __set_logicalType(const LogicalType& val); - - bool operator == (const SchemaElement & rhs) const - { - if (__isset.type != rhs.__isset.type) - return false; - else if (__isset.type && !(type == rhs.type)) - return false; - if (__isset.type_length != rhs.__isset.type_length) - return false; - else if (__isset.type_length && !(type_length == rhs.type_length)) - return false; - if (__isset.repetition_type != rhs.__isset.repetition_type) - return false; - else if (__isset.repetition_type && !(repetition_type == rhs.repetition_type)) - return false; - if (!(name == rhs.name)) - return false; - if (__isset.num_children != rhs.__isset.num_children) - return false; - else if (__isset.num_children && !(num_children == rhs.num_children)) - return false; - if (__isset.converted_type != rhs.__isset.converted_type) - return false; - else if (__isset.converted_type && !(converted_type == rhs.converted_type)) - return false; - if (__isset.scale != rhs.__isset.scale) - return false; - else if (__isset.scale && !(scale == rhs.scale)) - return false; - if (__isset.precision != rhs.__isset.precision) - return false; - else if (__isset.precision && !(precision == rhs.precision)) - return false; - if (__isset.field_id != rhs.__isset.field_id) - return false; - else if (__isset.field_id && !(field_id == rhs.field_id)) - return false; - if (__isset.logicalType != rhs.__isset.logicalType) - return false; - else if (__isset.logicalType && !(logicalType == rhs.logicalType)) - return false; - return true; - } - bool operator != (const SchemaElement &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SchemaElement & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(SchemaElement &a, SchemaElement &b); - -std::ostream& operator<<(std::ostream& out, const SchemaElement& obj); - -typedef struct _DataPageHeader__isset { - _DataPageHeader__isset() : statistics(false) {} - bool statistics :1; -} _DataPageHeader__isset; - -class DataPageHeader : public virtual ::apache::thrift::TBase { - public: - - DataPageHeader(const DataPageHeader&); - DataPageHeader& operator=(const DataPageHeader&); - DataPageHeader() : num_values(0), encoding((Encoding::type)0), definition_level_encoding((Encoding::type)0), repetition_level_encoding((Encoding::type)0) { - } - - virtual ~DataPageHeader() throw(); - int32_t num_values; - Encoding::type encoding; - Encoding::type definition_level_encoding; - Encoding::type repetition_level_encoding; - Statistics statistics; - - _DataPageHeader__isset __isset; - - void __set_num_values(const int32_t val); - - void __set_encoding(const Encoding::type val); - - void __set_definition_level_encoding(const Encoding::type val); - - void __set_repetition_level_encoding(const Encoding::type val); - - void __set_statistics(const Statistics& val); - - bool operator == (const DataPageHeader & rhs) const - { - if (!(num_values == rhs.num_values)) - return false; - if (!(encoding == rhs.encoding)) - return false; - if (!(definition_level_encoding == rhs.definition_level_encoding)) - return false; - if (!(repetition_level_encoding == rhs.repetition_level_encoding)) - return false; - if (__isset.statistics != rhs.__isset.statistics) - return false; - else if (__isset.statistics && !(statistics == rhs.statistics)) - return false; - return true; - } - bool operator != (const DataPageHeader &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DataPageHeader & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(DataPageHeader &a, DataPageHeader &b); - -std::ostream& operator<<(std::ostream& out, const DataPageHeader& obj); - - -class IndexPageHeader : public virtual ::apache::thrift::TBase { - public: - - IndexPageHeader(const IndexPageHeader&); - IndexPageHeader& operator=(const IndexPageHeader&); - IndexPageHeader() { - } - - virtual ~IndexPageHeader() throw(); - - bool operator == (const IndexPageHeader & /* rhs */) const - { - return true; - } - bool operator != (const IndexPageHeader &rhs) const { - return !(*this == rhs); - } - - bool operator < (const IndexPageHeader & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(IndexPageHeader &a, IndexPageHeader &b); - -std::ostream& operator<<(std::ostream& out, const IndexPageHeader& obj); - -typedef struct _DictionaryPageHeader__isset { - _DictionaryPageHeader__isset() : is_sorted(false) {} - bool is_sorted :1; -} _DictionaryPageHeader__isset; - -class DictionaryPageHeader : public virtual ::apache::thrift::TBase { - public: - - DictionaryPageHeader(const DictionaryPageHeader&); - DictionaryPageHeader& operator=(const DictionaryPageHeader&); - DictionaryPageHeader() : num_values(0), encoding((Encoding::type)0), is_sorted(0) { - } - - virtual ~DictionaryPageHeader() throw(); - int32_t num_values; - Encoding::type encoding; - bool is_sorted; - - _DictionaryPageHeader__isset __isset; - - void __set_num_values(const int32_t val); - - void __set_encoding(const Encoding::type val); - - void __set_is_sorted(const bool val); - - bool operator == (const DictionaryPageHeader & rhs) const - { - if (!(num_values == rhs.num_values)) - return false; - if (!(encoding == rhs.encoding)) - return false; - if (__isset.is_sorted != rhs.__isset.is_sorted) - return false; - else if (__isset.is_sorted && !(is_sorted == rhs.is_sorted)) - return false; - return true; - } - bool operator != (const DictionaryPageHeader &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DictionaryPageHeader & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(DictionaryPageHeader &a, DictionaryPageHeader &b); - -std::ostream& operator<<(std::ostream& out, const DictionaryPageHeader& obj); - -typedef struct _DataPageHeaderV2__isset { - _DataPageHeaderV2__isset() : is_compressed(true), statistics(false) {} - bool is_compressed :1; - bool statistics :1; -} _DataPageHeaderV2__isset; - -class DataPageHeaderV2 : public virtual ::apache::thrift::TBase { - public: - - DataPageHeaderV2(const DataPageHeaderV2&); - DataPageHeaderV2& operator=(const DataPageHeaderV2&); - DataPageHeaderV2() : num_values(0), num_nulls(0), num_rows(0), encoding((Encoding::type)0), definition_levels_byte_length(0), repetition_levels_byte_length(0), is_compressed(true) { - } - - virtual ~DataPageHeaderV2() throw(); - int32_t num_values; - int32_t num_nulls; - int32_t num_rows; - Encoding::type encoding; - int32_t definition_levels_byte_length; - int32_t repetition_levels_byte_length; - bool is_compressed; - Statistics statistics; - - _DataPageHeaderV2__isset __isset; - - void __set_num_values(const int32_t val); - - void __set_num_nulls(const int32_t val); - - void __set_num_rows(const int32_t val); - - void __set_encoding(const Encoding::type val); - - void __set_definition_levels_byte_length(const int32_t val); - - void __set_repetition_levels_byte_length(const int32_t val); - - void __set_is_compressed(const bool val); - - void __set_statistics(const Statistics& val); - - bool operator == (const DataPageHeaderV2 & rhs) const - { - if (!(num_values == rhs.num_values)) - return false; - if (!(num_nulls == rhs.num_nulls)) - return false; - if (!(num_rows == rhs.num_rows)) - return false; - if (!(encoding == rhs.encoding)) - return false; - if (!(definition_levels_byte_length == rhs.definition_levels_byte_length)) - return false; - if (!(repetition_levels_byte_length == rhs.repetition_levels_byte_length)) - return false; - if (__isset.is_compressed != rhs.__isset.is_compressed) - return false; - else if (__isset.is_compressed && !(is_compressed == rhs.is_compressed)) - return false; - if (__isset.statistics != rhs.__isset.statistics) - return false; - else if (__isset.statistics && !(statistics == rhs.statistics)) - return false; - return true; - } - bool operator != (const DataPageHeaderV2 &rhs) const { - return !(*this == rhs); - } - - bool operator < (const DataPageHeaderV2 & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(DataPageHeaderV2 &a, DataPageHeaderV2 &b); - -std::ostream& operator<<(std::ostream& out, const DataPageHeaderV2& obj); - -typedef struct _PageHeader__isset { - _PageHeader__isset() : crc(false), data_page_header(false), index_page_header(false), dictionary_page_header(false), data_page_header_v2(false) {} - bool crc :1; - bool data_page_header :1; - bool index_page_header :1; - bool dictionary_page_header :1; - bool data_page_header_v2 :1; -} _PageHeader__isset; - -class PageHeader : public virtual ::apache::thrift::TBase { - public: - - PageHeader(const PageHeader&); - PageHeader& operator=(const PageHeader&); - PageHeader() : type((PageType::type)0), uncompressed_page_size(0), compressed_page_size(0), crc(0) { - } - - virtual ~PageHeader() throw(); - PageType::type type; - int32_t uncompressed_page_size; - int32_t compressed_page_size; - int32_t crc; - DataPageHeader data_page_header; - IndexPageHeader index_page_header; - DictionaryPageHeader dictionary_page_header; - DataPageHeaderV2 data_page_header_v2; - - _PageHeader__isset __isset; - - void __set_type(const PageType::type val); - - void __set_uncompressed_page_size(const int32_t val); - - void __set_compressed_page_size(const int32_t val); - - void __set_crc(const int32_t val); - - void __set_data_page_header(const DataPageHeader& val); - - void __set_index_page_header(const IndexPageHeader& val); - - void __set_dictionary_page_header(const DictionaryPageHeader& val); - - void __set_data_page_header_v2(const DataPageHeaderV2& val); - - bool operator == (const PageHeader & rhs) const - { - if (!(type == rhs.type)) - return false; - if (!(uncompressed_page_size == rhs.uncompressed_page_size)) - return false; - if (!(compressed_page_size == rhs.compressed_page_size)) - return false; - if (__isset.crc != rhs.__isset.crc) - return false; - else if (__isset.crc && !(crc == rhs.crc)) - return false; - if (__isset.data_page_header != rhs.__isset.data_page_header) - return false; - else if (__isset.data_page_header && !(data_page_header == rhs.data_page_header)) - return false; - if (__isset.index_page_header != rhs.__isset.index_page_header) - return false; - else if (__isset.index_page_header && !(index_page_header == rhs.index_page_header)) - return false; - if (__isset.dictionary_page_header != rhs.__isset.dictionary_page_header) - return false; - else if (__isset.dictionary_page_header && !(dictionary_page_header == rhs.dictionary_page_header)) - return false; - if (__isset.data_page_header_v2 != rhs.__isset.data_page_header_v2) - return false; - else if (__isset.data_page_header_v2 && !(data_page_header_v2 == rhs.data_page_header_v2)) - return false; - return true; - } - bool operator != (const PageHeader &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PageHeader & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(PageHeader &a, PageHeader &b); - -std::ostream& operator<<(std::ostream& out, const PageHeader& obj); - -typedef struct _KeyValue__isset { - _KeyValue__isset() : value(false) {} - bool value :1; -} _KeyValue__isset; - -class KeyValue : public virtual ::apache::thrift::TBase { - public: - - KeyValue(const KeyValue&); - KeyValue& operator=(const KeyValue&); - KeyValue() : key(), value() { - } - - virtual ~KeyValue() throw(); - std::string key; - std::string value; - - _KeyValue__isset __isset; - - void __set_key(const std::string& val); - - void __set_value(const std::string& val); - - bool operator == (const KeyValue & rhs) const - { - if (!(key == rhs.key)) - return false; - if (__isset.value != rhs.__isset.value) - return false; - else if (__isset.value && !(value == rhs.value)) - return false; - return true; - } - bool operator != (const KeyValue &rhs) const { - return !(*this == rhs); - } - - bool operator < (const KeyValue & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(KeyValue &a, KeyValue &b); - -std::ostream& operator<<(std::ostream& out, const KeyValue& obj); - - -class SortingColumn : public virtual ::apache::thrift::TBase { - public: - - SortingColumn(const SortingColumn&); - SortingColumn& operator=(const SortingColumn&); - SortingColumn() : column_idx(0), descending(0), nulls_first(0) { - } - - virtual ~SortingColumn() throw(); - int32_t column_idx; - bool descending; - bool nulls_first; - - void __set_column_idx(const int32_t val); - - void __set_descending(const bool val); - - void __set_nulls_first(const bool val); - - bool operator == (const SortingColumn & rhs) const - { - if (!(column_idx == rhs.column_idx)) - return false; - if (!(descending == rhs.descending)) - return false; - if (!(nulls_first == rhs.nulls_first)) - return false; - return true; - } - bool operator != (const SortingColumn &rhs) const { - return !(*this == rhs); - } - - bool operator < (const SortingColumn & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(SortingColumn &a, SortingColumn &b); - -std::ostream& operator<<(std::ostream& out, const SortingColumn& obj); - - -class PageEncodingStats : public virtual ::apache::thrift::TBase { - public: - - PageEncodingStats(const PageEncodingStats&); - PageEncodingStats& operator=(const PageEncodingStats&); - PageEncodingStats() : page_type((PageType::type)0), encoding((Encoding::type)0), count(0) { - } - - virtual ~PageEncodingStats() throw(); - PageType::type page_type; - Encoding::type encoding; - int32_t count; - - void __set_page_type(const PageType::type val); - - void __set_encoding(const Encoding::type val); - - void __set_count(const int32_t val); - - bool operator == (const PageEncodingStats & rhs) const - { - if (!(page_type == rhs.page_type)) - return false; - if (!(encoding == rhs.encoding)) - return false; - if (!(count == rhs.count)) - return false; - return true; - } - bool operator != (const PageEncodingStats &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PageEncodingStats & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(PageEncodingStats &a, PageEncodingStats &b); - -std::ostream& operator<<(std::ostream& out, const PageEncodingStats& obj); - -typedef struct _ColumnMetaData__isset { - _ColumnMetaData__isset() : key_value_metadata(false), index_page_offset(false), dictionary_page_offset(false), statistics(false), encoding_stats(false) {} - bool key_value_metadata :1; - bool index_page_offset :1; - bool dictionary_page_offset :1; - bool statistics :1; - bool encoding_stats :1; -} _ColumnMetaData__isset; - -class ColumnMetaData : public virtual ::apache::thrift::TBase { - public: - - ColumnMetaData(const ColumnMetaData&); - ColumnMetaData& operator=(const ColumnMetaData&); - ColumnMetaData() : type((Type::type)0), codec((CompressionCodec::type)0), num_values(0), total_uncompressed_size(0), total_compressed_size(0), data_page_offset(0), index_page_offset(0), dictionary_page_offset(0) { - } - - virtual ~ColumnMetaData() throw(); - Type::type type; - std::vector encodings; - std::vector path_in_schema; - CompressionCodec::type codec; - int64_t num_values; - int64_t total_uncompressed_size; - int64_t total_compressed_size; - std::vector key_value_metadata; - int64_t data_page_offset; - int64_t index_page_offset; - int64_t dictionary_page_offset; - Statistics statistics; - std::vector encoding_stats; - - _ColumnMetaData__isset __isset; - - void __set_type(const Type::type val); - - void __set_encodings(const std::vector & val); - - void __set_path_in_schema(const std::vector & val); - - void __set_codec(const CompressionCodec::type val); - - void __set_num_values(const int64_t val); - - void __set_total_uncompressed_size(const int64_t val); - - void __set_total_compressed_size(const int64_t val); - - void __set_key_value_metadata(const std::vector & val); - - void __set_data_page_offset(const int64_t val); - - void __set_index_page_offset(const int64_t val); - - void __set_dictionary_page_offset(const int64_t val); - - void __set_statistics(const Statistics& val); - - void __set_encoding_stats(const std::vector & val); - - bool operator == (const ColumnMetaData & rhs) const - { - if (!(type == rhs.type)) - return false; - if (!(encodings == rhs.encodings)) - return false; - if (!(path_in_schema == rhs.path_in_schema)) - return false; - if (!(codec == rhs.codec)) - return false; - if (!(num_values == rhs.num_values)) - return false; - if (!(total_uncompressed_size == rhs.total_uncompressed_size)) - return false; - if (!(total_compressed_size == rhs.total_compressed_size)) - return false; - if (__isset.key_value_metadata != rhs.__isset.key_value_metadata) - return false; - else if (__isset.key_value_metadata && !(key_value_metadata == rhs.key_value_metadata)) - return false; - if (!(data_page_offset == rhs.data_page_offset)) - return false; - if (__isset.index_page_offset != rhs.__isset.index_page_offset) - return false; - else if (__isset.index_page_offset && !(index_page_offset == rhs.index_page_offset)) - return false; - if (__isset.dictionary_page_offset != rhs.__isset.dictionary_page_offset) - return false; - else if (__isset.dictionary_page_offset && !(dictionary_page_offset == rhs.dictionary_page_offset)) - return false; - if (__isset.statistics != rhs.__isset.statistics) - return false; - else if (__isset.statistics && !(statistics == rhs.statistics)) - return false; - if (__isset.encoding_stats != rhs.__isset.encoding_stats) - return false; - else if (__isset.encoding_stats && !(encoding_stats == rhs.encoding_stats)) - return false; - return true; - } - bool operator != (const ColumnMetaData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnMetaData & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(ColumnMetaData &a, ColumnMetaData &b); - -std::ostream& operator<<(std::ostream& out, const ColumnMetaData& obj); - - -class EncryptionWithFooterKey : public virtual ::apache::thrift::TBase { - public: - - EncryptionWithFooterKey(const EncryptionWithFooterKey&); - EncryptionWithFooterKey& operator=(const EncryptionWithFooterKey&); - EncryptionWithFooterKey() { - } - - virtual ~EncryptionWithFooterKey() throw(); - - bool operator == (const EncryptionWithFooterKey & /* rhs */) const - { - return true; - } - bool operator != (const EncryptionWithFooterKey &rhs) const { - return !(*this == rhs); - } - - bool operator < (const EncryptionWithFooterKey & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(EncryptionWithFooterKey &a, EncryptionWithFooterKey &b); - -std::ostream& operator<<(std::ostream& out, const EncryptionWithFooterKey& obj); - -typedef struct _EncryptionWithColumnKey__isset { - _EncryptionWithColumnKey__isset() : key_metadata(false) {} - bool key_metadata :1; -} _EncryptionWithColumnKey__isset; - -class EncryptionWithColumnKey : public virtual ::apache::thrift::TBase { - public: - - EncryptionWithColumnKey(const EncryptionWithColumnKey&); - EncryptionWithColumnKey& operator=(const EncryptionWithColumnKey&); - EncryptionWithColumnKey() : key_metadata() { - } - - virtual ~EncryptionWithColumnKey() throw(); - std::vector path_in_schema; - std::string key_metadata; - - _EncryptionWithColumnKey__isset __isset; - - void __set_path_in_schema(const std::vector & val); - - void __set_key_metadata(const std::string& val); - - bool operator == (const EncryptionWithColumnKey & rhs) const - { - if (!(path_in_schema == rhs.path_in_schema)) - return false; - if (__isset.key_metadata != rhs.__isset.key_metadata) - return false; - else if (__isset.key_metadata && !(key_metadata == rhs.key_metadata)) - return false; - return true; - } - bool operator != (const EncryptionWithColumnKey &rhs) const { - return !(*this == rhs); - } - - bool operator < (const EncryptionWithColumnKey & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(EncryptionWithColumnKey &a, EncryptionWithColumnKey &b); - -std::ostream& operator<<(std::ostream& out, const EncryptionWithColumnKey& obj); - -typedef struct _ColumnCryptoMetaData__isset { - _ColumnCryptoMetaData__isset() : ENCRYPTION_WITH_FOOTER_KEY(false), ENCRYPTION_WITH_COLUMN_KEY(false) {} - bool ENCRYPTION_WITH_FOOTER_KEY :1; - bool ENCRYPTION_WITH_COLUMN_KEY :1; -} _ColumnCryptoMetaData__isset; - -class ColumnCryptoMetaData : public virtual ::apache::thrift::TBase { - public: - - ColumnCryptoMetaData(const ColumnCryptoMetaData&); - ColumnCryptoMetaData& operator=(const ColumnCryptoMetaData&); - ColumnCryptoMetaData() { - } - - virtual ~ColumnCryptoMetaData() throw(); - EncryptionWithFooterKey ENCRYPTION_WITH_FOOTER_KEY; - EncryptionWithColumnKey ENCRYPTION_WITH_COLUMN_KEY; - - _ColumnCryptoMetaData__isset __isset; - - void __set_ENCRYPTION_WITH_FOOTER_KEY(const EncryptionWithFooterKey& val); - - void __set_ENCRYPTION_WITH_COLUMN_KEY(const EncryptionWithColumnKey& val); - - bool operator == (const ColumnCryptoMetaData & rhs) const - { - if (__isset.ENCRYPTION_WITH_FOOTER_KEY != rhs.__isset.ENCRYPTION_WITH_FOOTER_KEY) - return false; - else if (__isset.ENCRYPTION_WITH_FOOTER_KEY && !(ENCRYPTION_WITH_FOOTER_KEY == rhs.ENCRYPTION_WITH_FOOTER_KEY)) - return false; - if (__isset.ENCRYPTION_WITH_COLUMN_KEY != rhs.__isset.ENCRYPTION_WITH_COLUMN_KEY) - return false; - else if (__isset.ENCRYPTION_WITH_COLUMN_KEY && !(ENCRYPTION_WITH_COLUMN_KEY == rhs.ENCRYPTION_WITH_COLUMN_KEY)) - return false; - return true; - } - bool operator != (const ColumnCryptoMetaData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnCryptoMetaData & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(ColumnCryptoMetaData &a, ColumnCryptoMetaData &b); - -std::ostream& operator<<(std::ostream& out, const ColumnCryptoMetaData& obj); - -typedef struct _ColumnChunk__isset { - _ColumnChunk__isset() : file_path(false), meta_data(false), offset_index_offset(false), offset_index_length(false), column_index_offset(false), column_index_length(false), crypto_metadata(false), encrypted_column_metadata(false) {} - bool file_path :1; - bool meta_data :1; - bool offset_index_offset :1; - bool offset_index_length :1; - bool column_index_offset :1; - bool column_index_length :1; - bool crypto_metadata :1; - bool encrypted_column_metadata :1; -} _ColumnChunk__isset; - -class ColumnChunk : public virtual ::apache::thrift::TBase { - public: - - ColumnChunk(const ColumnChunk&); - ColumnChunk& operator=(const ColumnChunk&); - ColumnChunk() : file_path(), file_offset(0), offset_index_offset(0), offset_index_length(0), column_index_offset(0), column_index_length(0), encrypted_column_metadata() { - } - - virtual ~ColumnChunk() throw(); - std::string file_path; - int64_t file_offset; - ColumnMetaData meta_data; - int64_t offset_index_offset; - int32_t offset_index_length; - int64_t column_index_offset; - int32_t column_index_length; - ColumnCryptoMetaData crypto_metadata; - std::string encrypted_column_metadata; - - _ColumnChunk__isset __isset; - - void __set_file_path(const std::string& val); - - void __set_file_offset(const int64_t val); - - void __set_meta_data(const ColumnMetaData& val); - - void __set_offset_index_offset(const int64_t val); - - void __set_offset_index_length(const int32_t val); - - void __set_column_index_offset(const int64_t val); - - void __set_column_index_length(const int32_t val); - - void __set_crypto_metadata(const ColumnCryptoMetaData& val); - - void __set_encrypted_column_metadata(const std::string& val); - - bool operator == (const ColumnChunk & rhs) const - { - if (__isset.file_path != rhs.__isset.file_path) - return false; - else if (__isset.file_path && !(file_path == rhs.file_path)) - return false; - if (!(file_offset == rhs.file_offset)) - return false; - if (__isset.meta_data != rhs.__isset.meta_data) - return false; - else if (__isset.meta_data && !(meta_data == rhs.meta_data)) - return false; - if (__isset.offset_index_offset != rhs.__isset.offset_index_offset) - return false; - else if (__isset.offset_index_offset && !(offset_index_offset == rhs.offset_index_offset)) - return false; - if (__isset.offset_index_length != rhs.__isset.offset_index_length) - return false; - else if (__isset.offset_index_length && !(offset_index_length == rhs.offset_index_length)) - return false; - if (__isset.column_index_offset != rhs.__isset.column_index_offset) - return false; - else if (__isset.column_index_offset && !(column_index_offset == rhs.column_index_offset)) - return false; - if (__isset.column_index_length != rhs.__isset.column_index_length) - return false; - else if (__isset.column_index_length && !(column_index_length == rhs.column_index_length)) - return false; - if (__isset.crypto_metadata != rhs.__isset.crypto_metadata) - return false; - else if (__isset.crypto_metadata && !(crypto_metadata == rhs.crypto_metadata)) - return false; - if (__isset.encrypted_column_metadata != rhs.__isset.encrypted_column_metadata) - return false; - else if (__isset.encrypted_column_metadata && !(encrypted_column_metadata == rhs.encrypted_column_metadata)) - return false; - return true; - } - bool operator != (const ColumnChunk &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnChunk & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(ColumnChunk &a, ColumnChunk &b); - -std::ostream& operator<<(std::ostream& out, const ColumnChunk& obj); - -typedef struct _RowGroup__isset { - _RowGroup__isset() : sorting_columns(false), file_offset(false), total_compressed_size(false), ordinal(false) {} - bool sorting_columns :1; - bool file_offset :1; - bool total_compressed_size :1; - bool ordinal :1; -} _RowGroup__isset; - -class RowGroup : public virtual ::apache::thrift::TBase { - public: - - RowGroup(const RowGroup&); - RowGroup& operator=(const RowGroup&); - RowGroup() : total_byte_size(0), num_rows(0), file_offset(0), total_compressed_size(0), ordinal(0) { - } - - virtual ~RowGroup() throw(); - std::vector columns; - int64_t total_byte_size; - int64_t num_rows; - std::vector sorting_columns; - int64_t file_offset; - int64_t total_compressed_size; - int16_t ordinal; - - _RowGroup__isset __isset; - - void __set_columns(const std::vector & val); - - void __set_total_byte_size(const int64_t val); - - void __set_num_rows(const int64_t val); - - void __set_sorting_columns(const std::vector & val); - - void __set_file_offset(const int64_t val); - - void __set_total_compressed_size(const int64_t val); - - void __set_ordinal(const int16_t val); - - bool operator == (const RowGroup & rhs) const - { - if (!(columns == rhs.columns)) - return false; - if (!(total_byte_size == rhs.total_byte_size)) - return false; - if (!(num_rows == rhs.num_rows)) - return false; - if (__isset.sorting_columns != rhs.__isset.sorting_columns) - return false; - else if (__isset.sorting_columns && !(sorting_columns == rhs.sorting_columns)) - return false; - if (__isset.file_offset != rhs.__isset.file_offset) - return false; - else if (__isset.file_offset && !(file_offset == rhs.file_offset)) - return false; - if (__isset.total_compressed_size != rhs.__isset.total_compressed_size) - return false; - else if (__isset.total_compressed_size && !(total_compressed_size == rhs.total_compressed_size)) - return false; - if (__isset.ordinal != rhs.__isset.ordinal) - return false; - else if (__isset.ordinal && !(ordinal == rhs.ordinal)) - return false; - return true; - } - bool operator != (const RowGroup &rhs) const { - return !(*this == rhs); - } - - bool operator < (const RowGroup & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(RowGroup &a, RowGroup &b); - -std::ostream& operator<<(std::ostream& out, const RowGroup& obj); - - -class TypeDefinedOrder : public virtual ::apache::thrift::TBase { - public: - - TypeDefinedOrder(const TypeDefinedOrder&); - TypeDefinedOrder& operator=(const TypeDefinedOrder&); - TypeDefinedOrder() { - } - - virtual ~TypeDefinedOrder() throw(); - - bool operator == (const TypeDefinedOrder & /* rhs */) const - { - return true; - } - bool operator != (const TypeDefinedOrder &rhs) const { - return !(*this == rhs); - } - - bool operator < (const TypeDefinedOrder & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(TypeDefinedOrder &a, TypeDefinedOrder &b); - -std::ostream& operator<<(std::ostream& out, const TypeDefinedOrder& obj); - -typedef struct _ColumnOrder__isset { - _ColumnOrder__isset() : TYPE_ORDER(false) {} - bool TYPE_ORDER :1; -} _ColumnOrder__isset; - -class ColumnOrder : public virtual ::apache::thrift::TBase { - public: - - ColumnOrder(const ColumnOrder&); - ColumnOrder& operator=(const ColumnOrder&); - ColumnOrder() { - } - - virtual ~ColumnOrder() throw(); - TypeDefinedOrder TYPE_ORDER; - - _ColumnOrder__isset __isset; - - void __set_TYPE_ORDER(const TypeDefinedOrder& val); - - bool operator == (const ColumnOrder & rhs) const - { - if (__isset.TYPE_ORDER != rhs.__isset.TYPE_ORDER) - return false; - else if (__isset.TYPE_ORDER && !(TYPE_ORDER == rhs.TYPE_ORDER)) - return false; - return true; - } - bool operator != (const ColumnOrder &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnOrder & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(ColumnOrder &a, ColumnOrder &b); - -std::ostream& operator<<(std::ostream& out, const ColumnOrder& obj); - - -class PageLocation : public virtual ::apache::thrift::TBase { - public: - - PageLocation(const PageLocation&); - PageLocation& operator=(const PageLocation&); - PageLocation() : offset(0), compressed_page_size(0), first_row_index(0) { - } - - virtual ~PageLocation() throw(); - int64_t offset; - int32_t compressed_page_size; - int64_t first_row_index; - - void __set_offset(const int64_t val); - - void __set_compressed_page_size(const int32_t val); - - void __set_first_row_index(const int64_t val); - - bool operator == (const PageLocation & rhs) const - { - if (!(offset == rhs.offset)) - return false; - if (!(compressed_page_size == rhs.compressed_page_size)) - return false; - if (!(first_row_index == rhs.first_row_index)) - return false; - return true; - } - bool operator != (const PageLocation &rhs) const { - return !(*this == rhs); - } - - bool operator < (const PageLocation & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(PageLocation &a, PageLocation &b); - -std::ostream& operator<<(std::ostream& out, const PageLocation& obj); - - -class OffsetIndex : public virtual ::apache::thrift::TBase { - public: - - OffsetIndex(const OffsetIndex&); - OffsetIndex& operator=(const OffsetIndex&); - OffsetIndex() { - } - - virtual ~OffsetIndex() throw(); - std::vector page_locations; - - void __set_page_locations(const std::vector & val); - - bool operator == (const OffsetIndex & rhs) const - { - if (!(page_locations == rhs.page_locations)) - return false; - return true; - } - bool operator != (const OffsetIndex &rhs) const { - return !(*this == rhs); - } - - bool operator < (const OffsetIndex & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(OffsetIndex &a, OffsetIndex &b); - -std::ostream& operator<<(std::ostream& out, const OffsetIndex& obj); - -typedef struct _ColumnIndex__isset { - _ColumnIndex__isset() : null_counts(false) {} - bool null_counts :1; -} _ColumnIndex__isset; - -class ColumnIndex : public virtual ::apache::thrift::TBase { - public: - - ColumnIndex(const ColumnIndex&); - ColumnIndex& operator=(const ColumnIndex&); - ColumnIndex() : boundary_order((BoundaryOrder::type)0) { - } - - virtual ~ColumnIndex() throw(); - std::vector null_pages; - std::vector min_values; - std::vector max_values; - BoundaryOrder::type boundary_order; - std::vector null_counts; - - _ColumnIndex__isset __isset; - - void __set_null_pages(const std::vector & val); - - void __set_min_values(const std::vector & val); - - void __set_max_values(const std::vector & val); - - void __set_boundary_order(const BoundaryOrder::type val); - - void __set_null_counts(const std::vector & val); - - bool operator == (const ColumnIndex & rhs) const - { - if (!(null_pages == rhs.null_pages)) - return false; - if (!(min_values == rhs.min_values)) - return false; - if (!(max_values == rhs.max_values)) - return false; - if (!(boundary_order == rhs.boundary_order)) - return false; - if (__isset.null_counts != rhs.__isset.null_counts) - return false; - else if (__isset.null_counts && !(null_counts == rhs.null_counts)) - return false; - return true; - } - bool operator != (const ColumnIndex &rhs) const { - return !(*this == rhs); - } - - bool operator < (const ColumnIndex & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(ColumnIndex &a, ColumnIndex &b); - -std::ostream& operator<<(std::ostream& out, const ColumnIndex& obj); - -typedef struct _AesGcmV1__isset { - _AesGcmV1__isset() : aad_prefix(false), aad_file_unique(false), supply_aad_prefix(false) {} - bool aad_prefix :1; - bool aad_file_unique :1; - bool supply_aad_prefix :1; -} _AesGcmV1__isset; - -class AesGcmV1 : public virtual ::apache::thrift::TBase { - public: - - AesGcmV1(const AesGcmV1&); - AesGcmV1& operator=(const AesGcmV1&); - AesGcmV1() : aad_prefix(), aad_file_unique(), supply_aad_prefix(0) { - } - - virtual ~AesGcmV1() throw(); - std::string aad_prefix; - std::string aad_file_unique; - bool supply_aad_prefix; - - _AesGcmV1__isset __isset; - - void __set_aad_prefix(const std::string& val); - - void __set_aad_file_unique(const std::string& val); - - void __set_supply_aad_prefix(const bool val); - - bool operator == (const AesGcmV1 & rhs) const - { - if (__isset.aad_prefix != rhs.__isset.aad_prefix) - return false; - else if (__isset.aad_prefix && !(aad_prefix == rhs.aad_prefix)) - return false; - if (__isset.aad_file_unique != rhs.__isset.aad_file_unique) - return false; - else if (__isset.aad_file_unique && !(aad_file_unique == rhs.aad_file_unique)) - return false; - if (__isset.supply_aad_prefix != rhs.__isset.supply_aad_prefix) - return false; - else if (__isset.supply_aad_prefix && !(supply_aad_prefix == rhs.supply_aad_prefix)) - return false; - return true; - } - bool operator != (const AesGcmV1 &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AesGcmV1 & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(AesGcmV1 &a, AesGcmV1 &b); - -std::ostream& operator<<(std::ostream& out, const AesGcmV1& obj); - -typedef struct _AesGcmCtrV1__isset { - _AesGcmCtrV1__isset() : aad_prefix(false), aad_file_unique(false), supply_aad_prefix(false) {} - bool aad_prefix :1; - bool aad_file_unique :1; - bool supply_aad_prefix :1; -} _AesGcmCtrV1__isset; - -class AesGcmCtrV1 : public virtual ::apache::thrift::TBase { - public: - - AesGcmCtrV1(const AesGcmCtrV1&); - AesGcmCtrV1& operator=(const AesGcmCtrV1&); - AesGcmCtrV1() : aad_prefix(), aad_file_unique(), supply_aad_prefix(0) { - } - - virtual ~AesGcmCtrV1() throw(); - std::string aad_prefix; - std::string aad_file_unique; - bool supply_aad_prefix; - - _AesGcmCtrV1__isset __isset; - - void __set_aad_prefix(const std::string& val); - - void __set_aad_file_unique(const std::string& val); - - void __set_supply_aad_prefix(const bool val); - - bool operator == (const AesGcmCtrV1 & rhs) const - { - if (__isset.aad_prefix != rhs.__isset.aad_prefix) - return false; - else if (__isset.aad_prefix && !(aad_prefix == rhs.aad_prefix)) - return false; - if (__isset.aad_file_unique != rhs.__isset.aad_file_unique) - return false; - else if (__isset.aad_file_unique && !(aad_file_unique == rhs.aad_file_unique)) - return false; - if (__isset.supply_aad_prefix != rhs.__isset.supply_aad_prefix) - return false; - else if (__isset.supply_aad_prefix && !(supply_aad_prefix == rhs.supply_aad_prefix)) - return false; - return true; - } - bool operator != (const AesGcmCtrV1 &rhs) const { - return !(*this == rhs); - } - - bool operator < (const AesGcmCtrV1 & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(AesGcmCtrV1 &a, AesGcmCtrV1 &b); - -std::ostream& operator<<(std::ostream& out, const AesGcmCtrV1& obj); - -typedef struct _EncryptionAlgorithm__isset { - _EncryptionAlgorithm__isset() : AES_GCM_V1(false), AES_GCM_CTR_V1(false) {} - bool AES_GCM_V1 :1; - bool AES_GCM_CTR_V1 :1; -} _EncryptionAlgorithm__isset; - -class EncryptionAlgorithm : public virtual ::apache::thrift::TBase { - public: - - EncryptionAlgorithm(const EncryptionAlgorithm&); - EncryptionAlgorithm& operator=(const EncryptionAlgorithm&); - EncryptionAlgorithm() { - } - - virtual ~EncryptionAlgorithm() throw(); - AesGcmV1 AES_GCM_V1; - AesGcmCtrV1 AES_GCM_CTR_V1; - - _EncryptionAlgorithm__isset __isset; - - void __set_AES_GCM_V1(const AesGcmV1& val); - - void __set_AES_GCM_CTR_V1(const AesGcmCtrV1& val); - - bool operator == (const EncryptionAlgorithm & rhs) const - { - if (__isset.AES_GCM_V1 != rhs.__isset.AES_GCM_V1) - return false; - else if (__isset.AES_GCM_V1 && !(AES_GCM_V1 == rhs.AES_GCM_V1)) - return false; - if (__isset.AES_GCM_CTR_V1 != rhs.__isset.AES_GCM_CTR_V1) - return false; - else if (__isset.AES_GCM_CTR_V1 && !(AES_GCM_CTR_V1 == rhs.AES_GCM_CTR_V1)) - return false; - return true; - } - bool operator != (const EncryptionAlgorithm &rhs) const { - return !(*this == rhs); - } - - bool operator < (const EncryptionAlgorithm & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(EncryptionAlgorithm &a, EncryptionAlgorithm &b); - -std::ostream& operator<<(std::ostream& out, const EncryptionAlgorithm& obj); - -typedef struct _FileMetaData__isset { - _FileMetaData__isset() : key_value_metadata(false), created_by(false), column_orders(false), encryption_algorithm(false), footer_signing_key_metadata(false) {} - bool key_value_metadata :1; - bool created_by :1; - bool column_orders :1; - bool encryption_algorithm :1; - bool footer_signing_key_metadata :1; -} _FileMetaData__isset; - -class FileMetaData : public virtual ::apache::thrift::TBase { - public: - - FileMetaData(const FileMetaData&); - FileMetaData& operator=(const FileMetaData&); - FileMetaData() : version(0), num_rows(0), created_by(), footer_signing_key_metadata() { - } - - virtual ~FileMetaData() throw(); - int32_t version; - std::vector schema; - int64_t num_rows; - std::vector row_groups; - std::vector key_value_metadata; - std::string created_by; - std::vector column_orders; - EncryptionAlgorithm encryption_algorithm; - std::string footer_signing_key_metadata; - - _FileMetaData__isset __isset; - - void __set_version(const int32_t val); - - void __set_schema(const std::vector & val); - - void __set_num_rows(const int64_t val); - - void __set_row_groups(const std::vector & val); - - void __set_key_value_metadata(const std::vector & val); - - void __set_created_by(const std::string& val); - - void __set_column_orders(const std::vector & val); - - void __set_encryption_algorithm(const EncryptionAlgorithm& val); - - void __set_footer_signing_key_metadata(const std::string& val); - - bool operator == (const FileMetaData & rhs) const - { - if (!(version == rhs.version)) - return false; - if (!(schema == rhs.schema)) - return false; - if (!(num_rows == rhs.num_rows)) - return false; - if (!(row_groups == rhs.row_groups)) - return false; - if (__isset.key_value_metadata != rhs.__isset.key_value_metadata) - return false; - else if (__isset.key_value_metadata && !(key_value_metadata == rhs.key_value_metadata)) - return false; - if (__isset.created_by != rhs.__isset.created_by) - return false; - else if (__isset.created_by && !(created_by == rhs.created_by)) - return false; - if (__isset.column_orders != rhs.__isset.column_orders) - return false; - else if (__isset.column_orders && !(column_orders == rhs.column_orders)) - return false; - if (__isset.encryption_algorithm != rhs.__isset.encryption_algorithm) - return false; - else if (__isset.encryption_algorithm && !(encryption_algorithm == rhs.encryption_algorithm)) - return false; - if (__isset.footer_signing_key_metadata != rhs.__isset.footer_signing_key_metadata) - return false; - else if (__isset.footer_signing_key_metadata && !(footer_signing_key_metadata == rhs.footer_signing_key_metadata)) - return false; - return true; - } - bool operator != (const FileMetaData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const FileMetaData & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(FileMetaData &a, FileMetaData &b); - -std::ostream& operator<<(std::ostream& out, const FileMetaData& obj); - -typedef struct _FileCryptoMetaData__isset { - _FileCryptoMetaData__isset() : key_metadata(false) {} - bool key_metadata :1; -} _FileCryptoMetaData__isset; - -class FileCryptoMetaData : public virtual ::apache::thrift::TBase { - public: - - FileCryptoMetaData(const FileCryptoMetaData&); - FileCryptoMetaData& operator=(const FileCryptoMetaData&); - FileCryptoMetaData() : key_metadata() { - } - - virtual ~FileCryptoMetaData() throw(); - EncryptionAlgorithm encryption_algorithm; - std::string key_metadata; - - _FileCryptoMetaData__isset __isset; - - void __set_encryption_algorithm(const EncryptionAlgorithm& val); - - void __set_key_metadata(const std::string& val); - - bool operator == (const FileCryptoMetaData & rhs) const - { - if (!(encryption_algorithm == rhs.encryption_algorithm)) - return false; - if (__isset.key_metadata != rhs.__isset.key_metadata) - return false; - else if (__isset.key_metadata && !(key_metadata == rhs.key_metadata)) - return false; - return true; - } - bool operator != (const FileCryptoMetaData &rhs) const { - return !(*this == rhs); - } - - bool operator < (const FileCryptoMetaData & ) const; - - uint32_t read(::apache::thrift::protocol::TProtocol* iprot); - uint32_t write(::apache::thrift::protocol::TProtocol* oprot) const; - - virtual void printTo(std::ostream& out) const; -}; - -void swap(FileCryptoMetaData &a, FileCryptoMetaData &b); - -std::ostream& operator<<(std::ostream& out, const FileCryptoMetaData& obj); - -}} // namespace - -#endif diff --git a/contrib/arrow-cmake/cpp/src/parquet/parquet_version.h b/contrib/arrow-cmake/cpp/src/parquet/parquet_version.h index dd83e45028d..7404db1d381 100644 --- a/contrib/arrow-cmake/cpp/src/parquet/parquet_version.h +++ b/contrib/arrow-cmake/cpp/src/parquet/parquet_version.h @@ -18,6 +18,13 @@ #ifndef PARQUET_VERSION_H #define PARQUET_VERSION_H +#define PARQUET_VERSION_MAJOR 1 +#define PARQUET_VERSION_MINOR 5 +#define PARQUET_VERSION_PATCH 1 + +#define PARQUET_SO_VERSION 0 +#define PARQUET_FULL_SO_VERSION 0.17 + // define the parquet created by version #define CREATED_BY_VERSION "parquet-cpp version 1.5.1-SNAPSHOT" diff --git a/contrib/flatbuffers b/contrib/flatbuffers index bf9eb67ab93..6df40a24717 160000 --- a/contrib/flatbuffers +++ b/contrib/flatbuffers @@ -1 +1 @@ -Subproject commit bf9eb67ab9371755c6bcece13cadc7693bcbf264 +Subproject commit 6df40a2471737b27271bdd9b900ab5f3aec746c7 diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp index b975fa19068..0eaedfa7742 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.cpp @@ -24,10 +24,9 @@ arrow::Status ArrowBufferedOutputStream::Close() return arrow::Status::OK(); } -arrow::Status ArrowBufferedOutputStream::Tell(int64_t * position) const +arrow::Result ArrowBufferedOutputStream::Tell() const { - *position = total_length; - return arrow::Status::OK(); + return arrow::Result(total_length); } arrow::Status ArrowBufferedOutputStream::Write(const void * data, int64_t length) @@ -42,10 +41,9 @@ RandomAccessFileFromSeekableReadBuffer::RandomAccessFileFromSeekableReadBuffer(S { } -arrow::Status RandomAccessFileFromSeekableReadBuffer::GetSize(int64_t * size) +arrow::Result RandomAccessFileFromSeekableReadBuffer::GetSize() { - *size = file_size; - return arrow::Status::OK(); + return arrow::Result(file_size); } arrow::Status RandomAccessFileFromSeekableReadBuffer::Close() @@ -54,25 +52,25 @@ arrow::Status RandomAccessFileFromSeekableReadBuffer::Close() return arrow::Status::OK(); } -arrow::Status RandomAccessFileFromSeekableReadBuffer::Tell(int64_t * position) const +arrow::Result RandomAccessFileFromSeekableReadBuffer::Tell() const { - *position = in.getPosition(); - return arrow::Status::OK(); + return arrow::Result(in.getPosition()); } -arrow::Status RandomAccessFileFromSeekableReadBuffer::Read(int64_t nbytes, int64_t * bytes_read, void * out) +arrow::Result RandomAccessFileFromSeekableReadBuffer::Read(int64_t nbytes, void * out) { - *bytes_read = in.readBig(reinterpret_cast(out), nbytes); - return arrow::Status::OK(); + int64_t bytes_read = in.readBig(reinterpret_cast(out), nbytes); + return arrow::Result(bytes_read); } -arrow::Status RandomAccessFileFromSeekableReadBuffer::Read(int64_t nbytes, std::shared_ptr * out) +arrow::Result> RandomAccessFileFromSeekableReadBuffer::Read(int64_t nbytes) { std::shared_ptr buf; ARROW_RETURN_NOT_OK(arrow::AllocateBuffer(nbytes, &buf)); size_t n = in.readBig(reinterpret_cast(buf->mutable_data()), nbytes); - *out = arrow::SliceBuffer(buf, 0, n); - return arrow::Status::OK(); + + auto read_buffer = arrow::SliceBuffer(buf, 0, n); + return arrow::Result>(read_buffer); } arrow::Status RandomAccessFileFromSeekableReadBuffer::Seek(int64_t position) diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index de0922c8a31..bb94535549c 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -19,7 +19,7 @@ public: // FileInterface arrow::Status Close() override; - arrow::Status Tell(int64_t * position) const override; + arrow::Result Tell() const override; bool closed() const override { return !is_open; } @@ -39,17 +39,17 @@ class RandomAccessFileFromSeekableReadBuffer : public arrow::io::RandomAccessFil public: RandomAccessFileFromSeekableReadBuffer(SeekableReadBuffer & in_, off_t file_size_); - arrow::Status GetSize(int64_t * size) override; + arrow::Result GetSize() override; arrow::Status Close() override; - arrow::Status Tell(int64_t * position) const override; + arrow::Result Tell() const override; bool closed() const override { return !is_open; } - arrow::Status Read(int64_t nbytes, int64_t * bytes_read, void * out) override; + arrow::Result Read(int64_t nbytes, void * out) override; - arrow::Status Read(int64_t nbytes, std::shared_ptr * out) override; + arrow::Result> Read(int64_t nbytes) override; arrow::Status Seek(int64_t position) override; diff --git a/tests/queries/0_stateless/00900_parquet_load.reference b/tests/queries/0_stateless/00900_parquet_load.reference index 2930fcd3c14..f49b74e4c3c 100644 --- a/tests/queries/0_stateless/00900_parquet_load.reference +++ b/tests/queries/0_stateless/00900_parquet_load.reference @@ -39,7 +39,7 @@ 23.0 24.0 === Try load data from datapage_v2.snappy.parquet -Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Arrow error: IOError: Corrupt snappy compressed data. +Code: 33. DB::Ex---tion: Error while reading Parquet data: IOError: Not yet implemented: Unsupported encoding. === Try load data from fixed_length_decimal_1.parquet 1.0 From 5ef4e90dfb3a4c3f185903a43b5ae50a6693a3e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 Jul 2020 23:56:40 +0300 Subject: [PATCH 140/196] Fix minor issues after #12196 --- src/Functions/FunctionsLogical.cpp | 5 ++-- src/Functions/FunctionsLogical.h | 25 ++++++++++++++++--- .../0_stateless/01376_null_logical.reference | 8 ++++++ .../0_stateless/01376_null_logical.sql | 11 ++++++++ 4 files changed, 44 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/01376_null_logical.reference create mode 100644 tests/queries/0_stateless/01376_null_logical.sql diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 5443df33d79..27801ec5db6 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -336,7 +336,7 @@ static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAn const bool has_consts = extractConstColumnsAsTernary(arguments, const_3v_value); /// If the constant value uniquely determines the result, return it. - if (has_consts && (arguments.empty() || Op::isSaturatedValue(const_3v_value))) + if (has_consts && (arguments.empty() || Op::isSaturatedValueTernary(const_3v_value))) { result_info.column = ColumnConst::create( buildColumnFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()), @@ -498,7 +498,8 @@ DataTypePtr FunctionAnyArityLogical::getReturnTypeImpl(const DataTyp } template -void FunctionAnyArityLogical::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) +void FunctionAnyArityLogical::executeImpl( + Block & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) { ColumnRawPtrs args_in; for (const auto arg_index : arguments) diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 4751f0b7548..a6dc9865e6f 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -36,9 +36,21 @@ namespace Ternary { using ResultType = UInt8; - static constexpr UInt8 False = 0; - static constexpr UInt8 True = -1; - static constexpr UInt8 Null = 1; + /** These carefully picked values magically work so bitwise "and", "or" on them + * corresponds to the expected results in three-valued logic. + * + * False and True are represented by all-0 and all-1 bits, so all bitwise operations on them work as expected. + * Null is represented as single 1 bit. So, it is something in between False and True. + * And "or" works like maximum and "and" works like minimum: + * "or" keeps True as is and lifts False with Null to Null. + * "and" keeps False as is and downs True with Null to Null. + * + * This logic does not apply for "not" and "xor" - they work with default implementation for NULLs: + * anything with NULL returns NULL, otherwise use conventional two-valued logic. + */ + static constexpr UInt8 False = 0; /// All zero bits. + static constexpr UInt8 True = -1; /// All one bits. + static constexpr UInt8 Null = 1; /// Single one bit. template inline ResultType makeValue(T value) @@ -61,9 +73,16 @@ struct AndImpl using ResultType = UInt8; static inline constexpr bool isSaturable() { return true; } + + /// Final value in two-valued logic (no further operations with True, False will change this value) static inline constexpr bool isSaturatedValue(bool a) { return !a; } + + /// Final value in three-valued logic (no further operations with True, False, Null will change this value) static inline constexpr bool isSaturatedValueTernary(UInt8 a) { return a == Ternary::False; } + static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return a & b; } + + /// Will use three-valued logic for NULLs (see above) or default implementation (any operation with NULL returns NULL). static inline constexpr bool specialImplementationForNulls() { return true; } }; diff --git a/tests/queries/0_stateless/01376_null_logical.reference b/tests/queries/0_stateless/01376_null_logical.reference new file mode 100644 index 00000000000..4b849dd057d --- /dev/null +++ b/tests/queries/0_stateless/01376_null_logical.reference @@ -0,0 +1,8 @@ +1 +1 +0 +0 +\N +\N +\N +\N diff --git a/tests/queries/0_stateless/01376_null_logical.sql b/tests/queries/0_stateless/01376_null_logical.sql new file mode 100644 index 00000000000..d8c7a322445 --- /dev/null +++ b/tests/queries/0_stateless/01376_null_logical.sql @@ -0,0 +1,11 @@ +SELECT NULL OR 1; +SELECT materialize(NULL) OR materialize(1); + +SELECT NULL AND 0; +SELECT materialize(NULL) AND materialize(0); + +SELECT NULL OR 0; +SELECT materialize(NULL) OR materialize(0); + +SELECT NULL AND 1; +SELECT materialize(NULL) AND materialize(1); From 6e695809e4c8c87fdfd398f6cecf9908c37ece8f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 8 Jul 2020 00:19:46 +0300 Subject: [PATCH 141/196] Reset CurrentMetrics::MemoryTracking periodically to the process RSS --- src/Interpreters/AsynchronousMetrics.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 5c734f4da2a..ac71a88dc00 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -208,6 +208,7 @@ void AsynchronousMetrics::update() /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. /// See https://github.com/ClickHouse/ClickHouse/issues/10293 total_memory_tracker.set(data.resident); + CurrentMetrics::set(CurrentMetrics::MemoryTracking, data.resident); } #endif From 6a04de61b65d423f741a2f13336ff778afae73fc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 8 Jul 2020 00:26:09 +0300 Subject: [PATCH 142/196] Allow isInjective() with empty block (is function injective with any arguments) Since most of the time function will ignore it anyway, and creating arguments just for checking is function injective or not is overkill --- src/Functions/FunctionsExternalDictionaries.h | 4 ++++ src/Functions/IFunction.h | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index 425dcf8eec0..7f3979f2141 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -99,6 +99,10 @@ public: bool isDictGetFunctionInjective(const Block & sample_block) { + /// Assume non-injective by default + if (!sample_block) + return false; + if (sample_block.columns() != 3 && sample_block.columns() != 4) throw Exception{"Function dictGet... takes 3 or 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index b8873ea2671..9bed7b209bf 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -133,6 +133,10 @@ public: * But we assume, that it is injective. This could be documented as implementation-specific behaviour. * * sample_block should contain data types of arguments and values of constants, if relevant. + * NOTE: to check is function injective with any arguments, you can pass + * empty block as sample_block (since most of the time function will + * ignore it anyway, and creating arguments just for checking is + * function injective or not is overkill). */ virtual bool isInjective(const Block & /*sample_block*/) const { return false; } From 1f978022c589ebfb14c767e746eaa7d27824bcc7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 01:10:23 +0300 Subject: [PATCH 143/196] Implement getLeastSuperType for LowCardinality #8212 --- src/DataTypes/getLeastSupertype.cpp | 34 +++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 9c3ca679467..4e332f0d158 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -186,6 +187,39 @@ DataTypePtr getLeastSupertype(const DataTypes & types) } } + /// For LowCardinality + { + bool have_low_cardinality = false; + bool have_not_low_cardinality = false; + + DataTypes nested_types; + nested_types.reserve(types.size()); + + for (const auto & type : types) + { + if (const DataTypeLowCardinality * type_low_cardinality = typeid_cast(type.get())) + { + have_low_cardinality = true; + nested_types.emplace_back(type_low_cardinality->getDictionaryType()); + } + else + { + have_not_low_cardinality = true; + nested_types.emplace_back(type); + } + } + + /// All LowCardinality gives LowCardinality. + /// LowCardinality with high cardinality gives high cardinality. + if (have_low_cardinality) + { + if (have_not_low_cardinality) + return getLeastSupertype(nested_types); + else + return std::make_shared(getLeastSupertype(nested_types)); + } + } + /// Non-recursive rules std::unordered_set type_ids; From 60c1ffc892271d342343e05e0b4cb8c593b3a501 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 01:26:43 +0300 Subject: [PATCH 144/196] Implement supertype for LowCardinality --- src/DataTypes/getLeastSupertype.cpp | 54 +++++++------- .../01377_supertype_low_cardinality.reference | 71 +++++++++++++++++++ .../01377_supertype_low_cardinality.sql | 71 +++++++++++++++++++ 3 files changed, 169 insertions(+), 27 deletions(-) create mode 100644 tests/queries/0_stateless/01377_supertype_low_cardinality.reference create mode 100644 tests/queries/0_stateless/01377_supertype_low_cardinality.sql diff --git a/src/DataTypes/getLeastSupertype.cpp b/src/DataTypes/getLeastSupertype.cpp index 4e332f0d158..9cd3e09758e 100644 --- a/src/DataTypes/getLeastSupertype.cpp +++ b/src/DataTypes/getLeastSupertype.cpp @@ -161,33 +161,7 @@ DataTypePtr getLeastSupertype(const DataTypes & types) } } - /// For Nullable - { - bool have_nullable = false; - - DataTypes nested_types; - nested_types.reserve(types.size()); - - for (const auto & type : types) - { - if (const DataTypeNullable * type_nullable = typeid_cast(type.get())) - { - have_nullable = true; - - if (!type_nullable->onlyNull()) - nested_types.emplace_back(type_nullable->getNestedType()); - } - else - nested_types.emplace_back(type); - } - - if (have_nullable) - { - return std::make_shared(getLeastSupertype(nested_types)); - } - } - - /// For LowCardinality + /// For LowCardinality. This is above Nullable, because LowCardinality can contain Nullable but cannot be inside Nullable. { bool have_low_cardinality = false; bool have_not_low_cardinality = false; @@ -220,6 +194,32 @@ DataTypePtr getLeastSupertype(const DataTypes & types) } } + /// For Nullable + { + bool have_nullable = false; + + DataTypes nested_types; + nested_types.reserve(types.size()); + + for (const auto & type : types) + { + if (const DataTypeNullable * type_nullable = typeid_cast(type.get())) + { + have_nullable = true; + + if (!type_nullable->onlyNull()) + nested_types.emplace_back(type_nullable->getNestedType()); + } + else + nested_types.emplace_back(type); + } + + if (have_nullable) + { + return std::make_shared(getLeastSupertype(nested_types)); + } + } + /// Non-recursive rules std::unordered_set type_ids; diff --git a/tests/queries/0_stateless/01377_supertype_low_cardinality.reference b/tests/queries/0_stateless/01377_supertype_low_cardinality.reference new file mode 100644 index 00000000000..89c8606dd33 --- /dev/null +++ b/tests/queries/0_stateless/01377_supertype_low_cardinality.reference @@ -0,0 +1,71 @@ +hello +hello +String +String +--- +--- +hello +hello +hello +hello +--- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +- +hello +hello +--- +hello +hello +hello +- +hello +hello +hello +- +hello +hello +hello +- +hello +hello +hello +--- +hello +hello +hello +hello +--- +['abc','def'] +['abc','def'] +['abc','def'] diff --git a/tests/queries/0_stateless/01377_supertype_low_cardinality.sql b/tests/queries/0_stateless/01377_supertype_low_cardinality.sql new file mode 100644 index 00000000000..9e7ed1f98b2 --- /dev/null +++ b/tests/queries/0_stateless/01377_supertype_low_cardinality.sql @@ -0,0 +1,71 @@ +SELECT 'hello' UNION ALL SELECT toLowCardinality('hello'); +SELECT toTypeName(x) FROM (SELECT 'hello' AS x UNION ALL SELECT toLowCardinality('hello')); + +SELECT '---'; + +create temporary table t1(a String); +create temporary table t2(a LowCardinality(String)); +select a from t1 union all select a from t2; + +SELECT '---'; + +CREATE TEMPORARY TABLE a (x String); +CREATE TEMPORARY TABLE b (x LowCardinality(String)); +CREATE TEMPORARY TABLE c (x Nullable(String)); +CREATE TEMPORARY TABLE d (x LowCardinality(Nullable(String))); + +INSERT INTO a VALUES ('hello'); +INSERT INTO b VALUES ('hello'); +INSERT INTO c VALUES ('hello'); +INSERT INTO d VALUES ('hello'); + +SELECT x FROM a; +SELECT x FROM b; +SELECT x FROM c; +SELECT x FROM d; + +SELECT '---'; + +SELECT x FROM a UNION ALL SELECT x FROM b; +SELECT '-'; +SELECT x FROM a UNION ALL SELECT x FROM c; +SELECT '-'; +SELECT x FROM a UNION ALL SELECT x FROM d; +SELECT '-'; +SELECT x FROM b UNION ALL SELECT x FROM a; +SELECT '-'; +SELECT x FROM b UNION ALL SELECT x FROM c; +SELECT '-'; +SELECT x FROM b UNION ALL SELECT x FROM d; +SELECT '-'; +SELECT x FROM c UNION ALL SELECT x FROM a; +SELECT '-'; +SELECT x FROM c UNION ALL SELECT x FROM b; +SELECT '-'; +SELECT x FROM c UNION ALL SELECT x FROM d; +SELECT '-'; +SELECT x FROM d UNION ALL SELECT x FROM a; +SELECT '-'; +SELECT x FROM d UNION ALL SELECT x FROM c; +SELECT '-'; +SELECT x FROM d UNION ALL SELECT x FROM b; + +SELECT '---'; + +SELECT x FROM b UNION ALL SELECT x FROM c UNION ALL SELECT x FROM d; +SELECT '-'; +SELECT x FROM a UNION ALL SELECT x FROM c UNION ALL SELECT x FROM d; +SELECT '-'; +SELECT x FROM a UNION ALL SELECT x FROM b UNION ALL SELECT x FROM d; +SELECT '-'; +SELECT x FROM a UNION ALL SELECT x FROM b UNION ALL SELECT x FROM c; + +SELECT '---'; + +SELECT x FROM a UNION ALL SELECT x FROM b UNION ALL SELECT x FROM c UNION ALL SELECT x FROM d; + +SELECT '---'; + +SELECT [CAST('abc' AS LowCardinality(String)), CAST('def' AS Nullable(String))]; +SELECT [CAST('abc' AS LowCardinality(String)), CAST('def' AS FixedString(3))]; +SELECT [CAST('abc' AS LowCardinality(String)), CAST('def' AS LowCardinality(FixedString(3)))]; From 7138ccca2b106ed03d8f292c817d59a38498c213 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 01:44:29 +0300 Subject: [PATCH 145/196] Whitespace --- src/Functions/array/arrayIntersect.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/array/arrayIntersect.cpp b/src/Functions/array/arrayIntersect.cpp index 24db3c0cd08..ef216f96a8a 100644 --- a/src/Functions/array/arrayIntersect.cpp +++ b/src/Functions/array/arrayIntersect.cpp @@ -251,7 +251,6 @@ FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns( } else { - if (!arg.type->equals(*return_type) && !arg.type->equals(*nullable_return_type)) { /// If result has array type Array(T) still cast Array(Nullable(U)) to Array(Nullable(T)) From 454b0a2678131a93b6c6078aa80cdfea2d8bd1ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 01:58:22 +0300 Subject: [PATCH 146/196] Update test --- .../01372_wrong_order_by_removal.reference | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01372_wrong_order_by_removal.reference b/tests/queries/0_stateless/01372_wrong_order_by_removal.reference index c1b36bd53ca..8ed303c04aa 100644 --- a/tests/queries/0_stateless/01372_wrong_order_by_removal.reference +++ b/tests/queries/0_stateless/01372_wrong_order_by_removal.reference @@ -1 +1,16 @@ -SELECT \n k,\n groupArrayMovingSum(v)\nFROM \n(\n SELECT \n k,\n dt,\n v\n FROM moving_sum_num\n ORDER BY \n k ASC,\n dt ASC\n)\nGROUP BY k\nORDER BY k ASC +SELECT + k, + groupArrayMovingSum(v) +FROM +( + SELECT + k, + dt, + v + FROM moving_sum_num + ORDER BY + k ASC, + dt ASC +) +GROUP BY k +ORDER BY k ASC From 934d18b64968da0c7db55a69ce48ccd6ff23498d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 02:41:10 +0300 Subject: [PATCH 147/196] Added a test for PaddedPODArray just in case --- src/Common/tests/gtest_pod_array.cpp | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/src/Common/tests/gtest_pod_array.cpp b/src/Common/tests/gtest_pod_array.cpp index b77bc56b59a..988a3e649ba 100644 --- a/src/Common/tests/gtest_pod_array.cpp +++ b/src/Common/tests/gtest_pod_array.cpp @@ -44,3 +44,25 @@ TEST(Common, PODPushBackRawMany) EXPECT_EQ(15, chars.size()); EXPECT_EQ(std::string("first0123456789"), std::string(chars.data(), chars.size())); } + +TEST(Common, PODNoOverallocation) +{ + /// Check that PaddedPODArray allocates for smaller number of elements than the power of two due to padding. + /// NOTE: It's Ok to change these numbers if you will modify initial size or padding. + + PaddedPODArray chars; + std::vector capacities; + + size_t prev_capacity = 0; + for (size_t i = 0; i < 1000000; ++i) + { + chars.emplace_back(); + if (chars.capacity() != prev_capacity) + { + prev_capacity = chars.capacity(); + capacities.emplace_back(prev_capacity); + } + } + + EXPECT_EQ(capacities, (std::vector{4065, 8161, 16353, 32737, 65505, 131041, 262113, 524257, 1048545})); +} From abf1e015b3cfde220f8094ae2e29c6f6bd8fb751 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 04:44:24 +0300 Subject: [PATCH 148/196] Add documentation for arrayFill --- src/Functions/array/arrayFill.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Functions/array/arrayFill.cpp b/src/Functions/array/arrayFill.cpp index 4cc2caaaafa..110a00eaa69 100644 --- a/src/Functions/array/arrayFill.cpp +++ b/src/Functions/array/arrayFill.cpp @@ -11,6 +11,11 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } + +/** Replaces values where condition is met with the previous value that have condition not met + * (or with the first value if condition was true for all elements before). + * Looks somewhat similar to arrayFilter, but instead removing elements, it fills gaps with the value of previous element. + */ template struct ArrayFillImpl { From 34e67b521a1f3f63896cba806dc359ffbec870f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 04:56:50 +0300 Subject: [PATCH 149/196] Fix ugly ugliness --- src/Functions/array/arrayFill.cpp | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Functions/array/arrayFill.cpp b/src/Functions/array/arrayFill.cpp index 110a00eaa69..d4b36a89ba5 100644 --- a/src/Functions/array/arrayFill.cpp +++ b/src/Functions/array/arrayFill.cpp @@ -50,11 +50,11 @@ struct ArrayFillImpl for (auto in_offset : in_offsets) { - array_end = in_offset - 1; + array_end = in_offset; - for (; end <= array_end; ++end) + for (; end < array_end; ++end) { - if (end == array_end || fill[end + 1] != fill[begin]) + if (end + 1 == array_end || fill[end + 1] != fill[begin]) { if (fill[begin]) out_data.insertRangeFrom(in_data, begin, end + 1 - begin); @@ -62,8 +62,8 @@ struct ArrayFillImpl { if constexpr (reverse) { - if (end == array_end) - out_data.insertManyFrom(in_data, array_end, end + 1 - begin); + if (end + 1 == array_end) + out_data.insertManyFrom(in_data, end, end + 1 - begin); else out_data.insertManyFrom(in_data, end + 1, end + 1 - begin); } @@ -80,7 +80,7 @@ struct ArrayFillImpl } } - array_begin = array_end + 1; + array_begin = array_end; } } else @@ -93,8 +93,7 @@ struct ArrayFillImpl if (column_fill_const->getValue()) return ColumnArray::create( array.getDataPtr(), - array.getOffsetsPtr() - ); + array.getOffsetsPtr()); size_t array_begin = 0; size_t array_end = 0; From cea3de52434e421a77aec1a6796cca5f7204bdfe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 04:58:32 +0300 Subject: [PATCH 150/196] Added a test --- tests/queries/0_stateless/01376_array_fill_empty.reference | 4 ++++ tests/queries/0_stateless/01376_array_fill_empty.sql | 4 ++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01376_array_fill_empty.reference create mode 100644 tests/queries/0_stateless/01376_array_fill_empty.sql diff --git a/tests/queries/0_stateless/01376_array_fill_empty.reference b/tests/queries/0_stateless/01376_array_fill_empty.reference new file mode 100644 index 00000000000..70e8ac46074 --- /dev/null +++ b/tests/queries/0_stateless/01376_array_fill_empty.reference @@ -0,0 +1,4 @@ +[] +[] +[] +[] diff --git a/tests/queries/0_stateless/01376_array_fill_empty.sql b/tests/queries/0_stateless/01376_array_fill_empty.sql new file mode 100644 index 00000000000..38eea8a51c1 --- /dev/null +++ b/tests/queries/0_stateless/01376_array_fill_empty.sql @@ -0,0 +1,4 @@ +SELECT arrayFill(x -> (x < 10), []); +SELECT arrayFill(x -> (x < 10), emptyArrayUInt8()); +SELECT arrayFill(x -> 1, []); +SELECT arrayFill(x -> 0, []); From 9c53a90e3debf612fd089a696da44a0ef57ff237 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 06:16:01 +0300 Subject: [PATCH 151/196] Non-significant changes --- src/Columns/ColumnAggregateFunction.cpp | 58 +++++++++---------- .../Transforms/TotalsHavingTransform.cpp | 7 +-- 2 files changed, 30 insertions(+), 35 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 3374d171059..5880bb789a8 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -88,35 +88,35 @@ void ColumnAggregateFunction::addArena(ConstArenaPtr arena_) MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr column) { /** If the aggregate function returns an unfinalized/unfinished state, - * then you just need to copy pointers to it and also shared ownership of data. - * - * Also replace the aggregate function with the nested function. - * That is, if this column is the states of the aggregate function `aggState`, - * then we return the same column, but with the states of the aggregate function `agg`. - * These are the same states, changing only the function to which they correspond. - * - * Further is quite difficult to understand. - * Example when this happens: - * - * SELECT k, finalizeAggregation(quantileTimingState(0.5)(x)) FROM ... GROUP BY k WITH TOTALS - * - * This calculates the aggregate function `quantileTimingState`. - * Its return type AggregateFunction(quantileTiming(0.5), UInt64)`. - * Due to the presence of WITH TOTALS, during aggregation the states of this aggregate function will be stored - * in the ColumnAggregateFunction column of type - * AggregateFunction(quantileTimingState(0.5), UInt64). - * Then, in `TotalsHavingTransform`, it will be called `convertToValues` method, - * to get the "ready" values. - * But it just converts a column of type - * `AggregateFunction(quantileTimingState(0.5), UInt64)` - * into `AggregateFunction(quantileTiming(0.5), UInt64)` - * - in the same states. - *column_aggregate_func - * Then `finalizeAggregation` function will be calculated, which will call `convertToValues` already on the result. - * And this converts a column of type - * AggregateFunction(quantileTiming(0.5), UInt64) - * into UInt16 - already finished result of `quantileTiming`. - */ + * then you just need to copy pointers to it and also shared ownership of data. + * + * Also replace the aggregate function with the nested function. + * That is, if this column is the states of the aggregate function `aggState`, + * then we return the same column, but with the states of the aggregate function `agg`. + * These are the same states, changing only the function to which they correspond. + * + * Further is quite difficult to understand. + * Example when this happens: + * + * SELECT k, finalizeAggregation(quantileTimingState(0.5)(x)) FROM ... GROUP BY k WITH TOTALS + * + * This calculates the aggregate function `quantileTimingState`. + * Its return type AggregateFunction(quantileTiming(0.5), UInt64)`. + * Due to the presence of WITH TOTALS, during aggregation the states of this aggregate function will be stored + * in the ColumnAggregateFunction column of type + * AggregateFunction(quantileTimingState(0.5), UInt64). + * Then, in `TotalsHavingTransform`, it will be called `convertToValues` method, + * to get the "ready" values. + * But it just converts a column of type + * `AggregateFunction(quantileTimingState(0.5), UInt64)` + * into `AggregateFunction(quantileTiming(0.5), UInt64)` + * - in the same states. + * + * Then `finalizeAggregation` function will be calculated, which will call `convertToValues` already on the result. + * And this converts a column of type + * AggregateFunction(quantileTiming(0.5), UInt64) + * into UInt16 - already finished result of `quantileTiming`. + */ auto & column_aggregate_func = assert_cast(*column); auto & func = column_aggregate_func.func; auto & data = column_aggregate_func.data; diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index eb1cbf6e9d8..0c82c99680d 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -21,13 +21,8 @@ void finalizeChunk(Chunk & chunk) auto columns = chunk.detachColumns(); for (auto & column : columns) - { if (typeid_cast(column.get())) - { - auto mut_column = IColumn::mutate(std::move(column)); - column = ColumnAggregateFunction::convertToValues(std::move(mut_column)); - } - } + column = ColumnAggregateFunction::convertToValues(IColumn::mutate(std::move(column))); chunk.setColumns(std::move(columns), num_rows); } From c36bf6d93709d8914a3354beb533a177c5d89745 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 8 Jul 2020 10:18:51 +0300 Subject: [PATCH 152/196] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ef4e02c5434..9d5a24ecbbc 100644 --- a/README.md +++ b/README.md @@ -17,4 +17,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events * [ClickHouse for genetic data (in Russian)](https://cloud.yandex.ru/events/152) on July 14, 2020. -* [ClickHouse virtual office hours](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/271522978/) on July 15, 2020. +* [ClickHouse virtual office hours](https://www.eventbrite.com/e/clickhouse-july-virtual-meetup-tickets-111199787558) on July 15, 2020. From 9ea3bdd0547c1ede8ddad8be917211b725aa4938 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 Jul 2020 11:41:39 +0300 Subject: [PATCH 153/196] Fix skip lists for old branches --- docker/test/stateful/Dockerfile | 45 +--------------- docker/test/stateful/run.sh | 60 ++++++++++++++++++++++ docker/test/stateful_with_coverage/run.sh | 7 ++- docker/test/stateless/Dockerfile | 37 +------------ docker/test/stateless/run.sh | 54 +++++++++++++++++++ docker/test/stateless_with_coverage/run.sh | 6 ++- docker/test/stress/stress | 21 ++++++-- 7 files changed, 146 insertions(+), 84 deletions(-) create mode 100755 docker/test/stateful/run.sh create mode 100755 docker/test/stateless/run.sh diff --git a/docker/test/stateful/Dockerfile b/docker/test/stateful/Dockerfile index bdf397a70e0..8a7aca80653 100644 --- a/docker/test/stateful/Dockerfile +++ b/docker/test/stateful/Dockerfile @@ -11,46 +11,5 @@ COPY s3downloader /s3downloader ENV DATASETS="hits visits" -CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ - dpkg -i package_folder/clickhouse-common-static-dbg_*.deb; \ - dpkg -i package_folder/clickhouse-server_*.deb; \ - dpkg -i package_folder/clickhouse-client_*.deb; \ - dpkg -i package_folder/clickhouse-test_*.deb; \ - mkdir -p /etc/clickhouse-server/dict_examples; \ - ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ - ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ - ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ - if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ - if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ - echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ - echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-8/bin/llvm-symbolizer" >> /etc/environment; \ - echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ - echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \ - echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \ - echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment; \ - service zookeeper start; sleep 5; \ - service clickhouse-server start && sleep 5 \ - && /s3downloader --dataset-names $DATASETS \ - && chmod 777 -R /var/lib/clickhouse \ - && clickhouse-client --query "SHOW DATABASES" \ - && clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" \ - && clickhouse-client --query "CREATE DATABASE test" \ - && service clickhouse-server restart && sleep 5 \ - && clickhouse-client --query "SHOW TABLES FROM datasets" \ - && clickhouse-client --query "SHOW TABLES FROM test" \ - && clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" \ - && clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" \ - && clickhouse-client --query "SHOW TABLES FROM test" \ - && clickhouse-test --testname --shard --zookeeper --no-stateless --use-skip-list $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +COPY run.sh / +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh new file mode 100755 index 00000000000..44c6c217b73 --- /dev/null +++ b/docker/test/stateful/run.sh @@ -0,0 +1,60 @@ +#!/bin/bash + +set -e -x + +dpkg -i package_folder/clickhouse-common-static_*.deb; +dpkg -i package_folder/clickhouse-common-static-dbg_*.deb +dpkg -i package_folder/clickhouse-server_*.deb +dpkg -i package_folder/clickhouse-client_*.deb +dpkg -i package_folder/clickhouse-test_*.deb + +mkdir -p /etc/clickhouse-server/dict_examples +ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/ +ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/ +ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/ +ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ +ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ +ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ + +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then + ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ + ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +fi + +ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer +echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment +echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-8/bin/llvm-symbolizer" >> /etc/environment +echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment +echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment +echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment +echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment + +service zookeeper start +sleep 5 +service clickhouse-server start +sleep 5 +/s3downloader --dataset-names $DATASETS +chmod 777 -R /var/lib/clickhouse +clickhouse-client --query "SHOW DATABASES" +clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" +clickhouse-client --query "CREATE DATABASE test" +service clickhouse-server restart && sleep 5 +clickhouse-client --query "SHOW TABLES FROM datasets" +clickhouse-client --query "SHOW TABLES FROM test" +clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" +clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" +clickhouse-client --query "SHOW TABLES FROM test" + +if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then + SKIP_LIST_OPT="--use-skip-list" +fi + +clickhouse-test --testname --shard --zookeeper --no-stateless "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index 13b69c73b89..75ecf4d4cb7 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -105,7 +105,12 @@ LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW TABL LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW TABLES FROM test" -LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-test --testname --shard --zookeeper --no-stateless --use-skip-list $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt + +if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then + SKIP_LIST_OPT="--use-skip-list" +fi + +LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-test --testname --shard --zookeeper --no-stateless "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt kill_clickhouse diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index eee493d4430..541122a07a7 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -49,38 +49,5 @@ RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/ echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; -CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ - dpkg -i package_folder/clickhouse-common-static-dbg_*.deb; \ - dpkg -i package_folder/clickhouse-server_*.deb; \ - dpkg -i package_folder/clickhouse-client_*.deb; \ - dpkg -i package_folder/clickhouse-test_*.deb; \ - mkdir -p /etc/clickhouse-server/dict_examples; \ - ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ - ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ - ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/; \ - ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/; \ - if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/; fi; \ - if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/; fi; \ - if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/; fi; \ - ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml; \ - service zookeeper start; sleep 5; \ - service clickhouse-server start && sleep 5 && clickhouse-test --testname --shard --zookeeper --use-skip-list $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +COPY run.sh / +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh new file mode 100755 index 00000000000..f54b881f69c --- /dev/null +++ b/docker/test/stateless/run.sh @@ -0,0 +1,54 @@ +#!/bin/bash + +set -e -x + +dpkg -i package_folder/clickhouse-common-static_*.deb +dpkg -i package_folder/clickhouse-common-static-dbg_*.deb +dpkg -i package_folder/clickhouse-server_*.deb +dpkg -i package_folder/clickhouse-client_*.deb +dpkg -i package_folder/clickhouse-test_*.deb + +mkdir -p /etc/clickhouse-server/dict_examples +ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/ +ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/ +ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/ +ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ +ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ +ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/ +ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ + +if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then + ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ +fi +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then + ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ + ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +fi + +ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml + +service zookeeper start +sleep 5 +service clickhouse-server start && sleep 5 + +if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then + SKIP_LIST_OPT="--use-skip-list" +fi + +clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 37ad286e004..f6e1f79d1a2 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -76,7 +76,11 @@ start_clickhouse sleep 10 -LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-test --testname --shard --zookeeper --use-skip-list $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then + SKIP_LIST_OPT="--use-skip-list" +fi + +LLVM_PROFILE_FILE='client_coverage.profraw' clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt kill_clickhouse diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 46fa3e95f55..b5ed4dbd85d 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -1,5 +1,5 @@ #!/usr/bin/env python -#-*- coding: utf-8 -*- +# -*- coding: utf-8 -*- from multiprocessing import cpu_count from subprocess import Popen, check_call import os @@ -8,27 +8,39 @@ import argparse import logging import time + +def get_skip_list_cmd(path): + with open(path, 'r') as f: + for line in f: + if '--use-skip-list' in line: + return '--use-skip-list' + return '' + + def run_perf_test(cmd, xmls_path, output_folder): output_path = os.path.join(output_folder, "perf_stress_run.txt") f = open(output_path, 'w') p = Popen("{} --skip-tags=long --recursive --input-files {}".format(cmd, xmls_path), shell=True, stdout=f, stderr=f) return p + def run_func_test(cmd, output_prefix, num_processes, skip_tests_option): + skip_list_opt = get_skip_list_cmd(cmd) output_paths = [os.path.join(output_prefix, "stress_test_run_{}.txt".format(i)) for i in range(num_processes)] f = open(output_paths[0], 'w') - main_command = "{} --use-skip-list {}".format(cmd, skip_tests_option) + main_command = "{} {} {}".format(cmd, skip_list_opt, skip_tests_option) logging.info("Run func tests main cmd '%s'", main_command) pipes = [Popen(main_command, shell=True, stdout=f, stderr=f)] for output_path in output_paths[1:]: time.sleep(0.5) f = open(output_path, 'w') - full_command = "{} --use-skip-list --order=random {}".format(cmd, skip_tests_option) + full_command = "{} {} --order=random {}".format(cmd, skip_list_opt, skip_tests_option) logging.info("Run func tests '%s'", full_command) p = Popen(full_command, shell=True, stdout=f, stderr=f) pipes.append(p) return pipes + def check_clickhouse_alive(cmd): try: logging.info("Checking ClickHouse still alive") @@ -37,10 +49,11 @@ def check_clickhouse_alive(cmd): except: return False + if __name__ == "__main__": logging.basicConfig(level=logging.INFO, format='%(asctime)s %(message)s') parser = argparse.ArgumentParser(description="ClickHouse script for running stresstest") - parser.add_argument("--test-cmd", default='clickhouse-test') + parser.add_argument("--test-cmd", default='/usr/bin/clickhouse-test') parser.add_argument("--skip-func-tests", default='') parser.add_argument("--client-cmd", default='clickhouse-client') parser.add_argument("--perf-test-cmd", default='clickhouse-performance-test') From 68189db872eb0c321fe8bf267d895e652db58b53 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 8 Jul 2020 12:40:33 +0300 Subject: [PATCH 154/196] Update int_parsing.xml --- tests/performance/int_parsing.xml | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/tests/performance/int_parsing.xml b/tests/performance/int_parsing.xml index 3008ed06684..3b8620e46c3 100644 --- a/tests/performance/int_parsing.xml +++ b/tests/performance/int_parsing.xml @@ -1,15 +1,16 @@ hits_100m_single + hits_10m_single - SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(WatchID)) SETTINGS max_threads = 1 + SELECT count() FROM hits_10m_single WHERE NOT ignore(toString(WatchID)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(JavaEnable)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(GoodEvent)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(CounterID)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(ClientIP)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(RegionID)) SETTINGS max_threads = 1 - SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(UserID)) SETTINGS max_threads = 1 + SELECT count() FROM hits_10m_single WHERE NOT ignore(toString(UserID)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(CounterClass)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(OS)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(UserAgent)) SETTINGS max_threads = 1 @@ -43,7 +44,7 @@ SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(IsLink)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(IsDownload)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(IsNotBounce)) SETTINGS max_threads = 1 - SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(FUniqID)) SETTINGS max_threads = 1 + SELECT count() FROM hits_10m_single WHERE NOT ignore(toString(FUniqID)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(HID)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(IsOldCounter)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(IsEvent)) SETTINGS max_threads = 1 @@ -70,7 +71,7 @@ SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(ParamPrice)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(ParamCurrencyID)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(HasGCLID)) SETTINGS max_threads = 1 - SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(RefererHash)) SETTINGS max_threads = 1 - SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(URLHash)) SETTINGS max_threads = 1 + SELECT count() FROM hits_10m_single WHERE NOT ignore(toString(RefererHash)) SETTINGS max_threads = 1 + SELECT count() FROM hits_10m_single WHERE NOT ignore(toString(URLHash)) SETTINGS max_threads = 1 SELECT count() FROM hits_100m_single WHERE NOT ignore(toString(CLID)) SETTINGS max_threads = 1 From 08762dc8250463b0ed3d66a11d21b2d9719ff83e Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 8 Jul 2020 12:46:44 +0300 Subject: [PATCH 155/196] Update build.sh --- docker/packager/binary/build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 30a576a5d76..791fd18d80c 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -18,7 +18,7 @@ ccache --zero-stats ||: ln -s /usr/lib/x86_64-linux-gnu/libOpenCL.so.1.0.0 /usr/lib/libOpenCL.so ||: rm -f CMakeCache.txt cmake --debug-trycompile --verbose=1 -DCMAKE_VERBOSE_MAKEFILE=1 -LA -DCMAKE_BUILD_TYPE=$BUILD_TYPE -DSANITIZE=$SANITIZER $CMAKE_FLAGS .. -ninja clickhouse-bundle +ninja $NINJA_FLAGS clickhouse-bundle mv ./programs/clickhouse* /output mv ./src/unit_tests_dbms /output find . -name '*.so' -print -exec mv '{}' /output \; From dc8051f7371f3af13f3c16f67246888b1ea090d3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 Jul 2020 12:47:14 +0300 Subject: [PATCH 156/196] Fixes --- docker/test/stateful/run.sh | 11 +++++------ docker/test/stateful_with_coverage/run.sh | 5 ++--- docker/test/stateless/Dockerfile | 2 +- docker/test/stateless/run.sh | 9 ++++++++- docker/test/stateless_with_coverage/Dockerfile | 7 +++++++ docker/test/stateless_with_coverage/run.sh | 6 +++--- docker/test/stress/Dockerfile | 1 - tests/clickhouse-test | 2 +- 8 files changed, 27 insertions(+), 16 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 44c6c217b73..5be14970914 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -29,13 +29,12 @@ if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ fi -ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment -echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-8/bin/llvm-symbolizer" >> /etc/environment +echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment -echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment -echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment -echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-6.0/bin/llvm-symbolizer" >> /etc/environment +echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment +echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment +echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment service zookeeper start sleep 5 @@ -53,7 +52,7 @@ clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" clickhouse-client --query "SHOW TABLES FROM test" -if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then +if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then SKIP_LIST_OPT="--use-skip-list" fi diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index 75ecf4d4cb7..66877362d10 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -66,8 +66,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/; \ - ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/lib/llvm-8/bin/llvm-symbolizer /usr/bin/llvm-symbolizer + ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/; service zookeeper start @@ -106,7 +105,7 @@ LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TA LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" LLVM_PROFILE_FILE='client_%h_%p_%m.profraw' clickhouse-client --query "SHOW TABLES FROM test" -if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then +if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then SKIP_LIST_OPT="--use-skip-list" fi diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 541122a07a7..f15c2d38971 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -47,7 +47,7 @@ RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ - ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; + ln -s /usr/lib/llvm-10/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; COPY run.sh / CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index f54b881f69c..08adc94f535 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -43,11 +43,18 @@ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml +echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment +echo "TSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment +echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment +echo "ASAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment +echo "UBSAN_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment +echo "LLVM_SYMBOLIZER_PATH=/usr/lib/llvm-10/bin/llvm-symbolizer" >> /etc/environment + service zookeeper start sleep 5 service clickhouse-server start && sleep 5 -if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then +if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then SKIP_LIST_OPT="--use-skip-list" fi diff --git a/docker/test/stateless_with_coverage/Dockerfile b/docker/test/stateless_with_coverage/Dockerfile index ae4ab56e3d8..9a35f4f4b22 100644 --- a/docker/test/stateless_with_coverage/Dockerfile +++ b/docker/test/stateless_with_coverage/Dockerfile @@ -44,4 +44,11 @@ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone COPY run.sh /run.sh +# Sanitizer options +RUN echo "TSAN_OPTIONS='verbosity=1000 halt_on_error=1 history_size=7'" >> /etc/environment; \ + echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ + echo "MSAN_OPTIONS='abort_on_error=1'" >> /etc/environment; \ + ln -s /usr/lib/llvm-10/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; + + CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index f6e1f79d1a2..64e171b2869 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -66,8 +66,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/; \ - ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml; \ - ln -s /usr/lib/llvm-8/bin/llvm-symbolizer /usr/bin/llvm-symbolizer + ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml service zookeeper start sleep 5 @@ -76,7 +75,8 @@ start_clickhouse sleep 10 -if cat /usr/bin/clickhouse-test | grep -q '--use-skip-list'; then + +if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then SKIP_LIST_OPT="--use-skip-list" fi diff --git a/docker/test/stress/Dockerfile b/docker/test/stress/Dockerfile index a5aa3bbf004..12e2d5579b4 100644 --- a/docker/test/stress/Dockerfile +++ b/docker/test/stress/Dockerfile @@ -33,7 +33,6 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ dpkg -i package_folder/clickhouse-test_*.deb; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ - ln -s /usr/lib/llvm-9/bin/llvm-symbolizer /usr/bin/llvm-symbolizer; \ echo "TSAN_OPTIONS='halt_on_error=1 history_size=7 ignore_noninstrumented_modules=1 verbosity=1'" >> /etc/environment; \ echo "UBSAN_OPTIONS='print_stacktrace=1'" >> /etc/environment; \ echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment; \ diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fc3cfb0b432..07fe681c2c8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -467,7 +467,7 @@ def main(args): if args.use_skip_list: tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags) else: - tests_to_skip_from_list = {} + tests_to_skip_from_list = set([]) if args.skip: args.skip = set(args.skip) | tests_to_skip_from_list From 76ef8feb04b0b0dee76eb58b66dcf8db2458926b Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 8 Jul 2020 12:52:03 +0300 Subject: [PATCH 157/196] Update packager --- docker/packager/packager | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/packager/packager b/docker/packager/packager index c3e0778e10a..bc97429336c 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -149,6 +149,8 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ if clang_tidy: cmake_flags.append('-DENABLE_CLANG_TIDY=1') + # Don't stop on first error to find more clang-tidy errors in one run. + result.append('NINJA_FLAGS=-k0') if with_coverage: cmake_flags.append('-DWITH_COVERAGE=1') From c7729e465aa7673d3584441d010c5706085231df Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 Jul 2020 13:10:53 +0300 Subject: [PATCH 158/196] Hide nonzero error code on testflows runner --- tests/testflows/runner | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/testflows/runner b/tests/testflows/runner index eef51b3c20f..6522c8f47f7 100755 --- a/tests/testflows/runner +++ b/tests/testflows/runner @@ -118,4 +118,5 @@ if __name__ == "__main__": ) print("Running testflows container as: '" + cmd + "'.") - subprocess.check_call(cmd, shell=True) + # testflows return non zero error code on failed tests + subprocess.call(cmd, shell=True) From de0a6c08396e97a86ffb16fa13b98376be4e5b97 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 8 Jul 2020 13:17:06 +0300 Subject: [PATCH 159/196] fix builds --- programs/client/QueryFuzzer.cpp | 4 ++-- programs/client/QueryFuzzer.h | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/client/QueryFuzzer.cpp b/programs/client/QueryFuzzer.cpp index b24add1e733..d48b13f82dc 100644 --- a/programs/client/QueryFuzzer.cpp +++ b/programs/client/QueryFuzzer.cpp @@ -118,7 +118,7 @@ Field QueryFuzzer::fuzzField(Field field) str = str + str + str + str; break; case 4: - if (str.size() > 0) + if (!str.empty()) { str[fuzz_rand() % str.size()] = '\0'; } @@ -132,7 +132,7 @@ Field QueryFuzzer::fuzzField(Field field) { auto & arr = field.reinterpret(); - if (fuzz_rand() % 5 == 0 && arr.size() > 0) + if (fuzz_rand() % 5 == 0 && !arr.empty()) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); diff --git a/programs/client/QueryFuzzer.h b/programs/client/QueryFuzzer.h index 58168089b1b..db1102a94d8 100644 --- a/programs/client/QueryFuzzer.h +++ b/programs/client/QueryFuzzer.h @@ -30,11 +30,11 @@ struct QueryFuzzer std::unordered_set aliases_set; std::vector aliases; - std::unordered_map column_like_map; - std::vector column_like; + std::unordered_map column_like_map; + std::vector column_like; - std::unordered_map table_like_map; - std::vector table_like; + std::unordered_map table_like_map; + std::vector table_like; // This is the only function you have to call -- it will modify the passed // ASTPtr to point to new AST with some random changes. From 4e40965c8803c34c33ab5d054daef1aed7281a8b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 8 Jul 2020 13:55:39 +0300 Subject: [PATCH 160/196] Show error after TrieDictionary failed to load. --- src/Dictionaries/TrieDictionary.cpp | 14 ++------------ src/Dictionaries/TrieDictionary.h | 4 ---- 2 files changed, 2 insertions(+), 16 deletions(-) diff --git a/src/Dictionaries/TrieDictionary.cpp b/src/Dictionaries/TrieDictionary.cpp index 8bc45069bd5..4da52b1cd5a 100644 --- a/src/Dictionaries/TrieDictionary.cpp +++ b/src/Dictionaries/TrieDictionary.cpp @@ -64,18 +64,8 @@ TrieDictionary::TrieDictionary( { createAttributes(); trie = btrie_create(); - - try - { - loadData(); - calculateBytesAllocated(); - } - catch (...) - { - creation_exception = std::current_exception(); - } - - creation_time = std::chrono::system_clock::now(); + loadData(); + calculateBytesAllocated(); } TrieDictionary::~TrieDictionary() diff --git a/src/Dictionaries/TrieDictionary.h b/src/Dictionaries/TrieDictionary.h index 5f8b5df89bf..063a73b0346 100644 --- a/src/Dictionaries/TrieDictionary.h +++ b/src/Dictionaries/TrieDictionary.h @@ -249,10 +249,6 @@ private: size_t bucket_count = 0; mutable std::atomic query_count{0}; - std::chrono::time_point creation_time; - - std::exception_ptr creation_exception; - Poco::Logger * logger; }; From 26295f8695c1a103513b4107d2e3d09fa991cb0e Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Wed, 8 Jul 2020 14:24:50 +0300 Subject: [PATCH 161/196] try remove strange logic in DuplicateOrderByVisitor (#12267) --- src/Interpreters/DuplicateOrderByVisitor.h | 10 ---------- src/Interpreters/SyntaxAnalyzer.cpp | 2 +- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/src/Interpreters/DuplicateOrderByVisitor.h b/src/Interpreters/DuplicateOrderByVisitor.h index 72e0419f114..f5e5763c719 100644 --- a/src/Interpreters/DuplicateOrderByVisitor.h +++ b/src/Interpreters/DuplicateOrderByVisitor.h @@ -81,19 +81,9 @@ public: using TypeToVisit = ASTSelectQuery; const Context & context; - bool done = false; void visit(ASTSelectQuery & select_query, ASTPtr &) { - if (done) - return; - - for (const auto & elem : select_query.children) - { - if (elem->as() && !elem->as()->is_standalone) - return; - } - if (select_query.orderBy() || select_query.groupBy()) { for (auto & elem : select_query.children) diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index 9bc7ae055d2..932678e6955 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -512,7 +512,7 @@ void optimizeOrderBy(const ASTSelectQuery * select_query) /// Optimize duplicate ORDER BY and DISTINCT void optimizeDuplicateOrderByAndDistinct(ASTPtr & query, const Context & context) { - DuplicateOrderByVisitor::Data order_by_data{context, false}; + DuplicateOrderByVisitor::Data order_by_data{context}; DuplicateOrderByVisitor(order_by_data).visit(query); DuplicateDistinctVisitor::Data distinct_data{}; DuplicateDistinctVisitor(distinct_data).visit(query); From bd14d0647639eb4b82e3bb1c61cd8e4bab40bfee Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 8 Jul 2020 14:25:43 +0300 Subject: [PATCH 162/196] Update README.md --- docker/test/performance-comparison/README.md | 22 ++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/README.md b/docker/test/performance-comparison/README.md index b36bbc531c4..e0dd845fb1c 100644 --- a/docker/test/performance-comparison/README.md +++ b/docker/test/performance-comparison/README.md @@ -18,8 +18,26 @@ to have false positives. ### How to read the report -Should add inline comments there, because who reads the docs anyway. They must -be collapsible and I am afraid of Javascript, so I'm going to do it later. +The check status summarizes the report in a short text message like `1 faster, 10 unstable`: +* `1 faster` -- how many queries became faster, +* `1 slower` -- how many queries are slower, +* `1 too long` -- how many queries are taking too long to run, +* `1 unstable` -- how many queries have unstable results, +* `1 errors` -- how many errors there are in total. The number of errors includes slower tests, tests that are too long, errors while running the tests and building reports, etc. Please look at the main report page to investigate these errors. + +The report page itself constists of a several tables. Some of them always signify errors, e.g. "Run errors" -- the very presence of this table indicates that there were errors during the test, that are not normal and must be fixed. Some tables are mostly informational, e.g. "Test times" -- they reflect normal test results. But if a cell in such table is marked in red, this also means an error, e.g., a test is taking too long to run. + +#### Tested commits +Informational. Log messages for the commits that are tested. Note that for the right commit, we show nominal tested commit `pull/*/head` and real tested commit `pull/*/merge`, which is generated by GitHub by merging latest master to the `pull/*/head` and which we actually build and test in CI. + +#### Run errors +These are errors that must be fixed. The errors that ocurred when running some test queries. For more information about the error, download test output archive and see `test-name-err.log`. To reproduce, see 'How to run' below. + +#### Slow on client +These are errors that must be fixed. This table shows queries that take significantly longer to process on the client than on the server. A possible reason might be sending too much data to the client, e.g., a forgotten `format Null`. + +#### Partial queries +Informational, no action required if no red cells. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. ### How to run Run the entire docker container, specifying PR number (0 for master) From f16afa4deee6ba451ebda3909e329641cc4bfe2b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 14:49:44 +0300 Subject: [PATCH 163/196] Fix typo in setting name --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 08b03e78282..f0624fe9801 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -369,7 +369,7 @@ struct Settings : public SettingsCollection M(SettingBool, optimize_move_functions_out_of_any, true, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(SettingBool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ M(SettingBool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \ - M(SettingBool, optimize_if_chain_to_miltiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ + M(SettingBool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(SettingBool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ M(SettingBool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ \ From 344e2713fa0e916ab7adaee37c48faa5548107f4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 8 Jul 2020 14:52:06 +0300 Subject: [PATCH 164/196] Update README.md --- docker/test/performance-comparison/README.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/docker/test/performance-comparison/README.md b/docker/test/performance-comparison/README.md index e0dd845fb1c..b3bed40647b 100644 --- a/docker/test/performance-comparison/README.md +++ b/docker/test/performance-comparison/README.md @@ -39,6 +39,21 @@ These are errors that must be fixed. This table shows queries that take signific #### Partial queries Informational, no action required if no red cells. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. +#### Changes in performance +These are the queries for which we observe a statistically significant change in performance. Note that there will always be some false positives -- we try to filter by p < 0.001, and have 2000 queries, so two false positives per run are expected. In practice we have more -- e.g. code layout changed because of some unknowable jitter in compiler internals, so the change we observe is real, but it is a 'false positive' in the sense that it is not directly caused by your changes. If, based on your knowledge of ClickHouse internals, you can decide that the observed test changes are not relevant to the changes made in the tested PR, you can ignore them. + +You can find flame graphs for queries with performance changes in the test output archive, in files named as 'my_test_0_Cpu_SELECT 1 FROM....FORMAT Null.left.svg'. First goes the test name, then the query number in the test, then the trace type (same as in `system.trace_log`), and then the server version (left is old and right is new). + +#### Unstable queries +These are queries for which we did not observe a statistically significant change in performance, but for which the variance in query performance is very high. This means that we are likely to observe big changes in performance even in the absence of real changes, e.g. when comparing the server to itself. Such queries are going to have bad sensitivity as performance tests -- if a query has, say, 50% expected variability, this means we are going to see changes in performance up to 50%, even when there were no real changes in the code. And because of this, we won't be able to detect changes less than 50% with such a query, which is pretty bad. The reasons for the high variability must be investigated and fixed; ideally, the variability should be brought under 5-10%. + +The most frequent reason for instability is that the query is just too short -- e.g. below 0.1 seconds. Bringing query time to 0.2 seconds or above usually helps. +Other reasons may include: +* using a lot of memory which is allocated differently between servers, so the access time may vary. This may apply to your queries if you have a `Memory` engine table that is bigger than 1 GB. For example, this problem has plagued `arithmetic` and `logical_functions` tests for a long time. +* having some threshold behavior in the query, e.g. you insert to a Buffer table and it is flushed only on some query runs, so you get a much higher time for them. +Investigating the instablility is the hardest problem in performance testing, and we still have not been able to understand the reasons behind the instability of some queries. There are some data that can help you in the performance test output archive. Look for files named 'my_unstable_test_0_SELECT 1...FORMAT Null.{left,right}.metrics.rep'. They contain metrics from `system.query_log.ProfileEvents` and functions from stack traces from `system.trace_log`, that vary significantly between query runs. The second column is array of \[min, med, max] values for the metric. Say, if you see `PerfCacheMisses` there, it may mean that the code being tested has not-so-cache-local memory access pattern that is sensitive to memory layout. + + ### How to run Run the entire docker container, specifying PR number (0 for master) and SHA of the commit to test. The reference revision is determined as a nearest From 56b835effa906d39180453e8897e838ccb1e936b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Jul 2020 14:53:07 +0300 Subject: [PATCH 165/196] Fix typo in setting name --- src/Interpreters/SyntaxAnalyzer.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/SyntaxAnalyzer.cpp b/src/Interpreters/SyntaxAnalyzer.cpp index 932678e6955..6bacf0f9773 100644 --- a/src/Interpreters/SyntaxAnalyzer.cpp +++ b/src/Interpreters/SyntaxAnalyzer.cpp @@ -569,12 +569,12 @@ void optimizeUsing(const ASTSelectQuery * select_query) expression_list = uniq_expressions_list; } -void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_miltiif) +void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif) { /// Optimize if with constant condition after constants was substituted instead of scalar subqueries. OptimizeIfWithConstantConditionVisitor(aliases).visit(query); - if (if_chain_to_miltiif) + if (if_chain_to_multiif) OptimizeIfChainsVisitor().visit(query); } @@ -969,7 +969,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect( executeScalarSubqueries(query, context, subquery_depth, result.scalars, select_options.only_analyze); { - optimizeIf(query, result.aliases, settings.optimize_if_chain_to_miltiif); + optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); /// Move arithmetic operations out of aggregation functions if (settings.optimize_arithmetic_operations_in_aggregate_functions) @@ -1046,7 +1046,7 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Executing scalar subqueries. Column defaults could be a scalar subquery. executeScalarSubqueries(query, context, 0, result.scalars, false); - optimizeIf(query, result.aliases, settings.optimize_if_chain_to_miltiif); + optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); if (allow_aggregations) { From 306bad6b813d00d6033b1d3e2341d26b7fe9a813 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 Jul 2020 15:05:20 +0300 Subject: [PATCH 166/196] Add tests to skip list --- tests/queries/skip_list.json | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 2ef448e5d99..791886d6825 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -126,6 +126,8 @@ "01055_compact_parts", "01039_mergetree_exec_time", "00933_ttl_simple", - "00753_system_columns_and_system_tables" + "00753_system_columns_and_system_tables", + "01343_min_bytes_to_use_mmap_io", + "01344_min_bytes_to_use_mmap_io_index" ] } From c9162f6e779b27a4fd2cca4796bcde00589c2fa2 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 8 Jul 2020 15:05:49 +0300 Subject: [PATCH 167/196] Update README.md --- docker/test/performance-comparison/README.md | 29 ++++++++++++++++---- 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/docker/test/performance-comparison/README.md b/docker/test/performance-comparison/README.md index b3bed40647b..302b2225d34 100644 --- a/docker/test/performance-comparison/README.md +++ b/docker/test/performance-comparison/README.md @@ -28,31 +28,48 @@ The check status summarizes the report in a short text message like `1 faster, 1 The report page itself constists of a several tables. Some of them always signify errors, e.g. "Run errors" -- the very presence of this table indicates that there were errors during the test, that are not normal and must be fixed. Some tables are mostly informational, e.g. "Test times" -- they reflect normal test results. But if a cell in such table is marked in red, this also means an error, e.g., a test is taking too long to run. #### Tested commits -Informational. Log messages for the commits that are tested. Note that for the right commit, we show nominal tested commit `pull/*/head` and real tested commit `pull/*/merge`, which is generated by GitHub by merging latest master to the `pull/*/head` and which we actually build and test in CI. +Informational, no action required. Log messages for the commits that are tested. Note that for the right commit, we show nominal tested commit `pull/*/head` and real tested commit `pull/*/merge`, which is generated by GitHub by merging latest master to the `pull/*/head` and which we actually build and test in CI. #### Run errors -These are errors that must be fixed. The errors that ocurred when running some test queries. For more information about the error, download test output archive and see `test-name-err.log`. To reproduce, see 'How to run' below. +Action required for every item -- these are errors that must be fixed. The errors that ocurred when running some test queries. For more information about the error, download test output archive and see `test-name-err.log`. To reproduce, see 'How to run' below. #### Slow on client -These are errors that must be fixed. This table shows queries that take significantly longer to process on the client than on the server. A possible reason might be sending too much data to the client, e.g., a forgotten `format Null`. +Action required for every item -- these are errors that must be fixed. This table shows queries that take significantly longer to process on the client than on the server. A possible reason might be sending too much data to the client, e.g., a forgotten `format Null`. #### Partial queries -Informational, no action required if no red cells. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. +Action required for the cells marked in red. Shows the queries we are unable to run on an old server -- probably because they contain a new function. You should see this table when you add a new function and a performance test for it. Check that the run time and variance are acceptable (run time between 0.1 and 1 seconds, variance below 10%). If not, they will be highlighted in red. #### Changes in performance -These are the queries for which we observe a statistically significant change in performance. Note that there will always be some false positives -- we try to filter by p < 0.001, and have 2000 queries, so two false positives per run are expected. In practice we have more -- e.g. code layout changed because of some unknowable jitter in compiler internals, so the change we observe is real, but it is a 'false positive' in the sense that it is not directly caused by your changes. If, based on your knowledge of ClickHouse internals, you can decide that the observed test changes are not relevant to the changes made in the tested PR, you can ignore them. +Action required for the cells marked in red, and some cheering is appropriate for the cells marked in green. These are the queries for which we observe a statistically significant change in performance. Note that there will always be some false positives -- we try to filter by p < 0.001, and have 2000 queries, so two false positives per run are expected. In practice we have more -- e.g. code layout changed because of some unknowable jitter in compiler internals, so the change we observe is real, but it is a 'false positive' in the sense that it is not directly caused by your changes. If, based on your knowledge of ClickHouse internals, you can decide that the observed test changes are not relevant to the changes made in the tested PR, you can ignore them. You can find flame graphs for queries with performance changes in the test output archive, in files named as 'my_test_0_Cpu_SELECT 1 FROM....FORMAT Null.left.svg'. First goes the test name, then the query number in the test, then the trace type (same as in `system.trace_log`), and then the server version (left is old and right is new). #### Unstable queries -These are queries for which we did not observe a statistically significant change in performance, but for which the variance in query performance is very high. This means that we are likely to observe big changes in performance even in the absence of real changes, e.g. when comparing the server to itself. Such queries are going to have bad sensitivity as performance tests -- if a query has, say, 50% expected variability, this means we are going to see changes in performance up to 50%, even when there were no real changes in the code. And because of this, we won't be able to detect changes less than 50% with such a query, which is pretty bad. The reasons for the high variability must be investigated and fixed; ideally, the variability should be brought under 5-10%. +Action required for the cells marked in red. These are queries for which we did not observe a statistically significant change in performance, but for which the variance in query performance is very high. This means that we are likely to observe big changes in performance even in the absence of real changes, e.g. when comparing the server to itself. Such queries are going to have bad sensitivity as performance tests -- if a query has, say, 50% expected variability, this means we are going to see changes in performance up to 50%, even when there were no real changes in the code. And because of this, we won't be able to detect changes less than 50% with such a query, which is pretty bad. The reasons for the high variability must be investigated and fixed; ideally, the variability should be brought under 5-10%. The most frequent reason for instability is that the query is just too short -- e.g. below 0.1 seconds. Bringing query time to 0.2 seconds or above usually helps. Other reasons may include: * using a lot of memory which is allocated differently between servers, so the access time may vary. This may apply to your queries if you have a `Memory` engine table that is bigger than 1 GB. For example, this problem has plagued `arithmetic` and `logical_functions` tests for a long time. * having some threshold behavior in the query, e.g. you insert to a Buffer table and it is flushed only on some query runs, so you get a much higher time for them. + Investigating the instablility is the hardest problem in performance testing, and we still have not been able to understand the reasons behind the instability of some queries. There are some data that can help you in the performance test output archive. Look for files named 'my_unstable_test_0_SELECT 1...FORMAT Null.{left,right}.metrics.rep'. They contain metrics from `system.query_log.ProfileEvents` and functions from stack traces from `system.trace_log`, that vary significantly between query runs. The second column is array of \[min, med, max] values for the metric. Say, if you see `PerfCacheMisses` there, it may mean that the code being tested has not-so-cache-local memory access pattern that is sensitive to memory layout. +#### Skipped tests +Informational, no action required. Shows the tests that were skipped, and the reason for it. Normally it is because the data set required for the test was not loaded, or the test is marked as 'long' -- both cases mean that the test is too big to be ran per-commit. + +#### Test performance changes +Informational, no action required. This table summarizes the changes in performance of queries in each test -- how many queries have changed, how many are unstable, and what is the magnitude of the changes. + +#### Test times +Action required for the cells marked in red. This table shows the run times for all the tests. You may have to fix two kinds of errors in this table: +1) Average query run time is too long -- probalby means that the preparatory steps such as creating the table and filling them with data are taking too long. Try to make them faster. +2) Longest query run time is too long -- some particular queries are taking too long, try to make them faster. The ideal query run time is between 0.1 and 1 s. + +#### Concurrent benchmarks +No action required. This table shows the results of a concurrent behcmark where queries from `website` are ran in parallel using `clickhouse-benchmark`, and requests per second values are compared for old and new servers. It shows variability up to 20% for no apparent reason, so it's probably safe to disregard it. We have it for special cases like investigating concurrency effects in memory allocators, where it may be important. + +#### Metric changes +No action required. These are changes in median values of metrics from `system.asynchronous_metrics_log`. Again, they are prone to unexplained variation and you can safely ignore this table unless it's interesting to you for some particular reason (e.g. you want to compare memory usage). There are also graphs of these metrics in the performance test output archive, in the `metrics` folder. ### How to run Run the entire docker container, specifying PR number (0 for master) From cec35a0e1de2b9b9d5c498800d8f4f42f58d64f4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 8 Jul 2020 15:07:01 +0300 Subject: [PATCH 168/196] Update README.md --- docker/test/performance-comparison/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/README.md b/docker/test/performance-comparison/README.md index 302b2225d34..c0490371bce 100644 --- a/docker/test/performance-comparison/README.md +++ b/docker/test/performance-comparison/README.md @@ -23,7 +23,7 @@ The check status summarizes the report in a short text message like `1 faster, 1 * `1 slower` -- how many queries are slower, * `1 too long` -- how many queries are taking too long to run, * `1 unstable` -- how many queries have unstable results, -* `1 errors` -- how many errors there are in total. The number of errors includes slower tests, tests that are too long, errors while running the tests and building reports, etc. Please look at the main report page to investigate these errors. +* `1 errors` -- how many errors there are in total. Action is required for every error, this number must be zero. The number of errors includes slower tests, tests that are too long, errors while running the tests and building reports, etc. Please look at the main report page to investigate these errors. The report page itself constists of a several tables. Some of them always signify errors, e.g. "Run errors" -- the very presence of this table indicates that there were errors during the test, that are not normal and must be fixed. Some tables are mostly informational, e.g. "Test times" -- they reflect normal test results. But if a cell in such table is marked in red, this also means an error, e.g., a test is taking too long to run. From d199961e6e4f9b225393df39d1a6ebfb7396fd23 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 8 Jul 2020 15:12:23 +0300 Subject: [PATCH 169/196] Rerun tests --- src/Processors/Formats/Impl/ArrowBufferedStreams.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index bb94535549c..80bb6075e78 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -34,6 +34,7 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(ArrowBufferedOutputStream); }; + class RandomAccessFileFromSeekableReadBuffer : public arrow::io::RandomAccessFile { public: @@ -61,6 +62,7 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromSeekableReadBuffer); }; + std::shared_ptr asArrowFile(ReadBuffer & in); } From c0dea3ab1c463344439f63a1ef306e7f9e8329bf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 Jul 2020 15:52:54 +0300 Subject: [PATCH 170/196] Ignore testflows exit code --- docker/test/testflows/runner/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile index 71dca40bca6..b922864fefd 100644 --- a/docker/test/testflows/runner/Dockerfile +++ b/docker/test/testflows/runner/Dockerfile @@ -72,5 +72,5 @@ RUN set -x \ VOLUME /var/lib/docker EXPOSE 2375 ENTRYPOINT ["dockerd-entrypoint.sh"] -CMD ["sh", "-c", "python3 regression.py --no-color --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS} && cat test.log | tfs report results --format json > results.json"] +CMD ["sh", "-c", "python3 regression.py --no-color --local --clickhouse-binary-path ${CLICKHOUSE_TESTS_SERVER_BIN_PATH} --log test.log ${TESTFLOWS_OPTS}; cat test.log | tfs report results --format json > results.json"] From 88ff072da4ec833e1c30c6bdec4b1c98ac46b9d8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 Jul 2020 16:00:59 +0300 Subject: [PATCH 171/196] More skip checks --- tests/queries/skip_list.json | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 791886d6825..78896cb7efa 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -104,7 +104,10 @@ "00180_attach_materialized_view", "00116_storage_set", "00816_long_concurrent_alter_column", - "00992_system_parts_race_condition_zookeeper" + "00992_system_parts_race_condition_zookeeper", + "01320_create_sync_race_condition", + "01305_replica_create_drop_zookeeper", + "01193_metadata_loading" ], "polymorphic-parts": [ "avx", From c48ed67760078141abbc86d75389ac982a0ee9c6 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Jul 2020 17:39:48 +0300 Subject: [PATCH 172/196] get hostname without mutex --- src/Common/DNSResolver.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 2f530f2f2de..a3756d71c1d 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -269,6 +269,8 @@ bool DNSResolver::updateCache() LOG_DEBUG(log, "Updating DNS cache"); { + String updated_host_name = Poco::Net::DNS::hostName(); + std::lock_guard lock(impl->drop_mutex); for (const auto & host : impl->new_hosts) @@ -279,7 +281,7 @@ bool DNSResolver::updateCache() impl->known_addresses.insert(address); impl->new_addresses.clear(); - impl->host_name.emplace(Poco::Net::DNS::hostName()); + impl->host_name.emplace(updated_host_name); } /// FIXME Updating may take a long time becouse we cannot manage timeouts of getaddrinfo(...) and getnameinfo(...). From c2d4b02f5a29fde305a3e75173b7d41c3ba06e08 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 Jul 2020 18:26:54 +0300 Subject: [PATCH 173/196] Add more tests to skip --- tests/queries/skip_list.json | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 78896cb7efa..14aea9715c1 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -107,7 +107,8 @@ "00992_system_parts_race_condition_zookeeper", "01320_create_sync_race_condition", "01305_replica_create_drop_zookeeper", - "01193_metadata_loading" + "01193_metadata_loading", + "01130_in_memory_parts_partitons" ], "polymorphic-parts": [ "avx", @@ -131,6 +132,7 @@ "00933_ttl_simple", "00753_system_columns_and_system_tables", "01343_min_bytes_to_use_mmap_io", - "01344_min_bytes_to_use_mmap_io_index" + "01344_min_bytes_to_use_mmap_io_index", + "01213_alter_rename_with_default_zookeeper" ] } From d2d49972f15ab318b18115873182b4af502ce412 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 8 Jul 2020 21:18:49 +0300 Subject: [PATCH 174/196] Bump CI (after non-restartable inner CI issue) This reverts commit d199961e6e4f9b225393df39d1a6ebfb7396fd23. --- src/Processors/Formats/Impl/ArrowBufferedStreams.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowBufferedStreams.h b/src/Processors/Formats/Impl/ArrowBufferedStreams.h index 80bb6075e78..bb94535549c 100644 --- a/src/Processors/Formats/Impl/ArrowBufferedStreams.h +++ b/src/Processors/Formats/Impl/ArrowBufferedStreams.h @@ -34,7 +34,6 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(ArrowBufferedOutputStream); }; - class RandomAccessFileFromSeekableReadBuffer : public arrow::io::RandomAccessFile { public: @@ -62,7 +61,6 @@ private: ARROW_DISALLOW_COPY_AND_ASSIGN(RandomAccessFileFromSeekableReadBuffer); }; - std::shared_ptr asArrowFile(ReadBuffer & in); } From 35f0e68892bed172555cfb35ec4a44f34f874308 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 01:41:25 +0300 Subject: [PATCH 175/196] Added a test --- .../0_stateless/01388_multi_if_optimization.reference | 4 ++++ tests/queries/0_stateless/01388_multi_if_optimization.sql | 5 +++++ 2 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/01388_multi_if_optimization.reference create mode 100644 tests/queries/0_stateless/01388_multi_if_optimization.sql diff --git a/tests/queries/0_stateless/01388_multi_if_optimization.reference b/tests/queries/0_stateless/01388_multi_if_optimization.reference new file mode 100644 index 00000000000..6dbe0f0d96f --- /dev/null +++ b/tests/queries/0_stateless/01388_multi_if_optimization.reference @@ -0,0 +1,4 @@ +SELECT if(number = 1, \'hello\', if(number = 2, \'world\', \'xyz\')) +FROM numbers(10) +SELECT multiIf(number = 1, \'hello\', number = 2, \'world\', \'xyz\') +FROM numbers(10) diff --git a/tests/queries/0_stateless/01388_multi_if_optimization.sql b/tests/queries/0_stateless/01388_multi_if_optimization.sql new file mode 100644 index 00000000000..345fcfb6fcc --- /dev/null +++ b/tests/queries/0_stateless/01388_multi_if_optimization.sql @@ -0,0 +1,5 @@ +-- If you are reading this test please note that as of now this setting does not provide benefits in most of the cases. +SET optimize_if_chain_to_multiif = 0; +EXPLAIN SYNTAX SELECT number = 1 ? 'hello' : (number = 2 ? 'world' : 'xyz') FROM numbers(10); +SET optimize_if_chain_to_multiif = 1; +EXPLAIN SYNTAX SELECT number = 1 ? 'hello' : (number = 2 ? 'world' : 'xyz') FROM numbers(10); From eaa13d81769af71d6c620881e7064ff47285c9aa Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 02:04:42 +0300 Subject: [PATCH 176/196] Fix strange code CC @Enmk. Prove: g++ -xc++ -include vector - <<< 'int main() { return std::vector{1, 2, 3}.size(); }' --- src/Storages/tests/gtest_SplitTokenExtractor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/tests/gtest_SplitTokenExtractor.cpp b/src/Storages/tests/gtest_SplitTokenExtractor.cpp index 9255e5ca817..cb801cff808 100644 --- a/src/Storages/tests/gtest_SplitTokenExtractor.cpp +++ b/src/Storages/tests/gtest_SplitTokenExtractor.cpp @@ -35,7 +35,7 @@ public: { const auto & param = GetParam(); const auto & source = param.source; - data = std::make_unique>(source.data(), source.data() + source.size()); + data = std::make_unique>(source.data(), source.data() + source.size()); // add predefined padding that forms tokens to ensure no reads past end of buffer. const char extra_padding[] = "this is the end \xd1\x8d\xd1\x82\xd0\xbe\xd0\xba\xd0\xbe \xd0\xbd\xd0\xb5\xd1\x86"; @@ -44,7 +44,7 @@ public: data->resize(data->size() - sizeof(extra_padding)); } - std::unique_ptr> data; + std::unique_ptr> data; }; TEST_P(SplitTokenExtractorTest, next) From 0e4871eec2edf603b5f8eb06c4e1ae1cf9083b35 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 9 Jul 2020 02:05:56 +0300 Subject: [PATCH 177/196] fix TTL after renaming column --- src/Storages/TTLDescription.cpp | 23 ++++++++++++++++ src/Storages/TTLDescription.h | 4 +++ .../01378_alter_rename_with_ttl.reference | 3 +++ .../01378_alter_rename_with_ttl.sql | 26 +++++++++++++++++++ 4 files changed, 56 insertions(+) create mode 100644 tests/queries/0_stateless/01378_alter_rename_with_ttl.reference create mode 100644 tests/queries/0_stateless/01378_alter_rename_with_ttl.sql diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 6e0d323e8a0..2c29958250c 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -272,6 +272,29 @@ TTLDescription TTLDescription::getTTLFromAST( } +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) +{ +} + +TTLTableDescription & TTLTableDescription::operator=(const TTLTableDescription & other) +{ + if (&other == this) + return *this; + + if (other.definition_ast) + definition_ast = other.definition_ast->clone(); + else + definition_ast.reset(); + + rows_ttl = other.rows_ttl; + move_ttl = other.move_ttl; + + return *this; +} + TTLTableDescription TTLTableDescription::getTTLForTableFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 906cfb0e675..ab93967f114 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -102,6 +102,10 @@ struct TTLTableDescription /// Moving data TTL (to other disks or volumes) TTLDescriptions move_ttl; + TTLTableDescription() = default; + TTLTableDescription(const TTLTableDescription & other); + TTLTableDescription & operator=(const TTLTableDescription & other); + static TTLTableDescription getTTLForTableFromAST( const ASTPtr & definition_ast, const ColumnsDescription & columns, const Context & context, const KeyDescription & primary_key); }; diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl.reference b/tests/queries/0_stateless/01378_alter_rename_with_ttl.reference new file mode 100644 index 00000000000..bf8f7658af4 --- /dev/null +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl.reference @@ -0,0 +1,3 @@ +9 +9 +0 diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl.sql new file mode 100644 index 00000000000..98f29538380 --- /dev/null +++ b/tests/queries/0_stateless/01378_alter_rename_with_ttl.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS table_rename_with_ttl; + +CREATE TABLE table_rename_with_ttl +( + date1 Date, + value1 String +) +ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl', '1') +ORDER BY tuple(); + +INSERT INTO table_rename_with_ttl SELECT toDate('2018-10-01') + number % 3, toString(number) from numbers(9); + +SELECT count() FROM table_rename_with_ttl; + +SET materialize_ttl_after_modify = 0; +ALTER TABLE table_rename_with_ttl MODIFY TTL date1 + INTERVAL 1 MONTH; + +SELECT count() FROM table_rename_with_ttl; + +ALTER TABLE table_rename_with_ttl RENAME COLUMN date1 TO renamed_date1; + +ALTER TABLE table_rename_with_ttl materialize TTL settings mutations_sync=2; + +SELECT count() FROM table_rename_with_ttl; + +DROP TABLE IF EXISTS table_rename_with_ttl; From 1918d1d4176c0b8abc5ad8bbb26ea5b81a2cdb88 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 9 Jul 2020 03:49:21 +0400 Subject: [PATCH 178/196] Fix ORC build (#12258) * first try * change submodule * Update .gitmodules * include build directory * Update .gitmodules Co-authored-by: Nikita Mikhailov --- src/CMakeLists.txt | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f631732bad3..be5d3ba836d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -373,11 +373,8 @@ target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${DOUBLE_C target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${MSGPACK_INCLUDE_DIR}) if (USE_ORC) - target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR}) - configure_file ( - "${ORC_INCLUDE_DIR}/orc/orc-config.hh.in" - "${ORC_INCLUDE_DIR}/orc/orc-config.hh" - ) + dbms_target_link_libraries(PUBLIC ${ORC_LIBRARIES}) + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ORC_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/orc/c++/include) endif () if (ENABLE_TESTS AND USE_GTEST) From b78e1145e8764c84ba5447d36130a4072c4dfd86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 02:52:57 +0300 Subject: [PATCH 179/196] Fix filtering by virtual columns #12166 --- src/Storages/VirtualColumnUtils.cpp | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 2e1ce33951a..f0718a014b8 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include @@ -127,12 +128,21 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c /// Filter the block. String filter_column_name = expression_ast->getColumnName(); ColumnPtr filter_column = block_with_filter.getByName(filter_column_name).column->convertToFullColumnIfConst(); - const IColumn::Filter & filter = typeid_cast(*filter_column).getData(); + + ConstantFilterDescription constant_filter(*filter_column); + + if (constant_filter.always_true) + return; + + if (constant_filter.always_false) + block = block.cloneEmpty(); + + FilterDescription filter(*filter_column); for (size_t i = 0; i < block.columns(); ++i) { ColumnPtr & column = block.safeGetByPosition(i).column; - column = column->filter(filter, -1); + column = column->filter(*filter.data, -1); } } From c46ca164dd9067fe297ad2f3209af7d6ecfc22ac Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 02:54:15 +0300 Subject: [PATCH 180/196] Added a test --- .../0_stateless/01389_filter_by_virtual_columns.reference | 1 + tests/queries/0_stateless/01389_filter_by_virtual_columns.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/01389_filter_by_virtual_columns.reference create mode 100644 tests/queries/0_stateless/01389_filter_by_virtual_columns.sql diff --git a/tests/queries/0_stateless/01389_filter_by_virtual_columns.reference b/tests/queries/0_stateless/01389_filter_by_virtual_columns.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/01389_filter_by_virtual_columns.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/01389_filter_by_virtual_columns.sql b/tests/queries/0_stateless/01389_filter_by_virtual_columns.sql new file mode 100644 index 00000000000..43ce3ad40e5 --- /dev/null +++ b/tests/queries/0_stateless/01389_filter_by_virtual_columns.sql @@ -0,0 +1,2 @@ +SELECT count() FROM system.parts WHERE table = NULL AND database = currentDatabase(); +SELECT DISTINCT marks FROM system.parts WHERE (table = NULL) AND (database = currentDatabase()) AND (active = 1); From dbcd0fdd3e6b50e1a198d7ffa7efaffad26cabe5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 04:31:09 +0300 Subject: [PATCH 181/196] Fix "Arcadia" build --- tests/queries/0_stateless/arcadia_skip_list.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 63917583ae3..55c011e4884 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -129,3 +129,5 @@ 01326_build_id 01354_order_by_tuple_collate_const 01370_client_autocomplete_word_break_characters +01319_optimize_skip_unused_shards_nesting +01376_GROUP_BY_injective_elimination_dictGet From d07af62370cc9689041652c58a9719976114aacd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 05:00:48 +0300 Subject: [PATCH 182/196] Log sanitizer trap messages from separate thread --- base/daemon/BaseDaemon.cpp | 64 ++++++++++--------- base/daemon/SentryWriter.cpp | 37 +++++------ base/daemon/SentryWriter.h | 26 ++++---- .../arrow-cmake/cpp/src/arrow/util/config.h | 4 +- 4 files changed, 64 insertions(+), 67 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 711bbd0290a..c0f454107ab 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -163,7 +163,8 @@ public: enum Signals : int { StdTerminate = -1, - StopThread = -2 + StopThread = -2, + SanitizerTrap = -3, }; explicit SignalListener(BaseDaemon & daemon_) @@ -223,8 +224,12 @@ public: std::string query_id; DB::ThreadStatus * thread_ptr{}; - DB::readPODBinary(info, in); - DB::readPODBinary(context, in); + if (sig != SanitizerTrap) + { + DB::readPODBinary(info, in); + DB::readPODBinary(context, in); + } + DB::readPODBinary(stack_trace, in); DB::readBinary(thread_num, in); DB::readBinary(query_id, in); @@ -279,7 +284,14 @@ private: VERSION_STRING, VERSION_OFFICIAL, daemon.build_id_info, thread_num, query_id, strsignal(sig), sig); } - LOG_FATAL(log, signalToErrorMessage(sig, info, context)); + String error_message; + + if (sig != SanitizerTrap) + error_message = signalToErrorMessage(sig, info, context); + else + error_message = "Sanitizer trap."; + + LOG_FATAL(log, error_message); if (stack_trace.getSize()) { @@ -305,12 +317,12 @@ private: String build_id_hex{}; #endif - SentryWriter::onFault(sig, info, context, stack_trace, build_id_hex); + if (sig != SanitizerTrap) + SentryWriter::onFault(sig, error_message, stack_trace, build_id_hex); /// When everything is done, we will try to send these error messages to client. if (thread_ptr) thread_ptr->onFatalError(); - } }; @@ -320,35 +332,27 @@ extern "C" void __sanitizer_set_death_callback(void (*)()); static void sanitizerDeathCallback() { - Poco::Logger * log = &Poco::Logger::get("BaseDaemon"); + /// Also need to send data via pipe. Otherwise it may lead to deadlocks or failures in printing diagnostic info. - StringRef query_id = DB::CurrentThread::getQueryId(); /// This is signal safe. + char buf[signal_pipe_buf_size]; + DB::WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], signal_pipe_buf_size, buf); - if (query_id.size == 0) - { - LOG_FATAL(log, "(version {}{}) (from thread {}) (no query) Sanitizer trap.", - VERSION_STRING, VERSION_OFFICIAL, getThreadId()); - } - else - { - LOG_FATAL(log, "(version {}{}) (from thread {}) (query_id: {}) Sanitizer trap.", - VERSION_STRING, VERSION_OFFICIAL, getThreadId(), query_id); - } + const StackTrace stack_trace; - /// Just in case print our own stack trace. In case when llvm-symbolizer does not work. - StackTrace stack_trace; - if (stack_trace.getSize()) - { - std::stringstream bare_stacktrace; - bare_stacktrace << "Stack trace:"; - for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i) - bare_stacktrace << ' ' << stack_trace.getFramePointers()[i]; + StringRef query_id = DB::CurrentThread::getQueryId(); + query_id.size = std::min(query_id.size, max_query_id_size); - LOG_FATAL(log, bare_stacktrace.str()); - } + int sig = SignalListener::SanitizerTrap; + DB::writeBinary(sig, out); + DB::writePODBinary(stack_trace, out); + DB::writeBinary(UInt32(getThreadId()), out); + DB::writeStringBinary(query_id, out); + DB::writePODBinary(DB::current_thread, out); - /// Write symbolized stack trace line by line for better grep-ability. - stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, s); }); + out.next(); + + /// The time that is usually enough for separate thread to print info into log. + sleepForSeconds(10); } #endif diff --git a/base/daemon/SentryWriter.cpp b/base/daemon/SentryWriter.cpp index ea93d09f9aa..be432602f6e 100644 --- a/base/daemon/SentryWriter.cpp +++ b/base/daemon/SentryWriter.cpp @@ -2,23 +2,26 @@ #include #include +#include #include #include #include + +#include + #if !defined(ARCADIA_BUILD) # include "Common/config_version.h" # include #endif #if USE_SENTRY + # include // Y_IGNORE # include # include -#endif -#if USE_SENTRY namespace { @@ -76,12 +79,12 @@ void sentry_logger(sentry_level_t level, const char * message, va_list args) } } } + } -#endif + void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config) { -#if USE_SENTRY bool enabled = false; bool debug = config.getBool("send_crash_reports.debug", false); auto * logger = &Poco::Logger::get("SentryWriter"); @@ -146,28 +149,19 @@ void SentryWriter::initialize(Poco::Util::LayeredConfiguration & config) { LOG_INFO(logger, "Sending crash reports is disabled"); } -#else - UNUSED(config); -#endif } void SentryWriter::shutdown() { -#if USE_SENTRY if (initialized) - { sentry_shutdown(); - } -#endif } -void SentryWriter::onFault(int sig, const siginfo_t & info, const ucontext_t & context, const StackTrace & stack_trace, const String & build_id_hex) +void SentryWriter::onFault(int sig, const std::string & error_message, const StackTrace & stack_trace, const std::string & build_id_hex) { -#if USE_SENTRY auto * logger = &Poco::Logger::get("SentryWriter"); if (initialized) { - const std::string & error_message = signalToErrorMessage(sig, info, context); sentry_value_t event = sentry_value_new_message_event(SENTRY_LEVEL_FATAL, "fault", error_message.c_str()); sentry_set_tag("signal", strsignal(sig)); sentry_set_extra("signal_number", sentry_value_new_int32(sig)); @@ -240,11 +234,12 @@ void SentryWriter::onFault(int sig, const siginfo_t & info, const ucontext_t & c { LOG_INFO(logger, "Not sending crash report"); } -#else - UNUSED(sig); - UNUSED(info); - UNUSED(context); - UNUSED(stack_trace); - UNUSED(build_id_hex); -#endif } + +#else + +void SentryWriter::initialize(Poco::Util::LayeredConfiguration &) {} +void SentryWriter::shutdown() {} +void SentryWriter::onFault(int, const std::string &, const StackTrace &, const std::string &) {} + +#endif diff --git a/base/daemon/SentryWriter.h b/base/daemon/SentryWriter.h index a7b255e72bf..a5cc82651a9 100644 --- a/base/daemon/SentryWriter.h +++ b/base/daemon/SentryWriter.h @@ -1,12 +1,12 @@ #pragma once -#include -#include - -#include - #include + +namespace Poco { namespace Util { class LayeredConfiguration; }} +class StackTrace; + + /// \brief Sends crash reports to ClickHouse core developer team via https://sentry.io /// /// This feature can enabled with "send_crash_reports.enabled" server setting, @@ -14,20 +14,16 @@ /// /// It is possible to send those reports to your own sentry account or account of consulting company you hired /// by overriding "send_crash_reports.endpoint" setting. "send_crash_reports.debug" setting will allow to do that for -class SentryWriter +namespace SentryWriter { -public: - SentryWriter() = delete; - - static void initialize(Poco::Util::LayeredConfiguration & config); - static void shutdown(); + void initialize(Poco::Util::LayeredConfiguration & config); + void shutdown(); /// Not signal safe and can't be called from a signal handler - static void onFault( + void onFault( int sig, - const siginfo_t & info, - const ucontext_t & context, + const std::string & error_message, const StackTrace & stack_trace, - const String & build_id_hex + const std::string & build_id_hex ); }; diff --git a/contrib/arrow-cmake/cpp/src/arrow/util/config.h b/contrib/arrow-cmake/cpp/src/arrow/util/config.h index c803ebb5263..bf8ea581922 100644 --- a/contrib/arrow-cmake/cpp/src/arrow/util/config.h +++ b/contrib/arrow-cmake/cpp/src/arrow/util/config.h @@ -20,5 +20,7 @@ #define ARROW_VERSION_PATCH #define ARROW_VERSION ((ARROW_VERSION_MAJOR * 1000) + ARROW_VERSION_MINOR) * 1000 + ARROW_VERSION_PATCH -/* #undef DOUBLE_CONVERSION_HAS_CASE_INSENSIBILITY */ +#define ARROW_SO_VERSION "" +#define ARROW_FULL_SO_VERSION "" + /* #undef GRPCPP_PP_INCLUDE */ From 461a81455b5f629b1da5b30ecff8194eef225119 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 05:02:14 +0300 Subject: [PATCH 183/196] Revert strange file --- contrib/arrow-cmake/cpp/src/arrow/util/config.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/contrib/arrow-cmake/cpp/src/arrow/util/config.h b/contrib/arrow-cmake/cpp/src/arrow/util/config.h index bf8ea581922..c803ebb5263 100644 --- a/contrib/arrow-cmake/cpp/src/arrow/util/config.h +++ b/contrib/arrow-cmake/cpp/src/arrow/util/config.h @@ -20,7 +20,5 @@ #define ARROW_VERSION_PATCH #define ARROW_VERSION ((ARROW_VERSION_MAJOR * 1000) + ARROW_VERSION_MINOR) * 1000 + ARROW_VERSION_PATCH -#define ARROW_SO_VERSION "" -#define ARROW_FULL_SO_VERSION "" - +/* #undef DOUBLE_CONVERSION_HAS_CASE_INSENSIBILITY */ /* #undef GRPCPP_PP_INCLUDE */ From e17995cb75316417326b1f882409a571cbdf91c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 9 Jul 2020 05:24:59 +0300 Subject: [PATCH 184/196] Tested with "trap" function --- src/Functions/trap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/trap.cpp b/src/Functions/trap.cpp index 94aa56919b4..699335eb248 100644 --- a/src/Functions/trap.cpp +++ b/src/Functions/trap.cpp @@ -61,7 +61,7 @@ public: return std::make_shared(); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + [[clang::optnone]] void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { if (const ColumnConst * column = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get())) { From a645759a42f3f5cd45cce1a89e06a36b2c6ec772 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 9 Jul 2020 10:16:39 +0300 Subject: [PATCH 185/196] Set CMAKE_POLICY_DEFAULT_CMP0022/CMAKE_POLICY_DEFAULT_CMP0077 globally This will fix CMAKE_POLICY_DEFAULT_CMP0077 for snappy: CMake Warning (dev) at contrib/snappy/CMakeLists.txt:11 (option): Policy CMP0077 is not set: option() honors normal variables. Run "cmake --help-policy CMP0077" for policy details. Use the cmake_policy command to set the policy and suppress this warning. For compatibility with older versions of CMake, option is clearing the normal variable 'BUILD_SHARED_LIBS'. This warning is for project developers. Use -Wno-dev to suppress it. --- CMakeLists.txt | 14 ++++++++++++++ contrib/CMakeLists.txt | 27 +++++++-------------------- 2 files changed, 21 insertions(+), 20 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d4d325818e8..88dc287b201 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -12,6 +12,20 @@ foreach(policy endif() endforeach() +# set default policy +foreach(default_policy_var_name + # make option() honor normal variables for BUILD_SHARED_LIBS: + # - re2 + # - snappy + CMAKE_POLICY_DEFAULT_CMP0077 + # Google Test from sources uses too old cmake, 2.6.x, and CMP0022 should + # set, to avoid using deprecated LINK_INTERFACE_LIBRARIES(_)? over + # INTERFACE_LINK_LIBRARIES. + CMAKE_POLICY_DEFAULT_CMP0022 + ) + set(${default_policy_var_name} NEW) +endforeach() + project(ClickHouse) include (cmake/arch.cmake) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 72e48ffff19..df3cb8a774d 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -44,13 +44,8 @@ endif () if (USE_INTERNAL_RE2_LIBRARY) set(RE2_BUILD_TESTING 0 CACHE INTERNAL "") - function(re2_support) - # make option() honor normal variables for BUILD_SHARED_LIBS - set(CMAKE_POLICY_DEFAULT_CMP0077 NEW) - add_subdirectory (re2) - add_subdirectory (re2_st) - endfunction() - re2_support() + add_subdirectory (re2) + add_subdirectory (re2_st) endif () if (USE_INTERNAL_DOUBLE_CONVERSION_LIBRARY) @@ -227,19 +222,11 @@ if (USE_INTERNAL_AVRO_LIBRARY) endif() if(USE_INTERNAL_GTEST_LIBRARY) - # Wrap into function because of CMAKE_POLICY_DEFAULT_CMP0022 - function(googletest_support) - set(GOOGLETEST_VERSION 1.10.0) # master - # Google Test from sources uses too old cmake, 2.6.x, and CMP0022 should - # set, to avoid using deprecated LINK_INTERFACE_LIBRARIES(_)? over - # INTERFACE_LINK_LIBRARIES. - set(CMAKE_POLICY_DEFAULT_CMP0022 NEW) - # Google Test from sources - add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) - # avoid problems with - target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0) - endfunction() - googletest_support() + set(GOOGLETEST_VERSION 1.10.0) # master + # Google Test from sources + add_subdirectory(${ClickHouse_SOURCE_DIR}/contrib/googletest/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) + # avoid problems with + target_compile_definitions (gtest INTERFACE GTEST_HAS_POSIX_RE=0) elseif(GTEST_SRC_DIR) add_subdirectory(${GTEST_SRC_DIR}/googletest ${CMAKE_CURRENT_BINARY_DIR}/googletest) target_compile_definitions(gtest INTERFACE GTEST_HAS_POSIX_RE=0) From 0aa97af108e80f114be6542f8aaa2b07892ed4e6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 9 Jul 2020 10:17:39 +0300 Subject: [PATCH 186/196] Update config.h for arrow Yes ARROW_FULL_SO_VERSION/ARROW_SO_VERSION is empty right now, like other version variables (ARROW_VERSION_*) Refs: #12181 --- contrib/arrow-cmake/cpp/src/arrow/util/config.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/contrib/arrow-cmake/cpp/src/arrow/util/config.h b/contrib/arrow-cmake/cpp/src/arrow/util/config.h index c803ebb5263..bf8ea581922 100644 --- a/contrib/arrow-cmake/cpp/src/arrow/util/config.h +++ b/contrib/arrow-cmake/cpp/src/arrow/util/config.h @@ -20,5 +20,7 @@ #define ARROW_VERSION_PATCH #define ARROW_VERSION ((ARROW_VERSION_MAJOR * 1000) + ARROW_VERSION_MINOR) * 1000 + ARROW_VERSION_PATCH -/* #undef DOUBLE_CONVERSION_HAS_CASE_INSENSIBILITY */ +#define ARROW_SO_VERSION "" +#define ARROW_FULL_SO_VERSION "" + /* #undef GRPCPP_PP_INCLUDE */ From 3a519fe35a1832a45af2b2791c4da1f218956209 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 9 Jul 2020 10:25:13 +0300 Subject: [PATCH 187/196] Fix jemalloc enabled detection (should goes after contrib inclusion) --- CMakeLists.txt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 88dc287b201..c13691bdfbf 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -392,10 +392,6 @@ include (cmake/find/mysqlclient.cmake) # When testing for memory leaks with Valgrind, don't link tcmalloc or jemalloc. -if (OS_LINUX AND NOT ENABLE_JEMALLOC) - message (WARNING "Non default allocator is disabled. This is not recommended for production Linux builds.") -endif () - if (USE_OPENCL) if (OS_DARWIN) set(OPENCL_LINKER_FLAGS "-framework OpenCL") @@ -411,6 +407,10 @@ endif () add_subdirectory (contrib EXCLUDE_FROM_ALL) +if (OS_LINUX AND NOT ENABLE_JEMALLOC) + message (WARNING "Non default allocator is disabled. This is not recommended for production Linux builds.") +endif () + macro (add_executable target) # invoke built-in add_executable # explicitly acquire and interpose malloc symbols by clickhouse_malloc From 3ceefee22086bfa5c8c110f80e35f29c682007a8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 9 Jul 2020 10:25:42 +0300 Subject: [PATCH 188/196] Warn if jemalloc is not enabled for non-linux too Refs: #11897 (osx) Refs: #11774 (freebsd) --- CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index c13691bdfbf..7c213ace742 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -407,8 +407,8 @@ endif () add_subdirectory (contrib EXCLUDE_FROM_ALL) -if (OS_LINUX AND NOT ENABLE_JEMALLOC) - message (WARNING "Non default allocator is disabled. This is not recommended for production Linux builds.") +if (NOT ENABLE_JEMALLOC) + message (WARNING "Non default allocator is disabled. This is not recommended for production builds.") endif () macro (add_executable target) From 2eb6f1d0c16361a89d866a1bc98673824acade6e Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 9 Jul 2020 11:50:53 +0300 Subject: [PATCH 189/196] DOCS-679: netloc function (#12321) * DOCSUP-1377 (netloc function) (#135) * add EN description * changes in query * changes after review * add RU description * CLICKHOUSEDOCS-679: Text fixes. Co-authored-by: Sergei Shtykov Co-authored-by: emironyuk Co-authored-by: Evgenia Sudarikova <56156889+otrazhenia@users.noreply.github.com> --- .../sql-reference/functions/url-functions.md | 36 +++++++++++++++++++ .../sql-reference/functions/url-functions.md | 36 +++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/docs/en/sql-reference/functions/url-functions.md b/docs/en/sql-reference/functions/url-functions.md index 620c15d9d83..688fa6e84fc 100644 --- a/docs/en/sql-reference/functions/url-functions.md +++ b/docs/en/sql-reference/functions/url-functions.md @@ -184,6 +184,42 @@ SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS Decod └────────────────────────────────────────┘ ``` +### netloc {#netloc} + +Extracts network locality (`username:password@host:port`) from a URL. + +**Syntax** + +```sql +netloc(URL) +``` + +**Parameters** + +- `url` — URL. [String](../../sql-reference/data-types/string.md). + +**Returned value** + +- `username:password@host:port`. + +Type: `String`. + +**Example** + +Query: + +``` sql +SELECT netloc('http://paul@www.example.com:80/'); +``` + +Result: + +``` text +┌─netloc('http://paul@www.example.com:80/')─┐ +│ paul@www.example.com:80 │ +└───────────────────────────────────────────┘ +``` + ## Functions that Remove Part of a URL {#functions-that-remove-part-of-a-url} If the URL doesn’t have anything similar, the URL remains unchanged. diff --git a/docs/ru/sql-reference/functions/url-functions.md b/docs/ru/sql-reference/functions/url-functions.md index 6d749f6f143..843b75567ee 100644 --- a/docs/ru/sql-reference/functions/url-functions.md +++ b/docs/ru/sql-reference/functions/url-functions.md @@ -174,6 +174,42 @@ SELECT decodeURLComponent('http://127.0.0.1:8123/?query=SELECT%201%3B') AS Decod └────────────────────────────────────────┘ ``` +### netloc {#netloc} + +Извлекает сетевую локальность (`username:password@host:port`) из URL. + +**Синтаксис** + +```sql +netloc(URL) +``` + +**Параметры** + +- `url` — URL. Тип — [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +- `username:password@host:port`. + +Тип: `String`. + +**Пример** + +Запрос: + +``` sql +SELECT netloc('http://paul@www.example.com:80/'); +``` + +Результат: + +``` text +┌─netloc('http://paul@www.example.com:80/')─┐ +│ paul@www.example.com:80 │ +└───────────────────────────────────────────┘ +``` + ## Функции, удаляющие часть из URL-а {#funktsii-udaliaiushchie-chast-iz-url-a} Если в URL-е нет ничего похожего, то URL остаётся без изменений. From 761b03f17d656ddd2e821807749921142608313f Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 9 Jul 2020 14:21:23 +0300 Subject: [PATCH 190/196] add docker image for fuzzer --- docker/images.json | 4 ++ docker/test/fuzzer/Dockerfile | 35 ++++++++++++++ docker/test/fuzzer/run-fuzzer.sh | 82 ++++++++++++++++++++++++++++++++ 3 files changed, 121 insertions(+) create mode 100644 docker/test/fuzzer/Dockerfile create mode 100755 docker/test/fuzzer/run-fuzzer.sh diff --git a/docker/images.json b/docker/images.json index 45209fc775f..0ea8292e234 100644 --- a/docker/images.json +++ b/docker/images.json @@ -31,6 +31,10 @@ "name": "yandex/clickhouse-integration-test", "dependent": [] }, + "docker/test/fuzzer": { + "name": "yandex/clickhouse-fuzzer", + "dependent": [] + }, "docker/test/performance-comparison": { "name": "yandex/clickhouse-performance-comparison", "dependent": [] diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile new file mode 100644 index 00000000000..b161a5d5424 --- /dev/null +++ b/docker/test/fuzzer/Dockerfile @@ -0,0 +1,35 @@ +# docker build -t yandex/clickhouse-fuzzer . +FROM ubuntu:18.04 + +ENV LANG=C.UTF-8 +ENV TZ=Europe/Moscow +RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + +RUN apt-get update \ + && DEBIAN_FRONTEND=noninteractive apt-get install --yes --no-install-recommends \ + bash \ + ca-certificates \ + curl \ + gdb \ + git \ + libc6-dbg \ + moreutils \ + ncdu \ + p7zip-full \ + parallel \ + psmisc \ + rsync \ + tree \ + tzdata \ + vim \ + wget \ + && apt-get autoremove --yes \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + +COPY * / + +CMD cd /workspace && /run-fuzzer.sh + +# docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> yandex/clickhouse-fuzzer + diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh new file mode 100755 index 00000000000..8c94afd4040 --- /dev/null +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -0,0 +1,82 @@ +#!/bin/bash +set -ex +set -o pipefail +trap "exit" INT TERM +trap 'kill $(jobs -pr) ||:' EXIT + +stage=${stage:-} +script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + +# Doesn't work for clone stage, but should work after that +repo_dir=${repo_dir:-$(readlink -f "$script_dir/../../..")} + +function clone +{ +( + rm -rf ch ||: + mkdir ch + cd ch + + git init + git remote add origin https://github.com/ClickHouse/ClickHouse + git fetch --depth=1 origin "$SHA_TO_TEST" + + # If not master, try to fetch pull/.../{head,merge} + if [ "$PR_TO_TEST" != "0" ] + then + git fetch --depth=1 origin "refs/pull/$PR_TO_TEST/*:refs/heads/pull/$PR_TO_TEST/*" + fi + + git checkout "$SHA_TO_TEST" +) +} + +function download +{ +# wget -O- -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/performance/performance.tgz" \ +# | tar --strip-components=1 -zxv + + wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-10_debug_none_bundled_unsplitted_disable_False_binary/clickhouse" +} + +function configure +{ + mkdir db ||: + cp -av "$repo_dir"/programs/server/config* db + cp -av "$repo_dir"/programs/server/user* db + cp -av "$repo_dir"/tests/config db/config.d +} + +function fuzz +{ + ./clickhouse server --config-file db/config.xml -- --path db 2>&1 | tail -1000000 > server-log.txt & + server_pid=$! + kill -0 $server_pid + while ! ./clickhouse client --query "select 1" && kill -0 $server_pid ; do echo . ; sleep 1 ; done + ./clickhouse client --query "select 1" + echo Server started + + for f in $(ls ch/tests/queries/0_stateless/*.sql | sort -R); do cat $f; echo ';'; done \ + | ./clickhouse client --query-fuzzer-runs=10 2>&1 | tail -1000000 > fuzzer-log.txt +} + +case "$stage" in +"") + ;& +"clone") + time clone + stage=download time ch/docker/test/fuzzer/run-fuzzer.sh + ;; +"download") + time download + ;& +"configure") + time configure + ;& +"fuzz") + time fuzz + ;& +"report") + ;& +esac + From e06b675801cbc664c46d33baec19a342d627a95e Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 9 Jul 2020 14:55:54 +0300 Subject: [PATCH 191/196] fixes in fuzzer docker image --- docker/test/fuzzer/Dockerfile | 3 ++- docker/test/fuzzer/run-fuzzer.sh | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index b161a5d5424..e6f8053f5f6 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -29,7 +29,8 @@ RUN apt-get update \ COPY * / -CMD cd /workspace && /run-fuzzer.sh +CMD cd /workspace \ + && /run-fuzzer.sh 2>&1 | ts "$(printf '%%Y-%%m-%%d %%H:%%M:%%S\t')" | tee main.log # docker run --network=host --volume :/workspace -e PR_TO_TEST=<> -e SHA_TO_TEST=<> yandex/clickhouse-fuzzer diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 8c94afd4040..92d3c27f59b 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -49,7 +49,7 @@ function configure function fuzz { - ./clickhouse server --config-file db/config.xml -- --path db 2>&1 | tail -1000000 > server-log.txt & + ./clickhouse server --config-file db/config.xml -- --path db 2>&1 | tail -1000000 > server.log & server_pid=$! kill -0 $server_pid while ! ./clickhouse client --query "select 1" && kill -0 $server_pid ; do echo . ; sleep 1 ; done @@ -57,7 +57,7 @@ function fuzz echo Server started for f in $(ls ch/tests/queries/0_stateless/*.sql | sort -R); do cat $f; echo ';'; done \ - | ./clickhouse client --query-fuzzer-runs=10 2>&1 | tail -1000000 > fuzzer-log.txt + | ./clickhouse client --query-fuzzer-runs=10 2>&1 | tail -1000000 > fuzzer.log } case "$stage" in From e0d19d2aeaeafa3620305bce3a991fefc0e28695 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 9 Jul 2020 15:11:15 +0300 Subject: [PATCH 192/196] [docs] engine family introduction refactoring (#12268) * base refactoring * adjust links * Update index.md --- .../engines/table-engines/log-family/index.md | 39 +++++++++++++++ .../table-engines/log-family/log-family.md | 44 ----------------- .../engines/table-engines/log-family/log.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../table-engines/log-family/tinylog.md | 2 +- .../table-engines/mergetree-family/index.md | 11 +++++ docs/en/sql-reference/statements/create.md | 2 +- .../engines/table-engines/log-family/index.md | 39 +++++++++++++++ .../table-engines/log-family/log-family.md | 46 ------------------ .../engines/table-engines/log-family/log.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../table-engines/log-family/tinylog.md | 2 +- docs/es/sql-reference/statements/create.md | 2 +- .../engines/table-engines/log-family/index.md | 41 ++++++++++++++++ .../table-engines/log-family/log-family.md | 46 ------------------ .../engines/table-engines/log-family/log.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../table-engines/log-family/tinylog.md | 2 +- docs/fa/sql-reference/statements/create.md | 2 +- .../engines/table-engines/log-family/index.md | 40 +++++++++++++++- .../table-engines/log-family/log-family.md | 46 ------------------ .../engines/table-engines/log-family/log.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../table-engines/log-family/tinylog.md | 2 +- docs/fr/sql-reference/statements/create.md | 2 +- .../engines/table-engines/log-family/index.md | 39 +++++++++++++++ .../table-engines/log-family/log-family.md | 46 ------------------ .../engines/table-engines/log-family/log.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../table-engines/log-family/tinylog.md | 2 +- docs/ja/sql-reference/statements/create.md | 2 +- docs/redirects.txt | 1 + .../engines/table-engines/log-family/index.md | 41 +++++++++++++++- .../table-engines/log-family/log-family.md | 39 --------------- .../engines/table-engines/log-family/log.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../table-engines/log-family/tinylog.md | 2 +- .../engines/table-engines/log-family/index.md | 39 +++++++++++++++ .../table-engines/log-family/log-family.md | 46 ------------------ .../engines/table-engines/log-family/log.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../table-engines/log-family/tinylog.md | 2 +- docs/tr/sql-reference/statements/create.md | 2 +- .../engines/table-engines/log-family/index.md | 47 ++++++++++++++++++- .../table-engines/log-family/log-family.md | 45 ------------------ .../table-engines/log-family/stripelog.md | 2 +- 46 files changed, 361 insertions(+), 390 deletions(-) delete mode 100644 docs/en/engines/table-engines/log-family/log-family.md delete mode 100644 docs/es/engines/table-engines/log-family/log-family.md delete mode 100644 docs/fa/engines/table-engines/log-family/log-family.md delete mode 100644 docs/fr/engines/table-engines/log-family/log-family.md delete mode 100644 docs/ja/engines/table-engines/log-family/log-family.md delete mode 100644 docs/ru/engines/table-engines/log-family/log-family.md delete mode 100644 docs/tr/engines/table-engines/log-family/log-family.md delete mode 100644 docs/zh/engines/table-engines/log-family/log-family.md diff --git a/docs/en/engines/table-engines/log-family/index.md b/docs/en/engines/table-engines/log-family/index.md index a5d93c4c7c3..817e8088503 100644 --- a/docs/en/engines/table-engines/log-family/index.md +++ b/docs/en/engines/table-engines/log-family/index.md @@ -1,6 +1,45 @@ --- toc_folder_title: Log Family toc_priority: 29 +toc_title: Introduction --- +# Log Engine Family {#log-engine-family} +These engines were developed for scenarios when you need to quickly write many small tables (up to about 1 million rows) and read them later as a whole. + +Engines of the family: + +- [StripeLog](../../../engines/table-engines/log-family/stripelog.md) +- [Log](../../../engines/table-engines/log-family/log.md) +- [TinyLog](../../../engines/table-engines/log-family/tinylog.md) + +## Common Properties {#common-properties} + +Engines: + +- Store data on a disk. + +- Append data to the end of file when writing. + +- Support locks for concurrent data access. + + During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently. + +- Do not support [mutation](../../../sql-reference/statements/alter.md#alter-mutations) operations. + +- Do not support indexes. + + This means that `SELECT` queries for ranges of data are not efficient. + +- Do not write data atomically. + + You can get a table with corrupted data if something breaks the write operation, for example, abnormal server shutdown. + +## Differences {#differences} + +The `TinyLog` engine is the simplest in the family and provides the poorest functionality and lowest efficiency. The `TinyLog` engine doesn’t support parallel data reading by several threads. It reads data slower than other engines in the family that support parallel reading and it uses almost as many descriptors as the `Log` engine because it stores each column in a separate file. Use it in simple low-load scenarios. + +The `Log` and `StripeLog` engines support parallel data reading. When reading data, ClickHouse uses multiple threads. Each thread processes a separate data block. The `Log` engine uses a separate file for each column of the table. `StripeLog` stores all the data in one file. As a result, the `StripeLog` engine uses fewer descriptors in the operating system, but the `Log` engine provides higher efficiency when reading data. + +[Original article](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/en/engines/table-engines/log-family/log-family.md b/docs/en/engines/table-engines/log-family/log-family.md deleted file mode 100644 index 2a5096c62b3..00000000000 --- a/docs/en/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,44 +0,0 @@ ---- -toc_priority: 31 -toc_title: Introduction ---- - -# Log Engine Family {#log-engine-family} - -These engines were developed for scenarios when you need to quickly write many small tables (up to about 1 million rows) and read them later as a whole. - -Engines of the family: - -- [StripeLog](../../../engines/table-engines/log-family/stripelog.md) -- [Log](../../../engines/table-engines/log-family/log.md) -- [TinyLog](../../../engines/table-engines/log-family/tinylog.md) - -## Common Properties {#common-properties} - -Engines: - -- Store data on a disk. - -- Append data to the end of file when writing. - -- Support locks for concurrent data access. - - During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently. - -- Do not support [mutation](../../../sql-reference/statements/alter.md#alter-mutations) operations. - -- Do not support indexes. - - This means that `SELECT` queries for ranges of data are not efficient. - -- Do not write data atomically. - - You can get a table with corrupted data if something breaks the write operation, for example, abnormal server shutdown. - -## Differences {#differences} - -The `TinyLog` engine is the simplest in the family and provides the poorest functionality and lowest efficiency. The `TinyLog` engine doesn’t support parallel data reading by several threads. It reads data slower than other engines in the family that support parallel reading and it uses almost as many descriptors as the `Log` engine because it stores each column in a separate file. Use it in simple low-load scenarios. - -The `Log` and `StripeLog` engines support parallel data reading. When reading data, ClickHouse uses multiple threads. Each thread processes a separate data block. The `Log` engine uses a separate file for each column of the table. `StripeLog` stores all the data in one file. As a result, the `StripeLog` engine uses fewer descriptors in the operating system, but the `Log` engine provides higher efficiency when reading data. - -[Original article](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/en/engines/table-engines/log-family/log.md b/docs/en/engines/table-engines/log-family/log.md index faf4c5889f4..87cdb890e9f 100644 --- a/docs/en/engines/table-engines/log-family/log.md +++ b/docs/en/engines/table-engines/log-family/log.md @@ -5,7 +5,7 @@ toc_title: Log # Log {#log} -Engine belongs to the family of log engines. See the common properties of log engines and their differences in the [Log Engine Family](../../../engines/table-engines/log-family/log-family.md) article. +Engine belongs to the family of log engines. See the common properties of log engines and their differences in the [Log Engine Family](../../../engines/table-engines/log-family/index.md) article. Log differs from [TinyLog](../../../engines/table-engines/log-family/tinylog.md) in that a small file of “marks” resides with the column files. These marks are written on every data block and contain offsets that indicate where to start reading the file in order to skip the specified number of rows. This makes it possible to read table data in multiple threads. For concurrent data access, the read operations can be performed simultaneously, while write operations block reads and each other. diff --git a/docs/en/engines/table-engines/log-family/stripelog.md b/docs/en/engines/table-engines/log-family/stripelog.md index 68d8f52d68f..a2b9562a9f0 100644 --- a/docs/en/engines/table-engines/log-family/stripelog.md +++ b/docs/en/engines/table-engines/log-family/stripelog.md @@ -5,7 +5,7 @@ toc_title: StripeLog # Stripelog {#stripelog} -This engine belongs to the family of log engines. See the common properties of log engines and their differences in the [Log Engine Family](../../../engines/table-engines/log-family/log-family.md) article. +This engine belongs to the family of log engines. See the common properties of log engines and their differences in the [Log Engine Family](../../../engines/table-engines/log-family/index.md) article. Use this engine in scenarios when you need to write many tables with a small amount of data (less than 1 million rows). diff --git a/docs/en/engines/table-engines/log-family/tinylog.md b/docs/en/engines/table-engines/log-family/tinylog.md index 0a0d2ef3761..ea2138e36fe 100644 --- a/docs/en/engines/table-engines/log-family/tinylog.md +++ b/docs/en/engines/table-engines/log-family/tinylog.md @@ -5,7 +5,7 @@ toc_title: TinyLog # TinyLog {#tinylog} -The engine belongs to the log engine family. See [Log Engine Family](../../../engines/table-engines/log-family/log-family.md) for common properties of log engines and their differences. +The engine belongs to the log engine family. See [Log Engine Family](../../../engines/table-engines/log-family/index.md) for common properties of log engines and their differences. This table engine is typically used with the write-once method: write data one time, then read it as many times as necessary. For example, you can use `TinyLog`-type tables for intermediary data that is processed in small batches. Note that storing data in a large number of small tables is inefficient. diff --git a/docs/en/engines/table-engines/mergetree-family/index.md b/docs/en/engines/table-engines/mergetree-family/index.md index 9e989d807da..ded4a6e8707 100644 --- a/docs/en/engines/table-engines/mergetree-family/index.md +++ b/docs/en/engines/table-engines/mergetree-family/index.md @@ -1,6 +1,17 @@ --- toc_folder_title: MergeTree Family toc_priority: 28 +toc_title: Introduction --- +# MergeTree Engine Family +Table engines from the MergeTree family are the core of ClickHouse data storage capabilities. They provide most features for resilience and high-performance data retrieval: columnar storage, custom partitioning, sparse primary index, secondary data-skipping indexes, etc. + +Base [MergeTree](mergetree.md) table engine can be considered the default table engine for single-node ClickHouse instances because it is versatile and practical for a wide range of use cases. + +For production usage [ReplicatedMergeTree](replication.md) is the way to go, because it adds high-availability to all features of regular MergeTree engine. A bonus is automatic data deduplication on data ingestion, so the software can safely retry if there was some network issue during insert. + +All other engines of MergeTree family add extra functionality for some specific use cases. Usually, it's implemented as additional data manipulation in background. + +The main downside of MergeTree engines is that they are rather heavy-weight. So the typical pattern is to have not so many of them. If you need many small tables, for example for temporary data, consider [Log engine family](../../../engines/table-engines/log-family/index.md). diff --git a/docs/en/sql-reference/statements/create.md b/docs/en/sql-reference/statements/create.md index 051fa23e1bb..51c4a9db43a 100644 --- a/docs/en/sql-reference/statements/create.md +++ b/docs/en/sql-reference/statements/create.md @@ -153,7 +153,7 @@ If a codec is specified, the default codec doesn’t apply. Codecs can be combin Compression is supported for the following table engines: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family. Supports column compression codecs and selecting the default compression method by [compression](../../operations/server-configuration-parameters/settings.md#server-settings-compression) settings. -- [Log](../../engines/table-engines/log-family/log-family.md) family. Uses the `lz4` compression method by default and supports column compression codecs. +- [Log](../../engines/table-engines/log-family/index.md) family. Uses the `lz4` compression method by default and supports column compression codecs. - [Set](../../engines/table-engines/special/set.md). Only supported the default compression. - [Join](../../engines/table-engines/special/join.md). Only supported the default compression. diff --git a/docs/es/engines/table-engines/log-family/index.md b/docs/es/engines/table-engines/log-family/index.md index 42fd671a063..a7a3016f967 100644 --- a/docs/es/engines/table-engines/log-family/index.md +++ b/docs/es/engines/table-engines/log-family/index.md @@ -3,6 +3,45 @@ machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: Familia de registro toc_priority: 29 +toc_title: "Implantaci\xF3n" --- +# Familia del motor de registro {#log-engine-family} +Estos motores fueron desarrollados para escenarios en los que necesita escribir rápidamente muchas tablas pequeñas (hasta aproximadamente 1 millón de filas) y leerlas más tarde en su conjunto. + +Motores de la familia: + +- [StripeLog](stripelog.md) +- [Registro](log.md) +- [TinyLog](tinylog.md) + +## Propiedades comunes {#common-properties} + +Motor: + +- Almacenar datos en un disco. + +- Agregue datos al final del archivo al escribir. + +- Bloqueos de soporte para el acceso a datos simultáneos. + + Durante `INSERT` consultas, la tabla está bloqueada y otras consultas para leer y escribir datos esperan a que la tabla se desbloquee. Si no hay consultas de escritura de datos, se puede realizar cualquier número de consultas de lectura de datos simultáneamente. + +- No apoyo [mutación](../../../sql-reference/statements/alter.md#alter-mutations) operación. + +- No admite índices. + + Esto significa que `SELECT` las consultas para rangos de datos no son eficientes. + +- No escriba datos atómicamente. + + Puede obtener una tabla con datos dañados si algo rompe la operación de escritura, por ejemplo, un cierre anormal del servidor. + +## Diferencia {#differences} + +El `TinyLog` es el más simple de la familia y proporciona la funcionalidad más pobre y la eficiencia más baja. El `TinyLog` el motor no admite la lectura de datos paralelos por varios hilos. Lee datos más lentamente que otros motores de la familia que admiten lectura paralela y utiliza casi tantos descriptores como los `Log` motor porque almacena cada columna en un archivo separado. Úselo en escenarios simples de baja carga. + +El `Log` y `StripeLog` Los motores admiten lectura de datos paralela. Al leer datos, ClickHouse usa múltiples hilos. Cada subproceso procesa un bloque de datos separado. El `Log` utiliza un archivo separado para cada columna de la tabla. `StripeLog` almacena todos los datos en un archivo. Como resultado, el `StripeLog` el motor utiliza menos descriptores en el sistema operativo, pero el `Log` proporciona una mayor eficiencia al leer datos. + +[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/es/engines/table-engines/log-family/log-family.md b/docs/es/engines/table-engines/log-family/log-family.md deleted file mode 100644 index ee264f88d79..00000000000 --- a/docs/es/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: "Implantaci\xF3n" ---- - -# Familia del motor de registro {#log-engine-family} - -Estos motores fueron desarrollados para escenarios en los que necesita escribir rápidamente muchas tablas pequeñas (hasta aproximadamente 1 millón de filas) y leerlas más tarde en su conjunto. - -Motores de la familia: - -- [StripeLog](stripelog.md) -- [Registro](log.md) -- [TinyLog](tinylog.md) - -## Propiedades comunes {#common-properties} - -Motor: - -- Almacenar datos en un disco. - -- Agregue datos al final del archivo al escribir. - -- Bloqueos de soporte para el acceso a datos simultáneos. - - Durante `INSERT` consultas, la tabla está bloqueada y otras consultas para leer y escribir datos esperan a que la tabla se desbloquee. Si no hay consultas de escritura de datos, se puede realizar cualquier número de consultas de lectura de datos simultáneamente. - -- No apoyo [mutación](../../../sql-reference/statements/alter.md#alter-mutations) operación. - -- No admite índices. - - Esto significa que `SELECT` las consultas para rangos de datos no son eficientes. - -- No escriba datos atómicamente. - - Puede obtener una tabla con datos dañados si algo rompe la operación de escritura, por ejemplo, un cierre anormal del servidor. - -## Diferencia {#differences} - -El `TinyLog` es el más simple de la familia y proporciona la funcionalidad más pobre y la eficiencia más baja. El `TinyLog` el motor no admite la lectura de datos paralelos por varios hilos. Lee datos más lentamente que otros motores de la familia que admiten lectura paralela y utiliza casi tantos descriptores como los `Log` motor porque almacena cada columna en un archivo separado. Úselo en escenarios simples de baja carga. - -El `Log` y `StripeLog` Los motores admiten lectura de datos paralela. Al leer datos, ClickHouse usa múltiples hilos. Cada subproceso procesa un bloque de datos separado. El `Log` utiliza un archivo separado para cada columna de la tabla. `StripeLog` almacena todos los datos en un archivo. Como resultado, el `StripeLog` el motor utiliza menos descriptores en el sistema operativo, pero el `Log` proporciona una mayor eficiencia al leer datos. - -[Artículo Original](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/es/engines/table-engines/log-family/log.md b/docs/es/engines/table-engines/log-family/log.md index e9ce17214eb..1db374390e4 100644 --- a/docs/es/engines/table-engines/log-family/log.md +++ b/docs/es/engines/table-engines/log-family/log.md @@ -7,7 +7,7 @@ toc_title: Registro # Registro {#log} -El motor pertenece a la familia de motores de registro. Consulte las propiedades comunes de los motores de registro y sus diferencias en [Familia del motor de registro](log-family.md) artículo. +El motor pertenece a la familia de motores de registro. Consulte las propiedades comunes de los motores de registro y sus diferencias en [Familia del motor de registro](index.md) artículo. El registro difiere de [TinyLog](tinylog.md) en que un pequeño archivo de “marks” reside con los archivos de columna. Estas marcas se escriben en cada bloque de datos y contienen compensaciones que indican dónde comenzar a leer el archivo para omitir el número especificado de filas. Esto hace posible leer datos de tabla en múltiples hilos. Para el acceso a datos simultáneos, las operaciones de lectura se pueden realizar simultáneamente, mientras que las operaciones de escritura bloquean las lecturas entre sí. diff --git a/docs/es/engines/table-engines/log-family/stripelog.md b/docs/es/engines/table-engines/log-family/stripelog.md index 59a981a2fa6..0965e9a987c 100644 --- a/docs/es/engines/table-engines/log-family/stripelog.md +++ b/docs/es/engines/table-engines/log-family/stripelog.md @@ -7,7 +7,7 @@ toc_title: StripeLog # Lista de Stripelog {#stripelog} -Este motor pertenece a la familia de motores de registro. Consulte las propiedades comunes de los motores de registro y sus diferencias en [Familia del motor de registro](log-family.md) artículo. +Este motor pertenece a la familia de motores de registro. Consulte las propiedades comunes de los motores de registro y sus diferencias en [Familia del motor de registro](index.md) artículo. Utilice este motor en escenarios en los que necesite escribir muchas tablas con una pequeña cantidad de datos (menos de 1 millón de filas). diff --git a/docs/es/engines/table-engines/log-family/tinylog.md b/docs/es/engines/table-engines/log-family/tinylog.md index 1aa392fb831..a2cbf7257b6 100644 --- a/docs/es/engines/table-engines/log-family/tinylog.md +++ b/docs/es/engines/table-engines/log-family/tinylog.md @@ -7,7 +7,7 @@ toc_title: TinyLog # TinyLog {#tinylog} -El motor pertenece a la familia de motores de registro. Ver [Familia del motor de registro](log-family.md) para las propiedades comunes de los motores de registro y sus diferencias. +El motor pertenece a la familia de motores de registro. Ver [Familia del motor de registro](index.md) para las propiedades comunes de los motores de registro y sus diferencias. Este motor de tablas se usa normalmente con el método write-once: escribir datos una vez, luego leerlos tantas veces como sea necesario. Por ejemplo, puede usar `TinyLog`-type tablas para datos intermedios que se procesan en pequeños lotes. Tenga en cuenta que el almacenamiento de datos en un gran número de tablas pequeñas es ineficiente. diff --git a/docs/es/sql-reference/statements/create.md b/docs/es/sql-reference/statements/create.md index 0adc8f4b06e..9e1489ba2a4 100644 --- a/docs/es/sql-reference/statements/create.md +++ b/docs/es/sql-reference/statements/create.md @@ -155,7 +155,7 @@ Si se especifica un códec, el códec predeterminado no se aplica. Los códecs s La compresión es compatible con los siguientes motores de tablas: - [Método de codificación de datos:](../../engines/table-engines/mergetree-family/mergetree.md) familia. Admite códecs de compresión de columnas y selecciona el método de compresión predeterminado mediante [compresión](../../operations/server-configuration-parameters/settings.md#server-settings-compression) configuración. -- [Registro](../../engines/table-engines/log-family/log-family.md) familia. Utiliza el `lz4` método de compresión por defecto y soporta códecs de compresión de columna. +- [Registro](../../engines/table-engines/log-family/index.md) familia. Utiliza el `lz4` método de compresión por defecto y soporta códecs de compresión de columna. - [Establecer](../../engines/table-engines/special/set.md). Solo admite la compresión predeterminada. - [Unir](../../engines/table-engines/special/join.md). Solo admite la compresión predeterminada. diff --git a/docs/fa/engines/table-engines/log-family/index.md b/docs/fa/engines/table-engines/log-family/index.md index 6aaab539716..e7d22c37912 100644 --- a/docs/fa/engines/table-engines/log-family/index.md +++ b/docs/fa/engines/table-engines/log-family/index.md @@ -3,6 +3,47 @@ machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: "\u062B\u0628\u062A \u062E\u0627\u0646\u0648\u0627\u062F\u0647" toc_priority: 29 +toc_title: "\u0645\u0639\u0631\u0641\u06CC \u0634\u0631\u06A9\u062A" --- +# ورود خانواده موتور {#log-engine-family} + +هنگامی که شما نیاز به سرعت نوشتن بسیاری از جداول کوچک (تا حدود 1 میلیون ردیف) و بعد به عنوان یک کل خواندن این موتور برای حالات توسعه داده شد. + +موتورهای خانواده: + +- [خط زدن](stripelog.md) +- [ثبت](log.md) +- [جمع شدن](tinylog.md) + +## ویژگیهای مشترک {#common-properties} + +موتورها: + +- ذخیره داده ها بر روی یک دیسک. + +- اضافه کردن داده ها به پایان فایل هنگام نوشتن. + +- قفل پشتیبانی برای دسترسی همزمان داده ها. + + در طول `INSERT` نمایش داده شد, جدول قفل شده است, و دیگر نمایش داده شد برای خواندن و نوشتن داده ها هر دو منتظر جدول برای باز کردن. اگر هیچ نمایش داده شد نوشتن داده ها وجود دارد, هر تعداد از نمایش داده شد خواندن داده ها را می توان به صورت همزمان انجام. + +- پشتیبانی نمی کند [جهش](../../../sql-reference/statements/alter.md#alter-mutations) عملیات. + +- هنوز شاخص را پشتیبانی نمی کند. + + این به این معنی است که `SELECT` نمایش داده شد برای محدوده داده ها موثر نیست. + +- هنوز داده نوشتن نیست اتمی. + + شما می توانید یک جدول با داده های خراب اگر چیزی می شکند عملیات نوشتن, مثلا, خاموش کردن سرور غیر طبیعی. + +## تفاوت {#differences} + +این `TinyLog` موتور ساده ترین در خانواده است و فقیرترین قابلیت ها و کمترین بهره وری را فراهم می کند. این `TinyLog` موتور از خواندن داده های موازی با چندین موضوع پشتیبانی نمی کند. این اطلاعات کندتر از موتورهای دیگر در خانواده است که خواندن موازی را پشتیبانی می کند و تقریبا به عنوان بسیاری از توصیفگرها به عنوان `Log` موتور به دلیل ذخیره هر ستون در یک فایل جداگانه. در حالات کم بار ساده استفاده کنید. + +این `Log` و `StripeLog` موتورهای پشتیبانی خواندن داده های موازی. هنگام خواندن داده ها, تاتر با استفاده از موضوعات متعدد. هر موضوع یک بلوک داده جداگانه را پردازش می کند. این `Log` موتور با استفاده از یک فایل جداگانه برای هر ستون از جدول. `StripeLog` ذخیره تمام داده ها در یک فایل. در نتیجه `StripeLog` موتور با استفاده از توصیف کمتر در سیستم عامل, اما `Log` موتور فراهم می کند بهره وری بالاتر در هنگام خواندن داده ها. + +[مقاله اصلی](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) + diff --git a/docs/fa/engines/table-engines/log-family/log-family.md b/docs/fa/engines/table-engines/log-family/log-family.md deleted file mode 100644 index ae35d985de6..00000000000 --- a/docs/fa/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: "\u0645\u0639\u0631\u0641\u06CC \u0634\u0631\u06A9\u062A" ---- - -# ورود خانواده موتور {#log-engine-family} - -هنگامی که شما نیاز به سرعت نوشتن بسیاری از جداول کوچک (تا حدود 1 میلیون ردیف) و بعد به عنوان یک کل خواندن این موتور برای حالات توسعه داده شد. - -موتورهای خانواده: - -- [خط زدن](stripelog.md) -- [ثبت](log.md) -- [جمع شدن](tinylog.md) - -## ویژگیهای مشترک {#common-properties} - -موتورها: - -- ذخیره داده ها بر روی یک دیسک. - -- اضافه کردن داده ها به پایان فایل هنگام نوشتن. - -- قفل پشتیبانی برای دسترسی همزمان داده ها. - - در طول `INSERT` نمایش داده شد, جدول قفل شده است, و دیگر نمایش داده شد برای خواندن و نوشتن داده ها هر دو منتظر جدول برای باز کردن. اگر هیچ نمایش داده شد نوشتن داده ها وجود دارد, هر تعداد از نمایش داده شد خواندن داده ها را می توان به صورت همزمان انجام. - -- پشتیبانی نمی کند [جهش](../../../sql-reference/statements/alter.md#alter-mutations) عملیات. - -- هنوز شاخص را پشتیبانی نمی کند. - - این به این معنی است که `SELECT` نمایش داده شد برای محدوده داده ها موثر نیست. - -- هنوز داده نوشتن نیست اتمی. - - شما می توانید یک جدول با داده های خراب اگر چیزی می شکند عملیات نوشتن, مثلا, خاموش کردن سرور غیر طبیعی. - -## تفاوت {#differences} - -این `TinyLog` موتور ساده ترین در خانواده است و فقیرترین قابلیت ها و کمترین بهره وری را فراهم می کند. این `TinyLog` موتور از خواندن داده های موازی با چندین موضوع پشتیبانی نمی کند. این اطلاعات کندتر از موتورهای دیگر در خانواده است که خواندن موازی را پشتیبانی می کند و تقریبا به عنوان بسیاری از توصیفگرها به عنوان `Log` موتور به دلیل ذخیره هر ستون در یک فایل جداگانه. در حالات کم بار ساده استفاده کنید. - -این `Log` و `StripeLog` موتورهای پشتیبانی خواندن داده های موازی. هنگام خواندن داده ها, تاتر با استفاده از موضوعات متعدد. هر موضوع یک بلوک داده جداگانه را پردازش می کند. این `Log` موتور با استفاده از یک فایل جداگانه برای هر ستون از جدول. `StripeLog` ذخیره تمام داده ها در یک فایل. در نتیجه `StripeLog` موتور با استفاده از توصیف کمتر در سیستم عامل, اما `Log` موتور فراهم می کند بهره وری بالاتر در هنگام خواندن داده ها. - -[مقاله اصلی](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/fa/engines/table-engines/log-family/log.md b/docs/fa/engines/table-engines/log-family/log.md index ad51f85ee80..a8a871171db 100644 --- a/docs/fa/engines/table-engines/log-family/log.md +++ b/docs/fa/engines/table-engines/log-family/log.md @@ -7,7 +7,7 @@ toc_title: "\u062B\u0628\u062A" # ثبت {#log} -موتور متعلق به خانواده از موتورهای ورود به سیستم. مشاهده خواص مشترک از موتورهای ورود به سیستم و تفاوت های خود را در [ورود خانواده موتور](log-family.md) مقاله. +موتور متعلق به خانواده از موتورهای ورود به سیستم. مشاهده خواص مشترک از موتورهای ورود به سیستم و تفاوت های خود را در [ورود خانواده موتور](index.md) مقاله. ورود متفاوت از [جمع شدن](tinylog.md) در این فایل کوچک “marks” ساکن با فایل های ستون. این علامت ها در هر بلوک داده نوشته شده است و شامل شیپور خاموشی که نشان می دهد از کجا شروع به خواندن فایل به منظور جست و خیز تعداد مشخصی از ردیف. این باعث می شود امکان خواندن داده های جدول در موضوعات مختلف. برای همزمان دسترسی به داده ها, عملیات خواندن را می توان به طور همزمان انجام, در حالی که ارسال عملیات بلوک می خواند و هر یک از دیگر. diff --git a/docs/fa/engines/table-engines/log-family/stripelog.md b/docs/fa/engines/table-engines/log-family/stripelog.md index d4703e1434b..5c6fe14994e 100644 --- a/docs/fa/engines/table-engines/log-family/stripelog.md +++ b/docs/fa/engines/table-engines/log-family/stripelog.md @@ -7,7 +7,7 @@ toc_title: "\u062E\u0637 \u0632\u062F\u0646" # خط زدن {#stripelog} -این موتور متعلق به خانواده از موتورهای ورود به سیستم. مشاهده خواص مشترک از موتورهای ورود به سیستم و تفاوت های خود را در [ورود خانواده موتور](log-family.md) مقاله. +این موتور متعلق به خانواده از موتورهای ورود به سیستم. مشاهده خواص مشترک از موتورهای ورود به سیستم و تفاوت های خود را در [ورود خانواده موتور](index.md) مقاله. با استفاده از این موتور در حالات زمانی که شما نیاز به نوشتن بسیاری از جداول با مقدار کمی از داده ها (کمتر از 1 میلیون ردیف). diff --git a/docs/fa/engines/table-engines/log-family/tinylog.md b/docs/fa/engines/table-engines/log-family/tinylog.md index 20bee377116..ea06ba84c96 100644 --- a/docs/fa/engines/table-engines/log-family/tinylog.md +++ b/docs/fa/engines/table-engines/log-family/tinylog.md @@ -7,7 +7,7 @@ toc_title: "\u062C\u0645\u0639 \u0634\u062F\u0646" # جمع شدن {#tinylog} -موتور متعلق به خانواده موتور ورود به سیستم. ببینید [ورود خانواده موتور](log-family.md) برای خواص مشترک موتورهای ورود به سیستم و تفاوت های خود را. +موتور متعلق به خانواده موتور ورود به سیستم. ببینید [ورود خانواده موتور](index.md) برای خواص مشترک موتورهای ورود به سیستم و تفاوت های خود را. این موتور جدول معمولا با روش نوشتن یک بار استفاده می شود: نوشتن داده ها یک بار و سپس خواندن هر چند بار که لازم است. مثلا, شما می توانید استفاده کنید `TinyLog`- نوع جداول برای داده های واسطه است که در دسته های کوچک پردازش شده است. توجه داشته باشید که ذخیره سازی داده ها در تعداد زیادی از جداول کوچک بی اثر است. diff --git a/docs/fa/sql-reference/statements/create.md b/docs/fa/sql-reference/statements/create.md index f6217b8e87f..04c5d00e022 100644 --- a/docs/fa/sql-reference/statements/create.md +++ b/docs/fa/sql-reference/statements/create.md @@ -155,7 +155,7 @@ ENGINE = فشرده سازی برای موتورهای جدول زیر پشتیبانی می شود: - [ادغام](../../engines/table-engines/mergetree-family/mergetree.md) خانواده پشتیبانی از کدک های فشرده سازی ستون و انتخاب روش فشرده سازی پیش فرض توسط [فشردهسازی](../../operations/server-configuration-parameters/settings.md#server-settings-compression) تنظیمات. -- [ثبت](../../engines/table-engines/log-family/log-family.md) خانواده با استفاده از `lz4` روش فشرده سازی به طور پیش فرض و پشتیبانی از کدک های فشرده سازی ستون. +- [ثبت](../../engines/table-engines/log-family/index.md) خانواده با استفاده از `lz4` روش فشرده سازی به طور پیش فرض و پشتیبانی از کدک های فشرده سازی ستون. - [تنظیم](../../engines/table-engines/special/set.md). فقط فشرده سازی پیش فرض پشتیبانی می کند. - [پیوستن](../../engines/table-engines/special/join.md). فقط فشرده سازی پیش فرض پشتیبانی می کند. diff --git a/docs/fr/engines/table-engines/log-family/index.md b/docs/fr/engines/table-engines/log-family/index.md index 9b94463744c..a4005fcbd66 100644 --- a/docs/fr/engines/table-engines/log-family/index.md +++ b/docs/fr/engines/table-engines/log-family/index.md @@ -2,7 +2,45 @@ machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: Journal De La Famille -toc_priority: 29 +toc_title: Introduction --- +# Famille De Moteurs En Rondins {#log-engine-family} +Ces moteurs ont été développés pour les scénarios où vous devez écrire rapidement de nombreuses petites tables (jusqu'à environ 1 million de lignes) et les lire plus tard dans leur ensemble. + +Les moteurs de la famille: + +- [StripeLog](stripelog.md) +- [Journal](log.md) +- [TinyLog](tinylog.md) + +## Propriétés Communes {#common-properties} + +Moteur: + +- Stocker des données sur un disque. + +- Ajouter des données à la fin du fichier lors de l'écriture. + +- Bloque simultanées dans l'accès aux données. + + Lors `INSERT` requêtes, la table est verrouillée, et d'autres requêtes pour la lecture et l'écriture de données attendent que la table se déverrouille. S'il n'y a pas de requêtes d'écriture de données, un certain nombre de requêtes de lecture de données peuvent être effectuées simultanément. + +- Ne prennent pas en charge [mutation](../../../sql-reference/statements/alter.md#alter-mutations) opérations. + +- Ne prennent pas en charge les index. + + Cela signifie que `SELECT` les requêtes pour les plages de données ne sont pas efficaces. + +- N'écrivez pas de données de manière atomique. + + Vous pouvez obtenir une table avec des données corrompues si quelque chose interrompt l'opération d'écriture, par exemple, un arrêt anormal du serveur. + +## Différence {#differences} + +Le `TinyLog` le moteur est le plus simple de la famille et offre la fonctionnalité la plus pauvre et la plus faible efficacité. Le `TinyLog` le moteur ne prend pas en charge la lecture de données parallèles par plusieurs threads. Il lit les données plus lentement que les autres moteurs de la famille qui prennent en charge la lecture parallèle et utilise presque autant de descripteurs que `Log` moteur, car il stocke chaque colonne dans un fichier séparé. Utilisez-le dans des scénarios simples à faible charge. + +Le `Log` et `StripeLog` les moteurs prennent en charge la lecture de données parallèle. Lors de la lecture de données, ClickHouse utilise plusieurs threads. Chaque thread traite un bloc de données séparé. Le `Log` le moteur utilise un fichier distinct pour chaque colonne de la table. `StripeLog` stocke toutes les données dans un seul fichier. En conséquence, la `StripeLog` moteur utilise moins de descripteurs dans le système d'exploitation, mais le `Log` moteur fournit une plus grande efficacité lors de la lecture des données. + +[Article Original](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/fr/engines/table-engines/log-family/log-family.md b/docs/fr/engines/table-engines/log-family/log-family.md deleted file mode 100644 index 4ff840b9699..00000000000 --- a/docs/fr/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: Introduction ---- - -# Famille De Moteurs En Rondins {#log-engine-family} - -Ces moteurs ont été développés pour les scénarios où vous devez écrire rapidement de nombreuses petites tables (jusqu'à environ 1 million de lignes) et les lire plus tard dans leur ensemble. - -Les moteurs de la famille: - -- [StripeLog](stripelog.md) -- [Journal](log.md) -- [TinyLog](tinylog.md) - -## Propriétés Communes {#common-properties} - -Moteur: - -- Stocker des données sur un disque. - -- Ajouter des données à la fin du fichier lors de l'écriture. - -- Bloque simultanées dans l'accès aux données. - - Lors `INSERT` requêtes, la table est verrouillée, et d'autres requêtes pour la lecture et l'écriture de données attendent que la table se déverrouille. S'il n'y a pas de requêtes d'écriture de données, un certain nombre de requêtes de lecture de données peuvent être effectuées simultanément. - -- Ne prennent pas en charge [mutation](../../../sql-reference/statements/alter.md#alter-mutations) opérations. - -- Ne prennent pas en charge les index. - - Cela signifie que `SELECT` les requêtes pour les plages de données ne sont pas efficaces. - -- N'écrivez pas de données de manière atomique. - - Vous pouvez obtenir une table avec des données corrompues si quelque chose interrompt l'opération d'écriture, par exemple, un arrêt anormal du serveur. - -## Différence {#differences} - -Le `TinyLog` le moteur est le plus simple de la famille et offre la fonctionnalité la plus pauvre et la plus faible efficacité. Le `TinyLog` le moteur ne prend pas en charge la lecture de données parallèles par plusieurs threads. Il lit les données plus lentement que les autres moteurs de la famille qui prennent en charge la lecture parallèle et utilise presque autant de descripteurs que `Log` moteur, car il stocke chaque colonne dans un fichier séparé. Utilisez-le dans des scénarios simples à faible charge. - -Le `Log` et `StripeLog` les moteurs prennent en charge la lecture de données parallèle. Lors de la lecture de données, ClickHouse utilise plusieurs threads. Chaque thread traite un bloc de données séparé. Le `Log` le moteur utilise un fichier distinct pour chaque colonne de la table. `StripeLog` stocke toutes les données dans un seul fichier. En conséquence, la `StripeLog` moteur utilise moins de descripteurs dans le système d'exploitation, mais le `Log` moteur fournit une plus grande efficacité lors de la lecture des données. - -[Article Original](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/fr/engines/table-engines/log-family/log.md b/docs/fr/engines/table-engines/log-family/log.md index 33302e2b7f2..f0d06af03d6 100644 --- a/docs/fr/engines/table-engines/log-family/log.md +++ b/docs/fr/engines/table-engines/log-family/log.md @@ -7,7 +7,7 @@ toc_title: Journal # Journal {#log} -Moteur appartient à la famille de journal des moteurs. Voir les propriétés communes des moteurs de journal et leurs différences dans le [Famille De Moteurs En Rondins](log-family.md) article. +Moteur appartient à la famille de journal des moteurs. Voir les propriétés communes des moteurs de journal et leurs différences dans le [Famille De Moteurs En Rondins](index.md) article. Journal diffère de [TinyLog](tinylog.md) dans un petit fichier de “marks” réside avec les fichiers de colonne. Ces marques sont écrites sur chaque bloc de données et contiennent des décalages qui indiquent où commencer à lire le fichier afin d'ignorer le nombre de lignes spécifié. Cela permet de lire les données de table dans plusieurs threads. Pour l'accès aux données simultanées, les opérations de lecture peuvent être effectuées simultanément, tandis que les opérations d'écriture bloc lit et l'autre. diff --git a/docs/fr/engines/table-engines/log-family/stripelog.md b/docs/fr/engines/table-engines/log-family/stripelog.md index 390be1d34e3..2c2919aec21 100644 --- a/docs/fr/engines/table-engines/log-family/stripelog.md +++ b/docs/fr/engines/table-engines/log-family/stripelog.md @@ -7,7 +7,7 @@ toc_title: StripeLog # Stripelog {#stripelog} -Ce moteur appartient à la famille des moteurs en rondins. Voir les propriétés communes des moteurs de journal et leurs différences dans le [Famille De Moteurs En Rondins](log-family.md) article. +Ce moteur appartient à la famille des moteurs en rondins. Voir les propriétés communes des moteurs de journal et leurs différences dans le [Famille De Moteurs En Rondins](index.md) article. Utilisez ce moteur dans des scénarios lorsque vous devez écrire de nombreuses tables avec une petite quantité de données (moins de 1 million de lignes). diff --git a/docs/fr/engines/table-engines/log-family/tinylog.md b/docs/fr/engines/table-engines/log-family/tinylog.md index 54292730851..275b621718c 100644 --- a/docs/fr/engines/table-engines/log-family/tinylog.md +++ b/docs/fr/engines/table-engines/log-family/tinylog.md @@ -7,7 +7,7 @@ toc_title: TinyLog # TinyLog {#tinylog} -Le moteur appartient à la famille de moteurs en rondins. Voir [Famille De Moteurs En Rondins](log-family.md) pour les propriétés communes des moteurs en rondins et leurs différences. +Le moteur appartient à la famille de moteurs en rondins. Voir [Famille De Moteurs En Rondins](index.md) pour les propriétés communes des moteurs en rondins et leurs différences. Ce moteur de table est généralement utilisé avec la méthode write-once: écrivez des données une fois, puis lisez-les autant de fois que nécessaire. Par exemple, vous pouvez utiliser `TinyLog`- tapez des tables pour les données intermédiaires qui sont traitées en petits lots. Notez que le stockage des données dans un grand nombre de petites tables est inefficace. diff --git a/docs/fr/sql-reference/statements/create.md b/docs/fr/sql-reference/statements/create.md index 34f2953bcca..a9a9ce9b0b7 100644 --- a/docs/fr/sql-reference/statements/create.md +++ b/docs/fr/sql-reference/statements/create.md @@ -155,7 +155,7 @@ Si un codec est spécifié, le codec par défaut ne s'applique pas. Les Codecs p La Compression est prise en charge pour les moteurs de tableau suivants: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) famille. Prend en charge les codecs de compression de colonne et la sélection de la méthode de compression par défaut par [compression](../../operations/server-configuration-parameters/settings.md#server-settings-compression) paramètre. -- [Journal](../../engines/table-engines/log-family/log-family.md) famille. Utilise le `lz4` méthode de compression par défaut et prend en charge les codecs de compression de colonne. +- [Journal](../../engines/table-engines/log-family/index.md) famille. Utilise le `lz4` méthode de compression par défaut et prend en charge les codecs de compression de colonne. - [Définir](../../engines/table-engines/special/set.md). Uniquement pris en charge la compression par défaut. - [Rejoindre](../../engines/table-engines/special/join.md). Uniquement pris en charge la compression par défaut. diff --git a/docs/ja/engines/table-engines/log-family/index.md b/docs/ja/engines/table-engines/log-family/index.md index 40f23ead1c7..91bbdb0db47 100644 --- a/docs/ja/engines/table-engines/log-family/index.md +++ b/docs/ja/engines/table-engines/log-family/index.md @@ -3,6 +3,45 @@ machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: "\u30ED\u30B0\u30D5\u30A1\u30DF\u30EA" toc_priority: 29 +toc_title: "\u306F\u3058\u3081\u306B" --- +# ログエンジン家族 {#log-engine-family} +これらのエンジンは、多くの小さなテーブル(最大1万行)をすばやく書き込み、後で全体として読み込む必要があるシナリオ用に開発されました。 + +家族のエンジン: + +- [ストリップログ](stripelog.md) +- [ログ](log.md) +- [TinyLog](tinylog.md) + +## 共通プロパティ {#common-properties} + +エンジン: + +- ディスクにデータを格納します。 + +- 書き込み時にファイルの末尾にデータを追加します。 + +- 同時データアクセスのサポートロック。 + + 中 `INSERT` クエリのテーブルがロックされ、その他の質問を読み込みおよび書き込みデータの両方のテーブルを作成する データ書き込みクエリがない場合は、任意の数のデータ読み込みクエリを同時に実行できます。 + +- サポートしない [突然変異](../../../sql-reference/statements/alter.md#alter-mutations) 作戦だ + +- 索引をサポートしません。 + + つまり `SELECT` データ範囲のクエリは効率的ではありません。 + +- 書くわけではありませんデータを原子的に. + + 取得できるテーブルデータが破損した場合も破れ、書き込み操作は、例えば、異常サーバをシャットダウンしました。 + +## 違い {#differences} + +その `TinyLog` エンジンは家族の最も簡単、最も悪い機能性および最も低い効率を提供する。 その `TinyLog` エンジンをサポートしていない並列データの読み取りによる複数のスレッド)。 でデータを読み込む代わりに、各エンジンの家族を支援する並列読みでの使用がほとんど同じになりました記述子としての `Log` エンジンは、各列を別々のファイルに格納するためです。 単純な低負荷のシナリオで使用します。 + +その `Log` と `StripeLog` エンジンの支援並列データです。 読み込み時にデータClickHouse使複数のスレッド)。 各スレッドプロセス別データブロックです。 その `Log` エンジンは、テーブルの各列に個別のファイルを使用します。 `StripeLog` すべてのデータファイルです。 その結果、 `StripeLog` エンジンは、オペレーティングシス `Log` エンジンはデータを読むとき高性能を提供する。 + +[元の記事](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/ja/engines/table-engines/log-family/log-family.md b/docs/ja/engines/table-engines/log-family/log-family.md deleted file mode 100644 index 2606e6ad771..00000000000 --- a/docs/ja/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: "\u306F\u3058\u3081\u306B" ---- - -# ログエンジン家族 {#log-engine-family} - -これらのエンジンは、多くの小さなテーブル(最大1万行)をすばやく書き込み、後で全体として読み込む必要があるシナリオ用に開発されました。 - -家族のエンジン: - -- [ストリップログ](stripelog.md) -- [ログ](log.md) -- [TinyLog](tinylog.md) - -## 共通プロパティ {#common-properties} - -エンジン: - -- ディスクにデータを格納します。 - -- 書き込み時にファイルの末尾にデータを追加します。 - -- 同時データアクセスのサポートロック。 - - 中 `INSERT` クエリのテーブルがロックされ、その他の質問を読み込みおよび書き込みデータの両方のテーブルを作成する データ書き込みクエリがない場合は、任意の数のデータ読み込みクエリを同時に実行できます。 - -- サポートしない [突然変異](../../../sql-reference/statements/alter.md#alter-mutations) 作戦だ - -- 索引をサポートしません。 - - つまり `SELECT` データ範囲のクエリは効率的ではありません。 - -- 書くわけではありませんデータを原子的に. - - 取得できるテーブルデータが破損した場合も破れ、書き込み操作は、例えば、異常サーバをシャットダウンしました。 - -## 違い {#differences} - -その `TinyLog` エンジンは家族の最も簡単、最も悪い機能性および最も低い効率を提供する。 その `TinyLog` エンジンをサポートしていない並列データの読み取りによる複数のスレッド)。 でデータを読み込む代わりに、各エンジンの家族を支援する並列読みでの使用がほとんど同じになりました記述子としての `Log` エンジンは、各列を別々のファイルに格納するためです。 単純な低負荷のシナリオで使用します。 - -その `Log` と `StripeLog` エンジンの支援並列データです。 読み込み時にデータClickHouse使複数のスレッド)。 各スレッドプロセス別データブロックです。 その `Log` エンジンは、テーブルの各列に個別のファイルを使用します。 `StripeLog` すべてのデータファイルです。 その結果、 `StripeLog` エンジンは、オペレーティングシス `Log` エンジンはデータを読むとき高性能を提供する。 - -[元の記事](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/ja/engines/table-engines/log-family/log.md b/docs/ja/engines/table-engines/log-family/log.md index 412b92c5551..904c601257f 100644 --- a/docs/ja/engines/table-engines/log-family/log.md +++ b/docs/ja/engines/table-engines/log-family/log.md @@ -7,7 +7,7 @@ toc_title: "\u30ED\u30B0" # ログ {#log} -エンジンはログエンジンの系列に属します。 ログエンジンの共通のプロパティとその違いを参照してください [ログエンジン家族](log-family.md) 記事だ +エンジンはログエンジンの系列に属します。 ログエンジンの共通のプロパティとその違いを参照してください [ログエンジン家族](index.md) 記事だ ログとは異なります [TinyLog](tinylog.md) その中の小さなファイルの “marks” 列ファイルに存在します。 これらのマークはすべてのデータブロックに書き込まれ、指定された行数をスキップするためにファイルの読み取りを開始する場所を示すオフセット この読み取りを可能にする機能がありテーブルデータを複数のスレッド)。 同時データアクセスの場合、読み取り操作は同時に実行できますが、書き込み操作は読み取りをブロックします。 diff --git a/docs/ja/engines/table-engines/log-family/stripelog.md b/docs/ja/engines/table-engines/log-family/stripelog.md index d157368824e..bbecd50bbea 100644 --- a/docs/ja/engines/table-engines/log-family/stripelog.md +++ b/docs/ja/engines/table-engines/log-family/stripelog.md @@ -7,7 +7,7 @@ toc_title: "\u30B9\u30C8\u30EA\u30C3\u30D7\u30ED\u30B0" # ストリップログ {#stripelog} -このエンジンはログエンジンの系列に属します。 ログエンジンの共通のプロパティとその違いを参照してください [ログエンジン家族](log-family.md) 記事だ +このエンジンはログエンジンの系列に属します。 ログエンジンの共通のプロパティとその違いを参照してください [ログエンジン家族](index.md) 記事だ 少量のデータ(1万行未満)で多数のテーブルを記述する必要がある場合に、このエンジンを使用します。 diff --git a/docs/ja/engines/table-engines/log-family/tinylog.md b/docs/ja/engines/table-engines/log-family/tinylog.md index 3cfb05b5597..1bee48fc53e 100644 --- a/docs/ja/engines/table-engines/log-family/tinylog.md +++ b/docs/ja/engines/table-engines/log-family/tinylog.md @@ -7,7 +7,7 @@ toc_title: TinyLog # TinyLog {#tinylog} -エンジンはログエンジンファミリに属します。 見る [ログエンジン家族](log-family.md) ログエンジンの共通プロパティとその違い。 +エンジンはログエンジンファミリに属します。 見る [ログエンジン家族](index.md) ログエンジンの共通プロパティとその違い。 このテーブルエンジンは、通常、write-onceメソッドで使用されます。 たとえば、次のようにします `TinyLog`-小さなバッチで処理される中間データのテーブルを入力します。 多数の小さなテーブルにデータを格納するのは非効率的です。 diff --git a/docs/ja/sql-reference/statements/create.md b/docs/ja/sql-reference/statements/create.md index 0d7ed3b1bc7..fb40d5c63b4 100644 --- a/docs/ja/sql-reference/statements/create.md +++ b/docs/ja/sql-reference/statements/create.md @@ -155,7 +155,7 @@ ENGINE = 圧縮は、次の表エンジンでサポートされます: - [メルゲツリー](../../engines/table-engines/mergetree-family/mergetree.md) 家族だ 支柱の圧縮コーデックとの選択のデフォルトの圧縮メソッドによる [圧縮](../../operations/server-configuration-parameters/settings.md#server-settings-compression) 設定。 -- [ログ](../../engines/table-engines/log-family/log-family.md) 家族だ を使用して `lz4` 圧縮メソッドはデフォルト対応カラムの圧縮コーデック. +- [ログ](../../engines/table-engines/log-family/index.md) 家族だ を使用して `lz4` 圧縮メソッドはデフォルト対応カラムの圧縮コーデック. - [セット](../../engines/table-engines/special/set.md). 既定の圧縮のみをサポートしました。 - [参加](../../engines/table-engines/special/join.md). 既定の圧縮のみをサポートしました。 diff --git a/docs/redirects.txt b/docs/redirects.txt index b4c93ac4908..d0d4d4d6c2c 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -51,6 +51,7 @@ dicts/internal_dicts.md query-language/dicts/internal-dicts.md engines/database_engines/index.md engines/database-engines/index.md engines/database_engines/lazy.md engines/database-engines/lazy.md engines/database_engines/mysql.md engines/database-engines/mysql.md +engines/table-engines/log-family/log-family.md engines/table-engines/log-family/index.md engines/table_engines/index.md engines/table-engines/index.md engines/table_engines/integrations/hdfs.md engines/table-engines/integrations/hdfs.md engines/table_engines/integrations/index.md engines/table-engines/integrations/index.md diff --git a/docs/ru/engines/table-engines/log-family/index.md b/docs/ru/engines/table-engines/log-family/index.md index a5d93c4c7c3..866cee7d17e 100644 --- a/docs/ru/engines/table-engines/log-family/index.md +++ b/docs/ru/engines/table-engines/log-family/index.md @@ -1,6 +1,45 @@ --- -toc_folder_title: Log Family +toc_folder_title: Семейство Log +toc_title: Введение toc_priority: 29 --- +# Семейство Log {#semeistvo-log} +Движки разработаны для сценариев, когда необходимо быстро записывать много таблиц с небольшим объёмом данных (менее 1 миллиона строк), а затем читать их целиком. + +Движки семейства: + +- [StripeLog](stripelog.md) +- [Log](log.md) +- [TinyLog](tinylog.md) + +## Общие свойства {#obshchie-svoistva} + +Движки: + +- Хранят данные на диске. + +- Добавляют данные в конец файла при записи. + +- Поддерживают блокировки для конкурентного доступа к данным. + + Во время запросов `INSERT` таблица блокируется, а другие запросы на чтение и запись ожидают разблокировки таблицы. Если запросов на запись данных нет, то можно выполнять любое количество конкуретных запросов на чтение. + +- Не поддерживают операции [мутации](../../../engines/table-engines/log-family/index.md#alter-mutations). + +- Не поддерживают индексы. + + Это означает, что запросы `SELECT` не эффективны для выборки диапазонов данных. + +- Записывают данные не атомарно. + + Вы можете получить таблицу с повреждёнными данными, если что-то прервёт операцию записи (например, аварийное завершение работы сервера). + +## Отличия {#otlichiia} + +Движок `TinyLog` самый простой в семье и обеспечивает самые низкие функциональность и эффективность. Движок `TinyLog` не поддерживает параллельного чтения данных в несколько потоков. Движок читает данные медленнее, чем оба других движка с параллельным чтением, и использует почти столько же дескрипторов, сколько и движок `Log`, поскольку хранит каждый столбец в отдельном файле. Его можно использовать в простых сценариях с низкой нагрузкой. + +Движки `Log` и `StripeLog` поддерживают параллельное чтение. При чтении данных, ClickHouse использует множество потоков. Каждый поток обрабатывает отдельный блок данных. Движок `Log` сохраняет каждый столбец таблицы в отдельном файле. Движок `StripeLog` хранит все данные в одном файле. Таким образом, движок `StripeLog` использует меньше дескрипторов в операционной системе, а движок `Log` обеспечивает более эффективное считывание данных. + +[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/table_engines/log_family/) diff --git a/docs/ru/engines/table-engines/log-family/log-family.md b/docs/ru/engines/table-engines/log-family/log-family.md deleted file mode 100644 index e5156c95f9e..00000000000 --- a/docs/ru/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,39 +0,0 @@ -# Семейство Log {#semeistvo-log} - -Движки разработаны для сценариев, когда необходимо быстро записывать много таблиц с небольшим объёмом данных (менее 1 миллиона строк), а затем читать их целиком. - -Движки семейства: - -- [StripeLog](stripelog.md) -- [Log](log.md) -- [TinyLog](tinylog.md) - -## Общие свойства {#obshchie-svoistva} - -Движки: - -- Хранят данные на диске. - -- Добавляют данные в конец файла при записи. - -- Поддерживают блокировки для конкурентного доступа к данным. - - Во время запросов `INSERT` таблица блокируется, а другие запросы на чтение и запись ожидают разблокировки таблицы. Если запросов на запись данных нет, то можно выполнять любое количество конкуретных запросов на чтение. - -- Не поддерживают операции [мутации](../../../engines/table-engines/log-family/log-family.md#alter-mutations). - -- Не поддерживают индексы. - - Это означает, что запросы `SELECT` не эффективны для выборки диапазонов данных. - -- Записывают данные не атомарно. - - Вы можете получить таблицу с повреждёнными данными, если что-то прервёт операцию записи (например, аварийное завершение работы сервера). - -## Отличия {#otlichiia} - -Движок `TinyLog` самый простой в семье и обеспечивает самые низкие функциональность и эффективность. Движок `TinyLog` не поддерживает параллельного чтения данных в несколько потоков. Движок читает данные медленнее, чем оба других движка с параллельным чтением, и использует почти столько же дескрипторов, сколько и движок `Log`, поскольку хранит каждый столбец в отдельном файле. Его можно использовать в простых сценариях с низкой нагрузкой. - -Движки `Log` и `StripeLog` поддерживают параллельное чтение. При чтении данных, ClickHouse использует множество потоков. Каждый поток обрабатывает отдельный блок данных. Движок `Log` сохраняет каждый столбец таблицы в отдельном файле. Движок `StripeLog` хранит все данные в одном файле. Таким образом, движок `StripeLog` использует меньше дескрипторов в операционной системе, а движок `Log` обеспечивает более эффективное считывание данных. - -[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/table_engines/log_family/) diff --git a/docs/ru/engines/table-engines/log-family/log.md b/docs/ru/engines/table-engines/log-family/log.md index 84474b3f40a..826738d6a3d 100644 --- a/docs/ru/engines/table-engines/log-family/log.md +++ b/docs/ru/engines/table-engines/log-family/log.md @@ -1,6 +1,6 @@ # Log {#log} -Движок относится к семейству движков Log. Смотрите общие свойства и различия движков в статье [Семейство Log](log-family.md). +Движок относится к семейству движков Log. Смотрите общие свойства и различия движков в статье [Семейство Log](index.md). Отличается от [TinyLog](tinylog.md) тем, что вместе с файлами столбцов лежит небольшой файл «засечек». Засечки пишутся на каждый блок данных и содержат смещение - с какого места нужно читать файл, чтобы пропустить заданное количество строк. Это позволяет читать данные из таблицы в несколько потоков. При конкурентном доступе к данным, чтения могут выполняться одновременно, а записи блокируют чтения и друг друга. diff --git a/docs/ru/engines/table-engines/log-family/stripelog.md b/docs/ru/engines/table-engines/log-family/stripelog.md index d129cc055da..9523a354ee7 100644 --- a/docs/ru/engines/table-engines/log-family/stripelog.md +++ b/docs/ru/engines/table-engines/log-family/stripelog.md @@ -1,6 +1,6 @@ # StripeLog {#stripelog} -Движок относится к семейству движков Log. Смотрите общие свойства и различия движков в статье [Семейство Log](log-family.md). +Движок относится к семейству движков Log. Смотрите общие свойства и различия движков в статье [Семейство Log](index.md). Движок разработан для сценариев, когда необходимо записывать много таблиц с небольшим объёмом данных (менее 1 миллиона строк). diff --git a/docs/ru/engines/table-engines/log-family/tinylog.md b/docs/ru/engines/table-engines/log-family/tinylog.md index f9374272fbf..69bc88b133e 100644 --- a/docs/ru/engines/table-engines/log-family/tinylog.md +++ b/docs/ru/engines/table-engines/log-family/tinylog.md @@ -1,6 +1,6 @@ # TinyLog {#tinylog} -Движок относится к семейству движков Log. Смотрите общие свойства и различия движков в статье [Семейство Log](log-family.md). +Движок относится к семейству движков Log. Смотрите общие свойства и различия движков в статье [Семейство Log](index.md). Типичный способ использования этой движка — это write-once: сначала данные один раз записываются, а затем читаются столько раз, сколько это необходимо. Например, можно использовать таблицы с движком `TinyLog` для хранения промежуточных данных, которые обрабатываются небольшими блоками. Учтите, что хранить данные в большом количестве мелких таблиц неэффективно. diff --git a/docs/tr/engines/table-engines/log-family/index.md b/docs/tr/engines/table-engines/log-family/index.md index 39bc7c21f51..13d91686426 100644 --- a/docs/tr/engines/table-engines/log-family/index.md +++ b/docs/tr/engines/table-engines/log-family/index.md @@ -3,6 +3,45 @@ machine_translated: true machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: "G\xFCnl\xFCk Aile" toc_priority: 29 +toc_title: "Giri\u015F" --- +# Log Engine Ailesi {#log-engine-family} +Bu motorlar, birçok küçük tabloyu (yaklaşık 1 milyon satıra kadar) hızlı bir şekilde yazmanız ve daha sonra bir bütün olarak okumanız gerektiğinde senaryolar için geliştirilmiştir. + +Ailenin motorları: + +- [StripeLog](stripelog.md) +- [Günlük](log.md) +- [TinyLog](tinylog.md) + +## Ortak Özellikler {#common-properties} + +Motorlar: + +- Verileri bir diskte saklayın. + +- Yazarken dosyanın sonuna veri ekleyin. + +- Eşzamanlı veri erişimi için destek kilitleri. + + Sırasında `INSERT` sorgular, tablo kilitlenir ve veri okumak ve yazmak için diğer sorgular hem tablonun kilidini açmak için bekler. Veri yazma sorguları varsa, herhangi bir sayıda veri okuma sorguları aynı anda gerçekleştirilebilir. + +- Destek yok [mutasyon](../../../sql-reference/statements/alter.md#alter-mutations) harekat. + +- Dizinleri desteklemez. + + Bu demektir ki `SELECT` veri aralıkları için sorgular verimli değildir. + +- Atomik veri yazmayın. + + Bir şey yazma işlemini bozarsa, örneğin anormal sunucu kapatma gibi bozuk verilerle bir tablo alabilirsiniz. + +## Farklılıklar {#differences} + +Bu `TinyLog` motor, ailenin en basitidir ve en fakir işlevselliği ve en düşük verimliliği sağlar. Bu `TinyLog` motor, birkaç iş parçacığı tarafından paralel veri okumayı desteklemez. Paralel okumayı destekleyen ailedeki diğer motorlardan daha yavaş veri okur ve neredeyse birçok tanımlayıcı kullanır `Log` motor, her sütunu ayrı bir dosyada sakladığı için. Basit düşük yük senaryolarında kullanın. + +Bu `Log` ve `StripeLog` motorlar paralel veri okumayı destekler. Veri okurken, ClickHouse birden çok iş parçacığı kullanır. Her iş parçacığı ayrı bir veri bloğu işler. Bu `Log` engine, tablonun her sütunu için ayrı bir dosya kullanır. `StripeLog` tüm verileri tek bir dosyada saklar. Sonuç olarak, `StripeLog` motor işletim sisteminde daha az tanımlayıcı kullanır, ancak `Log` motor veri okurken daha yüksek verimlilik sağlar. + +[Orijinal makale](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/tr/engines/table-engines/log-family/log-family.md b/docs/tr/engines/table-engines/log-family/log-family.md deleted file mode 100644 index 074f9363e9f..00000000000 --- a/docs/tr/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,46 +0,0 @@ ---- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_priority: 31 -toc_title: "Giri\u015F" ---- - -# Log Engine Ailesi {#log-engine-family} - -Bu motorlar, birçok küçük tabloyu (yaklaşık 1 milyon satıra kadar) hızlı bir şekilde yazmanız ve daha sonra bir bütün olarak okumanız gerektiğinde senaryolar için geliştirilmiştir. - -Ailenin motorları: - -- [StripeLog](stripelog.md) -- [Günlük](log.md) -- [TinyLog](tinylog.md) - -## Ortak Özellikler {#common-properties} - -Motorlar: - -- Verileri bir diskte saklayın. - -- Yazarken dosyanın sonuna veri ekleyin. - -- Eşzamanlı veri erişimi için destek kilitleri. - - Sırasında `INSERT` sorgular, tablo kilitlenir ve veri okumak ve yazmak için diğer sorgular hem tablonun kilidini açmak için bekler. Veri yazma sorguları varsa, herhangi bir sayıda veri okuma sorguları aynı anda gerçekleştirilebilir. - -- Destek yok [mutasyon](../../../sql-reference/statements/alter.md#alter-mutations) harekat. - -- Dizinleri desteklemez. - - Bu demektir ki `SELECT` veri aralıkları için sorgular verimli değildir. - -- Atomik veri yazmayın. - - Bir şey yazma işlemini bozarsa, örneğin anormal sunucu kapatma gibi bozuk verilerle bir tablo alabilirsiniz. - -## Farklılıklar {#differences} - -Bu `TinyLog` motor, ailenin en basitidir ve en fakir işlevselliği ve en düşük verimliliği sağlar. Bu `TinyLog` motor, birkaç iş parçacığı tarafından paralel veri okumayı desteklemez. Paralel okumayı destekleyen ailedeki diğer motorlardan daha yavaş veri okur ve neredeyse birçok tanımlayıcı kullanır `Log` motor, her sütunu ayrı bir dosyada sakladığı için. Basit düşük yük senaryolarında kullanın. - -Bu `Log` ve `StripeLog` motorlar paralel veri okumayı destekler. Veri okurken, ClickHouse birden çok iş parçacığı kullanır. Her iş parçacığı ayrı bir veri bloğu işler. Bu `Log` engine, tablonun her sütunu için ayrı bir dosya kullanır. `StripeLog` tüm verileri tek bir dosyada saklar. Sonuç olarak, `StripeLog` motor işletim sisteminde daha az tanımlayıcı kullanır, ancak `Log` motor veri okurken daha yüksek verimlilik sağlar. - -[Orijinal makale](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/tr/engines/table-engines/log-family/log.md b/docs/tr/engines/table-engines/log-family/log.md index cdebf76515e..d0a585aca97 100644 --- a/docs/tr/engines/table-engines/log-family/log.md +++ b/docs/tr/engines/table-engines/log-family/log.md @@ -7,7 +7,7 @@ toc_title: "G\xFCnl\xFCk" # Günlük {#log} -Motor günlük motorları ailesine aittir. Günlük motorlarının ortak özelliklerini ve farklılıklarını görün [Log Engine Ailesi](log-family.md) Makale. +Motor günlük motorları ailesine aittir. Günlük motorlarının ortak özelliklerini ve farklılıklarını görün [Log Engine Ailesi](index.md) Makale. Log differsar differsit fromma [TinyLog](tinylog.md) bu küçük bir dosyada “marks” sütun dosyaları ile bulunur. Bu işaretler her veri bloğuna yazılır ve belirtilen satır sayısını atlamak için dosyayı okumaya nereden başlayacağınızı gösteren uzaklıklar içerir. Bu, tablo verilerini birden çok iş parçacığında okumayı mümkün kılar. Eşzamanlı veri erişimi için, okuma işlemleri aynı anda gerçekleştirilebilirken, yazma işlemleri okur ve birbirlerini engeller. diff --git a/docs/tr/engines/table-engines/log-family/stripelog.md b/docs/tr/engines/table-engines/log-family/stripelog.md index 038c6c28569..82b20138c5f 100644 --- a/docs/tr/engines/table-engines/log-family/stripelog.md +++ b/docs/tr/engines/table-engines/log-family/stripelog.md @@ -7,7 +7,7 @@ toc_title: StripeLog # Stripelog {#stripelog} -Bu motor günlük motor ailesine aittir. Günlük motorlarının ortak özelliklerini ve farklılıklarını görün [Log Engine Ailesi](log-family.md) Makale. +Bu motor günlük motor ailesine aittir. Günlük motorlarının ortak özelliklerini ve farklılıklarını görün [Log Engine Ailesi](index.md) Makale. Az miktarda veri içeren (1 milyondan az satır) birçok tablo yazmanız gerektiğinde, bu altyapıyı senaryolarda kullanın. diff --git a/docs/tr/engines/table-engines/log-family/tinylog.md b/docs/tr/engines/table-engines/log-family/tinylog.md index 6975e6e127c..dd21d35bd3b 100644 --- a/docs/tr/engines/table-engines/log-family/tinylog.md +++ b/docs/tr/engines/table-engines/log-family/tinylog.md @@ -7,7 +7,7 @@ toc_title: TinyLog # TinyLog {#tinylog} -Motor log engine ailesine aittir. Görmek [Log Engine Ailesi](log-family.md) günlük motorlarının ortak özellikleri ve farklılıkları için. +Motor log engine ailesine aittir. Görmek [Log Engine Ailesi](index.md) günlük motorlarının ortak özellikleri ve farklılıkları için. Bu tablo motoru genellikle write-once yöntemi ile kullanılır: verileri bir kez yazın, ardından gerektiği kadar okuyun. Örneğin, kullanabilirsiniz `TinyLog`- küçük gruplar halinde işlenen Ara veriler için tablolar yazın. Çok sayıda küçük tabloda veri depolamanın verimsiz olduğunu unutmayın. diff --git a/docs/tr/sql-reference/statements/create.md b/docs/tr/sql-reference/statements/create.md index d2c85ce6a4d..e52ccf0d9c6 100644 --- a/docs/tr/sql-reference/statements/create.md +++ b/docs/tr/sql-reference/statements/create.md @@ -155,7 +155,7 @@ Bir codec bileşeni belirtilmişse, varsayılan codec bileşeni geçerli değild Sıkıştırma Aşağıdaki tablo motorları için desteklenir: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) aile. Sütun sıkıştırma kodeklerini destekler ve varsayılan sıkıştırma yöntemini seçerek [sıkıştırma](../../operations/server-configuration-parameters/settings.md#server-settings-compression) ayarlar. -- [Günlük](../../engines/table-engines/log-family/log-family.md) aile. Kullanır `lz4` sıkıştırma yöntemi varsayılan olarak ve sütun sıkıştırma codec destekler. +- [Günlük](../../engines/table-engines/log-family/index.md) aile. Kullanır `lz4` sıkıştırma yöntemi varsayılan olarak ve sütun sıkıştırma codec destekler. - [Koymak](../../engines/table-engines/special/set.md). Yalnızca varsayılan sıkıştırmayı destekledi. - [Katmak](../../engines/table-engines/special/join.md). Yalnızca varsayılan sıkıştırmayı destekledi. diff --git a/docs/zh/engines/table-engines/log-family/index.md b/docs/zh/engines/table-engines/log-family/index.md index 30f6aac7442..64378a73daa 100644 --- a/docs/zh/engines/table-engines/log-family/index.md +++ b/docs/zh/engines/table-engines/log-family/index.md @@ -1,8 +1,51 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_folder_title: "\u65E5\u5FD7\u7CFB\u5217" +toc_title: 日志引擎系列 toc_priority: 29 --- +# 日志引擎系列 {#table_engines-log-engine-family} +这些引擎是为了需要写入许多小数据量(少于一百万行)的表的场景而开发的。 + +这系列的引擎有: + +- [StripeLog](stripelog.md) +- [日志](log.md) +- [TinyLog](tinylog.md) + +## 共同属性 {#table_engines-log-engine-family-common-properties} + +引擎: + +- 数据存储在磁盘上。 + +- 写入时将数据追加在文件末尾。 + +- 不支持[突变](../../../engines/table-engines/log-family/index.md#alter-mutations)操作。 + +- 不支持索引。 + + 这意味着 `SELECT` 在范围查询时效率不高。 + +- 非原子地写入数据。 + + 如果某些事情破坏了写操作,例如服务器的异常关闭,你将会得到一张包含了损坏数据的表。 + +## 差异 {#table_engines-log-engine-family-differences} + +`Log` 和 `StripeLog` 引擎支持: + +- 并发访问数据的锁。 + + `INSERT` 请求执行过程中表会被锁定,并且其他的读写数据的请求都会等待直到锁定被解除。如果没有写数据的请求,任意数量的读请求都可以并发执行。 + +- 并行读取数据。 + + 在读取数据时,ClickHouse 使用多线程。 每个线程处理不同的数据块。 + +`Log` 引擎为表中的每一列使用不同的文件。`StripeLog` 将所有的数据存储在一个文件中。因此 `StripeLog` 引擎在操作系统中使用更少的描述符,但是 `Log` 引擎提供更高的读性能。 + +`TingLog` 引擎是该系列中最简单的引擎并且提供了最少的功能和最低的性能。`TingLog` 引擎不支持并行读取和并发数据访问,并将每一列存储在不同的文件中。它比其余两种支持并行读取的引擎的读取速度更慢,并且使用了和 `Log` 引擎同样多的描述符。你可以在简单的低负载的情景下使用它。 + +[来源文章](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/zh/engines/table-engines/log-family/log-family.md b/docs/zh/engines/table-engines/log-family/log-family.md deleted file mode 100644 index b749cf42219..00000000000 --- a/docs/zh/engines/table-engines/log-family/log-family.md +++ /dev/null @@ -1,45 +0,0 @@ -# 日志引擎系列 {#table_engines-log-engine-family} - -这些引擎是为了需要写入许多小数据量(少于一百万行)的表的场景而开发的。 - -这系列的引擎有: - -- [StripeLog](stripelog.md) -- [日志](log.md) -- [TinyLog](tinylog.md) - -## 共同属性 {#table_engines-log-engine-family-common-properties} - -引擎: - -- 数据存储在磁盘上。 - -- 写入时将数据追加在文件末尾。 - -- 不支持[突变](../../../engines/table-engines/log-family/log-family.md#alter-mutations)操作。 - -- 不支持索引。 - - 这意味着 `SELECT` 在范围查询时效率不高。 - -- 非原子地写入数据。 - - 如果某些事情破坏了写操作,例如服务器的异常关闭,你将会得到一张包含了损坏数据的表。 - -## 差异 {#table_engines-log-engine-family-differences} - -`Log` 和 `StripeLog` 引擎支持: - -- 并发访问数据的锁。 - - `INSERT` 请求执行过程中表会被锁定,并且其他的读写数据的请求都会等待直到锁定被解除。如果没有写数据的请求,任意数量的读请求都可以并发执行。 - -- 并行读取数据。 - - 在读取数据时,ClickHouse 使用多线程。 每个线程处理不同的数据块。 - -`Log` 引擎为表中的每一列使用不同的文件。`StripeLog` 将所有的数据存储在一个文件中。因此 `StripeLog` 引擎在操作系统中使用更少的描述符,但是 `Log` 引擎提供更高的读性能。 - -`TingLog` 引擎是该系列中最简单的引擎并且提供了最少的功能和最低的性能。`TingLog` 引擎不支持并行读取和并发数据访问,并将每一列存储在不同的文件中。它比其余两种支持并行读取的引擎的读取速度更慢,并且使用了和 `Log` 引擎同样多的描述符。你可以在简单的低负载的情景下使用它。 - -[来源文章](https://clickhouse.tech/docs/en/operations/table_engines/log_family/) diff --git a/docs/zh/engines/table-engines/log-family/stripelog.md b/docs/zh/engines/table-engines/log-family/stripelog.md index 082880059c2..e10d9afd98d 100644 --- a/docs/zh/engines/table-engines/log-family/stripelog.md +++ b/docs/zh/engines/table-engines/log-family/stripelog.md @@ -1,6 +1,6 @@ # StripeLog {#stripelog} -该引擎属于日志引擎系列。请在[日志引擎系列](log-family.md)文章中查看引擎的共同属性和差异。 +该引擎属于日志引擎系列。请在[日志引擎系列](index.md)文章中查看引擎的共同属性和差异。 在你需要写入许多小数据量(小于一百万行)的表的场景下使用这个引擎。 From faf6be65769f42f5a79fff949ae26dcf6c9f9c58 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 9 Jul 2020 17:09:17 +0300 Subject: [PATCH 193/196] Implemented single part uploads for DiskS3 (#12026) * Implemented single part uploads for DiskS3. * Added `min_multi_part_upload_size` to disk configuration. --- src/Disks/S3/DiskS3.cpp | 14 +-- src/Disks/S3/DiskS3.h | 4 +- src/Disks/S3/registerDiskS3.cpp | 3 +- src/IO/WriteBufferFromS3.cpp | 89 +++++++++++++------- src/IO/WriteBufferFromS3.h | 3 + src/Storages/StorageS3.cpp | 2 +- tests/integration/test_s3_with_proxy/test.py | 6 +- 7 files changed, 81 insertions(+), 40 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 873b54353ad..57d38fcfa39 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -285,10 +285,11 @@ namespace const String & bucket_, Metadata metadata_, const String & s3_path_, + bool is_multipart, size_t min_upload_part_size, size_t buf_size_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) - , impl(WriteBufferFromS3(client_ptr_, bucket_, metadata_.s3_root_path + s3_path_, min_upload_part_size, buf_size_)) + , impl(WriteBufferFromS3(client_ptr_, bucket_, metadata_.s3_root_path + s3_path_, min_upload_part_size, is_multipart, buf_size_)) , metadata(std::move(metadata_)) , s3_path(s3_path_) { @@ -421,7 +422,8 @@ DiskS3::DiskS3( String bucket_, String s3_root_path_, String metadata_path_, - size_t min_upload_part_size_) + size_t min_upload_part_size_, + size_t min_multi_part_upload_size_) : name(std::move(name_)) , client(std::move(client_)) , proxy_configuration(std::move(proxy_configuration_)) @@ -429,6 +431,7 @@ DiskS3::DiskS3( , s3_root_path(std::move(s3_root_path_)) , metadata_path(std::move(metadata_path_)) , min_upload_part_size(min_upload_part_size_) + , min_multi_part_upload_size(min_multi_part_upload_size_) { } @@ -537,11 +540,12 @@ std::unique_ptr DiskS3::readFile(const String & path, si return std::make_unique(client, bucket, metadata, buf_size); } -std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t, size_t) +std::unique_ptr DiskS3::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t estimated_size, size_t) { bool exist = exists(path); /// Path to store new S3 object. auto s3_path = getRandomName(); + bool is_multipart = estimated_size >= min_multi_part_upload_size; if (!exist || mode == WriteMode::Rewrite) { /// If metadata file exists - remove and create new. @@ -554,7 +558,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Write to file by path: {} New S3 path: {}", backQuote(metadata_path + path), s3_root_path + s3_path); - return std::make_unique(client, bucket, metadata, s3_path, min_upload_part_size, buf_size); + return std::make_unique(client, bucket, metadata, s3_path, is_multipart, min_upload_part_size, buf_size); } else { @@ -563,7 +567,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Append to file by path: {}. New S3 path: {}. Existing S3 objects: {}.", backQuote(metadata_path + path), s3_root_path + s3_path, metadata.s3_objects.size()); - return std::make_unique(client, bucket, metadata, s3_path, min_upload_part_size, buf_size); + return std::make_unique(client, bucket, metadata, s3_path, is_multipart, min_upload_part_size, buf_size); } } diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 82168c55bb5..2323a12ba37 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -26,7 +26,8 @@ public: String bucket_, String s3_root_path_, String metadata_path_, - size_t min_upload_part_size_); + size_t min_upload_part_size_, + size_t min_multi_part_upload_size_); const String & getName() const override { return name; } @@ -109,6 +110,7 @@ private: const String s3_root_path; const String metadata_path; size_t min_upload_part_size; + size_t min_multi_part_upload_size; UInt64 reserved_bytes = 0; UInt64 reservation_count = 0; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index 119ba037c96..2935b039264 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -137,7 +137,8 @@ void registerDiskS3(DiskFactory & factory) uri.bucket, uri.key, metadata_path, - context.getSettingsRef().s3_min_upload_part_size); + context.getSettingsRef().s3_min_upload_part_size, + config.getUInt64(config_prefix + ".min_multi_part_upload_size", 10*1024*1024)); /// This code is used only to check access to the corresponding disk. checkWriteAccess(*s3disk); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 946871d9c63..9d51c503bd0 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -5,9 +5,12 @@ # include # include +# include +# include # include # include # include +# include # include # include @@ -33,8 +36,10 @@ WriteBufferFromS3::WriteBufferFromS3( const String & bucket_, const String & key_, size_t minimum_upload_part_size_, + bool is_multipart_, size_t buffer_size_) : BufferWithOwnMemory(buffer_size_, nullptr, 0) + , is_multipart(is_multipart_) , bucket(bucket_) , key(key_) , client_ptr(std::move(client_ptr_)) @@ -42,7 +47,8 @@ WriteBufferFromS3::WriteBufferFromS3( , temporary_buffer{std::make_unique()} , last_part_size{0} { - initiate(); + if (is_multipart) + initiate(); } @@ -53,14 +59,16 @@ void WriteBufferFromS3::nextImpl() temporary_buffer->write(working_buffer.begin(), offset()); - last_part_size += offset(); - - if (last_part_size > minimum_upload_part_size) + if (is_multipart) { - temporary_buffer->finalize(); - writePart(temporary_buffer->str()); - last_part_size = 0; - temporary_buffer = std::make_unique(); + last_part_size += offset(); + + if (last_part_size > minimum_upload_part_size) + { + writePart(temporary_buffer->str()); + last_part_size = 0; + temporary_buffer->restart(); + } } } @@ -69,8 +77,8 @@ void WriteBufferFromS3::finalize() { next(); - temporary_buffer->finalize(); - writePart(temporary_buffer->str()); + if (is_multipart) + writePart(temporary_buffer->str()); complete(); } @@ -144,31 +152,54 @@ void WriteBufferFromS3::writePart(const String & data) void WriteBufferFromS3::complete() { - LOG_DEBUG(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}", bucket, key, upload_id); - - Aws::S3::Model::CompleteMultipartUploadRequest req; - req.SetBucket(bucket); - req.SetKey(key); - req.SetUploadId(upload_id); - - if (!part_tags.empty()) + if (is_multipart) { - Aws::S3::Model::CompletedMultipartUpload multipart_upload; - for (size_t i = 0; i < part_tags.size(); ++i) + LOG_DEBUG(log, "Completing multipart upload. Bucket: {}, Key: {}, Upload_id: {}", bucket, key, upload_id); + + Aws::S3::Model::CompleteMultipartUploadRequest req; + req.SetBucket(bucket); + req.SetKey(key); + req.SetUploadId(upload_id); + + if (!part_tags.empty()) { - Aws::S3::Model::CompletedPart part; - multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); + Aws::S3::Model::CompletedMultipartUpload multipart_upload; + for (size_t i = 0; i < part_tags.size(); ++i) + { + Aws::S3::Model::CompletedPart part; + multipart_upload.AddParts(part.WithETag(part_tags[i]).WithPartNumber(i + 1)); + } + + req.SetMultipartUpload(multipart_upload); } - req.SetMultipartUpload(multipart_upload); + auto outcome = client_ptr->CompleteMultipartUpload(req); + + if (outcome.IsSuccess()) + LOG_DEBUG(log, "Multipart upload completed. Upload_id: {}", upload_id); + else + throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); } - - auto outcome = client_ptr->CompleteMultipartUpload(req); - - if (outcome.IsSuccess()) - LOG_DEBUG(log, "Multipart upload completed. Upload_id: {}", upload_id); else - throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + { + LOG_DEBUG(log, "Making single part upload. Bucket: {}, Key: {}", bucket, key); + + Aws::S3::Model::PutObjectRequest req; + req.SetBucket(bucket); + req.SetKey(key); + + /// This could be improved using an adapter to WriteBuffer. + const std::shared_ptr input_data = Aws::MakeShared("temporary buffer", temporary_buffer->str()); + temporary_buffer = std::make_unique(); + req.SetBody(input_data); + + auto outcome = client_ptr->PutObject(req); + + if (outcome.IsSuccess()) + LOG_DEBUG(log, "Single part upload has completed. Bucket: {}, Key: {}", bucket, key); + else + throw Exception(outcome.GetError().GetMessage(), ErrorCodes::S3_ERROR); + } } } diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 71c176749af..cabda4f2171 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -24,6 +24,8 @@ namespace DB class WriteBufferFromS3 : public BufferWithOwnMemory { private: + bool is_multipart; + String bucket; String key; std::shared_ptr client_ptr; @@ -44,6 +46,7 @@ public: const String & bucket_, const String & key_, size_t minimum_upload_part_size_, + bool is_multipart, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); void nextImpl() override; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index f8d3de3a238..d42dad42e47 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -153,7 +153,7 @@ namespace : sample_block(sample_block_) { write_buf = wrapWriteBufferWithCompressionMethod( - std::make_unique(client, bucket, key, min_upload_part_size), compression_method, 3); + std::make_unique(client, bucket, key, min_upload_part_size, true), compression_method, 3); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index dc326b719bf..0642cd88fe7 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -34,10 +34,10 @@ def cluster(): cluster.shutdown() -def check_proxy_logs(cluster, proxy_instance): +def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET", "DELETE"}): logs = cluster.get_container_logs(proxy_instance) # Check that all possible interactions with Minio are present - for http_method in ["POST", "PUT", "GET", "DELETE"]: + for http_method in http_methods: assert logs.find(http_method + " http://minio1") >= 0 @@ -65,4 +65,4 @@ def test_s3_with_proxy_list(cluster, policy): node.query("DROP TABLE IF EXISTS s3_test NO DELAY") for proxy in ["proxy1", "proxy2"]: - check_proxy_logs(cluster, proxy) + check_proxy_logs(cluster, proxy, ["PUT", "GET", "DELETE"]) From 26293e4640080a3d7c9c97a28f670717a87e746b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Jul 2020 17:18:50 +0300 Subject: [PATCH 194/196] Rename test 01378 --- ....reference => 01378_alter_rename_with_ttl_zookeeper.reference} | 0 ...ame_with_ttl.sql => 01378_alter_rename_with_ttl_zookeeper.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01378_alter_rename_with_ttl.reference => 01378_alter_rename_with_ttl_zookeeper.reference} (100%) rename tests/queries/0_stateless/{01378_alter_rename_with_ttl.sql => 01378_alter_rename_with_ttl_zookeeper.sql} (100%) diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl.reference b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference similarity index 100% rename from tests/queries/0_stateless/01378_alter_rename_with_ttl.reference rename to tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.reference diff --git a/tests/queries/0_stateless/01378_alter_rename_with_ttl.sql b/tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql similarity index 100% rename from tests/queries/0_stateless/01378_alter_rename_with_ttl.sql rename to tests/queries/0_stateless/01378_alter_rename_with_ttl_zookeeper.sql From 258d2fd499c88d73952d339ff1de651ff835c64d Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 9 Jul 2020 18:10:35 +0300 Subject: [PATCH 195/196] [docs] split various kinds of CREATE queries into separate articles (#12328) * normalize * split & adjust links * re-normalize * adjust ru links * adjust ja/tr links * partially apply e0d19d2aeaeafa3620305bce3a991fefc0e28695 * reset contribs --- docs/en/commercial/index.md | 9 +- docs/en/development/architecture.md | 18 +- docs/en/engines/index.md | 6 +- .../table-engines/integrations/index.md | 12 +- .../table-engines/integrations/mysql.md | 2 +- .../table-engines/integrations/odbc.md | 2 +- .../table-engines/log-family/stripelog.md | 2 +- .../mergetree-family/aggregatingmergetree.md | 2 +- .../mergetree-family/collapsingmergetree.md | 2 +- .../mergetree-family/graphitemergetree.md | 2 +- .../mergetree-family/mergetree.md | 2 +- .../mergetree-family/replacingmergetree.md | 2 +- .../mergetree-family/summingmergetree.md | 2 +- .../versionedcollapsingmergetree.md | 2 +- .../en/engines/table-engines/special/index.md | 2 +- docs/en/engines/table-engines/special/join.md | 2 +- .../table-engines/special/materializedview.md | 2 +- .../faq/general/why-clickhouse-is-so-fast.md | 2 +- docs/en/faq/use-cases/time-series.md | 4 +- docs/en/getting-started/install.md | 6 +- docs/en/getting-started/tutorial.md | 2 +- docs/en/interfaces/formats.md | 2 +- docs/en/interfaces/http.md | 28 +- docs/en/interfaces/third-party/index.md | 11 +- .../en/interfaces/third-party/integrations.md | 1 - docs/en/introduction/adopters.md | 2 +- docs/en/introduction/distinctive-features.md | 16 +- docs/en/operations/access-rights.md | 10 +- .../settings.md | 8 +- .../operations/settings/query-complexity.md | 3 +- docs/en/operations/settings/settings.md | 45 +- .../system-tables/asynchronous_metric_log.md | 2 +- .../system-tables/asynchronous_metrics.md | 6 +- .../system-tables/detached_parts.md | 2 +- docs/en/operations/system-tables/events.md | 6 +- docs/en/operations/system-tables/index.md | 3 +- .../en/operations/system-tables/metric_log.md | 6 +- docs/en/operations/system-tables/metrics.md | 6 +- .../en/operations/system-tables/numbers_mt.md | 2 +- docs/en/operations/system-tables/query_log.md | 6 +- .../system-tables/query_thread_log.md | 4 +- .../aggregate-functions/reference/anyheavy.md | 1 + .../aggregate-functions/reference/anylast.md | 2 +- .../aggregate-functions/reference/count.md | 2 +- .../reference/grouparrayinsertat.md | 2 +- .../reference/groupuniqarray.md | 4 +- .../aggregate-functions/reference/index.md | 47 +- .../aggregate-functions/reference/quantile.md | 8 +- .../reference/quantiledeterministic.md | 8 +- .../reference/quantileexact.md | 6 +- .../reference/quantileexactweighted.md | 9 +- .../reference/quantiletdigest.md | 8 +- .../reference/quantiletdigestweighted.md | 8 +- .../reference/quantiletiming.md | 12 +- .../reference/quantiletimingweighted.md | 8 +- .../reference/stddevpop.md | 2 +- .../reference/stddevsamp.md | 2 +- .../reference/stochasticlinearregression.md | 2 +- .../reference/stochasticlogisticregression.md | 2 +- .../reference/timeseriesgroupratesum.md | 2 +- .../aggregate-functions/reference/uniq.md | 8 +- .../reference/uniqcombined.md | 12 +- .../reference/uniqcombined64.md | 2 +- .../reference/uniqexact.md | 8 +- .../reference/uniqhll12.md | 8 +- .../data-types/aggregatefunction.md | 2 +- .../sql-reference/data-types/domains/index.md | 2 +- .../nested-data-structures/nested.md | 2 +- .../external-dicts-dict-layout.md | 4 +- .../external-dicts-dict-sources.md | 2 +- .../external-dicts-dict.md | 2 +- .../external-dictionaries/external-dicts.md | 4 +- docs/en/sql-reference/distributed-ddl.md | 21 + .../functions/array-functions.md | 11 +- .../functions/date-time-functions.md | 8 +- docs/en/sql-reference/functions/geo.md | 10 +- .../functions/other-functions.md | 36 +- .../functions/string-functions.md | 2 +- .../functions/string-replace-functions.md | 2 +- .../functions/string-search-functions.md | 2 +- .../functions/type-conversion-functions.md | 2 +- docs/en/sql-reference/index.md | 2 +- docs/en/sql-reference/operators/in.md | 2 +- docs/en/sql-reference/statements/alter.md | 4 +- docs/en/sql-reference/statements/create.md | 500 ------------- .../statements/create/database.md | 29 + .../statements/create/dictionary.md | 32 + .../sql-reference/statements/create/index.md | 21 + .../sql-reference/statements/create/quota.md | 29 + .../sql-reference/statements/create/role.md | 47 ++ .../statements/create/row-policy.md | 42 ++ .../statements/create/settings-profile.md | 25 + .../sql-reference/statements/create/table.md | 210 ++++++ .../sql-reference/statements/create/user.md | 81 +++ .../sql-reference/statements/create/view.md | 59 ++ docs/en/sql-reference/statements/grant.md | 2 +- docs/en/sql-reference/statements/index.md | 24 +- .../sql-reference/statements/insert-into.md | 2 +- docs/en/sql-reference/statements/misc.md | 2 +- docs/en/sql-reference/statements/show.md | 10 +- docs/en/sql-reference/statements/system.md | 8 +- .../sql-reference/table-functions/cluster.md | 5 +- .../en/sql-reference/table-functions/index.md | 2 +- docs/en/whats-new/index.md | 6 +- docs/ja/sql-reference/statements/grant.md | 477 ++++++++++++- docs/ru/getting-started/tutorial.md | 665 ++++++++++++++++- docs/ru/sql-reference/statements/grant.md | 667 +++++++++--------- docs/tr/sql-reference/statements/grant.md | 477 ++++++++++++- 108 files changed, 2853 insertions(+), 1116 deletions(-) create mode 100644 docs/en/sql-reference/distributed-ddl.md delete mode 100644 docs/en/sql-reference/statements/create.md create mode 100644 docs/en/sql-reference/statements/create/database.md create mode 100644 docs/en/sql-reference/statements/create/dictionary.md create mode 100644 docs/en/sql-reference/statements/create/index.md create mode 100644 docs/en/sql-reference/statements/create/quota.md create mode 100644 docs/en/sql-reference/statements/create/role.md create mode 100644 docs/en/sql-reference/statements/create/row-policy.md create mode 100644 docs/en/sql-reference/statements/create/settings-profile.md create mode 100644 docs/en/sql-reference/statements/create/table.md create mode 100644 docs/en/sql-reference/statements/create/user.md create mode 100644 docs/en/sql-reference/statements/create/view.md mode change 120000 => 100644 docs/ja/sql-reference/statements/grant.md mode change 120000 => 100644 docs/ru/getting-started/tutorial.md mode change 120000 => 100644 docs/tr/sql-reference/statements/grant.md diff --git a/docs/en/commercial/index.md b/docs/en/commercial/index.md index a8358f48b97..0f69df62c7b 100644 --- a/docs/en/commercial/index.md +++ b/docs/en/commercial/index.md @@ -4,15 +4,14 @@ toc_priority: 70 toc_title: Introduction --- -# ClickHouse Commercial Services +# ClickHouse Commercial Services {#clickhouse-commercial-services} This section is a directory of commercial service providers specializing in ClickHouse. They are independent companies not necessarily affiliated with Yandex. Service categories: -- [Cloud](cloud.md) -- [Support](support.md) - +- [Cloud](../commercial/cloud.md) +- [Support](../commercial/support.md) !!! note "For service providers" - If you happen to represent one of them, feel free to open a pull request adding your company to the respective section (or even adding a new section if the service doesn't fit into existing categories). The easiest way to open a pull-request for documentation page is by using a “pencil” edit button in the top-right corner. If your service available in some local market, make sure to mention it in a localized documentation page as well (or at least point it out in a pull-request description). + If you happen to represent one of them, feel free to open a pull request adding your company to the respective section (or even adding a new section if the service doesn’t fit into existing categories). The easiest way to open a pull-request for documentation page is by using a “pencil” edit button in the top-right corner. If your service available in some local market, make sure to mention it in a localized documentation page as well (or at least point it out in a pull-request description). diff --git a/docs/en/development/architecture.md b/docs/en/development/architecture.md index c2f7a86fc9f..f4b74c19f4e 100644 --- a/docs/en/development/architecture.md +++ b/docs/en/development/architecture.md @@ -5,7 +5,7 @@ toc_title: Architecture Overview # Overview of ClickHouse Architecture {#overview-of-clickhouse-architecture} -ClickHouse is a true column-oriented DBMS. Data is stored by columns, and during the execution of arrays (vectors or chunks of columns). Whenever possible, operations are dispatched on arrays, rather than on individual values. It is called "vectorized query execution" and it helps lower the cost of actual data processing. +ClickHouse is a true column-oriented DBMS. Data is stored by columns, and during the execution of arrays (vectors or chunks of columns). Whenever possible, operations are dispatched on arrays, rather than on individual values. It is called “vectorized query execution” and it helps lower the cost of actual data processing. > This idea is nothing new. It dates back to the `APL` (A programming language, 1957) and its descendants: `A +` (APL dialect), `J` (1990), `K` (1993), and `Q` (programming language from Kx Systems, 2003). Array programming is used in scientific data processing. Neither is this idea something new in relational databases: for example, it is used in the `VectorWise` system (also known as Actian Vector Analytic Database by Actian Corporation). @@ -21,11 +21,11 @@ Various `IColumn` implementations (`ColumnUInt8`, `ColumnString`, and so on) are Nevertheless, it is possible to work with individual values as well. To represent an individual value, the `Field` is used. `Field` is just a discriminated union of `UInt64`, `Int64`, `Float64`, `String` and `Array`. `IColumn` has the `operator []` method to get the n-th value as a `Field`, and the `insert` method to append a `Field` to the end of a column. These methods are not very efficient, because they require dealing with temporary `Field` objects representing an individual value. There are more efficient methods, such as `insertFrom`, `insertRangeFrom`, and so on. -`Field` doesn't have enough information about a specific data type for a table. For example, `UInt8`, `UInt16`, `UInt32`, and `UInt64` are all represented as `UInt64` in a `Field`. +`Field` doesn’t have enough information about a specific data type for a table. For example, `UInt8`, `UInt16`, `UInt32`, and `UInt64` are all represented as `UInt64` in a `Field`. ## Leaky Abstractions {#leaky-abstractions} -`IColumn` has methods for common relational transformations of data, but they don’t meet all needs. For example, `ColumnUInt64` doesn't have a method to calculate the sum of two columns, and `ColumnString` doesn't have a method to run a substring search. These countless routines are implemented outside of `IColumn`. +`IColumn` has methods for common relational transformations of data, but they don’t meet all needs. For example, `ColumnUInt64` doesn’t have a method to calculate the sum of two columns, and `ColumnString` doesn’t have a method to run a substring search. These countless routines are implemented outside of `IColumn`. Various functions on columns can be implemented in a generic, non-efficient way using `IColumn` methods to extract `Field` values, or in a specialized way using knowledge of inner memory layout of data in a specific `IColumn` implementation. It is implemented by casting functions to a specific `IColumn` type and deal with internal representation directly. For example, `ColumnUInt64` has the `getData` method that returns a reference to an internal array, then a separate routine reads or fills that array directly. We have “leaky abstractions” to allow efficient specializations of various routines. @@ -35,7 +35,7 @@ Various functions on columns can be implemented in a generic, non-efficient way `IDataType` and `IColumn` are only loosely related to each other. Different data types can be represented in memory by the same `IColumn` implementations. For example, `DataTypeUInt32` and `DataTypeDateTime` are both represented by `ColumnUInt32` or `ColumnConstUInt32`. In addition, the same data type can be represented by different `IColumn` implementations. For example, `DataTypeUInt8` can be represented by `ColumnUInt8` or `ColumnConstUInt8`. -`IDataType` only stores metadata. For instance, `DataTypeUInt8` doesn't store anything at all (except virtual pointer `vptr`) and `DataTypeFixedString` stores just `N` (the size of fixed-size strings). +`IDataType` only stores metadata. For instance, `DataTypeUInt8` doesn’t store anything at all (except virtual pointer `vptr`) and `DataTypeFixedString` stores just `N` (the size of fixed-size strings). `IDataType` has helper methods for various data formats. Examples are methods to serialize a value with possible quoting, to serialize a value for JSON, and to serialize a value as part of the XML format. There is no direct correspondence to data formats. For example, the different data formats `Pretty` and `TabSeparated` can use the same `serializeTextEscaped` helper method from the `IDataType` interface. @@ -120,9 +120,9 @@ There are ordinary functions and aggregate functions. For aggregate functions, s Ordinary functions don’t change the number of rows – they work as if they are processing each row independently. In fact, functions are not called for individual rows, but for `Block`’s of data to implement vectorized query execution. -There are some miscellaneous functions, like [blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), and [runningAccumulate](../sql-reference/functions/other-functions.md#runningaccumulatexploit block processing and violate the independence of rows. +There are some miscellaneous functions, like [blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), and \[runningAccumulate\](../sql-reference/functions/other-functions.md\#runningaccumulatexploit block processing and violate the independence of rows. -ClickHouse has strong typing, so there’s no implicit type conversion. If a function doesn't support a specific combination of types, it throws an exception. But functions can work (be overloaded) for many different combinations of types. For example, the `plus` function (to implement the `+` operator) works for any combination of numeric types: `UInt8` + `Float32`, `UInt16` + `Int8`, and so on. Also, some variadic functions can accept any number of arguments, such as the `concat` function. +ClickHouse has strong typing, so there’s no implicit type conversion. If a function doesn’t support a specific combination of types, it throws an exception. But functions can work (be overloaded) for many different combinations of types. For example, the `plus` function (to implement the `+` operator) works for any combination of numeric types: `UInt8` + `Float32`, `UInt16` + `Int8`, and so on. Also, some variadic functions can accept any number of arguments, such as the `concat` function. Implementing a function may be slightly inconvenient because a function explicitly dispatches supported data types and supported `IColumns`. For example, the `plus` function has code generated by instantiation of a C++ template for each combination of numeric types, and constant or non-constant left and right arguments. @@ -169,13 +169,13 @@ There is no global query plan for distributed query execution. Each node has its `MergeTree` is a family of storage engines that supports indexing by primary key. The primary key can be an arbitrary tuple of columns or expressions. Data in a `MergeTree` table is stored in “parts”. Each part stores data in the primary key order, so data is ordered lexicographically by the primary key tuple. All the table columns are stored in separate `column.bin` files in these parts. The files consist of compressed blocks. Each block is usually from 64 KB to 1 MB of uncompressed data, depending on the average value size. The blocks consist of column values placed contiguously one after the other. Column values are in the same order for each column (the primary key defines the order), so when you iterate by many columns, you get values for the corresponding rows. -The primary key itself is “sparse”. It doesn't address every single row, but only some ranges of data. A separate `primary.idx` file has the value of the primary key for each N-th row, where N is called `index_granularity` (usually, N = 8192). Also, for each column, we have `column.mrk` files with “marks,” which are offsets to each N-th row in the data file. Each mark is a pair: the offset in the file to the beginning of the compressed block, and the offset in the decompressed block to the beginning of data. Usually, compressed blocks are aligned by marks, and the offset in the decompressed block is zero. Data for `primary.idx` always resides in memory, and data for `column.mrk` files is cached. +The primary key itself is “sparse”. It doesn’t address every single row, but only some ranges of data. A separate `primary.idx` file has the value of the primary key for each N-th row, where N is called `index_granularity` (usually, N = 8192). Also, for each column, we have `column.mrk` files with “marks,” which are offsets to each N-th row in the data file. Each mark is a pair: the offset in the file to the beginning of the compressed block, and the offset in the decompressed block to the beginning of data. Usually, compressed blocks are aligned by marks, and the offset in the decompressed block is zero. Data for `primary.idx` always resides in memory, and data for `column.mrk` files is cached. When we are going to read something from a part in `MergeTree`, we look at `primary.idx` data and locate ranges that could contain requested data, then look at `column.mrk` data and calculate offsets for where to start reading those ranges. Because of sparseness, excess data may be read. ClickHouse is not suitable for a high load of simple point queries, because the entire range with `index_granularity` rows must be read for each key, and the entire compressed block must be decompressed for each column. We made the index sparse because we must be able to maintain trillions of rows per single server without noticeable memory consumption for the index. Also, because the primary key is sparse, it is not unique: it cannot check the existence of the key in the table at INSERT time. You could have many rows with the same key in a table. When you `INSERT` a bunch of data into `MergeTree`, that bunch is sorted by primary key order and forms a new part. There are background threads that periodically select some parts and merge them into a single sorted part to keep the number of parts relatively low. That’s why it is called `MergeTree`. Of course, merging leads to “write amplification”. All parts are immutable: they are only created and deleted, but not modified. When SELECT is executed, it holds a snapshot of the table (a set of parts). After merging, we also keep old parts for some time to make a recovery after failure easier, so if we see that some merged part is probably broken, we can replace it with its source parts. -`MergeTree` is not an LSM tree because it doesn't contain “memtable” and “log”: inserted data is written directly to the filesystem. This makes it suitable only to INSERT data in batches, not by individual row and not very frequently – about once per second is ok, but a thousand times a second is not. We did it this way for simplicity’s sake, and because we are already inserting data in batches in our applications. +`MergeTree` is not an LSM tree because it doesn’t contain “memtable” and “log”: inserted data is written directly to the filesystem. This makes it suitable only to INSERT data in batches, not by individual row and not very frequently – about once per second is ok, but a thousand times a second is not. We did it this way for simplicity’s sake, and because we are already inserting data in batches in our applications. > MergeTree tables can only have one (primary) index: there aren’t any secondary indices. It would be nice to allow multiple physical representations under one logical table, for example, to store data in more than one physical order or even to allow representations with pre-aggregated data along with original data. @@ -187,7 +187,7 @@ Replication in ClickHouse can be configured on a per-table basis. You could have Replication is implemented in the `ReplicatedMergeTree` storage engine. The path in `ZooKeeper` is specified as a parameter for the storage engine. All tables with the same path in `ZooKeeper` become replicas of each other: they synchronize their data and maintain consistency. Replicas can be added and removed dynamically simply by creating or dropping a table. -Replication uses an asynchronous multi-master scheme. You can insert data into any replica that has a session with `ZooKeeper`, and data is replicated to all other replicas asynchronously. Because ClickHouse doesn't support UPDATEs, replication is conflict-free. As there is no quorum acknowledgment of inserts, just-inserted data might be lost if one node fails. +Replication uses an asynchronous multi-master scheme. You can insert data into any replica that has a session with `ZooKeeper`, and data is replicated to all other replicas asynchronously. Because ClickHouse doesn’t support UPDATEs, replication is conflict-free. As there is no quorum acknowledgment of inserts, just-inserted data might be lost if one node fails. Metadata for replication is stored in ZooKeeper. There is a replication log that lists what actions to do. Actions are: get part; merge parts; drop a partition, and so on. Each replica copies the replication log to its queue and then executes the actions from the queue. For example, on insertion, the “get the part” action is created in the log, and every replica downloads that part. Merges are coordinated between replicas to get byte-identical results. All parts are merged in the same way on all replicas. It is achieved by electing one replica as the leader, and that replica initiates merges and writes “merge parts” actions to the log. diff --git a/docs/en/engines/index.md b/docs/en/engines/index.md index 66d2fdcb074..6b2b6509e8b 100644 --- a/docs/en/engines/index.md +++ b/docs/en/engines/index.md @@ -5,11 +5,11 @@ toc_priority: 25 toc_title: hidden --- -# ClickHouse Engines +# ClickHouse Engines {#clickhouse-engines} There are two key engine kinds in ClickHouse: -- [Table engines](table-engines/index.md) -- [Database engines](database-engines/index.md) +- [Table engines](../engines/table-engines/index.md) +- [Database engines](../engines/database-engines/index.md) {## [Original article](https://clickhouse.tech/docs/en/engines/) ##} diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index cfe94c8d305..ea1fda3a098 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -3,14 +3,14 @@ toc_folder_title: Integrations toc_priority: 30 --- -# Table Engines for Integrations +# Table Engines for Integrations {#table-engines-for-integrations} ClickHouse provides various means for integrating with external systems, including table engines. Like with all other table engines, the configuration is done using `CREATE TABLE` or `ALTER TABLE` queries. Then from a user perspective, the configured integration looks like a normal table, but queries to it are proxied to the external system. This transparent querying is one of the key advantages of this approach over alternative integration methods, like external dictionaries or table functions, which require to use custom query methods on each use. List of supported integrations: -- [ODBC](odbc.md) -- [JDBC](jdbc.md) -- [MySQL](mysql.md) -- [HDFS](hdfs.md) -- [Kafka](kafka.md) +- [ODBC](../../../engines/table-engines/integrations/odbc.md) +- [JDBC](../../../engines/table-engines/integrations/jdbc.md) +- [MySQL](../../../engines/table-engines/integrations/mysql.md) +- [HDFS](../../../engines/table-engines/integrations/hdfs.md) +- [Kafka](../../../engines/table-engines/integrations/kafka.md) diff --git a/docs/en/engines/table-engines/integrations/mysql.md b/docs/en/engines/table-engines/integrations/mysql.md index 805cb4817a5..2cb1facce91 100644 --- a/docs/en/engines/table-engines/integrations/mysql.md +++ b/docs/en/engines/table-engines/integrations/mysql.md @@ -18,7 +18,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE = MySQL('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']); ``` -See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) query. +See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. The table structure can differ from the original MySQL table structure: diff --git a/docs/en/engines/table-engines/integrations/odbc.md b/docs/en/engines/table-engines/integrations/odbc.md index b2924df4831..fffc125b0ff 100644 --- a/docs/en/engines/table-engines/integrations/odbc.md +++ b/docs/en/engines/table-engines/integrations/odbc.md @@ -23,7 +23,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ENGINE = ODBC(connection_settings, external_database, external_table) ``` -See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) query. +See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. The table structure can differ from the source table structure: diff --git a/docs/en/engines/table-engines/log-family/stripelog.md b/docs/en/engines/table-engines/log-family/stripelog.md index a2b9562a9f0..693f5a21db2 100644 --- a/docs/en/engines/table-engines/log-family/stripelog.md +++ b/docs/en/engines/table-engines/log-family/stripelog.md @@ -20,7 +20,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE = StripeLog ``` -See the detailed description of the [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) query. +See the detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. ## Writing the Data {#table_engines-stripelog-writing-the-data} diff --git a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md index abc5b19b930..1a997b6b237 100644 --- a/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md @@ -32,7 +32,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of request parameters, see [request description](../../../sql-reference/statements/create.md). +For a description of request parameters, see [request description](../../../sql-reference/statements/create/table.md). **Query clauses** diff --git a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md index d35d249120a..4bfb9dc200e 100644 --- a/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md @@ -26,7 +26,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of query parameters, see [query description](../../../sql-reference/statements/create.md). +For a description of query parameters, see [query description](../../../sql-reference/statements/create/table.md). **CollapsingMergeTree Parameters** diff --git a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md index b9f341892ad..ebbe2b0b5b7 100644 --- a/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/graphitemergetree.md @@ -28,7 +28,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) query. +See a detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. A table for the Graphite data should have the following columns for the following data: diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 92903f3167e..a9285021afc 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -49,7 +49,7 @@ ORDER BY expr [SETTINGS name=value, ...] ``` -For a description of parameters, see the [CREATE query description](../../../sql-reference/statements/create.md). +For a description of parameters, see the [CREATE query description](../../../sql-reference/statements/create/table.md). ### Query Clauses {#mergetree-query-clauses} diff --git a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md index a218fa1876f..684e7e28112 100644 --- a/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md @@ -27,7 +27,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of request parameters, see [request description](../../../sql-reference/statements/create.md). +For a description of request parameters, see [statement description](../../../sql-reference/statements/create/table.md). **ReplacingMergeTree Parameters** diff --git a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md index e8e60b629b3..1f23e4daf51 100644 --- a/docs/en/engines/table-engines/mergetree-family/summingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/summingmergetree.md @@ -24,7 +24,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of request parameters, see [request description](../../../sql-reference/statements/create.md). +For a description of request parameters, see [request description](../../../sql-reference/statements/create/table.md). **Parameters of SummingMergeTree** diff --git a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md index 986a1fc05b6..a010a395c64 100644 --- a/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/versionedcollapsingmergetree.md @@ -29,7 +29,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [SETTINGS name=value, ...] ``` -For a description of query parameters, see the [query description](../../../sql-reference/statements/create.md). +For a description of query parameters, see the [query description](../../../sql-reference/statements/create/table.md). **Engine Parameters** diff --git a/docs/en/engines/table-engines/special/index.md b/docs/en/engines/table-engines/special/index.md index 2c796a109da..872c01385e0 100644 --- a/docs/en/engines/table-engines/special/index.md +++ b/docs/en/engines/table-engines/special/index.md @@ -3,7 +3,7 @@ toc_folder_title: Special toc_priority: 31 --- -# Special Table Engines +# Special Table Engines {#special-table-engines} There are three main categories of table engines: diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 0f88f0a56e7..b8c3a6c83f1 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -20,7 +20,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE = Join(join_strictness, join_type, k1[, k2, ...]) ``` -See the detailed description of the [CREATE TABLE](../../../sql-reference/statements/create.md#create-table-query) query. +See the detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. **Engine Parameters** diff --git a/docs/en/engines/table-engines/special/materializedview.md b/docs/en/engines/table-engines/special/materializedview.md index 53ebf9641af..d450c8814a7 100644 --- a/docs/en/engines/table-engines/special/materializedview.md +++ b/docs/en/engines/table-engines/special/materializedview.md @@ -5,6 +5,6 @@ toc_title: MaterializedView # MaterializedView Table Engine {#materializedview} -Used for implementing materialized views (for more information, see [CREATE TABLE](../../../sql-reference/statements/create.md)). For storing data, it uses a different engine that was specified when creating the view. When reading from a table, it just uses that engine. +Used for implementing materialized views (for more information, see [CREATE TABLE](../../../sql-reference/statements/create/table.md)). For storing data, it uses a different engine that was specified when creating the view. When reading from a table, it just uses that engine. [Original article](https://clickhouse.tech/docs/en/operations/table_engines/materializedview/) diff --git a/docs/en/faq/general/why-clickhouse-is-so-fast.md b/docs/en/faq/general/why-clickhouse-is-so-fast.md index ff6b3ac0ff7..1ccf2595768 100644 --- a/docs/en/faq/general/why-clickhouse-is-so-fast.md +++ b/docs/en/faq/general/why-clickhouse-is-so-fast.md @@ -17,7 +17,7 @@ Indexes : ClickHouse keeps data structures in memory that allows reading not only used columns but only necessary row ranges of those columns. Data compression -: Storing different values of the same column together often leads to better compression ratios (compared to row-oriented systems) because in real data column often has the same or not so many different values for neighboring rows. In addition to general-purpose compression, ClickHouse supports [specialized codecs](../../sql-reference/statements/create.md#create-query-specialized-codecs) that can make data even more compact. +: Storing different values of the same column together often leads to better compression ratios (compared to row-oriented systems) because in real data column often has the same or not so many different values for neighboring rows. In addition to general-purpose compression, ClickHouse supports [specialized codecs](../../sql-reference/statements/create/table.md#create-query-specialized-codecs) that can make data even more compact. Vectorized query execution : ClickHouse not only stores data in columns but also processes data in columns. It leads to better CPU cache utilization and allows for [SIMD](https://en.wikipedia.org/wiki/SIMD) CPU instructions usage. diff --git a/docs/en/faq/use-cases/time-series.md b/docs/en/faq/use-cases/time-series.md index 6b916d51d1f..4fc53c0bea4 100644 --- a/docs/en/faq/use-cases/time-series.md +++ b/docs/en/faq/use-cases/time-series.md @@ -8,8 +8,8 @@ toc_priority: 101 ClickHouse is a generic data storage solution for [OLAP](../../faq/general/olap.md) workloads, while there are many specialized time-series database management systems. Nevertheless, ClickHouse’s [focus on query execution speed](../../faq/general/why-clickhouse-is-so-fast.md) allows it to outperform specialized systems in many cases. There are many independent benchmarks on this topic out there ([example](https://medium.com/@AltinityDB/clickhouse-for-time-series-scalability-benchmarks-e181132a895b)), so we’re not going to conduct one here. Instead, let’s focus on ClickHouse features that are important to use if that’s your use case. -First of all, there are **[specialized codecs](../../sql-reference/statements/create.md#create-query-specialized-codecs)** which make typical time-series. Either common algorithms like `DoubleDelta` and `Gorilla` or specific to ClickHouse like `T64`. +First of all, there are **[specialized codecs](../../sql-reference/statements/create/table.md#create-query-specialized-codecs)** which make typical time-series. Either common algorithms like `DoubleDelta` and `Gorilla` or specific to ClickHouse like `T64`. Second, time-series queries often hit only recent data, like one day or one week old. It makes sense to use servers that have both fast nVME/SSD drives and high-capacity HDD drives. ClickHouse [TTL](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) feature allows to configure keeping fresh hot data on fast drives and gradually move it to slower drives as it ages. Rollup or removal of even older data is also possible if your requirements demand it. -Even though it’s against ClickHouse philosophy of storing and processing raw data, you can use [materialized views](../../sql-reference/statements/create.md#create-view) to fit into even tighter latency or costs requirements. +Even though it’s against ClickHouse philosophy of storing and processing raw data, you can use [materialized views](../../sql-reference/statements/create/view.md) to fit into even tighter latency or costs requirements. diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 251d52373bf..c85284a946a 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -98,9 +98,9 @@ To run ClickHouse inside Docker follow the guide on [Docker Hub](https://hub.doc For non-Linux operating systems and for AArch64 CPU arhitecture, ClickHouse builds are provided as a cross-compiled binary from the latest commit of the `master` branch (with a few hours delay). -- [macOS](https://builds.clickhouse.tech/master/macos/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/macos/clickhouse' && chmod a+x ./clickhouse` -- [FreeBSD](https://builds.clickhouse.tech/master/freebsd/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/freebsd/clickhouse' && chmod a+x ./clickhouse` -- [AArch64](https://builds.clickhouse.tech/master/aarch64/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/aarch64/clickhouse' && chmod a+x ./clickhouse` +- [macOS](https://builds.clickhouse.tech/master/macos/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/macos/clickhouse' && chmod a+x ./clickhouse` +- [FreeBSD](https://builds.clickhouse.tech/master/freebsd/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/freebsd/clickhouse' && chmod a+x ./clickhouse` +- [AArch64](https://builds.clickhouse.tech/master/aarch64/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/aarch64/clickhouse' && chmod a+x ./clickhouse` After downloading, you can use the `clickhouse client` to connect to the server, or `clickhouse local` to process local data. To run `clickhouse server`, you have to additionally download [server](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.xml) and [users](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/users.xml) configuration files from GitHub. diff --git a/docs/en/getting-started/tutorial.md b/docs/en/getting-started/tutorial.md index bc71d762138..0fc267e497c 100644 --- a/docs/en/getting-started/tutorial.md +++ b/docs/en/getting-started/tutorial.md @@ -99,7 +99,7 @@ As in most databases management systems, ClickHouse logically groups tables into clickhouse-client --query "CREATE DATABASE IF NOT EXISTS tutorial" ``` -Syntax for creating tables is way more complicated compared to databases (see [reference](../sql-reference/statements/create.md). In general `CREATE TABLE` statement has to specify three key things: +Syntax for creating tables is way more complicated compared to databases (see [reference](../sql-reference/statements/create/table.md). In general `CREATE TABLE` statement has to specify three key things: 1. Name of table to create. 2. Table schema, i.e. list of columns and their [data types](../sql-reference/data-types/index.md). diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index ae96cb6dda4..81e71c668c3 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -979,7 +979,7 @@ message MessageType { } ``` -are not applied; the [table defaults](../sql-reference/statements/create.md#create-default-values) are used instead of them. +are not applied; the [table defaults](../sql-reference/statements/create/table.md#create-default-values) are used instead of them. ClickHouse inputs and outputs protobuf messages in the `length-delimited` format. It means before every message should be written its length as a [varint](https://developers.google.com/protocol-buffers/docs/encoding#varints). diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 69699be52ec..ab1cb4b4d7d 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -367,34 +367,34 @@ $ curl -v 'http://localhost:8123/predefined_query' As you can see from the example if `http_handlers` is configured in the config.xml file and `http_handlers` can contain many `rules`. ClickHouse will match the HTTP requests received to the predefined type in `rule` and the first matched runs the handler. Then ClickHouse will execute the corresponding predefined query if the match is successful. Now `rule` can configure `method`, `headers`, `url`, `handler`: - - `method` is responsible for matching the method part of the HTTP request. `method` fully conforms to the definition of [method](https://developer.mozilla.org/en-US/docs/Web/HTTP/Methods) in the HTTP protocol. It is an optional configuration. If it is not defined in the configuration file, it does not match the method portion of the HTTP request. +- `method` is responsible for matching the method part of the HTTP request. `method` fully conforms to the definition of [method](https://developer.mozilla.org/en-US/docs/Web/HTTP/Methods) in the HTTP protocol. It is an optional configuration. If it is not defined in the configuration file, it does not match the method portion of the HTTP request. - - `url` is responsible for matching the URL part of the HTTP request. It is compatible with [RE2](https://github.com/google/re2)’s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the URL portion of the HTTP request. +- `url` is responsible for matching the URL part of the HTTP request. It is compatible with [RE2](https://github.com/google/re2)’s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the URL portion of the HTTP request. - - `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2’s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request. +- `headers` are responsible for matching the header part of the HTTP request. It is compatible with RE2’s regular expressions. It is an optional configuration. If it is not defined in the configuration file, it does not match the header portion of the HTTP request. + +- `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`. + `type` currently supports three types: [predefined\_query\_handler](#predefined_query_handler), [dynamic\_query\_handler](#dynamic_query_handler), [static](#static). - - `handler` contains the main processing part. Now `handler` can configure `type`, `status`, `content_type`, `response_content`, `query`, `query_param_name`. - `type` currently supports three types: [predefined_query_handler](#predefined_query_handler), [dynamic_query_handler](#dynamic_query_handler), [static](#static). - - `query` — use with `predefined_query_handler` type, executes query when the handler is called. - + - `query_param_name` — use with `dynamic_query_handler` type, extracts and executes the value corresponding to the `query_param_name` value in HTTP request params. - + - `status` — use with `static` type, response status code. - + - `content_type` — use with `static` type, response [content-type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type). - `response_content` — use with `static` type, response content sent to client, when using the prefix ‘file://’ or ‘config://’, find the content from the file or configuration sends to client. Next are the configuration methods for different `type`. -### predefined_query_handler {#predefined_query_handler} +### predefined\_query\_handler {#predefined_query_handler} `predefined_query_handler` supports setting `Settings` and `query_params` values. You can configure `query` in the type of `predefined_query_handler`. `query` value is a predefined query of `predefined_query_handler`, which is executed by ClickHouse when an HTTP request is matched and the result of the query is returned. It is a must configuration. -The following example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` settings, then queries the system table to check whether these settings were set successfully. +The following example defines the values of [max\_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` settings, then queries the system table to check whether these settings were set successfully. Example: @@ -425,13 +425,13 @@ max_alter_threads 2 !!! note "caution" In one `predefined_query_handler` only supports one `query` of an insert type. -### dynamic_query_handler {#dynamic_query_handler} +### dynamic\_query\_handler {#dynamic_query_handler} In `dynamic_query_handler`, the query is written in the form of param of the HTTP request. The difference is that in `predefined_query_handler`, the query is written in the configuration file. You can configure `query_param_name` in `dynamic_query_handler`. ClickHouse extracts and executes the value corresponding to the `query_param_name` value in the URL of the HTTP request. The default value of `query_param_name` is `/query` . It is an optional configuration. If there is no definition in the configuration file, the param is not passed in. -To experiment with this functionality, the example defines the values of [max_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` and `queries` whether the settings were set successfully. +To experiment with this functionality, the example defines the values of [max\_threads](../operations/settings/settings.md#settings-max_threads) and `max_alter_threads` and `queries` whether the settings were set successfully. Example: @@ -456,7 +456,7 @@ max_alter_threads 2 ### static {#static} -`static` can return [content_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [status](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) and `response_content`. `response_content` can return the specified content. +`static` can return [content\_type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type), [status](https://developer.mozilla.org/en-US/docs/Web/HTTP/Status) and `response_content`. `response_content` can return the specified content. Example: diff --git a/docs/en/interfaces/third-party/index.md b/docs/en/interfaces/third-party/index.md index 9d7d9106e63..caf100681b4 100644 --- a/docs/en/interfaces/third-party/index.md +++ b/docs/en/interfaces/third-party/index.md @@ -3,15 +3,14 @@ toc_folder_title: Third-Party toc_priority: 24 --- -# Third-Party Interfaces +# Third-Party Interfaces {#third-party-interfaces} This is a collection of links to third-party tools that provide some sort of interface to ClickHouse. It can be either visual interface, command-line interface or an API: -- [Client libraries](client-libraries.md) -- [Integrations](integrations.md) -- [GUI](gui.md) -- [Proxies](proxy.md) - +- [Client libraries](../../interfaces/third-party/client-libraries.md) +- [Integrations](../../interfaces/third-party/integrations.md) +- [GUI](../../interfaces/third-party/gui.md) +- [Proxies](../../interfaces/third-party/proxy.md) !!! note "Note" Generic tools that support common API like [ODBC](../../interfaces/odbc.md) or [JDBC](../../interfaces/jdbc.md) usually can work with ClickHouse as well, but are not listed here because there are way too many of them. diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 4ec748ee7ee..e752725d727 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -104,6 +104,5 @@ toc_title: Integrations - [ActiveRecord](https://github.com/PNixx/clickhouse-activerecord) - [GraphQL](https://github.com/graphql) - [activecube-graphql](https://github.com/bitquery/activecube-graphql) - [Original article](https://clickhouse.tech/docs/en/interfaces/third-party/integrations/) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 01669e012d6..7487b212955 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -50,7 +50,7 @@ toc_title: Adopters | Pragma Innovation | Telemetry and Big Data Analysis | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/4_pragma_innovation.pdf) | | QINGCLOUD | Cloud services | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/4.%20Cloud%20%2B%20TSDB%20for%20ClickHouse%20张健%20QingCloud.pdf) | | Qrator | DDoS protection | Main product | — | — | [Blog Post, March 2019](https://blog.qrator.net/en/clickhouse-ddos-mitigation_37/) | -| Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | +| Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | | Plausible | Analytics | Main Product | — | — | [Blog post, June 2020](https://twitter.com/PlausibleHQ/status/1273889629087969280) | | Rambler | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) | | Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | diff --git a/docs/en/introduction/distinctive-features.md b/docs/en/introduction/distinctive-features.md index 686d73f1c06..4a8f31b8d56 100644 --- a/docs/en/introduction/distinctive-features.md +++ b/docs/en/introduction/distinctive-features.md @@ -17,11 +17,11 @@ It’s also worth noting that ClickHouse is a database management system, not a Some column-oriented DBMSs do not use data compression. However, data compression does play a key role in achieving excellent performance. -In addition to efficient general-purpose compression codecs with different trade-offs between disk space and CPU consumption, ClickHouse provides [specialized codecs](../sql-reference/statements/create.md#create-query-specialized-codecs) for specific kinds of data, which allow ClickHouse to compete with and outperform more niche databases, like time-series ones. +In addition to efficient general-purpose compression codecs with different trade-offs between disk space and CPU consumption, ClickHouse provides [specialized codecs](../sql-reference/statements/create/table.md#create-query-specialized-codecs) for specific kinds of data, which allow ClickHouse to compete with and outperform more niche databases, like time-series ones. ## Disk Storage of Data {#disk-storage-of-data} -Keeping data physically sorted by primary key makes it possible to extract data for its specific values or value ranges with low latency, less than a few dozen milliseconds. Some column-oriented DBMSs (such as SAP HANA and Google PowerDrill) can only work in RAM. This approach encourages the allocation of a larger hardware budget than is necessary for real-time analysis. +Keeping data physically sorted by primary key makes it possible to extract data for its specific values or value ranges with low latency, less than a few dozen milliseconds. Some column-oriented DBMSs (such as SAP HANA and Google PowerDrill) can only work in RAM. This approach encourages the allocation of a larger hardware budget than is necessary for real-time analysis. ClickHouse is designed to work on regular hard drives, which means the cost per GB of data storage is low, but SSD and additional RAM are also fully used if available. @@ -57,11 +57,11 @@ Having a data physically sorted by primary key makes it possible to extract data ## Secondary Indexes {#secondary-indexes} -Unlike other database management systems, secondary indexes in ClickHouse does not point to specific rows or row ranges. Instead, they allow the database to know in advance that all rows in some data parts wouldn't match the query filtering conditions and do not read them at all, thus they are called [data skipping indexes](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-data_skipping-indexes). +Unlike other database management systems, secondary indexes in ClickHouse does not point to specific rows or row ranges. Instead, they allow the database to know in advance that all rows in some data parts wouldn’t match the query filtering conditions and do not read them at all, thus they are called [data skipping indexes](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-data_skipping-indexes). ## Suitable for Online Queries {#suitable-for-online-queries} -Most OLAP database management systems don't aim for online queries with sub-second latencies. In alternative systems, report building time of tens of seconds or even minutes is often considered acceptable. Sometimes it takes even more which forces to prepare reports offline (in advance or by responding with "come back later"). +Most OLAP database management systems don’t aim for online queries with sub-second latencies. In alternative systems, report building time of tens of seconds or even minutes is often considered acceptable. Sometimes it takes even more which forces to prepare reports offline (in advance or by responding with “come back later”). In ClickHouse low latency means that queries can be processed without delay and without trying to prepare an answer in advance, right at the same moment while the user interface page is loading. In other words, online. @@ -73,9 +73,9 @@ ClickHouse provides various ways to trade accuracy for performance: 2. Running a query based on a part (sample) of data and getting an approximated result. In this case, proportionally less data is retrieved from the disk. 3. Running an aggregation for a limited number of random keys, instead of for all keys. Under certain conditions for key distribution in the data, this provides a reasonably accurate result while using fewer resources. -## Adaptive Join Algorithm +## Adaptive Join Algorithm {#adaptive-join-algorithm} -ClickHouse adaptively chooses how to [JOIN](../sql-reference/statements/select/join.md) multiple tables, by preferring hash-join algorithm and falling back to the merge-join algorithm if there's more than one large table. +ClickHouse adaptively chooses how to [JOIN](../sql-reference/statements/select/join.md) multiple tables, by preferring hash-join algorithm and falling back to the merge-join algorithm if there’s more than one large table. ## Data Replication and Data Integrity Support {#data-replication-and-data-integrity-support} @@ -83,9 +83,9 @@ ClickHouse uses asynchronous multi-master replication. After being written to an For more information, see the section [Data replication](../engines/table-engines/mergetree-family/replication.md). -## Role-Based Access Control +## Role-Based Access Control {#role-based-access-control} -ClickHouse implements user account management using SQL queries and allows for [role-based access control configuration](../operations/access-rights.md) similar to what can be found in ANSI SQL standard and popular relational database management systems. +ClickHouse implements user account management using SQL queries and allows for [role-based access control configuration](../operations/access-rights.md) similar to what can be found in ANSI SQL standard and popular relational database management systems. ## Features that Can Be Considered Disadvantages {#clickhouse-features-that-can-be-considered-disadvantages} diff --git a/docs/en/operations/access-rights.md b/docs/en/operations/access-rights.md index f7e1b98d6e3..a060a1c613e 100644 --- a/docs/en/operations/access-rights.md +++ b/docs/en/operations/access-rights.md @@ -58,7 +58,7 @@ Privileges can be granted to a user account by the [GRANT](../sql-reference/stat Management queries: -- [CREATE USER](../sql-reference/statements/create.md#create-user-statement) +- [CREATE USER](../sql-reference/statements/create/user.md) - [ALTER USER](../sql-reference/statements/alter.md#alter-user-statement) - [DROP USER](../sql-reference/statements/misc.md#drop-user-statement) - [SHOW CREATE USER](../sql-reference/statements/show.md#show-create-user-statement) @@ -84,7 +84,7 @@ Role contains: Management queries: -- [CREATE ROLE](../sql-reference/statements/create.md#create-role-statement) +- [CREATE ROLE](../sql-reference/statements/create/role.md) - [ALTER ROLE](../sql-reference/statements/alter.md#alter-role-statement) - [DROP ROLE](../sql-reference/statements/misc.md#drop-role-statement) - [SET ROLE](../sql-reference/statements/misc.md#set-role-statement) @@ -99,7 +99,7 @@ Row policy is a filter that defines which of the rows are available to a user or Management queries: -- [CREATE ROW POLICY](../sql-reference/statements/create.md#create-row-policy-statement) +- [CREATE ROW POLICY](../sql-reference/statements/create/row-policy.md) - [ALTER ROW POLICY](../sql-reference/statements/alter.md#alter-row-policy-statement) - [DROP ROW POLICY](../sql-reference/statements/misc.md#drop-row-policy-statement) - [SHOW CREATE ROW POLICY](../sql-reference/statements/show.md#show-create-row-policy-statement) @@ -110,7 +110,7 @@ Settings profile is a collection of [settings](../operations/settings/index.md). Management queries: -- [CREATE SETTINGS PROFILE](../sql-reference/statements/create.md#create-settings-profile-statement) +- [CREATE SETTINGS PROFILE](../sql-reference/statements/create/settings-profile.md#create-settings-profile-statement) - [ALTER SETTINGS PROFILE](../sql-reference/statements/alter.md#alter-settings-profile-statement) - [DROP SETTINGS PROFILE](../sql-reference/statements/misc.md#drop-settings-profile-statement) - [SHOW CREATE SETTINGS PROFILE](../sql-reference/statements/show.md#show-create-settings-profile-statement) @@ -123,7 +123,7 @@ Quota contains a set of limits for some durations, as well as a list of roles an Management queries: -- [CREATE QUOTA](../sql-reference/statements/create.md#create-quota-statement) +- [CREATE QUOTA](../sql-reference/statements/create/quota.md) - [ALTER QUOTA](../sql-reference/statements/alter.md#alter-quota-statement) - [DROP QUOTA](../sql-reference/statements/misc.md#drop-quota-statement) - [SHOW CREATE QUOTA](../sql-reference/statements/show.md#show-create-quota-statement) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index f1ffc011776..eb087bad698 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -348,7 +348,7 @@ Keys: Default value: `LOG_USER` if `address` is specified, `LOG_DAEMON otherwise.` - format – Message format. Possible values: `bsd` and `syslog.` -## send_crash_reports {#server_configuration_parameters-logger} +## send\_crash\_reports {#server_configuration_parameters-logger} Settings for opt-in sending crash reports to the ClickHouse core developers team via [Sentry](https://sentry.io). Enabling it, especially in pre-production environments, is greatly appreciated. @@ -398,8 +398,7 @@ The cache is shared for the server and memory is allocated as needed. The cache 5368709120 ``` - -## max_server_memory_usage {#max_server_memory_usage} +## max\_server\_memory\_usage {#max_server_memory_usage} Limits total RAM usage by the ClickHouse server. You can specify it only for the default profile. @@ -416,8 +415,7 @@ On hosts with low RAM and swap, you possibly need setting `max_server_memory_usa **See also** -- [max_memory_usage](../settings/query-complexity.md#settings_max_memory_usage) - +- [max\_memory\_usage](../../operations/settings/query-complexity.md#settings_max_memory_usage) ## max\_concurrent\_queries {#max-concurrent-queries} diff --git a/docs/en/operations/settings/query-complexity.md b/docs/en/operations/settings/query-complexity.md index 42ac9573b91..0486392d259 100644 --- a/docs/en/operations/settings/query-complexity.md +++ b/docs/en/operations/settings/query-complexity.md @@ -36,7 +36,7 @@ Memory usage is not monitored for the states of certain aggregate functions. Memory usage is not fully tracked for states of the aggregate functions `min`, `max`, `any`, `anyLast`, `argMin`, `argMax` from `String` and `Array` arguments. -Memory consumption is also restricted by the parameters `max_memory_usage_for_user` and [max_server_memory_usage](../server-configuration-parameters/settings.md#max_server_memory_usage). +Memory consumption is also restricted by the parameters `max_memory_usage_for_user` and [max\_server\_memory\_usage](../../operations/server-configuration-parameters/settings.md#max_server_memory_usage). ## max\_memory\_usage\_for\_user {#max-memory-usage-for-user} @@ -46,7 +46,6 @@ Default values are defined in [Settings.h](https://github.com/ClickHouse/ClickHo See also the description of [max\_memory\_usage](#settings_max_memory_usage). - ## max\_rows\_to\_read {#max-rows-to-read} The following restrictions can be checked on each block (instead of on each row). That is, the restrictions can be broken a little. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7d1f9a72a21..0c879f164d2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -129,7 +129,7 @@ Default value: 0. ## max\_http\_get\_redirects {#setting-max_http_get_redirects} -Limits the maximum number of HTTP GET redirect hops for [URL](../../engines/table-engines/special/url.md)-engine tables. The setting applies to both types of tables: those created by the [CREATE TABLE](../../sql-reference/statements/create.md#create-table-query) query and by the [url](../../sql-reference/table-functions/url.md) table function. +Limits the maximum number of HTTP GET redirect hops for [URL](../../engines/table-engines/special/url.md)-engine tables. The setting applies to both types of tables: those created by the [CREATE TABLE](../../sql-reference/statements/create/table.md) query and by the [url](../../sql-reference/table-functions/url.md) table function. Possible values: @@ -733,8 +733,8 @@ Limits maximum recursion depth in the recursive descent parser. Allows to contro Possible values: -- Positive integer. -- 0 — Recursion depth is unlimited. +- Positive integer. +- 0 — Recursion depth is unlimited. Default value: 1000. @@ -1427,20 +1427,20 @@ Possible values: Default value: 16. -## always_fetch_merged_part {#always_fetch_merged_part} +## always\_fetch\_merged\_part {#always_fetch_merged_part} -Prohibits data parts merging in [Replicated*MergeTree](../../engines/table-engines/mergetree-family/replication.md)-engine tables. +Prohibits data parts merging in [Replicated\*MergeTree](../../engines/table-engines/mergetree-family/replication.md)-engine tables. When merging is prohibited, the replica never merges parts and always downloads merged parts from other replicas. If there is no required data yet, the replica waits for it. CPU and disk load on the replica server decreases, but the network load on cluster increases. This setting can be useful on servers with relatively weak CPUs or slow disks, such as servers for backups storage. Possible values: - 0 — `Replicated*MergeTree`-engine tables merge data parts at the replica. -- 1 — `Replicated*MergeTree`-engine tables don't merge data parts at the replica. The tables download merged data parts from other replicas. +- 1 — `Replicated*MergeTree`-engine tables don’t merge data parts at the replica. The tables download merged data parts from other replicas. Default value: 0. -**See Also** +**See Also** - [Data Replication](../../engines/table-engines/mergetree-family/replication.md) @@ -1454,11 +1454,11 @@ Possible values: Default value: 16. -## transform_null_in {#transform_null_in} +## transform\_null\_in {#transform_null_in} Enables equality of [NULL](../../sql-reference/syntax.md#null-literal) values for [IN](../../sql-reference/operators/in.md) operator. -By default, `NULL` values can't be compared because `NULL` means undefined value. Thus, comparison `expr = NULL` must always return `false`. With this setting `NULL = NULL` returns `true` for `IN` operator. +By default, `NULL` values can’t be compared because `NULL` means undefined value. Thus, comparison `expr = NULL` must always return `false`. With this setting `NULL = NULL` returns `true` for `IN` operator. Possible values: @@ -1467,11 +1467,11 @@ Possible values: Default value: 0. -**Example** +**Example** Consider the `null_in` table: -```text +``` text ┌──idx─┬─────i─┐ │ 1 │ 1 │ │ 2 │ NULL │ @@ -1481,13 +1481,13 @@ Consider the `null_in` table: Query: -```sql +``` sql SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 0; ``` Result: -```text +``` text ┌──idx─┬────i─┐ │ 1 │ 1 │ └──────┴──────┘ @@ -1495,24 +1495,23 @@ Result: Query: -```sql +``` sql SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; ``` Result: -```text +``` text ┌──idx─┬─────i─┐ │ 1 │ 1 │ │ 2 │ NULL │ └──────┴───────┘ ``` -**See Also** +**See Also** - [NULL Processing in IN Operators](../../sql-reference/operators/in.md#in-null-processing) - ## low\_cardinality\_max\_dictionary\_size {#low_cardinality_max_dictionary_size} Sets a maximum size in rows of a shared global dictionary for the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type that can be written to a storage file system. This setting prevents issues with RAM in case of unlimited dictionary growth. All the data that can’t be encoded due to maximum dictionary size limitation ClickHouse writes in an ordinary method. @@ -1570,9 +1569,9 @@ Possible values: Default value: 0. -## min_insert_block_size_rows_for_materialized_views {#min-insert-block-size-rows-for-materialized-views} +## min\_insert\_block\_size\_rows\_for\_materialized\_views {#min-insert-block-size-rows-for-materialized-views} -Sets minimum number of rows in block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create.md#create-view). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. +Sets minimum number of rows in block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. Possible values: @@ -1583,11 +1582,11 @@ Default value: 1048576. **See Also** -- [min_insert_block_size_rows](#min-insert-block-size-rows) +- [min\_insert\_block\_size\_rows](#min-insert-block-size-rows) -## min_insert_block_size_bytes_for_materialized_views {#min-insert-block-size-bytes-for-materialized-views} +## min\_insert\_block\_size\_bytes\_for\_materialized\_views {#min-insert-block-size-bytes-for-materialized-views} -Sets minimum number of bytes in block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create.md#create-view). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. +Sets minimum number of bytes in block which can be inserted into a table by an `INSERT` query. Smaller-sized blocks are squashed into bigger ones. This setting is applied only for blocks inserted into [materialized view](../../sql-reference/statements/create/view.md). By adjusting this setting, you control blocks squashing while pushing to materialized view and avoid excessive memory usage. Possible values: @@ -1598,6 +1597,6 @@ Default value: 268435456. **See also** -- [min_insert_block_size_bytes](#min-insert-block-size-bytes) +- [min\_insert\_block\_size\_bytes](#min-insert-block-size-bytes) [Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) diff --git a/docs/en/operations/system-tables/asynchronous_metric_log.md b/docs/en/operations/system-tables/asynchronous_metric_log.md index afc71af1114..875217fdc49 100644 --- a/docs/en/operations/system-tables/asynchronous_metric_log.md +++ b/docs/en/operations/system-tables/asynchronous_metric_log.md @@ -1,3 +1,3 @@ ## system.asynchronous\_metric\_log {#system-tables-async-log} -Contains the historical values for `system.asynchronous_log` (see [system.asynchronous\_metrics](asynchronous_metrics.md#system_tables-asynchronous_metrics)) +Contains the historical values for `system.asynchronous_log` (see [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics)) diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index 438dfb62fb8..96c7e3c5319 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -31,6 +31,6 @@ SELECT * FROM system.asynchronous_metrics LIMIT 10 **See Also** - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. -- [system.metrics](metrics.md#system_tables-metrics) — Contains instantly calculated metrics. -- [system.events](events.md#system_tables-events) — Contains a number of events that have occurred. -- [system.metric\_log](metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. +- [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that have occurred. +- [system.metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. diff --git a/docs/en/operations/system-tables/detached_parts.md b/docs/en/operations/system-tables/detached_parts.md index 4c4c1f85413..94fd280d55a 100644 --- a/docs/en/operations/system-tables/detached_parts.md +++ b/docs/en/operations/system-tables/detached_parts.md @@ -4,6 +4,6 @@ Contains information about detached parts of [MergeTree](../../engines/table-eng For user-detached parts, the reason is empty. Such parts can be attached with [ALTER TABLE ATTACH PARTITION\|PART](../../sql-reference/statements/alter.md#alter_attach-partition) command. -For the description of other columns, see [system.parts](parts.md#system_tables-parts). +For the description of other columns, see [system.parts](../../operations/system-tables/parts.md#system_tables-parts). If part name is invalid, values of some columns may be `NULL`. Such parts can be deleted with [ALTER TABLE DROP DETACHED PART](../../sql-reference/statements/alter.md#alter_drop-detached). diff --git a/docs/en/operations/system-tables/events.md b/docs/en/operations/system-tables/events.md index 0a073a1ab1a..2cbc14f96f9 100644 --- a/docs/en/operations/system-tables/events.md +++ b/docs/en/operations/system-tables/events.md @@ -26,7 +26,7 @@ SELECT * FROM system.events LIMIT 5 **See Also** -- [system.asynchronous\_metrics](asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [system.metrics](metrics.md#system_tables-metrics) — Contains instantly calculated metrics. -- [system.metric\_log](metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. +- [system.metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index 395c98ee8b4..2ec3cc79763 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -20,7 +20,7 @@ System tables: Most of system tables store their data in RAM. A ClickHouse server creates such system tables at the start. -Unlike other system tables, the system tables [metric\_log](metric_log.md#system_tables-metric_log), [query\_log](query_log.md#system_tables-query_log), [query\_thread\_log](query_thread_log.md#system_tables-query_thread_log), [trace\_log](trace_log.md#system_tables-trace_log) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. +Unlike other system tables, the system tables [metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log), [query\_log](../../operations/system-tables/query_log.md#system_tables-query_log), [query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log), [trace\_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one. By default, table growth is unlimited. To control a size of a table, you can use [TTL](../../sql-reference/statements/alter.md#manipulations-with-table-ttl) settings for removing outdated log records. Also you can use the partitioning feature of `MergeTree`-engine tables. @@ -45,5 +45,4 @@ If procfs is supported and enabled on the system, ClickHouse server collects the - `OSReadBytes` - `OSWriteBytes` - [Original article](https://clickhouse.tech/docs/en/operations/system-tables/) diff --git a/docs/en/operations/system-tables/metric_log.md b/docs/en/operations/system-tables/metric_log.md index 028b5fe5065..ff15940d252 100644 --- a/docs/en/operations/system-tables/metric_log.md +++ b/docs/en/operations/system-tables/metric_log.md @@ -49,7 +49,7 @@ CurrentMetric_ReplicatedChecks: 0 **See also** -- [system.asynchronous\_metrics](asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [system.events](events.md#system_tables-events) — Contains a number of events that occurred. -- [system.metrics](metrics.md#system_tables-metrics) — Contains instantly calculated metrics. +- [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. +- [system.metrics](../../operations/system-tables/metrics.md#system_tables-metrics) — Contains instantly calculated metrics. - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/en/operations/system-tables/metrics.md b/docs/en/operations/system-tables/metrics.md index 2dc54167fb1..930cfa232e5 100644 --- a/docs/en/operations/system-tables/metrics.md +++ b/docs/en/operations/system-tables/metrics.md @@ -33,7 +33,7 @@ SELECT * FROM system.metrics LIMIT 10 **See Also** -- [system.asynchronous\_metrics](asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. -- [system.events](events.md#system_tables-events) — Contains a number of events that occurred. -- [system.metric\_log](metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. +- [system.asynchronous\_metrics](../../operations/system-tables/asynchronous_metrics.md#system_tables-asynchronous_metrics) — Contains periodically calculated metrics. +- [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred. +- [system.metric\_log](../../operations/system-tables/metric_log.md#system_tables-metric_log) — Contains a history of metrics values from tables `system.metrics` и `system.events`. - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/docs/en/operations/system-tables/numbers_mt.md b/docs/en/operations/system-tables/numbers_mt.md index c23bc810cfc..a09ee745341 100644 --- a/docs/en/operations/system-tables/numbers_mt.md +++ b/docs/en/operations/system-tables/numbers_mt.md @@ -1,5 +1,5 @@ # system.numbers\_mt {#system-numbers-mt} -The same as [system.numbers](numbers.md) but reads are parallelized. The numbers can be returned in any order. +The same as [system.numbers](../../operations/system-tables/numbers.md) but reads are parallelized. The numbers can be returned in any order. Used for tests. diff --git a/docs/en/operations/system-tables/query_log.md b/docs/en/operations/system-tables/query_log.md index 9ee34b0e516..3b96be068a5 100644 --- a/docs/en/operations/system-tables/query_log.md +++ b/docs/en/operations/system-tables/query_log.md @@ -11,7 +11,7 @@ You can disable queries logging by setting [log\_queries = 0](../../operations/s The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. -ClickHouse doesn’t delete data from the table automatically. See [Introduction](index.md#system-tables-introduction) for more details. +ClickHouse doesn’t delete data from the table automatically. See [Introduction](../../operations/system-tables/index.md#system-tables-introduction) for more details. The `system.query_log` table registers two kinds of queries: @@ -75,7 +75,7 @@ Columns: - `quota_key` ([String](../../sql-reference/data-types/string.md)) — The “quota key” specified in the [quotas](../../operations/quotas.md) setting (see `keyed`). - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse revision. - `thread_numbers` ([Array(UInt32)](../../sql-reference/data-types/array.md)) — Number of threads that are participating in query execution. -- `ProfileEvents.Names` ([Array(String)](../../sql-reference/data-types/array.md)) — Counters that measure different metrics. The description of them could be found in the table [system.events](events.md#system_tables-events) +- `ProfileEvents.Names` ([Array(String)](../../sql-reference/data-types/array.md)) — Counters that measure different metrics. The description of them could be found in the table [system.events](../../operations/system-tables/events.md#system_tables-events) - `ProfileEvents.Values` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — Values of metrics that are listed in the `ProfileEvents.Names` column. - `Settings.Names` ([Array(String)](../../sql-reference/data-types/array.md)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. - `Settings.Values` ([Array(String)](../../sql-reference/data-types/array.md)) — Values of settings that are listed in the `Settings.Names` column. @@ -135,4 +135,4 @@ Settings.Values: ['0','random','1','10000000000'] **See Also** -- [system.query\_thread\_log](query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. +- [system.query\_thread\_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — This table contains information about each query execution thread. diff --git a/docs/en/operations/system-tables/query_thread_log.md b/docs/en/operations/system-tables/query_thread_log.md index 370257639b3..88231f92dc2 100644 --- a/docs/en/operations/system-tables/query_thread_log.md +++ b/docs/en/operations/system-tables/query_thread_log.md @@ -9,7 +9,7 @@ To start logging: The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query\_thread\_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. -ClickHouse doesn’t delete data from the table automatically. See [Introduction](index.md#system-tables-introduction) for more details. +ClickHouse doesn’t delete data from the table automatically. See [Introduction](../../operations/system-tables/index.md#system-tables-introduction) for more details. Columns: @@ -110,4 +110,4 @@ ProfileEvents.Values: [1,97,81,5,81] **See Also** -- [system.query\_log](query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. +- [system.query\_log](../../operations/system-tables/query_log.md#system_tables-query_log) — Description of the `query_log` system table which contains common information about queries execution. diff --git a/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md b/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md index dbfda63c146..5c9d6875c51 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anyheavy.md @@ -1,6 +1,7 @@ --- toc_priority: 103 --- + # anyHeavy {#anyheavyx} Selects a frequently occurring value using the [heavy hitters](http://www.cs.umd.edu/~samir/498/karp.pdf) algorithm. If there is a value that occurs more than in half the cases in each of the query’s execution threads, this value is returned. Normally, the result is nondeterministic. diff --git a/docs/en/sql-reference/aggregate-functions/reference/anylast.md b/docs/en/sql-reference/aggregate-functions/reference/anylast.md index 298ec063924..43ac72318f2 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/anylast.md +++ b/docs/en/sql-reference/aggregate-functions/reference/anylast.md @@ -5,4 +5,4 @@ toc_priority: 104 ## anyLast {#anylastx} Selects the last value encountered. -The result is just as indeterminate as for the [any](any.md) function. +The result is just as indeterminate as for the [any](../../../sql-reference/aggregate-functions/reference/any.md) function. diff --git a/docs/en/sql-reference/aggregate-functions/reference/count.md b/docs/en/sql-reference/aggregate-functions/reference/count.md index 12b99814661..68ad69fe264 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/count.md +++ b/docs/en/sql-reference/aggregate-functions/reference/count.md @@ -26,7 +26,7 @@ In both cases the type of the returned value is [UInt64](../../../sql-reference/ **Details** -ClickHouse supports the `COUNT(DISTINCT ...)` syntax. The behavior of this construction depends on the [count\_distinct\_implementation](../../../operations/settings/settings.md#settings-count_distinct_implementation) setting. It defines which of the [uniq\*](uniq.md#agg_function-uniq) functions is used to perform the operation. The default is the [uniqExact](uniqexact.md#agg_function-uniqexact) function. +ClickHouse supports the `COUNT(DISTINCT ...)` syntax. The behavior of this construction depends on the [count\_distinct\_implementation](../../../operations/settings/settings.md#settings-count_distinct_implementation) setting. It defines which of the [uniq\*](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) functions is used to perform the operation. The default is the [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) function. The `SELECT count() FROM table` query is not optimized, because the number of entries in the table is not stored separately. It chooses a small column from the table and counts the number of values in it. diff --git a/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md b/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md index c35d3bd2004..f4b8665a0a4 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md +++ b/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md @@ -21,7 +21,7 @@ If in one query several values are inserted into the same position, the function - `x` — Value to be inserted. [Expression](../../../sql-reference/syntax.md#syntax-expressions) resulting in one of the [supported data types](../../../sql-reference/data-types/index.md). - `pos` — Position at which the specified element `x` is to be inserted. Index numbering in the array starts from zero. [UInt32](../../../sql-reference/data-types/int-uint.md#uint-ranges). -- `default_x`— Default value for substituting in empty positions. Optional parameter. [Expression](../../../sql-reference/syntax.md#syntax-expressions) resulting in the data type configured for the `x` parameter. If `default_x` is not defined, the [default values](../../../sql-reference/statements/create.md#create-default-values) are used. +- `default_x`— Default value for substituting in empty positions. Optional parameter. [Expression](../../../sql-reference/syntax.md#syntax-expressions) resulting in the data type configured for the `x` parameter. If `default_x` is not defined, the [default values](../../../sql-reference/statements/create/table.md#create-default-values) are used. - `size`— Length of the resulting array. Optional parameter. When using this parameter, the default value `default_x` must be specified. [UInt32](../../../sql-reference/data-types/int-uint.md#uint-ranges). **Returned value** diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md b/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md index 49f4d477d17..537212e5b94 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md @@ -2,11 +2,11 @@ toc_priority: 111 --- -# groupUniqArray +# groupUniqArray {#groupuniqarray} Syntax: `groupUniqArray(x)` or `groupUniqArray(max_size)(x)` -Creates an array from different argument values. Memory consumption is the same as for the [uniqExact](uniqexact.md) function. +Creates an array from different argument values. Memory consumption is the same as for the [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) function. The second version (with the `max_size` parameter) limits the size of the resulting array to `max_size` elements. For example, `groupUniqArray(1)(x)` is equivalent to `[any(x)]`. diff --git a/docs/en/sql-reference/aggregate-functions/reference/index.md b/docs/en/sql-reference/aggregate-functions/reference/index.md index 282642bc32d..0fce4c1d8b3 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/index.md +++ b/docs/en/sql-reference/aggregate-functions/reference/index.md @@ -1,7 +1,7 @@ --- +toc_folder_title: Reference toc_priority: 36 toc_title: Reference -toc_folder_title: Reference --- # Aggregate Function Reference {#aggregate-functions-reference} @@ -24,50 +24,95 @@ Standard aggregate functions: ClickHouse-specific aggregate functions: - [anyHeavy](../../../sql-reference/aggregate-functions/reference/anyheavy.md) + - [anyLast](../../../sql-reference/aggregate-functions/reference/anylast.md) + - [argMin](../../../sql-reference/aggregate-functions/reference/argmin.md) + - [argMax](../../../sql-reference/aggregate-functions/reference/argmax.md) + - [avgWeighted](../../../sql-reference/aggregate-functions/reference/avgweighted.md) + - [topK](../../../sql-reference/aggregate-functions/reference/topkweighted.md) + - [topKWeighted](../../../sql-reference/aggregate-functions/reference/topkweighted.md) + - [groupArray](../../../sql-reference/aggregate-functions/reference/grouparray.md) + - [groupUniqArray](../../../sql-reference/aggregate-functions/reference/groupuniqarray.md) + - [groupArrayInsertAt](../../../sql-reference/aggregate-functions/reference/grouparrayinsertat.md) + - [groupArrayMovingAvg](../../../sql-reference/aggregate-functions/reference/grouparraymovingavg.md) + - [groupArrayMovingSum](../../../sql-reference/aggregate-functions/reference/grouparraymovingsum.md) + - [groupBitAnd](../../../sql-reference/aggregate-functions/reference/groupbitand.md) + - [groupBitOr](../../../sql-reference/aggregate-functions/reference/groupbitor.md) + - [groupBitXor](../../../sql-reference/aggregate-functions/reference/groupbitxor.md) + - [groupBitmap](../../../sql-reference/aggregate-functions/reference/groupbitmap.md) + - [groupBitmapAnd](../../../sql-reference/aggregate-functions/reference/groupbitmapand.md) + - [groupBitmapOr](../../../sql-reference/aggregate-functions/reference/groupbitmapor.md) + - [groupBitmapXor](../../../sql-reference/aggregate-functions/reference/groupbitmapxor.md) + - [sumWithOverflow](../../../sql-reference/aggregate-functions/reference/sumwithoverflow.md) + - [sumMap](../../../sql-reference/aggregate-functions/reference/summap.md) + - [minMap](../../../sql-reference/aggregate-functions/reference/minmap.md) + - [maxMap](../../../sql-reference/aggregate-functions/reference/maxmap.md) + - [skewSamp](../../../sql-reference/aggregate-functions/reference/skewsamp.md) + - [skewPop](../../../sql-reference/aggregate-functions/reference/skewpop.md) + - [kurtSamp](../../../sql-reference/aggregate-functions/reference/kurtsamp.md) + - [kurtPop](../../../sql-reference/aggregate-functions/reference/kurtpop.md) + - [timeSeriesGroupSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupsum.md) + - [timeSeriesGroupRateSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md) + - [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md) + - [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md) + - [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) + - [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md) + - [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md) + - [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md) + - [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md) + - [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md) + - [quantileExactWeighted](../../../sql-reference/aggregate-functions/reference/quantileexactweighted.md) + - [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md) + - [quantileTimingWeighted](../../../sql-reference/aggregate-functions/reference/quantiletimingweighted.md) + - [quantileDeterministic](../../../sql-reference/aggregate-functions/reference/quantiledeterministic.md) + - [quantileTDigest](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) + - [quantileTDigestWeighted](../../../sql-reference/aggregate-functions/reference/quantiletdigestweighted.md) + - [simpleLinearRegression](../../../sql-reference/aggregate-functions/reference/simplelinearregression.md) + - [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md) + - [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md) - [categoricalInformationValue](../../../sql-reference/aggregate-functions/reference/categoricalinformationvalue.md) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantile.md b/docs/en/sql-reference/aggregate-functions/reference/quantile.md index 454925779f7..77f858a1735 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantile.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantile.md @@ -6,9 +6,9 @@ toc_priority: 200 Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence. -This function applies [reservoir sampling](https://en.wikipedia.org/wiki/Reservoir_sampling) with a reservoir size up to 8192 and a random number generator for sampling. The result is non-deterministic. To get an exact quantile, use the [quantileExact](quantileexact.md#quantileexact) function. +This function applies [reservoir sampling](https://en.wikipedia.org/wiki/Reservoir_sampling) with a reservoir size up to 8192 and a random number generator for sampling. The result is non-deterministic. To get an exact quantile, use the [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexact) function. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -62,5 +62,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiledeterministic.md b/docs/en/sql-reference/aggregate-functions/reference/quantiledeterministic.md index b413332de34..6046447dd10 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiledeterministic.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiledeterministic.md @@ -6,9 +6,9 @@ toc_priority: 206 Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence. -This function applies [reservoir sampling](https://en.wikipedia.org/wiki/Reservoir_sampling) with a reservoir size up to 8192 and deterministic algorithm of sampling. The result is deterministic. To get an exact quantile, use the [quantileExact](quantileexact.md#quantileexact) function. +This function applies [reservoir sampling](https://en.wikipedia.org/wiki/Reservoir_sampling) with a reservoir size up to 8192 and deterministic algorithm of sampling. The result is deterministic. To get an exact quantile, use the [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexact) function. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -63,5 +63,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md b/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md index e2780df6bbf..100d6ea129d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantileexact.md @@ -8,7 +8,7 @@ Exactly computes the [quantile](https://en.wikipedia.org/wiki/Quantile) of a num To get exact value, all the passed values ​​are combined into an array, which is then partially sorted. Therefore, the function consumes `O(n)` memory, where `n` is a number of values that were passed. However, for a small number of values, the function is very effective. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -51,5 +51,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantileexactweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantileexactweighted.md index 98152c9c73f..3251f8298a6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantileexactweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantileexactweighted.md @@ -2,14 +2,13 @@ toc_priority: 203 --- - # quantileExactWeighted {#quantileexactweighted} Exactly computes the [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence, taking into account the weight of each element. -To get exact value, all the passed values ​​are combined into an array, which is then partially sorted. Each value is counted with its weight, as if it is present `weight` times. A hash table is used in the algorithm. Because of this, if the passed values ​​are frequently repeated, the function consumes less RAM than [quantileExact](quantileexact.md#quantileexact). You can use this function instead of `quantileExact` and specify the weight 1. +To get exact value, all the passed values ​​are combined into an array, which is then partially sorted. Each value is counted with its weight, as if it is present `weight` times. A hash table is used in the algorithm. Because of this, if the passed values ​​are frequently repeated, the function consumes less RAM than [quantileExact](../../../sql-reference/aggregate-functions/reference/quantileexact.md#quantileexact). You can use this function instead of `quantileExact` and specify the weight 1. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -64,5 +63,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md index 18a4e5be4e0..bda98ea338d 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigest.md @@ -8,9 +8,9 @@ Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a The maximum error is 1%. Memory consumption is `log(n)`, where `n` is a number of values. The result depends on the order of running the query, and is nondeterministic. -The performance of the function is lower than performance of [quantile](quantile.md#quantile) or [quantileTiming](quantiletiming.md#quantiletiming). In terms of the ratio of State size to precision, this function is much better than `quantile`. +The performance of the function is lower than performance of [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile) or [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md#quantiletiming). In terms of the ratio of State size to precision, this function is much better than `quantile`. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -53,5 +53,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md index 712e49dfbcb..309cbe95e95 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md @@ -6,11 +6,11 @@ toc_priority: 208 Computes an approximate [quantile](https://en.wikipedia.org/wiki/Quantile) of a numeric data sequence using the [t-digest](https://github.com/tdunning/t-digest/blob/master/docs/t-digest-paper/histo.pdf) algorithm. The function takes into account the weight of each sequence member. The maximum error is 1%. Memory consumption is `log(n)`, where `n` is a number of values. -The performance of the function is lower than performance of [quantile](quantile.md#quantile) or [quantileTiming](quantiletiming.md#quantiletiming). In terms of the ratio of State size to precision, this function is much better than `quantile`. +The performance of the function is lower than performance of [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile) or [quantileTiming](../../../sql-reference/aggregate-functions/reference/quantiletiming.md#quantiletiming). In terms of the ratio of State size to precision, this function is much better than `quantile`. The result depends on the order of running the query, and is nondeterministic. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -54,5 +54,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletiming.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletiming.md index 7e887cba40d..867e8b87e74 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletiming.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletiming.md @@ -8,7 +8,7 @@ With the determined precision computes the [quantile](https://en.wikipedia.org/w The result is deterministic (it doesn’t depend on the query processing order). The function is optimized for working with sequences which describe distributions like loading web pages times or backend response times. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -24,8 +24,8 @@ Alias: `medianTiming`. - `expr` — [Expression](../../../sql-reference/syntax.md#syntax-expressions) over a column values returning a [Float\*](../../../sql-reference/data-types/float.md)-type number. - - If negative values are passed to the function, the behavior is undefined. - - If the value is greater than 30,000 (a page loading time of more than 30 seconds), it is assumed to be 30,000. + - If negative values are passed to the function, the behavior is undefined. + - If the value is greater than 30,000 (a page loading time of more than 30 seconds), it is assumed to be 30,000. **Accuracy** @@ -37,7 +37,7 @@ The calculation is accurate if: Otherwise, the result of the calculation is rounded to the nearest multiple of 16 ms. !!! note "Note" - For calculating page loading time quantiles, this function is more effective and accurate than [quantile](quantile.md#quantile). + For calculating page loading time quantiles, this function is more effective and accurate than [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile). **Returned value** @@ -82,5 +82,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletimingweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletimingweighted.md index 60853558e64..0f8606986c8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletimingweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletimingweighted.md @@ -8,7 +8,7 @@ With the determined precision computes the [quantile](https://en.wikipedia.org/w The result is deterministic (it doesn’t depend on the query processing order). The function is optimized for working with sequences which describe distributions like loading web pages times or backend response times. -When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](quantiles.md#quantiles) function. +When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. **Syntax** @@ -39,7 +39,7 @@ The calculation is accurate if: Otherwise, the result of the calculation is rounded to the nearest multiple of 16 ms. !!! note "Note" - For calculating page loading time quantiles, this function is more effective and accurate than [quantile](quantile.md#quantile). + For calculating page loading time quantiles, this function is more effective and accurate than [quantile](../../../sql-reference/aggregate-functions/reference/quantile.md#quantile). **Returned value** @@ -81,5 +81,5 @@ Result: **See Also** -- [median](median.md#median) -- [quantiles](quantiles.md#quantiles) +- [median](../../../sql-reference/aggregate-functions/reference/median.md#median) +- [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) diff --git a/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md b/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md index c64ebe1cbcf..58f8c27cd72 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stddevpop.md @@ -4,7 +4,7 @@ toc_priority: 30 # stddevPop {#stddevpop} -The result is equal to the square root of [varPop](varpop.md). +The result is equal to the square root of [varPop](../../../sql-reference/aggregate-functions/reference/varpop.md). !!! note "Note" This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the `stddevPopStable` function. It works slower but provides a lower computational error. diff --git a/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md b/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md index 78aa89a0757..4ec72881ae5 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md @@ -4,7 +4,7 @@ toc_priority: 31 # stddevSamp {#stddevsamp} -The result is equal to the square root of [varSamp](varsamp.md). +The result is equal to the square root of [varSamp](../../../sql-reference/aggregate-functions/reference/varsamp.md). !!! note "Note" This function uses a numerically unstable algorithm. If you need [numerical stability](https://en.wikipedia.org/wiki/Numerical_stability) in calculations, use the `stddevSampStable` function. It works slower but provides a lower computational error. diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md index 948614d4e68..7a37ed83e17 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md @@ -71,5 +71,5 @@ The query will return a column of predicted values. Note that first argument of **See Also** -- [stochasticLogisticRegression](stochasticlogisticregression.md#agg_functions-stochasticlogisticregression) +- [stochasticLogisticRegression](../../../sql-reference/aggregate-functions/reference/stochasticlogisticregression.md#agg_functions-stochasticlogisticregression) - [Difference between linear and logistic regressions](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression) diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md index 2e86eeed881..35d1e3899ac 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlogisticregression.md @@ -51,5 +51,5 @@ stochasticLogisticRegression(1.0, 1.0, 10, 'SGD') **See Also** -- [stochasticLinearRegression](stochasticlinearregression.md#agg_functions-stochasticlinearregression) +- [stochasticLinearRegression](../../../sql-reference/aggregate-functions/reference/stochasticlinearregression.md#agg_functions-stochasticlinearregression) - [Difference between linear and logistic regressions.](https://stackoverflow.com/questions/12146914/what-is-the-difference-between-linear-regression-and-logistic-regression) diff --git a/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md b/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md index 4cbe37bc635..d9b519a9cde 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md +++ b/docs/en/sql-reference/aggregate-functions/reference/timeseriesgroupratesum.md @@ -6,7 +6,7 @@ toc_priority: 171 Syntax: `timeSeriesGroupRateSum(uid, ts, val)` -Similarly to [timeSeriesGroupSum](timeseriesgroupsum.md), `timeSeriesGroupRateSum` calculates the rate of time-series and then sum rates together. +Similarly to [timeSeriesGroupSum](../../../sql-reference/aggregate-functions/reference/timeseriesgroupsum.md), `timeSeriesGroupRateSum` calculates the rate of time-series and then sum rates together. Also, timestamp should be in ascend order before use this function. Applying this function to the data from the `timeSeriesGroupSum` example, you get the following result: diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniq.md b/docs/en/sql-reference/aggregate-functions/reference/uniq.md index ed831b9c25b..81d1ec6761e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniq.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniq.md @@ -34,7 +34,7 @@ We recommend using this function in almost all scenarios. **See Also** -- [uniqCombined](uniqcombined.md#agg_function-uniqcombined) -- [uniqCombined64](uniqcombined64.md#agg_function-uniqcombined64) -- [uniqHLL12](uniqhll12.md#agg_function-uniqhll12) -- [uniqExact](uniqexact.md#agg_function-uniqexact) +- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined) +- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) +- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) +- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md index c7600334da8..c52486bc38f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined.md @@ -35,9 +35,9 @@ Function: - Provides the result deterministically (it doesn’t depend on the query processing order). !!! note "Note" - Since it uses 32-bit hash for non-`String` type, the result will have very high error for cardinalities significantly larger than `UINT_MAX` (error will raise quickly after a few tens of billions of distinct values), hence in this case you should use [uniqCombined64](uniqcombined64.md#agg_function-uniqcombined64) + Since it uses 32-bit hash for non-`String` type, the result will have very high error for cardinalities significantly larger than `UINT_MAX` (error will raise quickly after a few tens of billions of distinct values), hence in this case you should use [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) -Compared to the [uniq](uniq.md#agg_function-uniq) function, the `uniqCombined`: +Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined`: - Consumes several times less memory. - Calculates with several times higher accuracy. @@ -45,7 +45,7 @@ Compared to the [uniq](uniq.md#agg_function-uniq) function, the `uniqCombined`: **See Also** -- [uniq](uniq.md#agg_function-uniq) -- [uniqCombined64](uniqcombined64.md#agg_function-uniqcombined64) -- [uniqHLL12](uniqhll12.md#agg_function-uniqhll12) -- [uniqExact](uniqexact.md#agg_function-uniqexact) +- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) +- [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) +- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) +- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md index 5bf2e283e6e..6d060d82779 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqcombined64.md @@ -4,4 +4,4 @@ toc_priority: 193 # uniqCombined64 {#agg_function-uniqcombined64} -Same as [uniqCombined](uniqcombined.md#agg_function-uniqcombined), but uses 64-bit hash for all data types. +Same as [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined), but uses 64-bit hash for all data types. diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md b/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md index 9ccd065bb62..9a6224533c8 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqexact.md @@ -10,7 +10,7 @@ Calculates the exact number of different argument values. uniqExact(x[, ...]) ``` -Use the `uniqExact` function if you absolutely need an exact result. Otherwise use the [uniq](uniq.md#agg_function-uniq) function. +Use the `uniqExact` function if you absolutely need an exact result. Otherwise use the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function. The `uniqExact` function uses more memory than `uniq`, because the size of the state has unbounded growth as the number of different values increases. @@ -20,6 +20,6 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, ` **See Also** -- [uniq](uniq.md#agg_function-uniq) -- [uniqCombined](uniq.md#agg_function-uniqcombined) -- [uniqHLL12](uniq.md#agg_function-uniqhll12) +- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) +- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqcombined) +- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniqhll12) diff --git a/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md b/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md index 10263a65071..fcddc22cc46 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md +++ b/docs/en/sql-reference/aggregate-functions/reference/uniqhll12.md @@ -30,10 +30,10 @@ Function: - Provides the determinate result (it doesn’t depend on the query processing order). -We don’t recommend using this function. In most cases, use the [uniq](uniq.md#agg_function-uniq) or [uniqCombined](uniqcombined.md#agg_function-uniqcombined) function. +We don’t recommend using this function. In most cases, use the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) or [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined) function. **See Also** -- [uniq](uniq.md#agg_function-uniq) -- [uniqCombined](uniqcombined.md#agg_function-uniqcombined) -- [uniqExact](uniqexact.md#agg_function-uniqexact) +- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) +- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined) +- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) diff --git a/docs/en/sql-reference/data-types/aggregatefunction.md b/docs/en/sql-reference/data-types/aggregatefunction.md index 8bde9e8011b..5bc8ffc666b 100644 --- a/docs/en/sql-reference/data-types/aggregatefunction.md +++ b/docs/en/sql-reference/data-types/aggregatefunction.md @@ -5,7 +5,7 @@ toc_title: AggregateFunction # AggregateFunction {#data-type-aggregatefunction} -Aggregate functions can have an implementation-defined intermediate state that can be serialized to an AggregateFunction(…) data type and stored in a table, usually, by means of [a materialized view](../../sql-reference/statements/create.md#create-view). The common way to produce an aggregate function state is by calling the aggregate function with the `-State` suffix. To get the final result of aggregation in the future, you must use the same aggregate function with the `-Merge`suffix. +Aggregate functions can have an implementation-defined intermediate state that can be serialized to an `AggregateFunction(…)` data type and stored in a table, usually, by means of [a materialized view](../../sql-reference/statements/create/view.md). The common way to produce an aggregate function state is by calling the aggregate function with the `-State` suffix. To get the final result of aggregation in the future, you must use the same aggregate function with the `-Merge`suffix. `AggregateFunction(name, types_of_arguments…)` — parametric data type. diff --git a/docs/en/sql-reference/data-types/domains/index.md b/docs/en/sql-reference/data-types/domains/index.md index 30aca1eb059..143aafe4db3 100644 --- a/docs/en/sql-reference/data-types/domains/index.md +++ b/docs/en/sql-reference/data-types/domains/index.md @@ -1,6 +1,6 @@ --- -toc_priority: 56 toc_folder_title: Domains +toc_priority: 56 toc_title: Overview --- diff --git a/docs/en/sql-reference/data-types/nested-data-structures/nested.md b/docs/en/sql-reference/data-types/nested-data-structures/nested.md index 40946fb1bcf..302d41c1357 100644 --- a/docs/en/sql-reference/data-types/nested-data-structures/nested.md +++ b/docs/en/sql-reference/data-types/nested-data-structures/nested.md @@ -5,7 +5,7 @@ toc_title: Nested(Name1 Type1, Name2 Type2, ...) # Nested(name1 Type1, Name2 Type2, …) {#nestedname1-type1-name2-type2} -A nested data structure is like a table inside a cell. The parameters of a nested data structure – the column names and types – are specified the same way as in a [CREATE TABLE](../../../sql-reference/statements/create.md) query. Each table row can correspond to any number of rows in a nested data structure. +A nested data structure is like a table inside a cell. The parameters of a nested data structure – the column names and types – are specified the same way as in a [CREATE TABLE](../../../sql-reference/statements/create/table.md) query. Each table row can correspond to any number of rows in a nested data structure. Example: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 3cf0ac050a9..251ec532e17 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -39,7 +39,7 @@ The configuration looks like this: ``` -Corresponding [DDL-query](../../../sql-reference/statements/create.md#create-dictionary-query): +Corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md): ``` sql CREATE DICTIONARY (...) @@ -330,7 +330,7 @@ LAYOUT(CACHE(BLOCK_SIZE 4096 FILE_SIZE 16777216 READ_BUFFER_SIZE 1048576 ### complex\_key\_ssd\_cache {#complex-key-ssd-cache} -This type of storage is for use with composite [keys](external-dicts-dict-structure.md). Similar to `ssd\_cache`. +This type of storage is for use with composite [keys](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md). Similar to `ssd\_cache`. ### direct {#direct} diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 29c76223059..98f0a5ffb4c 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -24,7 +24,7 @@ If dictionary is configured using xml-file, the configuration looks like this: ``` -In case of [DDL-query](../../../sql-reference/statements/create.md#create-dictionary-query), equal configuration will looks like: +In case of [DDL-query](../../../sql-reference/statements/create/dictionary.md), equal configuration will looks like: ``` sql CREATE DICTIONARY dict_name (...) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md index 8dad24df6a7..17ad110aa19 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md @@ -29,7 +29,7 @@ If dictionary is configured using xml file, than dictionary configuration has th ``` -Corresponding [DDL-query](../../../sql-reference/statements/create.md#create-dictionary-query) has the following structure: +Corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md) has the following structure: ``` sql CREATE DICTIONARY dict_name diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index d279ecdd165..c6eb2cb3f62 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -11,7 +11,7 @@ ClickHouse: - Fully or partially stores dictionaries in RAM. - Periodically updates dictionaries and dynamically loads missing values. In other words, dictionaries can be loaded dynamically. -- Allows to create external dictionaries with xml files or [DDL queries](../../../sql-reference/statements/create.md#create-dictionary-query). +- Allows to create external dictionaries with xml files or [DDL queries](../../../sql-reference/statements/create/dictionary.md). The configuration of external dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries\_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parameter. @@ -43,7 +43,7 @@ The dictionary configuration file has the following format: You can [configure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md) any number of dictionaries in the same file. -[DDL queries for dictionaries](../../../sql-reference/statements/create.md#create-dictionary-query) doesn’t require any additional records in server configuration. They allow to work with dictionaries as first-class entities, like tables or views. +[DDL queries for dictionaries](../../../sql-reference/statements/create/dictionary.md) doesn’t require any additional records in server configuration. They allow to work with dictionaries as first-class entities, like tables or views. !!! attention "Attention" You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../../sql-reference/functions/other-functions.md) function). This functionality is not related to external dictionaries. diff --git a/docs/en/sql-reference/distributed-ddl.md b/docs/en/sql-reference/distributed-ddl.md new file mode 100644 index 00000000000..c291c85fa7a --- /dev/null +++ b/docs/en/sql-reference/distributed-ddl.md @@ -0,0 +1,21 @@ +--- +toc_priority: 32 +toc_title: Distributed DDL +--- + +# Distributed DDL Queries (ON CLUSTER Clause) {#distributed-ddl-queries-on-cluster-clause} + +By default the `CREATE`, `DROP`, `ALTER`, and `RENAME` queries affect only the current server where they are executed. In a cluster setup, it is possible to run such queries in a distributed manner with the `ON CLUSTER` clause. + +For example, the following query creates the `all_hits` `Distributed` table on each host in `cluster`: + +``` sql +CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) +``` + +In order to run these queries correctly, each host must have the same cluster definition (to simplify syncing configs, you can use substitutions from ZooKeeper). They must also connect to the ZooKeeper servers. + +The local version of the query will eventually be executed on each host in the cluster, even if some hosts are currently not available. + +!!! warning "Warning" + The order for executing queries within a single host is guaranteed. diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 02da307a0f3..91ecc963b1f 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -184,12 +184,12 @@ Checks whether all the elements of array2 appear in array1 in the same exact ord hasSubstr(array1, array2) ``` -In other words, the functions will check whether all the elements of `array2` are contained in `array1` like +In other words, the functions will check whether all the elements of `array2` are contained in `array1` like the `hasAll` function. In addition, it will check that the elements are observed in the same order in both `array1` and `array2`. -For Example: - - `hasSubstr([1,2,3,4], [2,3])` returns 1. However, `hasSubstr([1,2,3,4], [3,2])` will return `0`. - - `hasSubstr([1,2,3,4], [1,2,3])` returns 1. However, `hasSubstr([1,2,3,4], [1,2,4])` will return `0`. +For Example: +- `hasSubstr([1,2,3,4], [2,3])` returns 1. However, `hasSubstr([1,2,3,4], [3,2])` will return `0`. +- `hasSubstr([1,2,3,4], [1,2,3])` returns 1. However, `hasSubstr([1,2,3,4], [1,2,4])` will return `0`. **Parameters** @@ -223,7 +223,6 @@ For Example: `SELECT hasSubstr([[1, 2], [3, 4], [5, 6]], [[1, 2], [3, 4]])` returns 1. - ## indexOf(arr, x) {#indexofarr-x} Returns the index of the first ‘x’ element (starting from 1) if it is in the array, or 0 if it is not. @@ -939,7 +938,7 @@ arrayReduceInRanges(agg_func, ranges, arr1, arr2, ..., arrN) **Returned value** -- Array containing results of the aggregate function over specified ranges. +- Array containing results of the aggregate function over specified ranges. Type: [Array](../../sql-reference/data-types/array.md). diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4eb316e0455..5593221f74c 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -161,14 +161,14 @@ toStartOfSecond(value[, timezone]) **Parameters** -- `value` — Date and time. [DateTime64](../data-types/datetime64.md). -- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../data-types/string.md). +- `value` — Date and time. [DateTime64](../../sql-reference/data-types/datetime64.md). +- `timezone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../../sql-reference/data-types/string.md). **Returned value** -- Input value without sub-seconds. +- Input value without sub-seconds. -Type: [DateTime64](../data-types/datetime64.md). +Type: [DateTime64](../../sql-reference/data-types/datetime64.md). **Examples** diff --git a/docs/en/sql-reference/functions/geo.md b/docs/en/sql-reference/functions/geo.md index 0e8deb7a2c7..7c85af4e9d1 100644 --- a/docs/en/sql-reference/functions/geo.md +++ b/docs/en/sql-reference/functions/geo.md @@ -281,7 +281,7 @@ h3GetBaseCell(index) **Returned value** -- Hexagon base cell number. +- Hexagon base cell number. Type: [UInt8](../../sql-reference/data-types/int-uint.md). @@ -317,7 +317,7 @@ h3HexAreaM2(resolution) **Returned value** -- Area in square meters. +- Area in square meters. Type: [Float64](../../sql-reference/data-types/float.md). @@ -392,7 +392,7 @@ h3ToChildren(index, resolution) **Returned values** -- Array of the child H3-indexes. +- Array of the child H3-indexes. Type: [Array](../../sql-reference/data-types/array.md)([UInt64](../../sql-reference/data-types/int-uint.md)). @@ -429,7 +429,7 @@ h3ToParent(index, resolution) **Returned value** -- Parent H3 index. +- Parent H3 index. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -463,7 +463,7 @@ h3ToString(index) **Returned value** -- String representation of the H3 index. +- String representation of the H3 index. Type: [String](../../sql-reference/data-types/string.md). diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 55ddc5c029c..9c72f0611b2 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -799,7 +799,7 @@ Query: SELECT blockSerializedSize(maxState(1)) as x ``` -Result: +Result: ``` text ┌─x─┐ @@ -1056,25 +1056,25 @@ Takes state of aggregate function. Returns result of aggregation (finalized stat ## runningAccumulate {#runningaccumulate} -Accumulates states of an aggregate function for each row of a data block. +Accumulates states of an aggregate function for each row of a data block. !!! warning "Warning" The state is reset for each new data block. **Syntax** -```sql +``` sql runningAccumulate(agg_state[, grouping]); ``` **Parameters** -- `agg_state` — State of the aggregate function. [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction). -- `grouping` — Grouping key. Optional. The state of the function is reset if the `grouping` value is changed. It can be any of the [supported data types](../../sql-reference/data-types/index.md) for which the equality operator is defined. +- `agg_state` — State of the aggregate function. [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction). +- `grouping` — Grouping key. Optional. The state of the function is reset if the `grouping` value is changed. It can be any of the [supported data types](../../sql-reference/data-types/index.md) for which the equality operator is defined. **Returned value** -- Each resulting row contains a result of the aggregate function, accumulated for all the input rows from 0 to the current position. `runningAccumulate` resets states for each new data block or when the `grouping` value changes. +- Each resulting row contains a result of the aggregate function, accumulated for all the input rows from 0 to the current position. `runningAccumulate` resets states for each new data block or when the `grouping` value changes. Type depends on the aggregate function used. @@ -1084,13 +1084,13 @@ Consider how you can use `runningAccumulate` to find the cumulative sum of numbe Query: -```sql +``` sql SELECT k, runningAccumulate(sum_k) AS res FROM (SELECT number as k, sumState(k) AS sum_k FROM numbers(10) GROUP BY k ORDER BY k); ``` Result: -```text +``` text ┌─k─┬─res─┐ │ 0 │ 0 │ │ 1 │ 1 │ @@ -1105,27 +1105,27 @@ Result: └───┴─────┘ ``` -The subquery generates `sumState` for every number from `0` to `9`. `sumState` returns the state of the [sum](../aggregate-functions/reference/sum.md) function that contains the sum of a single number. +The subquery generates `sumState` for every number from `0` to `9`. `sumState` returns the state of the [sum](../../sql-reference/aggregate-functions/reference/sum.md) function that contains the sum of a single number. The whole query does the following: -1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. -2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. -3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. -4. The actions are repeated until the block ends. +1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. +2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. +3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. +4. The actions are repeated until the block ends. The following example shows the `groupping` parameter usage: Query: -```sql -SELECT +``` sql +SELECT grouping, item, runningAccumulate(state, grouping) AS res -FROM +FROM ( - SELECT + SELECT toInt8(number / 4) AS grouping, number AS item, sumState(number) AS state @@ -1137,7 +1137,7 @@ FROM Result: -```text +``` text ┌─grouping─┬─item─┬─res─┐ │ 0 │ 0 │ 0 │ │ 0 │ 1 │ 1 │ diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index a19544f868f..8ed3b073fa8 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -6,7 +6,7 @@ toc_title: Strings # Functions for Working with Strings {#functions-for-working-with-strings} !!! note "Note" - Functions for [searching](string-search-functions.md) and [replacing](string-replace-functions.md) in strings are described separately. + Functions for [searching](../../sql-reference/functions/string-search-functions.md) and [replacing](../../sql-reference/functions/string-replace-functions.md) in strings are described separately. ## empty {#empty} diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 98b9943cdab..731215277f5 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -6,7 +6,7 @@ toc_title: For Replacing in Strings # Functions for Searching and Replacing in Strings {#functions-for-searching-and-replacing-in-strings} !!! note "Note" - Functions for [searching](string-search-functions.md) and [other manipulations with strings](string-functions.md) are described separately. + Functions for [searching](../../sql-reference/functions/string-search-functions.md) and [other manipulations with strings](../../sql-reference/functions/string-functions.md) are described separately. ## replaceOne(haystack, pattern, replacement) {#replaceonehaystack-pattern-replacement} diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 815d74a2ccb..067644c30b2 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -8,7 +8,7 @@ toc_title: For Searching in Strings The search is case-sensitive by default in all these functions. There are separate variants for case insensitive search. !!! note "Note" - Functions for [replacing](string-replace-functions.md) and [other manipulations with strings](string-functions.md) are described separately. + Functions for [replacing](../../sql-reference/functions/string-replace-functions.md) and [other manipulations with strings](../../sql-reference/functions/string-functions.md) are described separately. ## position(haystack, needle), locate(haystack, needle) {#position} diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 929c861281b..67361c350c7 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -523,7 +523,7 @@ Result: ## parseDateTimeBestEffortUS {#parsedatetimebesteffortUS} -This function is similar to ['parseDateTimeBestEffort'](#parsedatetimebesteffort), the only difference is that this function prefers US style (`MM/DD/YYYY` etc) in case of ambiguouty. +This function is similar to [‘parseDateTimeBestEffort’](#parsedatetimebesteffort), the only difference is that this function prefers US style (`MM/DD/YYYY` etc) in case of ambiguouty. **Syntax** diff --git a/docs/en/sql-reference/index.md b/docs/en/sql-reference/index.md index aeb3fb3ccdb..22a89d001af 100644 --- a/docs/en/sql-reference/index.md +++ b/docs/en/sql-reference/index.md @@ -11,7 +11,7 @@ ClickHouse supports the following types of queries: - [SELECT](../sql-reference/statements/select/index.md) - [INSERT INTO](../sql-reference/statements/insert-into.md) -- [CREATE](../sql-reference/statements/create.md) +- [CREATE](../sql-reference/statements/create/index.md) - [ALTER](../sql-reference/statements/alter.md#query_language_queries_alter) - [Other types of queries](../sql-reference/statements/misc.md) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 58797853454..d67f1474f10 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -64,7 +64,7 @@ A subquery in the IN clause is always run just one time on a single server. Ther ## NULL Processing {#in-null-processing} -During request processing, the `IN` operator assumes that the result of an operation with [NULL](../../sql-reference/syntax.md#null-literal) always equals `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared if [transform_null_in = 0](../../operations/settings/settings.md#transform_null_in). +During request processing, the `IN` operator assumes that the result of an operation with [NULL](../../sql-reference/syntax.md#null-literal) always equals `0`, regardless of whether `NULL` is on the right or left side of the operator. `NULL` values are not included in any dataset, do not correspond to each other and cannot be compared if [transform\_null\_in = 0](../../operations/settings/settings.md#transform_null_in). Here is an example with the `t_null` table: diff --git a/docs/en/sql-reference/statements/alter.md b/docs/en/sql-reference/statements/alter.md index 52d821a8ab8..9c6d9703e13 100644 --- a/docs/en/sql-reference/statements/alter.md +++ b/docs/en/sql-reference/statements/alter.md @@ -34,7 +34,7 @@ These actions are described in detail below. ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after] ``` -Adds a new column to the table with the specified `name`, `type`, [`codec`](../../sql-reference/statements/create.md#codecs) and `default_expr` (see the section [Default expressions](../../sql-reference/statements/create.md#create-default-values)). +Adds a new column to the table with the specified `name`, `type`, [`codec`](../../sql-reference/statements/create/table.md#codecs) and `default_expr` (see the section [Default expressions](../../sql-reference/statements/create/table.md#create-default-values)). If the `IF NOT EXISTS` clause is included, the query won’t return an error if the column already exists. If you specify `AFTER name_after` (the name of another column), the column is added after the specified one in the list of table columns. Otherwise, the column is added to the end of the table. Note that there is no way to add a column to the beginning of a table. For a chain of actions, `name_after` can be the name of a column that is added in one of the previous actions. @@ -182,7 +182,7 @@ Also, they are replicated (syncing indices metadata through ZooKeeper). ### Manipulations with Constraints {#manipulations-with-constraints} -See more on [constraints](../../sql-reference/statements/create.md#constraints) +See more on [constraints](../../sql-reference/statements/create/table.md#constraints) Constraints could be added or deleted using following syntax: diff --git a/docs/en/sql-reference/statements/create.md b/docs/en/sql-reference/statements/create.md deleted file mode 100644 index 51c4a9db43a..00000000000 --- a/docs/en/sql-reference/statements/create.md +++ /dev/null @@ -1,500 +0,0 @@ ---- -toc_priority: 35 -toc_title: CREATE ---- - -# CREATE Queries {#create-queries} - -## CREATE DATABASE {#query-language-create-database} - -Creates database. - -``` sql -CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] -``` - -### Clauses {#clauses} - -- `IF NOT EXISTS` - If the `db_name` database already exists, then ClickHouse doesn’t create a new database and: - - - Doesn’t throw an exception if clause is specified. - - Throws an exception if clause isn’t specified. - -- `ON CLUSTER` - ClickHouse creates the `db_name` database on all the servers of a specified cluster. - -- `ENGINE` - - - [MySQL](../../engines/database-engines/mysql.md) - Allows you to retrieve data from the remote MySQL server. - By default, ClickHouse uses its own [database engine](../../engines/database-engines/index.md). - -## CREATE TABLE {#create-table-query} - -The `CREATE TABLE` query can have several forms. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec] [TTL expr2], - ... -) ENGINE = engine -``` - -Creates a table named ‘name’ in the ‘db’ database or the current database if ‘db’ is not set, with the structure specified in brackets and the ‘engine’ engine. -The structure of the table is a list of column descriptions, secondary indexes and constraints . If primary key is supported by the engine, it will be indicated as parameter for the table engine. - -A column description is `name type` in the simplest case. Example: `RegionID UInt32`. -Expressions can also be defined for default values (see below). - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine] -``` - -Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table. - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name AS table_function() -``` - -Creates a table with the structure and data returned by a [table function](../../sql-reference/table-functions/index.md#table-functions). - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... -``` - -Creates a table with a structure like the result of the `SELECT` query, with the ‘engine’ engine, and fills it with data from SELECT. - -In all cases, if `IF NOT EXISTS` is specified, the query won’t return an error if the table already exists. In this case, the query won’t do anything. - -There can be other clauses after the `ENGINE` clause in the query. See detailed documentation on how to create tables in the descriptions of [table engines](../../engines/table-engines/index.md#table_engines). - -### Default Values {#create-default-values} - -The column description can specify an expression for a default value, in one of the following ways: `DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`. -Example: `URLDomain String DEFAULT domain(URL)`. - -If an expression for the default value is not defined, the default values will be set to zeros for numbers, empty strings for strings, empty arrays for arrays, and `0000-00-00` for dates or `0000-00-00 00:00:00` for dates with time. NULLs are not supported. - -If the default expression is defined, the column type is optional. If there isn’t an explicitly defined type, the default expression type is used. Example: `EventDate DEFAULT toDate(EventTime)` – the ‘Date’ type will be used for the ‘EventDate’ column. - -If the data type and default expression are defined explicitly, this expression will be cast to the specified type using type casting functions. Example: `Hits UInt32 DEFAULT 0` means the same thing as `Hits UInt32 DEFAULT toUInt32(0)`. - -Default expressions may be defined as an arbitrary expression from table constants and columns. When creating and changing the table structure, it checks that expressions don’t contain loops. For INSERT, it checks that expressions are resolvable – that all columns they can be calculated from have been passed. - -`DEFAULT expr` - -Normal default value. If the INSERT query doesn’t specify the corresponding column, it will be filled in by computing the corresponding expression. - -`MATERIALIZED expr` - -Materialized expression. Such a column can’t be specified for INSERT, because it is always calculated. -For an INSERT without a list of columns, these columns are not considered. -In addition, this column is not substituted when using an asterisk in a SELECT query. This is to preserve the invariant that the dump obtained using `SELECT *` can be inserted back into the table using INSERT without specifying the list of columns. - -`ALIAS expr` - -Synonym. Such a column isn’t stored in the table at all. -Its values can’t be inserted in a table, and it is not substituted when using an asterisk in a SELECT query. -It can be used in SELECTs if the alias is expanded during query parsing. - -When using the ALTER query to add new columns, old data for these columns is not written. Instead, when reading old data that does not have values for the new columns, expressions are computed on the fly by default. However, if running the expressions requires different columns that are not indicated in the query, these columns will additionally be read, but only for the blocks of data that need it. - -If you add a new column to a table but later change its default expression, the values used for old data will change (for data where values were not stored on the disk). Note that when running background merges, data for columns that are missing in one of the merging parts is written to the merged part. - -It is not possible to set default values for elements in nested data structures. - -### Constraints {#constraints} - -Along with columns descriptions constraints could be defined: - -``` sql -CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], - ... - CONSTRAINT constraint_name_1 CHECK boolean_expr_1, - ... -) ENGINE = engine -``` - -`boolean_expr_1` could by any boolean expression. If constraints are defined for the table, each of them will be checked for every row in `INSERT` query. If any constraint is not satisfied — server will raise an exception with constraint name and checking expression. - -Adding large amount of constraints can negatively affect performance of big `INSERT` queries. - -### TTL Expression {#ttl-expression} - -Defines storage time for values. Can be specified only for MergeTree-family tables. For the detailed description, see [TTL for columns and tables](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). - -### Column Compression Codecs {#codecs} - -By default, ClickHouse applies the `lz4` compression method. For `MergeTree`-engine family you can change the default compression method in the [compression](../../operations/server-configuration-parameters/settings.md#server-settings-compression) section of a server configuration. You can also define the compression method for each individual column in the `CREATE TABLE` query. - -``` sql -CREATE TABLE codec_example -( - dt Date CODEC(ZSTD), - ts DateTime CODEC(LZ4HC), - float_value Float32 CODEC(NONE), - double_value Float64 CODEC(LZ4HC(9)) - value Float32 CODEC(Delta, ZSTD) -) -ENGINE = -... -``` - -If a codec is specified, the default codec doesn’t apply. Codecs can be combined in a pipeline, for example, `CODEC(Delta, ZSTD)`. To select the best codec combination for you project, pass benchmarks similar to described in the Altinity [New Encodings to Improve ClickHouse Efficiency](https://www.altinity.com/blog/2019/7/new-encodings-to-improve-clickhouse) article. - -!!! warning "Warning" - You can’t decompress ClickHouse database files with external utilities like `lz4`. Instead, use the special [clickhouse-compressor](https://github.com/ClickHouse/ClickHouse/tree/master/programs/compressor) utility. - -Compression is supported for the following table engines: - -- [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family. Supports column compression codecs and selecting the default compression method by [compression](../../operations/server-configuration-parameters/settings.md#server-settings-compression) settings. -- [Log](../../engines/table-engines/log-family/index.md) family. Uses the `lz4` compression method by default and supports column compression codecs. -- [Set](../../engines/table-engines/special/set.md). Only supported the default compression. -- [Join](../../engines/table-engines/special/join.md). Only supported the default compression. - -ClickHouse supports common purpose codecs and specialized codecs. - -#### Specialized Codecs {#create-query-specialized-codecs} - -These codecs are designed to make compression more effective by using specific features of data. Some of these codecs don’t compress data themself. Instead, they prepare the data for a common purpose codec, which compresses it better than without this preparation. - -Specialized codecs: - -- `Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` are used for storing delta values, so `delta_bytes` is the maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. The default value for `delta_bytes` is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, it’s 1. -- `DoubleDelta` — Calculates delta of deltas and writes it in compact binary form. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-byte deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). -- `Gorilla` — Calculates XOR between current and previous value and writes it in compact binary form. Efficient when storing a series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see Compressing Values in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). -- `T64` — Compression approach that crops unused high bits of values in integer data types (including `Enum`, `Date` and `DateTime`). At each step of its algorithm, codec takes a block of 64 values, puts them into 64x64 bit matrix, transposes it, crops the unused bits of values and returns the rest as a sequence. Unused bits are the bits, that don’t differ between maximum and minimum values in the whole data part for which the compression is used. - -`DoubleDelta` and `Gorilla` codecs are used in Gorilla TSDB as the components of its compressing algorithm. Gorilla approach is effective in scenarios when there is a sequence of slowly changing values with their timestamps. Timestamps are effectively compressed by the `DoubleDelta` codec, and values are effectively compressed by the `Gorilla` codec. For example, to get an effectively stored table, you can create it in the following configuration: - -``` sql -CREATE TABLE codec_example -( - timestamp DateTime CODEC(DoubleDelta), - slow_values Float32 CODEC(Gorilla) -) -ENGINE = MergeTree() -``` - -#### General Purpose Codecs {#create-query-general-purpose-codecs} - -Codecs: - -- `NONE` — No compression. -- `LZ4` — Lossless [data compression algorithm](https://github.com/lz4/lz4) used by default. Applies LZ4 fast compression. -- `LZ4HC[(level)]` — LZ4 HC (high compression) algorithm with configurable level. Default level: 9. Setting `level <= 0` applies the default level. Possible levels: \[1, 12\]. Recommended level range: \[4, 9\]. -- `ZSTD[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable `level`. Possible levels: \[1, 22\]. Default value: 1. - -High compression levels are useful for asymmetric scenarios, like compress once, decompress repeatedly. Higher levels mean better compression and higher CPU usage. - -## Temporary Tables {#temporary-tables} - -ClickHouse supports temporary tables which have the following characteristics: - -- Temporary tables disappear when the session ends, including if the connection is lost. -- A temporary table uses the Memory engine only. -- The DB can’t be specified for a temporary table. It is created outside of databases. -- Impossible to create a temporary table with distributed DDL query on all cluster servers (by using `ON CLUSTER`): this table exists only in the current session. -- If a temporary table has the same name as another one and a query specifies the table name without specifying the DB, the temporary table will be used. -- For distributed query processing, temporary tables used in a query are passed to remote servers. - -To create a temporary table, use the following syntax: - -``` sql -CREATE TEMPORARY TABLE [IF NOT EXISTS] table_name -( - name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], - name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], - ... -) -``` - -In most cases, temporary tables are not created manually, but when using external data for a query, or for distributed `(GLOBAL) IN`. For more information, see the appropriate sections - -It’s possible to use tables with [ENGINE = Memory](../../engines/table-engines/special/memory.md) instead of temporary tables. - -## Distributed DDL Queries (ON CLUSTER Clause) {#distributed-ddl-queries-on-cluster-clause} - -The `CREATE`, `DROP`, `ALTER`, and `RENAME` queries support distributed execution on a cluster. -For example, the following query creates the `all_hits` `Distributed` table on each host in `cluster`: - -``` sql -CREATE TABLE IF NOT EXISTS all_hits ON CLUSTER cluster (p Date, i Int32) ENGINE = Distributed(cluster, default, hits) -``` - -In order to run these queries correctly, each host must have the same cluster definition (to simplify syncing configs, you can use substitutions from ZooKeeper). They must also connect to the ZooKeeper servers. -The local version of the query will eventually be implemented on each host in the cluster, even if some hosts are currently not available. The order for executing queries within a single host is guaranteed. - -## CREATE VIEW {#create-view} - -``` sql -CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]table_name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... -``` - -Creates a view. There are two types of views: normal and MATERIALIZED. - -Normal views don’t store any data, but just perform a read from another table. In other words, a normal view is nothing more than a saved query. When reading from a view, this saved query is used as a subquery in the FROM clause. - -As an example, assume you’ve created a view: - -``` sql -CREATE VIEW view AS SELECT ... -``` - -and written a query: - -``` sql -SELECT a, b, c FROM view -``` - -This query is fully equivalent to using the subquery: - -``` sql -SELECT a, b, c FROM (SELECT ...) -``` - -Materialized views store data transformed by the corresponding SELECT query. - -When creating a materialized view without `TO [db].[table]`, you must specify ENGINE – the table engine for storing data. - -When creating a materialized view with `TO [db].[table]`, you must not use `POPULATE`. - -A materialized view is arranged as follows: when inserting data to the table specified in SELECT, part of the inserted data is converted by this SELECT query, and the result is inserted in the view. - -If you specify POPULATE, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We don’t recommend using POPULATE, since data inserted in the table during the view creation will not be inserted in it. - -A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an ENGINE that independently performs data aggregation, such as `SummingMergeTree`. - -The execution of `ALTER` queries on materialized views has not been fully developed, so they might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view. - -Views look the same as normal tables. For example, they are listed in the result of the `SHOW TABLES` query. - -There isn’t a separate query for deleting views. To delete a view, use `DROP TABLE`. - -## CREATE DICTIONARY {#create-dictionary-query} - -``` sql -CREATE DICTIONARY [IF NOT EXISTS] [db.]dictionary_name [ON CLUSTER cluster] -( - key1 type1 [DEFAULT|EXPRESSION expr1] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], - key2 type2 [DEFAULT|EXPRESSION expr2] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], - attr1 type2 [DEFAULT|EXPRESSION expr3], - attr2 type2 [DEFAULT|EXPRESSION expr4] -) -PRIMARY KEY key1, key2 -SOURCE(SOURCE_NAME([param1 value1 ... paramN valueN])) -LAYOUT(LAYOUT_NAME([param_name param_value])) -LIFETIME([MIN val1] MAX val2) -``` - -Creates [external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) with given [structure](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md), [source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md), [layout](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) and [lifetime](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). - -External dictionary structure consists of attributes. Dictionary attributes are specified similarly to table columns. The only required attribute property is its type, all other properties may have default values. - -Depending on dictionary [layout](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) one or more attributes can be specified as dictionary keys. - -For more information, see [External Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. - -## CREATE USER {#create-user-statement} - -Creates a [user account](../../operations/access-rights.md#user-account-management). - -### Syntax {#create-user-syntax} - -``` sql -CREATE USER [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] - [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] - [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] - [DEFAULT ROLE role [,...]] - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] -``` - -#### Identification {#identification} - -There are multiple ways of user identification: - -- `IDENTIFIED WITH no_password` -- `IDENTIFIED WITH plaintext_password BY 'qwerty'` -- `IDENTIFIED WITH sha256_password BY 'qwerty'` or `IDENTIFIED BY 'password'` -- `IDENTIFIED WITH sha256_hash BY 'hash'` -- `IDENTIFIED WITH double_sha1_password BY 'qwerty'` -- `IDENTIFIED WITH double_sha1_hash BY 'hash'` - -#### User Host {#user-host} - -User host is a host from which a connection to ClickHouse server could be established. The host can be specified in the `HOST` query section in the following ways: - -- `HOST IP 'ip_address_or_subnetwork'` — User can connect to ClickHouse server only from the specified IP address or a [subnetwork](https://en.wikipedia.org/wiki/Subnetwork). Examples: `HOST IP '192.168.0.0/16'`, `HOST IP '2001:DB8::/32'`. For use in production, only specify `HOST IP` elements (IP addresses and their masks), since using `host` and `host_regexp` might cause extra latency. -- `HOST ANY` — User can connect from any location. This is a default option. -- `HOST LOCAL` — User can connect only locally. -- `HOST NAME 'fqdn'` — User host can be specified as FQDN. For example, `HOST NAME 'mysite.com'`. -- `HOST NAME REGEXP 'regexp'` — You can use [pcre](http://www.pcre.org/) regular expressions when specifying user hosts. For example, `HOST NAME REGEXP '.*\.mysite\.com'`. -- `HOST LIKE 'template'` — Allows you to use the [LIKE](../../sql-reference/functions/string-search-functions.md#function-like) operator to filter the user hosts. For example, `HOST LIKE '%'` is equivalent to `HOST ANY`, `HOST LIKE '%.mysite.com'` filters all the hosts in the `mysite.com` domain. - -Another way of specifying host is to use `@` syntax following the username. Examples: - -- `CREATE USER mira@'127.0.0.1'` — Equivalent to the `HOST IP` syntax. -- `CREATE USER mira@'localhost'` — Equivalent to the `HOST LOCAL` syntax. -- `CREATE USER mira@'192.168.%.%'` — Equivalent to the `HOST LIKE` syntax. - -!!! info "Warning" - ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technically you can create multiple users with the same `user_name` and different constructions after `@`. However, we don’t recommend to do so. - -### Examples {#create-user-examples} - -Create the user account `mira` protected by the password `qwerty`: - -``` sql -CREATE USER mira HOST IP '127.0.0.1' IDENTIFIED WITH sha256_password BY 'qwerty' -``` - -`mira` should start client app at the host where the ClickHouse server runs. - -Create the user account `john`, assign roles to it and make this roles default: - -``` sql -CREATE USER john DEFAULT ROLE role1, role2 -``` - -Create the user account `john` and make all his future roles default: - -``` sql -ALTER USER user DEFAULT ROLE ALL -``` - -When some role is assigned to `john` in the future, it will become default automatically. - -Create the user account `john` and make all his future roles default excepting `role1` and `role2`: - -``` sql -ALTER USER john DEFAULT ROLE ALL EXCEPT role1, role2 -``` - -## CREATE ROLE {#create-role-statement} - -Creates a [role](../../operations/access-rights.md#role-management). - -### Syntax {#create-role-syntax} - -``` sql -CREATE ROLE [IF NOT EXISTS | OR REPLACE] name - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] -``` - -### Description {#create-role-description} - -Role is a set of [privileges](../../sql-reference/statements/grant.md#grant-privileges). A user assigned a role gets all the privileges of this role. - -A user can be assigned multiple roles. Users can apply their assigned roles in arbitrary combinations by the [SET ROLE](../../sql-reference/statements/misc.md#set-role-statement) statement. The final scope of privileges is a combined set of all the privileges of all the applied roles. If a user has privileges granted directly to it’s user account, they are also combined with the privileges granted by roles. - -User can have default roles which apply at user login. To set default roles, use the [SET DEFAULT ROLE](../../sql-reference/statements/misc.md#set-default-role-statement) statement or the [ALTER USER](../../sql-reference/statements/alter.md#alter-user-statement) statement. - -To revoke a role, use the [REVOKE](../../sql-reference/statements/revoke.md) statement. - -To delete role, use the [DROP ROLE](../../sql-reference/statements/misc.md#drop-role-statement) statement. The deleted role is being automatically revoked from all the users and roles to which it was assigned. - -### Examples {#create-role-examples} - -``` sql -CREATE ROLE accountant; -GRANT SELECT ON db.* TO accountant; -``` - -This sequence of queries creates the role `accountant` that has the privilege of reading data from the `accounting` database. - -Assigning the role to the user `mira`: - -``` sql -GRANT accountant TO mira; -``` - -After the role is assigned, the user can apply it and execute the allowed queries. For example: - -``` sql -SET ROLE accountant; -SELECT * FROM db.*; -``` - -## CREATE ROW POLICY {#create-row-policy-statement} - -Creates a [filter for rows](../../operations/access-rights.md#row-policy-management), which a user can read from a table. - -### Syntax {#create-row-policy-syntax} - -``` sql -CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name [ON CLUSTER cluster_name] ON [db.]table - [AS {PERMISSIVE | RESTRICTIVE}] - [FOR SELECT] - [USING condition] - [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] -``` - -#### Section AS {#create-row-policy-as} - -Using this section you can create permissive or restrictive policies. - -Permissive policy grants access to rows. Permissive policies which apply to the same table are combined together using the boolean `OR` operator. Policies are permissive by default. - -Restrictive policy restricts access to rows. Restrictive policies which apply to the same table are combined together using the boolean `AND` operator. - -Restrictive policies apply to rows that passed the permissive filters. If you set restrictive policies but no permissive policies, the user can’t get any row from the table. - -#### Section TO {#create-row-policy-to} - -In the section `TO` you can provide a mixed list of roles and users, for example, `CREATE ROW POLICY ... TO accountant, john@localhost`. - -Keyword `ALL` means all the ClickHouse users including current user. Keywords `ALL EXCEPT` allow to exclude some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` - -### Examples {#examples} - -- `CREATE ROW POLICY filter ON mydb.mytable FOR SELECT USING a<1000 TO accountant, john@localhost` -- `CREATE ROW POLICY filter ON mydb.mytable FOR SELECT USING a<1000 TO ALL EXCEPT mira` - -## CREATE QUOTA {#create-quota-statement} - -Creates a [quota](../../operations/access-rights.md#quotas-management) that can be assigned to a user or a role. - -### Syntax {#create-quota-syntax} - -``` sql -CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] - [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] - [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} - {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | - NO LIMITS | TRACKING ONLY} [,...]] - [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] -``` - -### Example {#create-quota-example} - -Limit the maximum number of queries for the current user with 123 queries in 15 months constraint: - -``` sql -CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER -``` - -## CREATE SETTINGS PROFILE {#create-settings-profile-statement} - -Creates a [settings profile](../../operations/access-rights.md#settings-profiles-management) that can be assigned to a user or a role. - -### Syntax {#create-settings-profile-syntax} - -``` sql -CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] - [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...] -``` - -### Example {#create-settings-profile-syntax} - -Create the `max_memory_usage_profile` settings profile with value and constraints for the `max_memory_usage` setting. Assign it to `robin`: - -``` sql -CREATE SETTINGS PROFILE max_memory_usage_profile SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin -``` - -[Original article](https://clickhouse.tech/docs/en/query_language/create/) diff --git a/docs/en/sql-reference/statements/create/database.md b/docs/en/sql-reference/statements/create/database.md new file mode 100644 index 00000000000..e874672471d --- /dev/null +++ b/docs/en/sql-reference/statements/create/database.md @@ -0,0 +1,29 @@ +--- +toc_priority: 1 +toc_title: DATABASE +--- + +# CREATE DATABASE {#query-language-create-database} + +Creates a new database. + +``` sql +CREATE DATABASE [IF NOT EXISTS] db_name [ON CLUSTER cluster] [ENGINE = engine(...)] +``` + +## Clauses {#clauses} + +### IF NOT EXISTS {#if-not-exists} + +If the `db_name` database already exists, then ClickHouse doesn’t create a new database and: + +- Doesn’t throw an exception if clause is specified. +- Throws an exception if clause isn’t specified. + +### ON CLUSTER {#on-cluster} + +ClickHouse creates the `db_name` database on all the servers of a specified cluster. More details in a [Distributed DDL](../../../sql-reference/distributed-ddl.md) article. + +### ENGINE {#engine} + +[MySQL](../../../engines/database-engines/mysql.md) allows you to retrieve data from the remote MySQL server. By default, ClickHouse uses its own [database engine](../../../engines/database-engines/index.md). There’s also a [lazy](../../../engines/database-engines/lazy.md) engine. diff --git a/docs/en/sql-reference/statements/create/dictionary.md b/docs/en/sql-reference/statements/create/dictionary.md new file mode 100644 index 00000000000..5313c59396e --- /dev/null +++ b/docs/en/sql-reference/statements/create/dictionary.md @@ -0,0 +1,32 @@ +--- +toc_priority: 4 +toc_title: DICTIONARY +--- + +# CREATE DICTIONARY {#create-dictionary-query} + +Creates a new [external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) with given [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md), [source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md), [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) and [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). + +Syntax: + +``` sql +CREATE DICTIONARY [IF NOT EXISTS] [db.]dictionary_name [ON CLUSTER cluster] +( + key1 type1 [DEFAULT|EXPRESSION expr1] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], + key2 type2 [DEFAULT|EXPRESSION expr2] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], + attr1 type2 [DEFAULT|EXPRESSION expr3], + attr2 type2 [DEFAULT|EXPRESSION expr4] +) +PRIMARY KEY key1, key2 +SOURCE(SOURCE_NAME([param1 value1 ... paramN valueN])) +LAYOUT(LAYOUT_NAME([param_name param_value])) +LIFETIME([MIN val1] MAX val2) +``` + +External dictionary structure consists of attributes. Dictionary attributes are specified similarly to table columns. The only required attribute property is its type, all other properties may have default values. + +`ON CLUSTER` clause allows creating dictionary on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md). + +Depending on dictionary [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) one or more attributes can be specified as dictionary keys. + +For more information, see [External Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. diff --git a/docs/en/sql-reference/statements/create/index.md b/docs/en/sql-reference/statements/create/index.md new file mode 100644 index 00000000000..71ace2b664c --- /dev/null +++ b/docs/en/sql-reference/statements/create/index.md @@ -0,0 +1,21 @@ +--- +toc_folder_title: CREATE +toc_priority: 35 +toc_title: Overview +--- + +# CREATE Queries {#create-queries} + +Create queries make a new entity of one of the following kinds: + +- [DATABASE](../../../sql-reference/statements/create/database.md) +- [TABLE](../../../sql-reference/statements/create/table.md) +- [VIEW](../../../sql-reference/statements/create/view.md) +- [DICTIONARY](../../../sql-reference/statements/create/dictionary.md) +- [USER](../../../sql-reference/statements/create/user.md) +- [ROLE](../../../sql-reference/statements/create/role.md) +- [ROW POLICY](../../../sql-reference/statements/create/row-policy.md) +- [QUOTA](../../../sql-reference/statements/create/quota.md) +- [SETTINGS PROFILE](../../../sql-reference/statements/create/settings-profile.md) + +[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/create/) diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md new file mode 100644 index 00000000000..209c421a631 --- /dev/null +++ b/docs/en/sql-reference/statements/create/quota.md @@ -0,0 +1,29 @@ +--- +toc_priority: 8 +toc_title: QUOTA +--- + +# CREATE QUOTA {#create-quota-statement} + +Creates a [quota](../../../operations/access-rights.md#quotas-management) that can be assigned to a user or a role. + +Syntax: + +``` sql +CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] + [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] + [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] | + NO LIMITS | TRACKING ONLY} [,...]] + [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] +``` + +`ON CLUSTER` clause allows creating quotas on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md). + +## Example {#create-quota-example} + +Limit the maximum number of queries for the current user with 123 queries in 15 months constraint: + +``` sql +CREATE QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES 123 TO CURRENT_USER +``` diff --git a/docs/en/sql-reference/statements/create/role.md b/docs/en/sql-reference/statements/create/role.md new file mode 100644 index 00000000000..81be1e9b341 --- /dev/null +++ b/docs/en/sql-reference/statements/create/role.md @@ -0,0 +1,47 @@ +--- +toc_priority: 6 +toc_title: ROLE +--- + +# CREATE ROLE {#create-role-statement} + +Creates a new [role](../../../operations/access-rights.md#role-management). Role is a set of [privileges](../../../sql-reference/statements/grant.md#grant-privileges). A [user](../../../sql-reference/statements/create/user.md) assigned a role gets all the privileges of this role. + +Syntax: + +``` sql +CREATE ROLE [IF NOT EXISTS | OR REPLACE] name + [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] +``` + +## Managing Roles {#managing-roles} + +A user can be assigned multiple roles. Users can apply their assigned roles in arbitrary combinations by the [SET ROLE](../../../sql-reference/statements/misc.md#set-role-statement) statement. The final scope of privileges is a combined set of all the privileges of all the applied roles. If a user has privileges granted directly to it’s user account, they are also combined with the privileges granted by roles. + +User can have default roles which apply at user login. To set default roles, use the [SET DEFAULT ROLE](../../../sql-reference/statements/misc.md#set-default-role-statement) statement or the [ALTER USER](../../../sql-reference/statements/alter.md#alter-user-statement) statement. + +To revoke a role, use the [REVOKE](../../../sql-reference/statements/revoke.md) statement. + +To delete role, use the [DROP ROLE](../../../sql-reference/statements/misc.md#drop-role-statement) statement. The deleted role is being automatically revoked from all the users and roles to which it was assigned. + +## Examples {#create-role-examples} + +``` sql +CREATE ROLE accountant; +GRANT SELECT ON db.* TO accountant; +``` + +This sequence of queries creates the role `accountant` that has the privilege of reading data from the `accounting` database. + +Assigning the role to the user `mira`: + +``` sql +GRANT accountant TO mira; +``` + +After the role is assigned, the user can apply it and execute the allowed queries. For example: + +``` sql +SET ROLE accountant; +SELECT * FROM db.*; +``` diff --git a/docs/en/sql-reference/statements/create/row-policy.md b/docs/en/sql-reference/statements/create/row-policy.md new file mode 100644 index 00000000000..b1e7c8e9006 --- /dev/null +++ b/docs/en/sql-reference/statements/create/row-policy.md @@ -0,0 +1,42 @@ +--- +toc_priority: 7 +toc_title: ROW POLICY +--- + +# CREATE ROW POLICY {#create-row-policy-statement} + +Creates a [filter for rows](../../../operations/access-rights.md#row-policy-management), which a user can read from a table. + +Syntax: + +``` sql +CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name [ON CLUSTER cluster_name] ON [db.]table + [AS {PERMISSIVE | RESTRICTIVE}] + [FOR SELECT] + [USING condition] + [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] +``` + +`ON CLUSTER` clause allows creating row policies on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md). + +## AS Clause {#create-row-policy-as} + +Using this section you can create permissive or restrictive policies. + +Permissive policy grants access to rows. Permissive policies which apply to the same table are combined together using the boolean `OR` operator. Policies are permissive by default. + +Restrictive policy restricts access to rows. Restrictive policies which apply to the same table are combined together using the boolean `AND` operator. + +Restrictive policies apply to rows that passed the permissive filters. If you set restrictive policies but no permissive policies, the user can’t get any row from the table. + +## TO Clause {#create-row-policy-to} + +In the section `TO` you can provide a mixed list of roles and users, for example, `CREATE ROW POLICY ... TO accountant, john@localhost`. + +Keyword `ALL` means all the ClickHouse users including current user. Keywords `ALL EXCEPT` allow to exclude some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost` + +## Examples {#examples} + +`CREATE ROW POLICY filter ON mydb.mytable FOR SELECT USING a<1000 TO accountant, john@localhost` + +`CREATE ROW POLICY filter ON mydb.mytable FOR SELECT USING a<1000 TO ALL EXCEPT mira` diff --git a/docs/en/sql-reference/statements/create/settings-profile.md b/docs/en/sql-reference/statements/create/settings-profile.md new file mode 100644 index 00000000000..6489daebc98 --- /dev/null +++ b/docs/en/sql-reference/statements/create/settings-profile.md @@ -0,0 +1,25 @@ +--- +toc_priority: 9 +toc_title: SETTINGS PROFILE +--- + +# CREATE SETTINGS PROFILE {#create-settings-profile-statement} + +Creates a [settings profile](../../../operations/access-rights.md#settings-profiles-management) that can be assigned to a user or a role. + +Syntax: + +``` sql +CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] + [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...] +``` + +`ON CLUSTER` clause allows creating settings profiles on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md). + +## Example {#create-settings-profile-syntax} + +Create the `max_memory_usage_profile` settings profile with value and constraints for the `max_memory_usage` setting and assign it to user `robin`: + +``` sql +CREATE SETTINGS PROFILE max_memory_usage_profile SETTINGS max_memory_usage = 100000001 MIN 90000000 MAX 110000000 TO robin +``` diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md new file mode 100644 index 00000000000..566d280b012 --- /dev/null +++ b/docs/en/sql-reference/statements/create/table.md @@ -0,0 +1,210 @@ +--- +toc_priority: 2 +toc_title: TABLE +--- + +# CREATE TABLE {#create-table-query} + +Creates a new table. This query can have various syntax forms depending on a use case. + +By default, tables are created only on the current server. Distributed DDL queries are implemented as `ON CLUSTER` clause, which is [described separately](../../../sql-reference/distributed-ddl.md). + +## Syntax Forms {#syntax-forms} + +### With Explicit Schema {#with-explicit-schema} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2] [compression_codec] [TTL expr2], + ... +) ENGINE = engine +``` + +Creates a table named `name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine. +The structure of the table is a list of column descriptions, secondary indexes and constraints . If primary key is supported by the engine, it will be indicated as parameter for the table engine. + +A column description is `name type` in the simplest case. Example: `RegionID UInt32`. + +Expressions can also be defined for default values (see below). + +### With a Schema Similar to Other Table {#with-a-schema-similar-to-other-table} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name AS [db2.]name2 [ENGINE = engine] +``` + +Creates a table with the same structure as another table. You can specify a different engine for the table. If the engine is not specified, the same engine will be used as for the `db2.name2` table. + +## From a Table Function {#from-a-table-function} + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name AS table_function() +``` + +Creates a table with the structure and data returned by a [table function](../../../sql-reference/table-functions/index.md#table-functions). + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name ENGINE = engine AS SELECT ... +``` + +Creates a table with a structure like the result of the `SELECT` query, with the `engine` engine, and fills it with data from SELECT. + +In all cases, if `IF NOT EXISTS` is specified, the query won’t return an error if the table already exists. In this case, the query won’t do anything. + +There can be other clauses after the `ENGINE` clause in the query. See detailed documentation on how to create tables in the descriptions of [table engines](../../../engines/table-engines/index.md#table_engines). + +## Default Values {#create-default-values} + +The column description can specify an expression for a default value, in one of the following ways: `DEFAULT expr`, `MATERIALIZED expr`, `ALIAS expr`. + +Example: `URLDomain String DEFAULT domain(URL)`. + +If an expression for the default value is not defined, the default values will be set to zeros for numbers, empty strings for strings, empty arrays for arrays, and `0000-00-00` for dates or `0000-00-00 00:00:00` for dates with time. NULLs are not supported. + +If the default expression is defined, the column type is optional. If there isn’t an explicitly defined type, the default expression type is used. Example: `EventDate DEFAULT toDate(EventTime)` – the ‘Date’ type will be used for the ‘EventDate’ column. + +If the data type and default expression are defined explicitly, this expression will be cast to the specified type using type casting functions. Example: `Hits UInt32 DEFAULT 0` means the same thing as `Hits UInt32 DEFAULT toUInt32(0)`. + +Default expressions may be defined as an arbitrary expression from table constants and columns. When creating and changing the table structure, it checks that expressions don’t contain loops. For INSERT, it checks that expressions are resolvable – that all columns they can be calculated from have been passed. + +### DEFAULT {#default} + +`DEFAULT expr` + +Normal default value. If the INSERT query doesn’t specify the corresponding column, it will be filled in by computing the corresponding expression. + +### MATERIALIZED {#materialized} + +`MATERIALIZED expr` + +Materialized expression. Such a column can’t be specified for INSERT, because it is always calculated. +For an INSERT without a list of columns, these columns are not considered. +In addition, this column is not substituted when using an asterisk in a SELECT query. This is to preserve the invariant that the dump obtained using `SELECT *` can be inserted back into the table using INSERT without specifying the list of columns. + +### ALIAS {#alias} + +`ALIAS expr` + +Synonym. Such a column isn’t stored in the table at all. +Its values can’t be inserted in a table, and it is not substituted when using an asterisk in a SELECT query. +It can be used in SELECTs if the alias is expanded during query parsing. + +When using the ALTER query to add new columns, old data for these columns is not written. Instead, when reading old data that does not have values for the new columns, expressions are computed on the fly by default. However, if running the expressions requires different columns that are not indicated in the query, these columns will additionally be read, but only for the blocks of data that need it. + +If you add a new column to a table but later change its default expression, the values used for old data will change (for data where values were not stored on the disk). Note that when running background merges, data for columns that are missing in one of the merging parts is written to the merged part. + +It is not possible to set default values for elements in nested data structures. + +## Constraints {#constraints} + +Along with columns descriptions constraints could be defined: + +``` sql +CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1] [compression_codec] [TTL expr1], + ... + CONSTRAINT constraint_name_1 CHECK boolean_expr_1, + ... +) ENGINE = engine +``` + +`boolean_expr_1` could by any boolean expression. If constraints are defined for the table, each of them will be checked for every row in `INSERT` query. If any constraint is not satisfied — server will raise an exception with constraint name and checking expression. + +Adding large amount of constraints can negatively affect performance of big `INSERT` queries. + +## TTL Expression {#ttl-expression} + +Defines storage time for values. Can be specified only for MergeTree-family tables. For the detailed description, see [TTL for columns and tables](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). + +## Column Compression Codecs {#codecs} + +By default, ClickHouse applies the `lz4` compression method. For `MergeTree`-engine family you can change the default compression method in the [compression](../../../operations/server-configuration-parameters/settings.md#server-settings-compression) section of a server configuration. You can also define the compression method for each individual column in the `CREATE TABLE` query. + +``` sql +CREATE TABLE codec_example +( + dt Date CODEC(ZSTD), + ts DateTime CODEC(LZ4HC), + float_value Float32 CODEC(NONE), + double_value Float64 CODEC(LZ4HC(9)) + value Float32 CODEC(Delta, ZSTD) +) +ENGINE = +... +``` + +If a codec is specified, the default codec doesn’t apply. Codecs can be combined in a pipeline, for example, `CODEC(Delta, ZSTD)`. To select the best codec combination for you project, pass benchmarks similar to described in the Altinity [New Encodings to Improve ClickHouse Efficiency](https://www.altinity.com/blog/2019/7/new-encodings-to-improve-clickhouse) article. + +!!! warning "Warning" + You can’t decompress ClickHouse database files with external utilities like `lz4`. Instead, use the special [clickhouse-compressor](https://github.com/ClickHouse/ClickHouse/tree/master/programs/compressor) utility. + +Compression is supported for the following table engines: + +- [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md) family. Supports column compression codecs and selecting the default compression method by [compression](../../../operations/server-configuration-parameters/settings.md#server-settings-compression) settings. +- [Log](../../../engines/table-engines/log-family/index.md) family. Uses the `lz4` compression method by default and supports column compression codecs. +- [Set](../../../engines/table-engines/special/set.md). Only supported the default compression. +- [Join](../../../engines/table-engines/special/join.md). Only supported the default compression. + +ClickHouse supports general purpose codecs and specialized codecs. + +### General Purpose Codecs {#create-query-general-purpose-codecs} + +Codecs: + +- `NONE` — No compression. +- `LZ4` — Lossless [data compression algorithm](https://github.com/lz4/lz4) used by default. Applies LZ4 fast compression. +- `LZ4HC[(level)]` — LZ4 HC (high compression) algorithm with configurable level. Default level: 9. Setting `level <= 0` applies the default level. Possible levels: \[1, 12\]. Recommended level range: \[4, 9\]. +- `ZSTD[(level)]` — [ZSTD compression algorithm](https://en.wikipedia.org/wiki/Zstandard) with configurable `level`. Possible levels: \[1, 22\]. Default value: 1. + +High compression levels are useful for asymmetric scenarios, like compress once, decompress repeatedly. Higher levels mean better compression and higher CPU usage. + +### Specialized Codecs {#create-query-specialized-codecs} + +These codecs are designed to make compression more effective by using specific features of data. Some of these codecs don’t compress data themself. Instead, they prepare the data for a common purpose codec, which compresses it better than without this preparation. + +Specialized codecs: + +- `Delta(delta_bytes)` — Compression approach in which raw values are replaced by the difference of two neighboring values, except for the first value that stays unchanged. Up to `delta_bytes` are used for storing delta values, so `delta_bytes` is the maximum size of raw values. Possible `delta_bytes` values: 1, 2, 4, 8. The default value for `delta_bytes` is `sizeof(type)` if equal to 1, 2, 4, or 8. In all other cases, it’s 1. +- `DoubleDelta` — Calculates delta of deltas and writes it in compact binary form. Optimal compression rates are achieved for monotonic sequences with a constant stride, such as time series data. Can be used with any fixed-width type. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. Uses 1 extra bit for 32-byte deltas: 5-bit prefixes instead of 4-bit prefixes. For additional information, see Compressing Time Stamps in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). +- `Gorilla` — Calculates XOR between current and previous value and writes it in compact binary form. Efficient when storing a series of floating point values that change slowly, because the best compression rate is achieved when neighboring values are binary equal. Implements the algorithm used in Gorilla TSDB, extending it to support 64-bit types. For additional information, see Compressing Values in [Gorilla: A Fast, Scalable, In-Memory Time Series Database](http://www.vldb.org/pvldb/vol8/p1816-teller.pdf). +- `T64` — Compression approach that crops unused high bits of values in integer data types (including `Enum`, `Date` and `DateTime`). At each step of its algorithm, codec takes a block of 64 values, puts them into 64x64 bit matrix, transposes it, crops the unused bits of values and returns the rest as a sequence. Unused bits are the bits, that don’t differ between maximum and minimum values in the whole data part for which the compression is used. + +`DoubleDelta` and `Gorilla` codecs are used in Gorilla TSDB as the components of its compressing algorithm. Gorilla approach is effective in scenarios when there is a sequence of slowly changing values with their timestamps. Timestamps are effectively compressed by the `DoubleDelta` codec, and values are effectively compressed by the `Gorilla` codec. For example, to get an effectively stored table, you can create it in the following configuration: + +``` sql +CREATE TABLE codec_example +( + timestamp DateTime CODEC(DoubleDelta), + slow_values Float32 CODEC(Gorilla) +) +ENGINE = MergeTree() +``` + +## Temporary Tables {#temporary-tables} + +ClickHouse supports temporary tables which have the following characteristics: + +- Temporary tables disappear when the session ends, including if the connection is lost. +- A temporary table uses the Memory engine only. +- The DB can’t be specified for a temporary table. It is created outside of databases. +- Impossible to create a temporary table with distributed DDL query on all cluster servers (by using `ON CLUSTER`): this table exists only in the current session. +- If a temporary table has the same name as another one and a query specifies the table name without specifying the DB, the temporary table will be used. +- For distributed query processing, temporary tables used in a query are passed to remote servers. + +To create a temporary table, use the following syntax: + +``` sql +CREATE TEMPORARY TABLE [IF NOT EXISTS] table_name +( + name1 [type1] [DEFAULT|MATERIALIZED|ALIAS expr1], + name2 [type2] [DEFAULT|MATERIALIZED|ALIAS expr2], + ... +) +``` + +In most cases, temporary tables are not created manually, but when using external data for a query, or for distributed `(GLOBAL) IN`. For more information, see the appropriate sections + +It’s possible to use tables with [ENGINE = Memory](../../../engines/table-engines/special/memory.md) instead of temporary tables. diff --git a/docs/en/sql-reference/statements/create/user.md b/docs/en/sql-reference/statements/create/user.md new file mode 100644 index 00000000000..98c29363102 --- /dev/null +++ b/docs/en/sql-reference/statements/create/user.md @@ -0,0 +1,81 @@ +--- +toc_priority: 5 +toc_title: USER +--- + +# CREATE USER {#create-user-statement} + +Creates a [user account](../../../operations/access-rights.md#user-account-management). + +Syntax: + +``` sql +CREATE USER [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name] + [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}] + [HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE] + [DEFAULT ROLE role [,...]] + [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...] +``` + +`ON CLUSTER` clause allows creating users on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md). + +## Identification {#identification} + +There are multiple ways of user identification: + +- `IDENTIFIED WITH no_password` +- `IDENTIFIED WITH plaintext_password BY 'qwerty'` +- `IDENTIFIED WITH sha256_password BY 'qwerty'` or `IDENTIFIED BY 'password'` +- `IDENTIFIED WITH sha256_hash BY 'hash'` +- `IDENTIFIED WITH double_sha1_password BY 'qwerty'` +- `IDENTIFIED WITH double_sha1_hash BY 'hash'` + +## User Host {#user-host} + +User host is a host from which a connection to ClickHouse server could be established. The host can be specified in the `HOST` query section in the following ways: + +- `HOST IP 'ip_address_or_subnetwork'` — User can connect to ClickHouse server only from the specified IP address or a [subnetwork](https://en.wikipedia.org/wiki/Subnetwork). Examples: `HOST IP '192.168.0.0/16'`, `HOST IP '2001:DB8::/32'`. For use in production, only specify `HOST IP` elements (IP addresses and their masks), since using `host` and `host_regexp` might cause extra latency. +- `HOST ANY` — User can connect from any location. This is a default option. +- `HOST LOCAL` — User can connect only locally. +- `HOST NAME 'fqdn'` — User host can be specified as FQDN. For example, `HOST NAME 'mysite.com'`. +- `HOST NAME REGEXP 'regexp'` — You can use [pcre](http://www.pcre.org/) regular expressions when specifying user hosts. For example, `HOST NAME REGEXP '.*\.mysite\.com'`. +- `HOST LIKE 'template'` — Allows you to use the [LIKE](../../../sql-reference/functions/string-search-functions.md#function-like) operator to filter the user hosts. For example, `HOST LIKE '%'` is equivalent to `HOST ANY`, `HOST LIKE '%.mysite.com'` filters all the hosts in the `mysite.com` domain. + +Another way of specifying host is to use `@` syntax following the username. Examples: + +- `CREATE USER mira@'127.0.0.1'` — Equivalent to the `HOST IP` syntax. +- `CREATE USER mira@'localhost'` — Equivalent to the `HOST LOCAL` syntax. +- `CREATE USER mira@'192.168.%.%'` — Equivalent to the `HOST LIKE` syntax. + +!!! info "Warning" + ClickHouse treats `user_name@'address'` as a username as a whole. Thus, technically you can create multiple users with the same `user_name` and different constructions after `@`. However, we don’t recommend to do so. + +## Examples {#create-user-examples} + +Create the user account `mira` protected by the password `qwerty`: + +``` sql +CREATE USER mira HOST IP '127.0.0.1' IDENTIFIED WITH sha256_password BY 'qwerty' +``` + +`mira` should start client app at the host where the ClickHouse server runs. + +Create the user account `john`, assign roles to it and make this roles default: + +``` sql +CREATE USER john DEFAULT ROLE role1, role2 +``` + +Create the user account `john` and make all his future roles default: + +``` sql +ALTER USER user DEFAULT ROLE ALL +``` + +When some role is assigned to `john` in the future, it will become default automatically. + +Create the user account `john` and make all his future roles default excepting `role1` and `role2`: + +``` sql +ALTER USER john DEFAULT ROLE ALL EXCEPT role1, role2 +``` diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md new file mode 100644 index 00000000000..1f77dcb61d1 --- /dev/null +++ b/docs/en/sql-reference/statements/create/view.md @@ -0,0 +1,59 @@ +--- +toc_priority: 3 +toc_title: VIEW +--- + +# CREATE VIEW {#create-view} + +Creates a new view. There are two types of views: normal and materialized. + +Syntax: + +``` sql +CREATE [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]table_name [TO[db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... +``` + +## Normal {#normal} + +Normal views don’t store any data, they just perform a read from another table on each access. In other words, a normal view is nothing more than a saved query. When reading from a view, this saved query is used as a subquery in the [FROM](../../../sql-reference/statements/select/from.md) clause. + +As an example, assume you’ve created a view: + +``` sql +CREATE VIEW view AS SELECT ... +``` + +and written a query: + +``` sql +SELECT a, b, c FROM view +``` + +This query is fully equivalent to using the subquery: + +``` sql +SELECT a, b, c FROM (SELECT ...) +``` + +## Materialized {#materialized} + +Materialized views store data transformed by the corresponding [SELECT](../../../sql-reference/statements/select/index.md) query. + +When creating a materialized view without `TO [db].[table]`, you must specify `ENGINE` – the table engine for storing data. + +When creating a materialized view with `TO [db].[table]`, you must not use `POPULATE`. + +A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view. + +!!! important "Important" + Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) doesn’t change the materialized view. + +If you specify `POPULATE`, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **don’t recommend** using POPULATE, since data inserted in the table during the view creation will not be inserted in it. + +A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`. + +The execution of [ALTER](../../../sql-reference/statements/alter.md) queries on materialized views has limitations, so they might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view. + +Views look the same as normal tables. For example, they are listed in the result of the `SHOW TABLES` query. + +There isn’t a separate query for deleting views. To delete a view, use [DROP TABLE](../../../sql-reference/statements/misc.md#drop). diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 72266561c1b..99cc4b431b1 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -297,7 +297,7 @@ Examples of how this hierarchy is treated: ### CREATE {#grant-create} -Allows executing [CREATE](../../sql-reference/statements/create.md) and [ATTACH](../../sql-reference/statements/misc.md#attach) DDL-queries according to the following hierarchy of privileges: +Allows executing [CREATE](../../sql-reference/statements/create/index.md) and [ATTACH](../../sql-reference/statements/misc.md#attach) DDL-queries according to the following hierarchy of privileges: - `CREATE`. Level: `GROUP` - `CREATE DATABASE`. Level: `DATABASE` diff --git a/docs/en/sql-reference/statements/index.md b/docs/en/sql-reference/statements/index.md index 3a2c09d94cb..1bea6f01b2e 100644 --- a/docs/en/sql-reference/statements/index.md +++ b/docs/en/sql-reference/statements/index.md @@ -1,19 +1,19 @@ --- toc_folder_title: Statements -toc_priority: 31 toc_hidden: true +toc_priority: 31 --- -# ClickHouse SQL Statements +# ClickHouse SQL Statements {#clickhouse-sql-statements} -Statements represent various kinds of action you can perform using SQL queries. Each kind of statement has it's own syntax and usage details that are described separately: +Statements represent various kinds of action you can perform using SQL queries. Each kind of statement has it’s own syntax and usage details that are described separately: -- [SELECT](select/index.md) -- [INSERT INTO](insert-into.md) -- [CREATE](create.md) -- [ALTER](alter.md) -- [SYSTEM](system.md) -- [SHOW](show.md) -- [GRANT](grant.md) -- [REVOKE](revoke.md) -- [Other](misc.md) +- [SELECT](../../sql-reference/statements/select/index.md) +- [INSERT INTO](../../sql-reference/statements/insert-into.md) +- [CREATE](../../sql-reference/statements/create/index.md) +- [ALTER](../../sql-reference/statements/alter.md) +- [SYSTEM](../../sql-reference/statements/system.md) +- [SHOW](../../sql-reference/statements/show.md) +- [GRANT](../../sql-reference/statements/grant.md) +- [REVOKE](../../sql-reference/statements/revoke.md) +- [Other](../../sql-reference/statements/misc.md) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 28a05ff2870..d60437345ef 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -46,7 +46,7 @@ You can insert data separately from the query by using the command-line client o ### Constraints {#constraints} -If table has [constraints](../../sql-reference/statements/create.md#constraints), their expressions will be checked for each row of inserted data. If any of those constraints is not satisfied — server will raise an exception containing constraint name and expression, the query will be stopped. +If table has [constraints](../../sql-reference/statements/create/table.md#constraints), their expressions will be checked for each row of inserted data. If any of those constraints is not satisfied — server will raise an exception containing constraint name and expression, the query will be stopped. ### Inserting the Results of `SELECT` {#insert_query_insert-select} diff --git a/docs/en/sql-reference/statements/misc.md b/docs/en/sql-reference/statements/misc.md index 59d07f5eebb..d26583f4fbe 100644 --- a/docs/en/sql-reference/statements/misc.md +++ b/docs/en/sql-reference/statements/misc.md @@ -69,7 +69,7 @@ Returns the following `String` type columns: - `name` — Column name. - `type`— Column type. -- `default_type` — Clause that is used in [default expression](../../sql-reference/statements/create.md#create-default-values) (`DEFAULT`, `MATERIALIZED` or `ALIAS`). Column contains an empty string, if the default expression isn’t specified. +- `default_type` — Clause that is used in [default expression](../../sql-reference/statements/create/table.md#create-default-values) (`DEFAULT`, `MATERIALIZED` or `ALIAS`). Column contains an empty string, if the default expression isn’t specified. - `default_expression` — Value specified in the `DEFAULT` clause. - `comment_expression` — Comment text. diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 65af3f75e73..27dcb5742fd 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -114,7 +114,7 @@ If user is not specified, the query returns privileges for the current user. ## SHOW CREATE USER {#show-create-user-statement} -Shows parameters that were used at a [user creation](../../sql-reference/statements/create.md#create-user-statement). +Shows parameters that were used at a [user creation](../../sql-reference/statements/create/user.md). `SHOW CREATE USER` doesn’t output user passwords. @@ -126,7 +126,7 @@ SHOW CREATE USER [name | CURRENT_USER] ## SHOW CREATE ROLE {#show-create-role-statement} -Shows parameters that were used at a [role creation](../../sql-reference/statements/create.md#create-role-statement). +Shows parameters that were used at a [role creation](../../sql-reference/statements/create/role.md). ### Syntax {#show-create-role-syntax} @@ -136,7 +136,7 @@ SHOW CREATE ROLE name ## SHOW CREATE ROW POLICY {#show-create-row-policy-statement} -Shows parameters that were used at a [row policy creation](../../sql-reference/statements/create.md#create-row-policy-statement). +Shows parameters that were used at a [row policy creation](../../sql-reference/statements/create/row-policy.md). ### Syntax {#show-create-row-policy-syntax} @@ -146,7 +146,7 @@ SHOW CREATE [ROW] POLICY name ON [database.]table ## SHOW CREATE QUOTA {#show-create-quota-statement} -Shows parameters that were used at a [quota creation](../../sql-reference/statements/create.md#create-quota-statement). +Shows parameters that were used at a [quota creation](../../sql-reference/statements/create/quota.md). ### Syntax {#show-create-row-policy-syntax} @@ -156,7 +156,7 @@ SHOW CREATE QUOTA [name | CURRENT] ## SHOW CREATE SETTINGS PROFILE {#show-create-settings-profile-statement} -Shows parameters that were used at a [settings profile creation](../../sql-reference/statements/create.md#create-settings-profile-statement). +Shows parameters that were used at a [settings profile creation](../../sql-reference/statements/create/settings-profile.md). ### Syntax {#show-create-row-policy-syntax} diff --git a/docs/en/sql-reference/statements/system.md b/docs/en/sql-reference/statements/system.md index a0aac0cfd8e..e0cbba3a39e 100644 --- a/docs/en/sql-reference/statements/system.md +++ b/docs/en/sql-reference/statements/system.md @@ -72,19 +72,19 @@ Resets the mark cache. Used in development of ClickHouse and performance tests. Dead replicas can be dropped using following syntax: -```sql +``` sql SYSTEM DROP REPLICA 'replica_name' FROM TABLE database.table; SYSTEM DROP REPLICA 'replica_name' FROM DATABASE database; SYSTEM DROP REPLICA 'replica_name'; SYSTEM DROP REPLICA 'replica_name' FROM ZKPATH '/path/to/table/in/zk'; ``` -Queries will remove the replica path in ZooKeeper. It's useful when replica is dead and its metadata cannot be removed from ZooKeeper by `DROP TABLE` because there is no such table anymore. It will only drop the inactive/stale replica, and it can't drop local replica, please use `DROP TABLE` for that. `DROP REPLICA` does not drop any tables and does not remove any data or metadata from disk. +Queries will remove the replica path in ZooKeeper. It’s useful when replica is dead and its metadata cannot be removed from ZooKeeper by `DROP TABLE` because there is no such table anymore. It will only drop the inactive/stale replica, and it can’t drop local replica, please use `DROP TABLE` for that. `DROP REPLICA` does not drop any tables and does not remove any data or metadata from disk. -The first one removes metadata of `'replica_name'` replica of `database.table` table. +The first one removes metadata of `'replica_name'` replica of `database.table` table. The second one does the same for all replicated tables in the database. The third one does the same for all replicated tables on local server. -The forth one is useful to remove metadata of dead replica when all other replicas of a table were dropped. It requires the table path to be specified explicitly. It must be the same path as was passed to the first argument of `ReplicatedMergeTree` engine on table creation. +The forth one is useful to remove metadata of dead replica when all other replicas of a table were dropped. It requires the table path to be specified explicitly. It must be the same path as was passed to the first argument of `ReplicatedMergeTree` engine on table creation. ## DROP UNCOMPRESSED CACHE {#query_language-system-drop-uncompressed-cache} diff --git a/docs/en/sql-reference/table-functions/cluster.md b/docs/en/sql-reference/table-functions/cluster.md index 2cd5b14caa4..9fe50a2ad2e 100644 --- a/docs/en/sql-reference/table-functions/cluster.md +++ b/docs/en/sql-reference/table-functions/cluster.md @@ -6,12 +6,11 @@ toc_title: cluster # cluster, clusterAllReplicas {#cluster-clusterallreplicas} Allows to access all shards in an existing cluster which configured in `remote_servers` section without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. One replica of each shard is queried. -`clusterAllReplicas` - same as `cluster` but all replicas are queried. Each replica in a cluster is used as separate shard/connection. +`clusterAllReplicas` - same as `cluster` but all replicas are queried. Each replica in a cluster is used as separate shard/connection. !!! note "Note" All available clusters are listed in the `system.clusters` table. - Signatures: ``` sql @@ -21,7 +20,7 @@ clusterAllReplicas('cluster_name', db.table) clusterAllReplicas('cluster_name', db, table) ``` -`cluster_name` – Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. +`cluster_name` – Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. Using the `cluster` and `clusterAllReplicas` table functions are less efficient than creating a `Distributed` table because in this case, the server connection is re-established for every request. When processing a large number of queries, please always create the `Distributed` table ahead of time, and don’t use the `cluster` and `clusterAllReplicas` table functions. diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index 630bf6d8c27..c3208d132b3 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -14,7 +14,7 @@ You can use table functions in: The method for creating a temporary table that is available only in the current query. The table is deleted when the query finishes. -- [CREATE TABLE AS \](../../sql-reference/statements/create.md#create-table-query) query. +- [CREATE TABLE AS \](../../sql-reference/statements/create/table.md) query. It's one of the methods of creating a table. diff --git a/docs/en/whats-new/index.md b/docs/en/whats-new/index.md index b523f9b3df0..8033fdf71d9 100644 --- a/docs/en/whats-new/index.md +++ b/docs/en/whats-new/index.md @@ -3,8 +3,6 @@ toc_folder_title: What's New toc_priority: 82 --- -# What's New In ClickHouse? - -There's a short high-level [roadmap](roadmap.md) and a detailed [changelog](changelog/index.md) for releases that have already been published. - +# What’s New in ClickHouse? {#whats-new-in-clickhouse} +There’s a short high-level [roadmap](../whats-new/roadmap.md) and a detailed [changelog](../whats-new/changelog/index.md) for releases that have already been published. diff --git a/docs/ja/sql-reference/statements/grant.md b/docs/ja/sql-reference/statements/grant.md deleted file mode 120000 index f2acbe125b4..00000000000 --- a/docs/ja/sql-reference/statements/grant.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/sql-reference/statements/grant.md \ No newline at end of file diff --git a/docs/ja/sql-reference/statements/grant.md b/docs/ja/sql-reference/statements/grant.md new file mode 100644 index 00000000000..63ac7f3aafa --- /dev/null +++ b/docs/ja/sql-reference/statements/grant.md @@ -0,0 +1,476 @@ +--- +machine_translated: true +machine_translated_rev: cbd8aa9052361a7ee11c209560cff7175c2b8e42 +toc_priority: 39 +toc_title: GRANT +--- + +# GRANT {#grant} + +- 助成金 [特権](#grant-privileges) ClickHouseユーザーアカウントまたはロールへ。 +- 員の役割をユーザーアカウントまたはその他の役割です。 + +権限を取り消すには [REVOKE](../../sql-reference/statements/revoke.md) 声明。 また、付与された権限を [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants-statement) 声明。 + +## 権限構文の付与 {#grant-privigele-syntax} + +``` sql +GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO {user | role | CURRENT_USER} [,...] [WITH GRANT OPTION] +``` + +- `privilege` — Type of privilege. +- `role` — ClickHouse user role. +- `user` — ClickHouse user account. + +この `WITH GRANT OPTION` 句の付与 `user` または `role` 実行する許可を得て `GRANT` クエリ。 ユーザーは、持っているスコープとそれ以下の権限を付与できます。 + +## ロール構文の割り当て {#assign-role-syntax} + +``` sql +GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_USER} [,...] [WITH ADMIN OPTION] +``` + +- `role` — ClickHouse user role. +- `user` — ClickHouse user account. + +この `WITH ADMIN OPTION` 句の付与 [ADMIN OPTION](#admin-option-privilege) への特権 `user` または `role`. + +## 使用法 {#grant-usage} + +使用するには `GRANT` アカウントには `GRANT OPTION` 特権だ 権限を付与できるのは、アカウント権限の範囲内でのみです。 + +たとえば、管理者は `john` クエリによるアカウント: + +``` sql +GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION +``` + +つまり `john` 実行する権限があります: + +- `SELECT x,y FROM db.table`. +- `SELECT x FROM db.table`. +- `SELECT y FROM db.table`. + +`john` 実行できない `SELECT z FROM db.table`. この `SELECT * FROM db.table` また、利用できません。 このクエリを処理すると、ClickHouseはデータを返しません。 `x` そして `y`. 唯一の例外は、テーブルにのみ含まれている場合です `x` そして `y` 列。 この場合、ClickHouseはすべてのデータを返します。 + +また `john` は、 `GRANT OPTION` そのため、同じまたはより小さいスコープの特権を持つ他のユーザーに付与できます。 + +アスタリスクを使用できる権限の指定 (`*`)テーブルまたはデータベース名の代わりに。 例えば、 `GRANT SELECT ON db.* TO john` クエリ許可 `john` 実行するには `SELECT` すべてのテーブルに対するクエリ `db` データベース。 また、データベース名を省略できます。 この場合、現在のデータベースに特権が付与されます。 例えば, `GRANT SELECT ON * TO john` 現在のデータベ, `GRANT SELECT ON mytable TO john` の権限を付与します。 `mytable` 現在のデータベース内のテーブル。 + +へのアクセス `system` databaseは常に許可されます(このデータベースはクエリの処理に使用されるため)。 + +一つのクエリで複数のアカウントに複数の権限を付与できます。 クエリ `GRANT SELECT, INSERT ON *.* TO john, robin` アカウントを許可 `john` そして `robin` 実行するには `INSERT` そして `SELECT` クエリのテーブルのすべてのデータベースに、サーバーにコピーします。 + +## 特権 {#grant-privileges} + +特権は、特定の種類のクエリを実行する権限です。 + +権限には階層構造があります。 許可されるクエリのセットは、特権スコープに依存します。 + +特権の階層: + +- [SELECT](#grant-select) +- [INSERT](#grant-insert) +- [ALTER](#grant-alter) + - `ALTER TABLE` + - `ALTER UPDATE` + - `ALTER DELETE` + - `ALTER COLUMN` + - `ALTER ADD COLUMN` + - `ALTER DROP COLUMN` + - `ALTER MODIFY COLUMN` + - `ALTER COMMENT COLUMN` + - `ALTER CLEAR COLUMN` + - `ALTER RENAME COLUMN` + - `ALTER INDEX` + - `ALTER ORDER BY` + - `ALTER ADD INDEX` + - `ALTER DROP INDEX` + - `ALTER MATERIALIZE INDEX` + - `ALTER CLEAR INDEX` + - `ALTER CONSTRAINT` + - `ALTER ADD CONSTRAINT` + - `ALTER DROP CONSTRAINT` + - `ALTER TTL` + - `ALTER MATERIALIZE TTL` + - `ALTER SETTINGS` + - `ALTER MOVE PARTITION` + - `ALTER FETCH PARTITION` + - `ALTER FREEZE PARTITION` + - `ALTER VIEW` + - `ALTER VIEW REFRESH` + - `ALTER VIEW MODIFY QUERY` +- [CREATE](#grant-create) + - `CREATE DATABASE` + - `CREATE TABLE` + - `CREATE VIEW` + - `CREATE DICTIONARY` + - `CREATE TEMPORARY TABLE` +- [DROP](#grant-drop) + - `DROP DATABASE` + - `DROP TABLE` + - `DROP VIEW` + - `DROP DICTIONARY` +- [TRUNCATE](#grant-truncate) +- [OPTIMIZE](#grant-optimize) +- [SHOW](#grant-show) + - `SHOW DATABASES` + - `SHOW TABLES` + - `SHOW COLUMNS` + - `SHOW DICTIONARIES` +- [KILL QUERY](#grant-kill-query) +- [ACCESS MANAGEMENT](#grant-access-management) + - `CREATE USER` + - `ALTER USER` + - `DROP USER` + - `CREATE ROLE` + - `ALTER ROLE` + - `DROP ROLE` + - `CREATE ROW POLICY` + - `ALTER ROW POLICY` + - `DROP ROW POLICY` + - `CREATE QUOTA` + - `ALTER QUOTA` + - `DROP QUOTA` + - `CREATE SETTINGS PROFILE` + - `ALTER SETTINGS PROFILE` + - `DROP SETTINGS PROFILE` + - `SHOW ACCESS` + - `SHOW_USERS` + - `SHOW_ROLES` + - `SHOW_ROW_POLICIES` + - `SHOW_QUOTAS` + - `SHOW_SETTINGS_PROFILES` + - `ROLE ADMIN` +- [SYSTEM](#grant-system) + - `SYSTEM SHUTDOWN` + - `SYSTEM DROP CACHE` + - `SYSTEM DROP DNS CACHE` + - `SYSTEM DROP MARK CACHE` + - `SYSTEM DROP UNCOMPRESSED CACHE` + - `SYSTEM RELOAD` + - `SYSTEM RELOAD CONFIG` + - `SYSTEM RELOAD DICTIONARY` + - `SYSTEM RELOAD EMBEDDED DICTIONARIES` + - `SYSTEM MERGES` + - `SYSTEM TTL MERGES` + - `SYSTEM FETCHES` + - `SYSTEM MOVES` + - `SYSTEM SENDS` + - `SYSTEM DISTRIBUTED SENDS` + - `SYSTEM REPLICATED SENDS` + - `SYSTEM REPLICATION QUEUES` + - `SYSTEM SYNC REPLICA` + - `SYSTEM RESTART REPLICA` + - `SYSTEM FLUSH` + - `SYSTEM FLUSH DISTRIBUTED` + - `SYSTEM FLUSH LOGS` +- [INTROSPECTION](#grant-introspection) + - `addressToLine` + - `addressToSymbol` + - `demangle` +- [SOURCES](#grant-sources) + - `FILE` + - `URL` + - `REMOTE` + - `YSQL` + - `ODBC` + - `JDBC` + - `HDFS` + - `S3` +- [dictGet](#grant-dictget) + +この階層がどのように扱われるかの例: + +- この `ALTER` 特典を含む他のすべての `ALTER*` 特権だ +- `ALTER CONSTRAINT` 含む `ALTER ADD CONSTRAINT` そして `ALTER DROP CONSTRAINT` 特権だ + +特権は異なるレベルで適用されます。 レベルを知ることは、特権に利用可能な構文を示唆しています。 + +レベル(下位から上位へ): + +- `COLUMN` — Privilege can be granted for column, table, database, or globally. +- `TABLE` — Privilege can be granted for table, database, or globally. +- `VIEW` — Privilege can be granted for view, database, or globally. +- `DICTIONARY` — Privilege can be granted for dictionary, database, or globally. +- `DATABASE` — Privilege can be granted for database or globally. +- `GLOBAL` — Privilege can be granted only globally. +- `GROUP` — Groups privileges of different levels. When `GROUP`-レベルの特権が付与され、使用される構文に対応するグループからの特権のみが付与されます。 + +許可される構文の例: + +- `GRANT SELECT(x) ON db.table TO user` +- `GRANT SELECT ON db.* TO user` + +禁止された構文の例: + +- `GRANT CREATE USER(x) ON db.table TO user` +- `GRANT CREATE USER ON db.* TO user` + +特別特典 [ALL](#grant-all) ユーザーアカウントまたはロールにすべての権限を付与します。 + +既定では、ユーザーアカウントまたはロールには特権はありません。 + +ユーザーまたはロールに権限がない場合は、次のように表示されます [NONE](#grant-none) 特権だ + +実装によるクエリには、一連の特権が必要です。 たとえば、 [RENAME](../../sql-reference/statements/misc.md#misc_operations-rename) クエリには次の権限が必要です: `SELECT`, `CREATE TABLE`, `INSERT` そして `DROP TABLE`. + +### SELECT {#grant-select} + +実行を許可する [SELECT](../../sql-reference/statements/select/index.md) クエリ。 + +特権レベル: `COLUMN`. + +**説明** + +ユーザーの交付を受けるこの権限での実行 `SELECT` 指定した表およびデータベース内の指定した列のリストに対するクエリ。 ユーザーが他の列を含む場合、クエリはデータを返しません。 + +次の特権を考慮してください: + +``` sql +GRANT SELECT(x,y) ON db.table TO john +``` + +この特権は `john` 実行するには `SELECT` データを含むクエリ `x` および/または `y` の列 `db.table` 例えば, `SELECT x FROM db.table`. `john` 実行できない `SELECT z FROM db.table`. この `SELECT * FROM db.table` また、利用できません。 このクエリを処理すると、ClickHouseはデータを返しません。 `x` そして `y`. 唯一の例外は、テーブルにのみ含まれている場合です `x` そして `y` この場合、ClickHouseはすべてのデータを返します。 + +### INSERT {#grant-insert} + +実行を許可する [INSERT](../../sql-reference/statements/insert-into.md) クエリ。 + +特権レベル: `COLUMN`. + +**説明** + +ユーザーの交付を受けるこの権限での実行 `INSERT` 指定した表およびデータベース内の指定した列のリストに対するクエリ。 ユーザーが他の列を含む場合、指定されたクエリはデータを挿入しません。 + +**例** + +``` sql +GRANT INSERT(x,y) ON db.table TO john +``` + +許可された特権は `john` にデータを挿入するには `x` および/または `y` の列 `db.table`. + +### ALTER {#grant-alter} + +実行を許可する [ALTER](../../sql-reference/statements/alter.md) 次の特権の階層に従ってクエリを実行します: + +- `ALTER`. レベル: `COLUMN`. + - `ALTER TABLE`. レベル: `GROUP` + - `ALTER UPDATE`. レベル: `COLUMN`. 別名: `UPDATE` + - `ALTER DELETE`. レベル: `COLUMN`. 別名: `DELETE` + - `ALTER COLUMN`. レベル: `GROUP` + - `ALTER ADD COLUMN`. レベル: `COLUMN`. 別名: `ADD COLUMN` + - `ALTER DROP COLUMN`. レベル: `COLUMN`. 別名: `DROP COLUMN` + - `ALTER MODIFY COLUMN`. レベル: `COLUMN`. 別名: `MODIFY COLUMN` + - `ALTER COMMENT COLUMN`. レベル: `COLUMN`. 別名: `COMMENT COLUMN` + - `ALTER CLEAR COLUMN`. レベル: `COLUMN`. 別名: `CLEAR COLUMN` + - `ALTER RENAME COLUMN`. レベル: `COLUMN`. 別名: `RENAME COLUMN` + - `ALTER INDEX`. レベル: `GROUP`. 別名: `INDEX` + - `ALTER ORDER BY`. レベル: `TABLE`. 別名: `ALTER MODIFY ORDER BY`, `MODIFY ORDER BY` + - `ALTER ADD INDEX`. レベル: `TABLE`. 別名: `ADD INDEX` + - `ALTER DROP INDEX`. レベル: `TABLE`. 別名: `DROP INDEX` + - `ALTER MATERIALIZE INDEX`. レベル: `TABLE`. 別名: `MATERIALIZE INDEX` + - `ALTER CLEAR INDEX`. レベル: `TABLE`. 別名: `CLEAR INDEX` + - `ALTER CONSTRAINT`. レベル: `GROUP`. 別名: `CONSTRAINT` + - `ALTER ADD CONSTRAINT`. レベル: `TABLE`. 別名: `ADD CONSTRAINT` + - `ALTER DROP CONSTRAINT`. レベル: `TABLE`. 別名: `DROP CONSTRAINT` + - `ALTER TTL`. レベル: `TABLE`. 別名: `ALTER MODIFY TTL`, `MODIFY TTL` + - `ALTER MATERIALIZE TTL`. レベル: `TABLE`. 別名: `MATERIALIZE TTL` + - `ALTER SETTINGS`. レベル: `TABLE`. 別名: `ALTER SETTING`, `ALTER MODIFY SETTING`, `MODIFY SETTING` + - `ALTER MOVE PARTITION`. レベル: `TABLE`. 別名: `ALTER MOVE PART`, `MOVE PARTITION`, `MOVE PART` + - `ALTER FETCH PARTITION`. レベル: `TABLE`. 別名: `FETCH PARTITION` + - `ALTER FREEZE PARTITION`. レベル: `TABLE`. 別名: `FREEZE PARTITION` + - `ALTER VIEW` レベル: `GROUP` + - `ALTER VIEW REFRESH`. レベル: `VIEW`. 別名: `ALTER LIVE VIEW REFRESH`, `REFRESH VIEW` + - `ALTER VIEW MODIFY QUERY`. レベル: `VIEW`. 別名: `ALTER TABLE MODIFY QUERY` + +この階層がどのように扱われるかの例: + +- この `ALTER` 特典を含む他のすべての `ALTER*` 特権だ +- `ALTER CONSTRAINT` 含む `ALTER ADD CONSTRAINT` そして `ALTER DROP CONSTRAINT` 特権だ + +**ノート** + +- この `MODIFY SETTING` 権限を変更できるテーブルエンジンを設定します。 設定やサーバー構成パラメーターには影響しません。 +- この `ATTACH` 操作は必要とします [CREATE](#grant-create) 特権だ +- この `DETACH` 操作は必要とします [DROP](#grant-drop) 特権だ +- によって突然変異を停止するには [KILL MUTATION](../../sql-reference/statements/misc.md#kill-mutation) クエリ、あなたはこの突然変異を開始する権限を持っている必要があります。 たとえば、 `ALTER UPDATE` 問い合わせ、必要とします `ALTER UPDATE`, `ALTER TABLE`,または `ALTER` 特権だ + +### CREATE {#grant-create} + +実行を許可する [CREATE](../../sql-reference/statements/create.md) そして [ATTACH](../../sql-reference/statements/misc.md#attach) DDL-次の特権の階層に従ったクエリ: + +- `CREATE`. レベル: `GROUP` + - `CREATE DATABASE`. レベル: `DATABASE` + - `CREATE TABLE`. レベル: `TABLE` + - `CREATE VIEW`. レベル: `VIEW` + - `CREATE DICTIONARY`. レベル: `DICTIONARY` + - `CREATE TEMPORARY TABLE`. レベル: `GLOBAL` + +**ノート** + +- 削除し、作成したテーブルは、ユーザーニーズ [DROP](#grant-drop). + +### DROP {#grant-drop} + +実行を許可する [DROP](../../sql-reference/statements/misc.md#drop) そして [DETACH](../../sql-reference/statements/misc.md#detach) 次の特権の階層に従ってクエリを実行します: + +- `DROP`. レベル: + - `DROP DATABASE`. レベル: `DATABASE` + - `DROP TABLE`. レベル: `TABLE` + - `DROP VIEW`. レベル: `VIEW` + - `DROP DICTIONARY`. レベル: `DICTIONARY` + +### TRUNCATE {#grant-truncate} + +実行を許可する [TRUNCATE](../../sql-reference/statements/misc.md#truncate-statement) クエリ。 + +特権レベル: `TABLE`. + +### OPTIMIZE {#grant-optimize} + +実行を許可する [OPTIMIZE TABLE](../../sql-reference/statements/misc.md#misc_operations-optimize) クエリ。 + +特権レベル: `TABLE`. + +### SHOW {#grant-show} + +実行を許可する `SHOW`, `DESCRIBE`, `USE`,and `EXISTS` 次の特権の階層に従ってクエリを実行します: + +- `SHOW`. レベル: `GROUP` + - `SHOW DATABASES`. レベル: `DATABASE`. 実行を許可する `SHOW DATABASES`, `SHOW CREATE DATABASE`, `USE ` クエリ。 + - `SHOW TABLES`. レベル: `TABLE`. 実行を許可する `SHOW TABLES`, `EXISTS `, `CHECK
    ` クエリ。 + - `SHOW COLUMNS`. レベル: `COLUMN`. 実行を許可する `SHOW CREATE TABLE`, `DESCRIBE` クエリ。 + - `SHOW DICTIONARIES`. レベル: `DICTIONARY`. 実行を許可する `SHOW DICTIONARIES`, `SHOW CREATE DICTIONARY`, `EXISTS ` クエリ。 + +**ノート** + +ユーザは、 `SHOW` 特典の場合は、その他の特典に関する指定されたテーブル、辞書やデータベースです。 + +### KILL QUERY {#grant-kill-query} + +実行を許可する [KILL](../../sql-reference/statements/misc.md#kill-query-statement) 次の特権の階層に従ってクエリを実行します: + +特権レベル: `GLOBAL`. + +**ノート** + +`KILL QUERY` 特典でユーザを殺クエリのその他のユーザー + +### ACCESS MANAGEMENT {#grant-access-management} + +ユーザー、ロール、および行ポリシーを管理するクエリを実行できます。 + +- `ACCESS MANAGEMENT`. レベル: `GROUP` + - `CREATE USER`. レベル: `GLOBAL` + - `ALTER USER`. レベル: `GLOBAL` + - `DROP USER`. レベル: `GLOBAL` + - `CREATE ROLE`. レベル: `GLOBAL` + - `ALTER ROLE`. レベル: `GLOBAL` + - `DROP ROLE`. レベル: `GLOBAL` + - `ROLE ADMIN`. レベル: `GLOBAL` + - `CREATE ROW POLICY`. レベル: `GLOBAL`. 別名: `CREATE POLICY` + - `ALTER ROW POLICY`. レベル: `GLOBAL`. 別名: `ALTER POLICY` + - `DROP ROW POLICY`. レベル: `GLOBAL`. 別名: `DROP POLICY` + - `CREATE QUOTA`. レベル: `GLOBAL` + - `ALTER QUOTA`. レベル: `GLOBAL` + - `DROP QUOTA`. レベル: `GLOBAL` + - `CREATE SETTINGS PROFILE`. レベル: `GLOBAL`. 別名: `CREATE PROFILE` + - `ALTER SETTINGS PROFILE`. レベル: `GLOBAL`. 別名: `ALTER PROFILE` + - `DROP SETTINGS PROFILE`. レベル: `GLOBAL`. 別名: `DROP PROFILE` + - `SHOW ACCESS`. レベル: `GROUP` + - `SHOW_USERS`. レベル: `GLOBAL`. 別名: `SHOW CREATE USER` + - `SHOW_ROLES`. レベル: `GLOBAL`. 別名: `SHOW CREATE ROLE` + - `SHOW_ROW_POLICIES`. レベル: `GLOBAL`. 別名: `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY` + - `SHOW_QUOTAS`. レベル: `GLOBAL`. 別名: `SHOW CREATE QUOTA` + - `SHOW_SETTINGS_PROFILES`. レベル: `GLOBAL`. 別名: `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW CREATE PROFILE` + +この `ROLE ADMIN` 特典できるユーザーに割り当ておよび取り消す為の役割を含めるものではありませんので、ユーザーの管理のオプションです。 + +### SYSTEM {#grant-system} + +ユーザーに実行を許可する [SYSTEM](../../sql-reference/statements/system.md) 次の特権の階層に従ってクエリを実行します。 + +- `SYSTEM`. レベル: `GROUP` + - `SYSTEM SHUTDOWN`. レベル: `GLOBAL`. 別名: `SYSTEM KILL`, `SHUTDOWN` + - `SYSTEM DROP CACHE`. 別名: `DROP CACHE` + - `SYSTEM DROP DNS CACHE`. レベル: `GLOBAL`. 別名: `SYSTEM DROP DNS`, `DROP DNS CACHE`, `DROP DNS` + - `SYSTEM DROP MARK CACHE`. レベル: `GLOBAL`. 別名: `SYSTEM DROP MARK`, `DROP MARK CACHE`, `DROP MARKS` + - `SYSTEM DROP UNCOMPRESSED CACHE`. レベル: `GLOBAL`. 別名: `SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, `DROP UNCOMPRESSED` + - `SYSTEM RELOAD`. レベル: `GROUP` + - `SYSTEM RELOAD CONFIG`. レベル: `GLOBAL`. 別名: `RELOAD CONFIG` + - `SYSTEM RELOAD DICTIONARY`. レベル: `GLOBAL`. 別名: `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARY`, `RELOAD DICTIONARIES` + - `SYSTEM RELOAD EMBEDDED DICTIONARIES`. レベル: `GLOBAL`. 別名:R`ELOAD EMBEDDED DICTIONARIES` + - `SYSTEM MERGES`. レベル: `TABLE`. 別名: `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, `START MERGES` + - `SYSTEM TTL MERGES`. レベル: `TABLE`. 別名: `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, `START TTL MERGES` + - `SYSTEM FETCHES`. レベル: `TABLE`. 別名: `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, `START FETCHES` + - `SYSTEM MOVES`. レベル: `TABLE`. 別名: `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, `START MOVES` + - `SYSTEM SENDS`. レベル: `GROUP`. 別名: `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, `START SENDS` + - `SYSTEM DISTRIBUTED SENDS`. レベル: `TABLE`. 別名: `SYSTEM STOP DISTRIBUTED SENDS`, `SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, `START DISTRIBUTED SENDS` + - `SYSTEM REPLICATED SENDS`. レベル: `TABLE`. 別名: `SYSTEM STOP REPLICATED SENDS`, `SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, `START REPLICATED SENDS` + - `SYSTEM REPLICATION QUEUES`. レベル: `TABLE`. 別名: `SYSTEM STOP REPLICATION QUEUES`, `SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, `START REPLICATION QUEUES` + - `SYSTEM SYNC REPLICA`. レベル: `TABLE`. 別名: `SYNC REPLICA` + - `SYSTEM RESTART REPLICA`. レベル: `TABLE`. 別名: `RESTART REPLICA` + - `SYSTEM FLUSH`. レベル: `GROUP` + - `SYSTEM FLUSH DISTRIBUTED`. レベル: `TABLE`. 別名: `FLUSH DISTRIBUTED` + - `SYSTEM FLUSH LOGS`. レベル: `GLOBAL`. 別名: `FLUSH LOGS` + +この `SYSTEM RELOAD EMBEDDED DICTIONARIES` によって暗黙的に付与される特権 `SYSTEM RELOAD DICTIONARY ON *.*` 特権だ + +### INTROSPECTION {#grant-introspection} + +使用を許可する [内省](../../operations/optimizing-performance/sampling-query-profiler.md) 機能。 + +- `INTROSPECTION`. レベル: `GROUP`. 別名: `INTROSPECTION FUNCTIONS` + - `addressToLine`. レベル: `GLOBAL` + - `addressToSymbol`. レベル: `GLOBAL` + - `demangle`. レベル: `GLOBAL` + +### SOURCES {#grant-sources} + +外部データソースの使用を許可します。 に適用されます [表エンジン](../../engines/table-engines/index.md) そして [テーブル関数](../../sql-reference/table-functions/index.md#table-functions). + +- `SOURCES`. レベル: `GROUP` + - `FILE`. レベル: `GLOBAL` + - `URL`. レベル: `GLOBAL` + - `REMOTE`. レベル: `GLOBAL` + - `YSQL`. レベル: `GLOBAL` + - `ODBC`. レベル: `GLOBAL` + - `JDBC`. レベル: `GLOBAL` + - `HDFS`. レベル: `GLOBAL` + - `S3`. レベル: `GLOBAL` + +この `SOURCES` 特権は、すべてのソースの使用を可能にします。 また、各ソースに個別に権限を付与することもできます。 ソースを使用するには、追加の権限が必要です。 + +例: + +- テーブルを作成するには [MySQLテーブルエンジン](../../engines/table-engines/integrations/mysql.md)、必要とします `CREATE TABLE (ON db.table_name)` そして `MYSQL` 特権だ +- を使用するには [mysqlテーブル関数](../../sql-reference/table-functions/mysql.md)、必要とします `CREATE TEMPORARY TABLE` そして `MYSQL` 特権だ + +### dictGet {#grant-dictget} + +- `dictGet`. 別名: `dictHas`, `dictGetHierarchy`, `dictIsIn` + +ユーザーに実行を許可する [dictGet](../../sql-reference/functions/ext-dict-functions.md#dictget), [ディクタス](../../sql-reference/functions/ext-dict-functions.md#dicthas), [dictGetHierarchy](../../sql-reference/functions/ext-dict-functions.md#dictgethierarchy), [ジクチシン](../../sql-reference/functions/ext-dict-functions.md#dictisin) 機能。 + +特権レベル: `DICTIONARY`. + +**例** + +- `GRANT dictGet ON mydb.mydictionary TO john` +- `GRANT dictGet ON mydictionary TO john` + +### ALL {#grant-all} + +助成金の全ての権限を規制組織のユーザーアカウントまたは役割を担う。 + +### NONE {#grant-none} + +権限は付与されません。 + +### ADMIN OPTION {#admin-option-privilege} + +この `ADMIN OPTION` 特典できるユーザー補助金の役割を他のユーザーです。 + +[元の記事](https://clickhouse.tech/docs/en/query_language/grant/) diff --git a/docs/ru/getting-started/tutorial.md b/docs/ru/getting-started/tutorial.md deleted file mode 120000 index 7af416c9878..00000000000 --- a/docs/ru/getting-started/tutorial.md +++ /dev/null @@ -1 +0,0 @@ -../../en/getting-started/tutorial.md \ No newline at end of file diff --git a/docs/ru/getting-started/tutorial.md b/docs/ru/getting-started/tutorial.md new file mode 100644 index 00000000000..bc71d762138 --- /dev/null +++ b/docs/ru/getting-started/tutorial.md @@ -0,0 +1,664 @@ +--- +toc_priority: 12 +toc_title: Tutorial +--- + +# ClickHouse Tutorial {#clickhouse-tutorial} + +## What to Expect from This Tutorial? {#what-to-expect-from-this-tutorial} + +By going through this tutorial, you’ll learn how to set up a simple ClickHouse cluster. It’ll be small, but fault-tolerant and scalable. Then we will use one of the example datasets to fill it with data and execute some demo queries. + +## Single Node Setup {#single-node-setup} + +To postpone the complexities of a distributed environment, we’ll start with deploying ClickHouse on a single server or virtual machine. ClickHouse is usually installed from [deb](../getting-started/install.md#install-from-deb-packages) or [rpm](../getting-started/install.md#from-rpm-packages) packages, but there are [alternatives](../getting-started/install.md#from-docker-image) for the operating systems that do no support them. + +For example, you have chosen `deb` packages and executed: + +``` bash +{% include 'install/deb.sh' %} +``` + +What do we have in the packages that got installed: + +- `clickhouse-client` package contains [clickhouse-client](../interfaces/cli.md) application, interactive ClickHouse console client. +- `clickhouse-common` package contains a ClickHouse executable file. +- `clickhouse-server` package contains configuration files to run ClickHouse as a server. + +Server config files are located in `/etc/clickhouse-server/`. Before going further, please notice the `` element in `config.xml`. Path determines the location for data storage, so it should be located on volume with large disk capacity; the default value is `/var/lib/clickhouse/`. If you want to adjust the configuration, it’s not handy to directly edit `config.xml` file, considering it might get rewritten on future package updates. The recommended way to override the config elements is to create [files in config.d directory](../operations/configuration-files.md) which serve as “patches” to config.xml. + +As you might have noticed, `clickhouse-server` is not launched automatically after package installation. It won’t be automatically restarted after updates, either. The way you start the server depends on your init system, usually, it is: + +``` bash +sudo service clickhouse-server start +``` + +or + +``` bash +sudo /etc/init.d/clickhouse-server start +``` + +The default location for server logs is `/var/log/clickhouse-server/`. The server is ready to handle client connections once it logs the `Ready for connections` message. + +Once the `clickhouse-server` is up and running, we can use `clickhouse-client` to connect to the server and run some test queries like `SELECT "Hello, world!";`. + +
    + +Quick tips for clickhouse-client + +Interactive mode: + +``` bash +clickhouse-client +clickhouse-client --host=... --port=... --user=... --password=... +``` + +Enable multiline queries: + +``` bash +clickhouse-client -m +clickhouse-client --multiline +``` + +Run queries in batch-mode: + +``` bash +clickhouse-client --query='SELECT 1' +echo 'SELECT 1' | clickhouse-client +clickhouse-client <<< 'SELECT 1' +``` + +Insert data from a file in specified format: + +``` bash +clickhouse-client --query='INSERT INTO table VALUES' < data.txt +clickhouse-client --query='INSERT INTO table FORMAT TabSeparated' < data.tsv +``` + +
    + +## Import Sample Dataset {#import-sample-dataset} + +Now it’s time to fill our ClickHouse server with some sample data. In this tutorial, we’ll use the anonymized data of Yandex.Metrica, the first service that runs ClickHouse in production way before it became open-source (more on that in [history section](../introduction/history.md)). There are [multiple ways to import Yandex.Metrica dataset](../getting-started/example-datasets/metrica.md), and for the sake of the tutorial, we’ll go with the most realistic one. + +### Download and Extract Table Data {#download-and-extract-table-data} + +``` bash +curl https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv +curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv +``` + +The extracted files are about 10GB in size. + +### Create Tables {#create-tables} + +As in most databases management systems, ClickHouse logically groups tables into “databases”. There’s a `default` database, but we’ll create a new one named `tutorial`: + +``` bash +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS tutorial" +``` + +Syntax for creating tables is way more complicated compared to databases (see [reference](../sql-reference/statements/create.md). In general `CREATE TABLE` statement has to specify three key things: + +1. Name of table to create. +2. Table schema, i.e. list of columns and their [data types](../sql-reference/data-types/index.md). +3. [Table engine](../engines/table-engines/index.md) and its settings, which determines all the details on how queries to this table will be physically executed. + +Yandex.Metrica is a web analytics service, and sample dataset doesn’t cover its full functionality, so there are only two tables to create: + +- `hits` is a table with each action done by all users on all websites covered by the service. +- `visits` is a table that contains pre-built sessions instead of individual actions. + +Let’s see and execute the real create table queries for these tables: + +``` sql +CREATE TABLE tutorial.hits_v1 +( + `WatchID` UInt64, + `JavaEnable` UInt8, + `Title` String, + `GoodEvent` Int16, + `EventTime` DateTime, + `EventDate` Date, + `CounterID` UInt32, + `ClientIP` UInt32, + `ClientIP6` FixedString(16), + `RegionID` UInt32, + `UserID` UInt64, + `CounterClass` Int8, + `OS` UInt8, + `UserAgent` UInt8, + `URL` String, + `Referer` String, + `URLDomain` String, + `RefererDomain` String, + `Refresh` UInt8, + `IsRobot` UInt8, + `RefererCategories` Array(UInt16), + `URLCategories` Array(UInt16), + `URLRegions` Array(UInt32), + `RefererRegions` Array(UInt32), + `ResolutionWidth` UInt16, + `ResolutionHeight` UInt16, + `ResolutionDepth` UInt8, + `FlashMajor` UInt8, + `FlashMinor` UInt8, + `FlashMinor2` String, + `NetMajor` UInt8, + `NetMinor` UInt8, + `UserAgentMajor` UInt16, + `UserAgentMinor` FixedString(2), + `CookieEnable` UInt8, + `JavascriptEnable` UInt8, + `IsMobile` UInt8, + `MobilePhone` UInt8, + `MobilePhoneModel` String, + `Params` String, + `IPNetworkID` UInt32, + `TraficSourceID` Int8, + `SearchEngineID` UInt16, + `SearchPhrase` String, + `AdvEngineID` UInt8, + `IsArtifical` UInt8, + `WindowClientWidth` UInt16, + `WindowClientHeight` UInt16, + `ClientTimeZone` Int16, + `ClientEventTime` DateTime, + `SilverlightVersion1` UInt8, + `SilverlightVersion2` UInt8, + `SilverlightVersion3` UInt32, + `SilverlightVersion4` UInt16, + `PageCharset` String, + `CodeVersion` UInt32, + `IsLink` UInt8, + `IsDownload` UInt8, + `IsNotBounce` UInt8, + `FUniqID` UInt64, + `HID` UInt32, + `IsOldCounter` UInt8, + `IsEvent` UInt8, + `IsParameter` UInt8, + `DontCountHits` UInt8, + `WithHash` UInt8, + `HitColor` FixedString(1), + `UTCEventTime` DateTime, + `Age` UInt8, + `Sex` UInt8, + `Income` UInt8, + `Interests` UInt16, + `Robotness` UInt8, + `GeneralInterests` Array(UInt16), + `RemoteIP` UInt32, + `RemoteIP6` FixedString(16), + `WindowName` Int32, + `OpenerName` Int32, + `HistoryLength` Int16, + `BrowserLanguage` FixedString(2), + `BrowserCountry` FixedString(2), + `SocialNetwork` String, + `SocialAction` String, + `HTTPError` UInt16, + `SendTiming` Int32, + `DNSTiming` Int32, + `ConnectTiming` Int32, + `ResponseStartTiming` Int32, + `ResponseEndTiming` Int32, + `FetchTiming` Int32, + `RedirectTiming` Int32, + `DOMInteractiveTiming` Int32, + `DOMContentLoadedTiming` Int32, + `DOMCompleteTiming` Int32, + `LoadEventStartTiming` Int32, + `LoadEventEndTiming` Int32, + `NSToDOMContentLoadedTiming` Int32, + `FirstPaintTiming` Int32, + `RedirectCount` Int8, + `SocialSourceNetworkID` UInt8, + `SocialSourcePage` String, + `ParamPrice` Int64, + `ParamOrderID` String, + `ParamCurrency` FixedString(3), + `ParamCurrencyID` UInt16, + `GoalsReached` Array(UInt32), + `OpenstatServiceName` String, + `OpenstatCampaignID` String, + `OpenstatAdID` String, + `OpenstatSourceID` String, + `UTMSource` String, + `UTMMedium` String, + `UTMCampaign` String, + `UTMContent` String, + `UTMTerm` String, + `FromTag` String, + `HasGCLID` UInt8, + `RefererHash` UInt64, + `URLHash` UInt64, + `CLID` UInt32, + `YCLID` UInt64, + `ShareService` String, + `ShareURL` String, + `ShareTitle` String, + `ParsedParams` Nested( + Key1 String, + Key2 String, + Key3 String, + Key4 String, + Key5 String, + ValueDouble Float64), + `IslandID` FixedString(16), + `RequestNum` UInt32, + `RequestTry` UInt8 +) +ENGINE = MergeTree() +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 +``` + +``` sql +CREATE TABLE tutorial.visits_v1 +( + `CounterID` UInt32, + `StartDate` Date, + `Sign` Int8, + `IsNew` UInt8, + `VisitID` UInt64, + `UserID` UInt64, + `StartTime` DateTime, + `Duration` UInt32, + `UTCStartTime` DateTime, + `PageViews` Int32, + `Hits` Int32, + `IsBounce` UInt8, + `Referer` String, + `StartURL` String, + `RefererDomain` String, + `StartURLDomain` String, + `EndURL` String, + `LinkURL` String, + `IsDownload` UInt8, + `TraficSourceID` Int8, + `SearchEngineID` UInt16, + `SearchPhrase` String, + `AdvEngineID` UInt8, + `PlaceID` Int32, + `RefererCategories` Array(UInt16), + `URLCategories` Array(UInt16), + `URLRegions` Array(UInt32), + `RefererRegions` Array(UInt32), + `IsYandex` UInt8, + `GoalReachesDepth` Int32, + `GoalReachesURL` Int32, + `GoalReachesAny` Int32, + `SocialSourceNetworkID` UInt8, + `SocialSourcePage` String, + `MobilePhoneModel` String, + `ClientEventTime` DateTime, + `RegionID` UInt32, + `ClientIP` UInt32, + `ClientIP6` FixedString(16), + `RemoteIP` UInt32, + `RemoteIP6` FixedString(16), + `IPNetworkID` UInt32, + `SilverlightVersion3` UInt32, + `CodeVersion` UInt32, + `ResolutionWidth` UInt16, + `ResolutionHeight` UInt16, + `UserAgentMajor` UInt16, + `UserAgentMinor` UInt16, + `WindowClientWidth` UInt16, + `WindowClientHeight` UInt16, + `SilverlightVersion2` UInt8, + `SilverlightVersion4` UInt16, + `FlashVersion3` UInt16, + `FlashVersion4` UInt16, + `ClientTimeZone` Int16, + `OS` UInt8, + `UserAgent` UInt8, + `ResolutionDepth` UInt8, + `FlashMajor` UInt8, + `FlashMinor` UInt8, + `NetMajor` UInt8, + `NetMinor` UInt8, + `MobilePhone` UInt8, + `SilverlightVersion1` UInt8, + `Age` UInt8, + `Sex` UInt8, + `Income` UInt8, + `JavaEnable` UInt8, + `CookieEnable` UInt8, + `JavascriptEnable` UInt8, + `IsMobile` UInt8, + `BrowserLanguage` UInt16, + `BrowserCountry` UInt16, + `Interests` UInt16, + `Robotness` UInt8, + `GeneralInterests` Array(UInt16), + `Params` Array(String), + `Goals` Nested( + ID UInt32, + Serial UInt32, + EventTime DateTime, + Price Int64, + OrderID String, + CurrencyID UInt32), + `WatchIDs` Array(UInt64), + `ParamSumPrice` Int64, + `ParamCurrency` FixedString(3), + `ParamCurrencyID` UInt16, + `ClickLogID` UInt64, + `ClickEventID` Int32, + `ClickGoodEvent` Int32, + `ClickEventTime` DateTime, + `ClickPriorityID` Int32, + `ClickPhraseID` Int32, + `ClickPageID` Int32, + `ClickPlaceID` Int32, + `ClickTypeID` Int32, + `ClickResourceID` Int32, + `ClickCost` UInt32, + `ClickClientIP` UInt32, + `ClickDomainID` UInt32, + `ClickURL` String, + `ClickAttempt` UInt8, + `ClickOrderID` UInt32, + `ClickBannerID` UInt32, + `ClickMarketCategoryID` UInt32, + `ClickMarketPP` UInt32, + `ClickMarketCategoryName` String, + `ClickMarketPPName` String, + `ClickAWAPSCampaignName` String, + `ClickPageName` String, + `ClickTargetType` UInt16, + `ClickTargetPhraseID` UInt64, + `ClickContextType` UInt8, + `ClickSelectType` Int8, + `ClickOptions` String, + `ClickGroupBannerID` Int32, + `OpenstatServiceName` String, + `OpenstatCampaignID` String, + `OpenstatAdID` String, + `OpenstatSourceID` String, + `UTMSource` String, + `UTMMedium` String, + `UTMCampaign` String, + `UTMContent` String, + `UTMTerm` String, + `FromTag` String, + `HasGCLID` UInt8, + `FirstVisit` DateTime, + `PredLastVisit` Date, + `LastVisit` Date, + `TotalVisits` UInt32, + `TraficSource` Nested( + ID Int8, + SearchEngineID UInt16, + AdvEngineID UInt8, + PlaceID UInt16, + SocialSourceNetworkID UInt8, + Domain String, + SearchPhrase String, + SocialSourcePage String), + `Attendance` FixedString(16), + `CLID` UInt32, + `YCLID` UInt64, + `NormalizedRefererHash` UInt64, + `SearchPhraseHash` UInt64, + `RefererDomainHash` UInt64, + `NormalizedStartURLHash` UInt64, + `StartURLDomainHash` UInt64, + `NormalizedEndURLHash` UInt64, + `TopLevelDomain` UInt64, + `URLScheme` UInt64, + `OpenstatServiceNameHash` UInt64, + `OpenstatCampaignIDHash` UInt64, + `OpenstatAdIDHash` UInt64, + `OpenstatSourceIDHash` UInt64, + `UTMSourceHash` UInt64, + `UTMMediumHash` UInt64, + `UTMCampaignHash` UInt64, + `UTMContentHash` UInt64, + `UTMTermHash` UInt64, + `FromHash` UInt64, + `WebVisorEnabled` UInt8, + `WebVisorActivity` UInt32, + `ParsedParams` Nested( + Key1 String, + Key2 String, + Key3 String, + Key4 String, + Key5 String, + ValueDouble Float64), + `Market` Nested( + Type UInt8, + GoalID UInt32, + OrderID String, + OrderPrice Int64, + PP UInt32, + DirectPlaceID UInt32, + DirectOrderID UInt32, + DirectBannerID UInt32, + GoodID String, + GoodName String, + GoodQuantity Int32, + GoodPrice Int64), + `IslandID` FixedString(16) +) +ENGINE = CollapsingMergeTree(Sign) +PARTITION BY toYYYYMM(StartDate) +ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 +``` + +You can execute those queries using the interactive mode of `clickhouse-client` (just launch it in a terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you want. + +As we can see, `hits_v1` uses the [basic MergeTree engine](../engines/table-engines/mergetree-family/mergetree.md), while the `visits_v1` uses the [Collapsing](../engines/table-engines/mergetree-family/collapsingmergetree.md) variant. + +### Import Data {#import-data} + +Data import to ClickHouse is done via [INSERT INTO](../sql-reference/statements/insert-into.md) query like in many other SQL databases. However, data is usually provided in one of the [supported serialization formats](../interfaces/formats.md) instead of `VALUES` clause (which is also supported). + +The files we downloaded earlier are in tab-separated format, so here’s how to import them via console client: + +``` bash +clickhouse-client --query "INSERT INTO tutorial.hits_v1 FORMAT TSV" --max_insert_block_size=100000 < hits_v1.tsv +clickhouse-client --query "INSERT INTO tutorial.visits_v1 FORMAT TSV" --max_insert_block_size=100000 < visits_v1.tsv +``` + +ClickHouse has a lot of [settings to tune](../operations/settings/index.md) and one way to specify them in console client is via arguments, as we can see with `--max_insert_block_size`. The easiest way to figure out what settings are available, what do they mean and what the defaults are is to query the `system.settings` table: + +``` sql +SELECT name, value, changed, description +FROM system.settings +WHERE name LIKE '%max_insert_b%' +FORMAT TSV + +max_insert_block_size 1048576 0 "The maximum block size for insertion, if we control the creation of blocks for insertion." +``` + +Optionally you can [OPTIMIZE](../sql-reference/statements/misc.md#misc_operations-optimize) the tables after import. Tables that are configured with an engine from MergeTree-family always do merges of data parts in the background to optimize data storage (or at least check if it makes sense). These queries force the table engine to do storage optimization right now instead of some time later: + +``` bash +clickhouse-client --query "OPTIMIZE TABLE tutorial.hits_v1 FINAL" +clickhouse-client --query "OPTIMIZE TABLE tutorial.visits_v1 FINAL" +``` + +These queries start an I/O and CPU intensive operation, so if the table consistently receives new data, it’s better to leave it alone and let merges run in the background. + +Now we can check if the table import was successful: + +``` bash +clickhouse-client --query "SELECT COUNT(*) FROM tutorial.hits_v1" +clickhouse-client --query "SELECT COUNT(*) FROM tutorial.visits_v1" +``` + +## Example Queries {#example-queries} + +``` sql +SELECT + StartURL AS URL, + AVG(Duration) AS AvgDuration +FROM tutorial.visits_v1 +WHERE StartDate BETWEEN '2014-03-23' AND '2014-03-30' +GROUP BY URL +ORDER BY AvgDuration DESC +LIMIT 10 +``` + +``` sql +SELECT + sum(Sign) AS visits, + sumIf(Sign, has(Goals.ID, 1105530)) AS goal_visits, + (100. * goal_visits) / visits AS goal_percent +FROM tutorial.visits_v1 +WHERE (CounterID = 912887) AND (toYYYYMM(StartDate) = 201403) AND (domain(StartURL) = 'yandex.ru') +``` + +## Cluster Deployment {#cluster-deployment} + +ClickHouse cluster is a homogenous cluster. Steps to set up: + +1. Install ClickHouse server on all machines of the cluster +2. Set up cluster configs in configuration files +3. Create local tables on each instance +4. Create a [Distributed table](../engines/table-engines/special/distributed.md) + +[Distributed table](../engines/table-engines/special/distributed.md) is actually a kind of “view” to local tables of ClickHouse cluster. SELECT query from a distributed table executes using resources of all cluster’s shards. You may specify configs for multiple clusters and create multiple distributed tables providing views to different clusters. + +Example config for a cluster with three shards, one replica each: + +``` xml + + + + + example-perftest01j.yandex.ru + 9000 + + + + + example-perftest02j.yandex.ru + 9000 + + + + + example-perftest03j.yandex.ru + 9000 + + + + +``` + +For further demonstration, let’s create a new local table with the same `CREATE TABLE` query that we used for `hits_v1`, but different table name: + +``` sql +CREATE TABLE tutorial.hits_local (...) ENGINE = MergeTree() ... +``` + +Creating a distributed table providing a view into local tables of the cluster: + +``` sql +CREATE TABLE tutorial.hits_all AS tutorial.hits_local +ENGINE = Distributed(perftest_3shards_1replicas, tutorial, hits_local, rand()); +``` + +A common practice is to create similar Distributed tables on all machines of the cluster. It allows running distributed queries on any machine of the cluster. Also there’s an alternative option to create temporary distributed table for a given SELECT query using [remote](../sql-reference/table-functions/remote.md) table function. + +Let’s run [INSERT SELECT](../sql-reference/statements/insert-into.md) into the Distributed table to spread the table to multiple servers. + +``` sql +INSERT INTO tutorial.hits_all SELECT * FROM tutorial.hits_v1; +``` + +!!! warning "Notice" + This approach is not suitable for the sharding of large tables. There’s a separate tool [clickhouse-copier](../operations/utilities/clickhouse-copier.md) that can re-shard arbitrary large tables. + +As you could expect, computationally heavy queries run N times faster if they utilize 3 servers instead of one. + +In this case, we have used a cluster with 3 shards, and each contains a single replica. + +To provide resilience in a production environment, we recommend that each shard should contain 2-3 replicas spread between multiple availability zones or datacenters (or at least racks). Note that ClickHouse supports an unlimited number of replicas. + +Example config for a cluster of one shard containing three replicas: + +``` xml + + ... + + + + example-perftest01j.yandex.ru + 9000 + + + example-perftest02j.yandex.ru + 9000 + + + example-perftest03j.yandex.ru + 9000 + + + + +``` + +To enable native replication [ZooKeeper](http://zookeeper.apache.org/) is required. ClickHouse takes care of data consistency on all replicas and runs restore procedure after failure automatically. It’s recommended to deploy the ZooKeeper cluster on separate servers (where no other processes including ClickHouse are running). + +!!! note "Note" + ZooKeeper is not a strict requirement: in some simple cases, you can duplicate the data by writing it into all the replicas from your application code. This approach is **not** recommended, in this case, ClickHouse won’t be able to guarantee data consistency on all replicas. Thus it becomes the responsibility of your application. + +ZooKeeper locations are specified in the configuration file: + +``` xml + + + zoo01.yandex.ru + 2181 + + + zoo02.yandex.ru + 2181 + + + zoo03.yandex.ru + 2181 + + +``` + +Also, we need to set macros for identifying each shard and replica which are used on table creation: + +``` xml + + 01 + 01 + +``` + +If there are no replicas at the moment on replicated table creation, a new first replica is instantiated. If there are already live replicas, the new replica clones data from existing ones. You have an option to create all replicated tables first, and then insert data to it. Another option is to create some replicas and add the others after or during data insertion. + +``` sql +CREATE TABLE tutorial.hits_replica (...) +ENGINE = ReplcatedMergeTree( + '/clickhouse_perftest/tables/{shard}/hits', + '{replica}' +) +... +``` + +Here we use [ReplicatedMergeTree](../engines/table-engines/mergetree-family/replication.md) table engine. In parameters we specify ZooKeeper path containing shard and replica identifiers. + +``` sql +INSERT INTO tutorial.hits_replica SELECT * FROM tutorial.hits_local; +``` + +Replication operates in multi-master mode. Data can be loaded into any replica, and the system then syncs it with other instances automatically. Replication is asynchronous so at a given moment, not all replicas may contain recently inserted data. At least one replica should be up to allow data ingestion. Others will sync up data and repair consistency once they will become active again. Note that this approach allows for the low possibility of a loss of recently inserted data. + +[Original article](https://clickhouse.tech/docs/en/getting_started/tutorial/) diff --git a/docs/ru/sql-reference/statements/grant.md b/docs/ru/sql-reference/statements/grant.md index a328975ec12..72266561c1b 100644 --- a/docs/ru/sql-reference/statements/grant.md +++ b/docs/ru/sql-reference/statements/grant.md @@ -1,479 +1,474 @@ -# GRANT +--- +toc_priority: 39 +toc_title: GRANT +--- -- Присваивает [привилегии](#grant-privileges) пользователям или ролям ClickHouse. -- Назначает роли пользователям или другим ролям. +# GRANT {#grant} -Отозвать привилегию можно с помощью выражения [REVOKE](revoke.md). Чтобы вывести список присвоенных привилегий, воспользуйтесь выражением [SHOW GRANTS](show.md#show-grants-statement). +- Grants [privileges](#grant-privileges) to ClickHouse user accounts or roles. +- Assigns roles to user accounts or to the other roles. -## Синтаксис присвоения привилегий {#grant-privigele-syntax} +To revoke privileges, use the [REVOKE](../../sql-reference/statements/revoke.md) statement. Also you can list granted privileges with the [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants-statement) statement. -```sql +## Granting Privilege Syntax {#grant-privigele-syntax} + +``` sql GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO {user | role | CURRENT_USER} [,...] [WITH GRANT OPTION] ``` -- `privilege` — Тип привилегии -- `role` — Роль пользователя ClickHouse. -- `user` — Пользователь ClickHouse. +- `privilege` — Type of privilege. +- `role` — ClickHouse user role. +- `user` — ClickHouse user account. -`WITH GRANT OPTION` разрешает пользователю или роли выполнять запрос `GRANT`. Пользователь может выдавать только те привилегии, которые есть у него, той же или меньшей области действий. +The `WITH GRANT OPTION` clause grants `user` or `role` with permission to execute the `GRANT` query. Users can grant privileges of the same scope they have and less. +## Assigning Role Syntax {#assign-role-syntax} -## Синтаксис назначения ролей {#assign-role-syntax} - -```sql +``` sql GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_USER} [,...] [WITH ADMIN OPTION] ``` -- `role` — Роль пользователя ClickHouse. -- `user` — Пользователь ClickHouse. +- `role` — ClickHouse user role. +- `user` — ClickHouse user account. -`WITH ADMIN OPTION` присваивает привилегию [ADMIN OPTION](#admin-option-privilege) пользователю или роли. +The `WITH ADMIN OPTION` clause grants [ADMIN OPTION](#admin-option-privilege) privilege to `user` or `role`. -## Использование {#grant-usage} +## Usage {#grant-usage} -Для использования `GRANT` пользователь должен иметь привилегию `GRANT OPTION`. Пользователь может выдавать привилегии только внутри области действий назначенных ему самому привилегий. +To use `GRANT`, your account must have the `GRANT OPTION` privilege. You can grant privileges only inside the scope of your account privileges. -Например, администратор выдал привилегию пользователю `john`: +For example, administrator has granted privileges to the `john` account by the query: -```sql +``` sql GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION ``` -Это означает, что пользователю `john` разрешено выполнять: +It means that `john` has the permission to execute: -- `SELECT x,y FROM db.table`. -- `SELECT x FROM db.table`. -- `SELECT y FROM db.table`. +- `SELECT x,y FROM db.table`. +- `SELECT x FROM db.table`. +- `SELECT y FROM db.table`. -`john` не может выполнить `SELECT z FROM db.table` или `SELECT * FROM db.table`. После обработки данных запросов ClickHouse ничего не вернет — даже `x` или `y`. Единственное исключение — если таблица содержит только столбцы `x` и `y`. В таком случае ClickHouse вернет все данные. +`john` can’t execute `SELECT z FROM db.table`. The `SELECT * FROM db.table` also is not available. Processing this query, ClickHouse doesn’t return any data, even `x` and `y`. The only exception is if a table contains only `x` and `y` columns. In this case ClickHouse returns all the data. -Также у `john` есть привилегия `GRANT OPTION`. `john` может выдать другим пользователям привилегии той же или меньшей области действий из тех, которые есть у него. +Also `john` has the `GRANT OPTION` privilege, so it can grant other users with privileges of the same or smaller scope. -При присвоении привилегий допускается использовать астериск (`*`) вместо имени таблицы или базы данных. Например, запрос `GRANT SELECT ON db.* TO john` позволит пользователю `john` выполнять `SELECT` над всеми таблицам в базе данных `db`. Также вы можете опускать имя базы данных. В таком случае привилегии позволят совершать операции над текущей базой данных. Например, запрос `GRANT SELECT ON * TO john` выдаст привилегию на выполнение `SELECT` над всеми таблицами в текущей базе данных; `GRANT SELECT ON mytable TO john` — только над таблицей `mytable` в текущей базе данных. +Specifying privileges you can use asterisk (`*`) instead of a table or a database name. For example, the `GRANT SELECT ON db.* TO john` query allows `john` to execute the `SELECT` query over all the tables in `db` database. Also, you can omit database name. In this case privileges are granted for current database. For example, `GRANT SELECT ON * TO john` grants the privilege on all the tables in the current database, `GRANT SELECT ON mytable TO john` grants the privilege on the `mytable` table in the current database. -Доступ к базе данных `system` разрешен всегда (данная база данных используется при обработке запросов). +Access to the `system` database is always allowed (since this database is used for processing queries). -Вы можете присвоить несколько привилегий нескольким пользователям в одном запросе. Запрос `GRANT SELECT, INSERT ON *.* TO john, robin` позволит пользователям `john` и `robin` выполнять `INSERT` и `SELECT` над всеми таблицами всех баз данных на сервере. +You can grant multiple privileges to multiple accounts in one query. The query `GRANT SELECT, INSERT ON *.* TO john, robin` allows accounts `john` and `robin` to execute the `INSERT` and `SELECT` queries over all the tables in all the databases on the server. +## Privileges {#grant-privileges} -## Привилегии {#grant-privileges} +Privilege is a permission to execute specific kind of queries. -Привилегия — это разрешение на выполнение определенного типа запросов. +Privileges have a hierarchical structure. A set of permitted queries depends on the privilege scope. -Привилегии имеют иерархическую структуру. Набор разрешенных запросов зависит от области действия привилегии. +Hierarchy of privileges: -Иерархия привилегий: +- [SELECT](#grant-select) +- [INSERT](#grant-insert) +- [ALTER](#grant-alter) + - `ALTER TABLE` + - `ALTER UPDATE` + - `ALTER DELETE` + - `ALTER COLUMN` + - `ALTER ADD COLUMN` + - `ALTER DROP COLUMN` + - `ALTER MODIFY COLUMN` + - `ALTER COMMENT COLUMN` + - `ALTER CLEAR COLUMN` + - `ALTER RENAME COLUMN` + - `ALTER INDEX` + - `ALTER ORDER BY` + - `ALTER ADD INDEX` + - `ALTER DROP INDEX` + - `ALTER MATERIALIZE INDEX` + - `ALTER CLEAR INDEX` + - `ALTER CONSTRAINT` + - `ALTER ADD CONSTRAINT` + - `ALTER DROP CONSTRAINT` + - `ALTER TTL` + - `ALTER MATERIALIZE TTL` + - `ALTER SETTINGS` + - `ALTER MOVE PARTITION` + - `ALTER FETCH PARTITION` + - `ALTER FREEZE PARTITION` + - `ALTER VIEW` + - `ALTER VIEW REFRESH` + - `ALTER VIEW MODIFY QUERY` +- [CREATE](#grant-create) + - `CREATE DATABASE` + - `CREATE TABLE` + - `CREATE VIEW` + - `CREATE DICTIONARY` + - `CREATE TEMPORARY TABLE` +- [DROP](#grant-drop) + - `DROP DATABASE` + - `DROP TABLE` + - `DROP VIEW` + - `DROP DICTIONARY` +- [TRUNCATE](#grant-truncate) +- [OPTIMIZE](#grant-optimize) +- [SHOW](#grant-show) + - `SHOW DATABASES` + - `SHOW TABLES` + - `SHOW COLUMNS` + - `SHOW DICTIONARIES` +- [KILL QUERY](#grant-kill-query) +- [ACCESS MANAGEMENT](#grant-access-management) + - `CREATE USER` + - `ALTER USER` + - `DROP USER` + - `CREATE ROLE` + - `ALTER ROLE` + - `DROP ROLE` + - `CREATE ROW POLICY` + - `ALTER ROW POLICY` + - `DROP ROW POLICY` + - `CREATE QUOTA` + - `ALTER QUOTA` + - `DROP QUOTA` + - `CREATE SETTINGS PROFILE` + - `ALTER SETTINGS PROFILE` + - `DROP SETTINGS PROFILE` + - `SHOW ACCESS` + - `SHOW_USERS` + - `SHOW_ROLES` + - `SHOW_ROW_POLICIES` + - `SHOW_QUOTAS` + - `SHOW_SETTINGS_PROFILES` + - `ROLE ADMIN` +- [SYSTEM](#grant-system) + - `SYSTEM SHUTDOWN` + - `SYSTEM DROP CACHE` + - `SYSTEM DROP DNS CACHE` + - `SYSTEM DROP MARK CACHE` + - `SYSTEM DROP UNCOMPRESSED CACHE` + - `SYSTEM RELOAD` + - `SYSTEM RELOAD CONFIG` + - `SYSTEM RELOAD DICTIONARY` + - `SYSTEM RELOAD EMBEDDED DICTIONARIES` + - `SYSTEM MERGES` + - `SYSTEM TTL MERGES` + - `SYSTEM FETCHES` + - `SYSTEM MOVES` + - `SYSTEM SENDS` + - `SYSTEM DISTRIBUTED SENDS` + - `SYSTEM REPLICATED SENDS` + - `SYSTEM REPLICATION QUEUES` + - `SYSTEM SYNC REPLICA` + - `SYSTEM RESTART REPLICA` + - `SYSTEM FLUSH` + - `SYSTEM FLUSH DISTRIBUTED` + - `SYSTEM FLUSH LOGS` +- [INTROSPECTION](#grant-introspection) + - `addressToLine` + - `addressToSymbol` + - `demangle` +- [SOURCES](#grant-sources) + - `FILE` + - `URL` + - `REMOTE` + - `YSQL` + - `ODBC` + - `JDBC` + - `HDFS` + - `S3` +- [dictGet](#grant-dictget) -- [SELECT](#grant-select) -- [INSERT](#grant-insert) -- [ALTER](#grant-alter) - - `ALTER TABLE` - - `ALTER UPDATE` - - `ALTER DELETE` - - `ALTER COLUMN` - - `ALTER ADD COLUMN` - - `ALTER DROP COLUMN` - - `ALTER MODIFY COLUMN` - - `ALTER COMMENT COLUMN` - - `ALTER CLEAR COLUMN` - - `ALTER RENAME COLUMN` - - `ALTER INDEX` - - `ALTER ORDER BY` - - `ALTER ADD INDEX` - - `ALTER DROP INDEX` - - `ALTER MATERIALIZE INDEX` - - `ALTER CLEAR INDEX` - - `ALTER CONSTRAINT` - - `ALTER ADD CONSTRAINT` - - `ALTER DROP CONSTRAINT` - - `ALTER TTL` - - `ALTER MATERIALIZE TTL` - - `ALTER SETTINGS` - - `ALTER MOVE PARTITION` - - `ALTER FETCH PARTITION` - - `ALTER FREEZE PARTITION` - - `ALTER VIEW` - - `ALTER VIEW REFRESH ` - - `ALTER VIEW MODIFY QUERY` -- [CREATE](#grant-create) - - `CREATE DATABASE` - - `CREATE TABLE` - - `CREATE VIEW` - - `CREATE DICTIONARY` - - `CREATE TEMPORARY TABLE` -- [DROP](#grant-drop) - - `DROP DATABASE` - - `DROP TABLE` - - `DROP VIEW` - - `DROP DICTIONARY` -- [TRUNCATE](#grant-truncate) -- [OPTIMIZE](#grant-optimize) -- [SHOW](#grant-show) - - `SHOW DATABASES` - - `SHOW TABLES` - - `SHOW COLUMNS` - - `SHOW DICTIONARIES` -- [KILL QUERY](#grant-kill-query) -- [ACCESS MANAGEMENT](#grant-access-management) - - `CREATE USER` - - `ALTER USER` - - `DROP USER` - - `CREATE ROLE` - - `ALTER ROLE` - - `DROP ROLE` - - `CREATE ROW POLICY` - - `ALTER ROW POLICY` - - `DROP ROW POLICY` - - `CREATE QUOTA` - - `ALTER QUOTA` - - `DROP QUOTA` - - `CREATE SETTINGS PROFILE` - - `ALTER SETTINGS PROFILE` - - `DROP SETTINGS PROFILE` - - `SHOW ACCESS` - - `SHOW_USERS` - - `SHOW_ROLES` - - `SHOW_ROW_POLICIES` - - `SHOW_QUOTAS` - - `SHOW_SETTINGS_PROFILES` - - `ROLE ADMIN` -- [SYSTEM](#grant-system) - - `SYSTEM SHUTDOWN` - - `SYSTEM DROP CACHE` - - `SYSTEM DROP DNS CACHE` - - `SYSTEM DROP MARK CACHE` - - `SYSTEM DROP UNCOMPRESSED CACHE` - - `SYSTEM RELOAD` - - `SYSTEM RELOAD CONFIG` - - `SYSTEM RELOAD DICTIONARY` - - `SYSTEM RELOAD EMBEDDED DICTIONARIES` - - `SYSTEM MERGES` - - `SYSTEM TTL MERGES` - - `SYSTEM FETCHES` - - `SYSTEM MOVES` - - `SYSTEM SENDS` - - `SYSTEM DISTRIBUTED SENDS` - - `SYSTEM REPLICATED SENDS` - - `SYSTEM REPLICATION QUEUES` - - `SYSTEM SYNC REPLICA` - - `SYSTEM RESTART REPLICA` - - `SYSTEM FLUSH` - - `SYSTEM FLUSH DISTRIBUTED` - - `SYSTEM FLUSH LOGS` -- [INTROSPECTION](#grant-introspection) - - `addressToLine` - - `addressToSymbol` - - `demangle` -- [SOURCES](#grant-sources) - - `FILE` - - `URL` - - `REMOTE` - - `YSQL` - - `ODBC` - - `JDBC` - - `HDFS` - - `S3` -- [dictGet](#grant-dictget) +Examples of how this hierarchy is treated: -Примеры того, как трактуется данная иерархия: +- The `ALTER` privilege includes all other `ALTER*` privileges. +- `ALTER CONSTRAINT` includes `ALTER ADD CONSTRAINT` and `ALTER DROP CONSTRAINT` privileges. -- Привилегия `ALTER` включает все остальные `ALTER*` привилегии. -- `ALTER CONSTRAINT` включает `ALTER ADD CONSTRAINT` и `ALTER DROP CONSTRAINT`. +Privileges are applied at different levels. Knowing of a level suggests syntax available for privilege. -Привилегии применяются на разных уровнях. Уровень определяет синтаксис присваивания привилегии. +Levels (from lower to higher): -Уровни (от низшего к высшему): +- `COLUMN` — Privilege can be granted for column, table, database, or globally. +- `TABLE` — Privilege can be granted for table, database, or globally. +- `VIEW` — Privilege can be granted for view, database, or globally. +- `DICTIONARY` — Privilege can be granted for dictionary, database, or globally. +- `DATABASE` — Privilege can be granted for database or globally. +- `GLOBAL` — Privilege can be granted only globally. +- `GROUP` — Groups privileges of different levels. When `GROUP`-level privilege is granted, only that privileges from the group are granted which correspond to the used syntax. -- `COLUMN` — Привилегия присваивается для столбца, таблицы, базы данных или глобально. -- `TABLE` — Привилегия присваивается для таблицы, базы данных или глобально. -- `VIEW` — Привилегия присваивается для представления, базы данных или глобально. -- `DICTIONARY` — Привилегия присваивается для словаря, базы данных или глобально. -- `DATABASE` — Привилегия присваивается для базы данных или глобально. -- `GLOBAL` — Привилегия присваивается только глобально. -- `GROUP` — Группирует привилегии разных уровней. При присвоении привилегии уровня `GROUP` присваиваются только привилегии из группы в соответствии с используемым синтаксисом. +Examples of allowed syntax: -Примеры допустимого синтаксиса: +- `GRANT SELECT(x) ON db.table TO user` +- `GRANT SELECT ON db.* TO user` -- `GRANT SELECT(x) ON db.table TO user` -- `GRANT SELECT ON db.* TO user` +Examples of disallowed syntax: -Примеры недопустимого синтаксиса: +- `GRANT CREATE USER(x) ON db.table TO user` +- `GRANT CREATE USER ON db.* TO user` -- `GRANT CREATE USER(x) ON db.table TO user` -- `GRANT CREATE USER ON db.* TO user` +The special privilege [ALL](#grant-all) grants all the privileges to a user account or a role. -Специальная привилегия [ALL](#grant-all) присваивает все привилегии пользователю или роли. +By default, a user account or a role has no privileges. -По умолчанию пользователь или роль не имеют привилегий. - -Отсутствие привилегий у пользователя или роли отображается как привилегия [NONE](#grant-none). - -Выполнение некоторых запросов требует определенного набора привилегий. Например, чтобы выполнить запрос [RENAME](misc.md#misc_operations-rename), нужны следующие привилегии: `SELECT`, `CREATE TABLE`, `INSERT` и `DROP TABLE`. +If a user or a role has no privileges, it is displayed as [NONE](#grant-none) privilege. +Some queries by their implementation require a set of privileges. For example, to execute the [RENAME](../../sql-reference/statements/misc.md#misc_operations-rename) query you need the following privileges: `SELECT`, `CREATE TABLE`, `INSERT` and `DROP TABLE`. ### SELECT {#grant-select} -Разрешает выполнять запросы [SELECT](select/index.md). +Allows executing [SELECT](../../sql-reference/statements/select/index.md) queries. -Уровень: `COLUMN`. +Privilege level: `COLUMN`. -**Описание** +**Description** -Пользователь с данной привилегией может выполнять запросы `SELECT` над определенными столбцами из определенной таблицы и базы данных. При включении в запрос других столбцов запрос ничего не вернет. +User granted with this privilege can execute `SELECT` queries over a specified list of columns in the specified table and database. If user includes other columns then specified a query returns no data. -Рассмотрим следующую привилегию: +Consider the following privilege: -```sql +``` sql GRANT SELECT(x,y) ON db.table TO john ``` -Данная привилегия позволяет пользователю `john` выполнять выборку данных из столбцов `x` и/или `y` в `db.table`, например, `SELECT x FROM db.table`. `john` не может выполнить `SELECT z FROM db.table` или `SELECT * FROM db.table`. После обработки данных запросов ClickHouse ничего не вернет — даже `x` или `y`. Единственное исключение — если таблица содержит только столбцы `x` и `y`. В таком случае ClickHouse вернет все данные. +This privilege allows `john` to execute any `SELECT` query that involves data from the `x` and/or `y` columns in `db.table`, for example, `SELECT x FROM db.table`. `john` can’t execute `SELECT z FROM db.table`. The `SELECT * FROM db.table` also is not available. Processing this query, ClickHouse doesn’t return any data, even `x` and `y`. The only exception is if a table contains only `x` and `y` columns, in this case ClickHouse returns all the data. ### INSERT {#grant-insert} -Разрешает выполнять запросы [INSERT](insert-into.md). +Allows executing [INSERT](../../sql-reference/statements/insert-into.md) queries. -Уровень: `COLUMN`. +Privilege level: `COLUMN`. -**Описание** +**Description** -Пользователь с данной привилегией может выполнять запросы `INSERT` над определенными столбцами из определенной таблицы и базы данных. При включении в запрос других столбцов запрос не добавит никаких данных. +User granted with this privilege can execute `INSERT` queries over a specified list of columns in the specified table and database. If user includes other columns then specified a query doesn’t insert any data. -**Пример** +**Example** -```sql +``` sql GRANT INSERT(x,y) ON db.table TO john ``` -Присвоенная привилегия позволит пользователю `john` вставить данные в столбцы `x` и/или `y` в `db.table`. +The granted privilege allows `john` to insert data to the `x` and/or `y` columns in `db.table`. ### ALTER {#grant-alter} -Разрешает выполнять запросы [ALTER](alter.md) в соответствии со следующей иерархией привилегий: +Allows executing [ALTER](../../sql-reference/statements/alter.md) queries according to the following hierarchy of privileges: -- `ALTER`. Уровень: `COLUMN`. - - `ALTER TABLE`. Уровень: `GROUP` - - `ALTER UPDATE`. Уровень: `COLUMN`. Алиасы: `UPDATE` - - `ALTER DELETE`. Уровень: `COLUMN`. Алиасы: `DELETE` - - `ALTER COLUMN`. Уровень: `GROUP` - - `ALTER ADD COLUMN`. Уровень: `COLUMN`. Алиасы: `ADD COLUMN` - - `ALTER DROP COLUMN`. Уровень: `COLUMN`. Алиасы: `DROP COLUMN` - - `ALTER MODIFY COLUMN`. Уровень: `COLUMN`. Алиасы: `MODIFY COLUMN` - - `ALTER COMMENT COLUMN`. Уровень: `COLUMN`. Алиасы: `COMMENT COLUMN` - - `ALTER CLEAR COLUMN`. Уровень: `COLUMN`. Алиасы: `CLEAR COLUMN` - - `ALTER RENAME COLUMN`. Уровень: `COLUMN`. Алиасы: `RENAME COLUMN` - - `ALTER INDEX`. Уровень: `GROUP`. Алиасы: `INDEX` - - `ALTER ORDER BY`. Уровень: `TABLE`. Алиасы: `ALTER MODIFY ORDER BY`, `MODIFY ORDER BY` - - `ALTER ADD INDEX`. Уровень: `TABLE`. Алиасы: `ADD INDEX` - - `ALTER DROP INDEX`. Уровень: `TABLE`. Алиасы: `DROP INDEX` - - `ALTER MATERIALIZE INDEX`. Уровень: `TABLE`. Алиасы: `MATERIALIZE INDEX` - - `ALTER CLEAR INDEX`. Уровень: `TABLE`. Алиасы: `CLEAR INDEX` - - `ALTER CONSTRAINT`. Уровень: `GROUP`. Алиасы: `CONSTRAINT` - - `ALTER ADD CONSTRAINT`. Уровень: `TABLE`. Алиасы: `ADD CONSTRAINT` - - `ALTER DROP CONSTRAINT`. Уровень: `TABLE`. Алиасы: `DROP CONSTRAINT` - - `ALTER TTL`. Уровень: `TABLE`. Алиасы: `ALTER MODIFY TTL`, `MODIFY TTL` - - `ALTER MATERIALIZE TTL`. Уровень: `TABLE`. Алиасы: `MATERIALIZE TTL` - - `ALTER SETTINGS`. Уровень: `TABLE`. Алиасы: `ALTER SETTING`, `ALTER MODIFY SETTING`, `MODIFY SETTING` - - `ALTER MOVE PARTITION`. Уровень: `TABLE`. Алиасы: `ALTER MOVE PART`, `MOVE PARTITION`, `MOVE PART` - - `ALTER FETCH PARTITION`. Уровень: `TABLE`. Алиасы: `FETCH PARTITION` - - `ALTER FREEZE PARTITION`. Уровень: `TABLE`. Алиасы: `FREEZE PARTITION` - - `ALTER VIEW` Уровень: `GROUP` - - `ALTER VIEW REFRESH `. Уровень: `VIEW`. Алиасы: `ALTER LIVE VIEW REFRESH`, `REFRESH VIEW` - - `ALTER VIEW MODIFY QUERY`. Уровень: `VIEW`. Алиасы: `ALTER TABLE MODIFY QUERY` +- `ALTER`. Level: `COLUMN`. + - `ALTER TABLE`. Level: `GROUP` + - `ALTER UPDATE`. Level: `COLUMN`. Aliases: `UPDATE` + - `ALTER DELETE`. Level: `COLUMN`. Aliases: `DELETE` + - `ALTER COLUMN`. Level: `GROUP` + - `ALTER ADD COLUMN`. Level: `COLUMN`. Aliases: `ADD COLUMN` + - `ALTER DROP COLUMN`. Level: `COLUMN`. Aliases: `DROP COLUMN` + - `ALTER MODIFY COLUMN`. Level: `COLUMN`. Aliases: `MODIFY COLUMN` + - `ALTER COMMENT COLUMN`. Level: `COLUMN`. Aliases: `COMMENT COLUMN` + - `ALTER CLEAR COLUMN`. Level: `COLUMN`. Aliases: `CLEAR COLUMN` + - `ALTER RENAME COLUMN`. Level: `COLUMN`. Aliases: `RENAME COLUMN` + - `ALTER INDEX`. Level: `GROUP`. Aliases: `INDEX` + - `ALTER ORDER BY`. Level: `TABLE`. Aliases: `ALTER MODIFY ORDER BY`, `MODIFY ORDER BY` + - `ALTER ADD INDEX`. Level: `TABLE`. Aliases: `ADD INDEX` + - `ALTER DROP INDEX`. Level: `TABLE`. Aliases: `DROP INDEX` + - `ALTER MATERIALIZE INDEX`. Level: `TABLE`. Aliases: `MATERIALIZE INDEX` + - `ALTER CLEAR INDEX`. Level: `TABLE`. Aliases: `CLEAR INDEX` + - `ALTER CONSTRAINT`. Level: `GROUP`. Aliases: `CONSTRAINT` + - `ALTER ADD CONSTRAINT`. Level: `TABLE`. Aliases: `ADD CONSTRAINT` + - `ALTER DROP CONSTRAINT`. Level: `TABLE`. Aliases: `DROP CONSTRAINT` + - `ALTER TTL`. Level: `TABLE`. Aliases: `ALTER MODIFY TTL`, `MODIFY TTL` + - `ALTER MATERIALIZE TTL`. Level: `TABLE`. Aliases: `MATERIALIZE TTL` + - `ALTER SETTINGS`. Level: `TABLE`. Aliases: `ALTER SETTING`, `ALTER MODIFY SETTING`, `MODIFY SETTING` + - `ALTER MOVE PARTITION`. Level: `TABLE`. Aliases: `ALTER MOVE PART`, `MOVE PARTITION`, `MOVE PART` + - `ALTER FETCH PARTITION`. Level: `TABLE`. Aliases: `FETCH PARTITION` + - `ALTER FREEZE PARTITION`. Level: `TABLE`. Aliases: `FREEZE PARTITION` + - `ALTER VIEW` Level: `GROUP` + - `ALTER VIEW REFRESH`. Level: `VIEW`. Aliases: `ALTER LIVE VIEW REFRESH`, `REFRESH VIEW` + - `ALTER VIEW MODIFY QUERY`. Level: `VIEW`. Aliases: `ALTER TABLE MODIFY QUERY` -Примеры того, как трактуется данная иерархия: +Examples of how this hierarchy is treated: -- Привилегия `ALTER` включает все остальные `ALTER*` привилегии. -- `ALTER CONSTRAINT` включает `ALTER ADD CONSTRAINT` и `ALTER DROP CONSTRAINT`. +- The `ALTER` privilege includes all other `ALTER*` privileges. +- `ALTER CONSTRAINT` includes `ALTER ADD CONSTRAINT` and `ALTER DROP CONSTRAINT` privileges. -**Дополнительно** +**Notes** -- Привилегия `MODIFY SETTING` позволяет изменять настройки движков таблиц. Не влияет на настройки или конфигурационные параметры сервера. -- Операция `ATTACH` требует наличие привилегии [CREATE](#grant-create). -- Операция `DETACH` требует наличие привилегии [DROP](#grant-drop). -- Для остановки мутации с помощью [KILL MUTATION](misc.md#kill-mutation-statement), необходима привилегия на выполнение данной мутации. Например, чтобы остановить запрос `ALTER UPDATE`, необходима одна из привилегий: `ALTER UPDATE`, `ALTER TABLE` или `ALTER`. +- The `MODIFY SETTING` privilege allows modifying table engine settings. It doesn’t affect settings or server configuration parameters. +- The `ATTACH` operation needs the [CREATE](#grant-create) privilege. +- The `DETACH` operation needs the [DROP](#grant-drop) privilege. +- To stop mutation by the [KILL MUTATION](../../sql-reference/statements/misc.md#kill-mutation) query, you need to have a privilege to start this mutation. For example, if you want to stop the `ALTER UPDATE` query, you need the `ALTER UPDATE`, `ALTER TABLE`, or `ALTER` privilege. ### CREATE {#grant-create} -Разрешает выполнять DDL-запросы [CREATE](create.md) и [ATTACH](misc.md#attach) в соответствии со следующей иерархией привилегий: +Allows executing [CREATE](../../sql-reference/statements/create.md) and [ATTACH](../../sql-reference/statements/misc.md#attach) DDL-queries according to the following hierarchy of privileges: -- `CREATE`. Уровень: `GROUP` - - `CREATE DATABASE`. Уровень: `DATABASE` - - `CREATE TABLE`. Уровень: `TABLE` - - `CREATE VIEW`. Уровень: `VIEW` - - `CREATE DICTIONARY`. Уровень: `DICTIONARY` - - `CREATE TEMPORARY TABLE`. Уровень: `GLOBAL` +- `CREATE`. Level: `GROUP` + - `CREATE DATABASE`. Level: `DATABASE` + - `CREATE TABLE`. Level: `TABLE` + - `CREATE VIEW`. Level: `VIEW` + - `CREATE DICTIONARY`. Level: `DICTIONARY` + - `CREATE TEMPORARY TABLE`. Level: `GLOBAL` -**Дополнительно** +**Notes** -- Для удаления созданной таблицы пользователю необходима привилегия [DROP](#grant-drop). +- To delete the created table, a user needs [DROP](#grant-drop). ### DROP {#grant-drop} -Разрешает выполнять запросы [DROP](misc.md#drop) и [DETACH](misc.md#detach-statement) в соответствии со следующей иерархией привилегий: - -- `DROP`. Уровень: - - `DROP DATABASE`. Уровень: `DATABASE` - - `DROP TABLE`. Уровень: `TABLE` - - `DROP VIEW`. Уровень: `VIEW` - - `DROP DICTIONARY`. Уровень: `DICTIONARY` +Allows executing [DROP](../../sql-reference/statements/misc.md#drop) and [DETACH](../../sql-reference/statements/misc.md#detach) queries according to the following hierarchy of privileges: +- `DROP`. Level: + - `DROP DATABASE`. Level: `DATABASE` + - `DROP TABLE`. Level: `TABLE` + - `DROP VIEW`. Level: `VIEW` + - `DROP DICTIONARY`. Level: `DICTIONARY` ### TRUNCATE {#grant-truncate} -Разрешает выполнять запросы [TRUNCATE](misc.md#truncate-statement). +Allows executing [TRUNCATE](../../sql-reference/statements/misc.md#truncate-statement) queries. -Уровень: `TABLE`. +Privilege level: `TABLE`. ### OPTIMIZE {#grant-optimize} -Разрешает выполнять запросы [OPTIMIZE TABLE](misc.md#misc_operations-optimize). +Allows executing [OPTIMIZE TABLE](../../sql-reference/statements/misc.md#misc_operations-optimize) queries. -Уровень: `TABLE`. +Privilege level: `TABLE`. ### SHOW {#grant-show} -Разрешает выполнять запросы `SHOW`, `DESCRIBE`, `USE` и `EXISTS` в соответствии со следующей иерархией привилегий: +Allows executing `SHOW`, `DESCRIBE`, `USE`, and `EXISTS` queries according to the following hierarchy of privileges: -- `SHOW`. Уровень: `GROUP` - - `SHOW DATABASES`. Уровень: `DATABASE`. Разрешает выполнять запросы `SHOW DATABASES`, `SHOW CREATE DATABASE`, `USE `. - - `SHOW TABLES`. Уровень: `TABLE`. Разрешает выполнять запросы `SHOW TABLES`, `EXISTS
    `, `CHECK
    `. - - `SHOW COLUMNS`. Уровень: `COLUMN`. Разрешает выполнять запросы `SHOW CREATE TABLE`, `DESCRIBE`. - - `SHOW DICTIONARIES`. Уровень: `DICTIONARY`. Разрешает выполнять запросы `SHOW DICTIONARIES`, `SHOW CREATE DICTIONARY`, `EXISTS `. +- `SHOW`. Level: `GROUP` + - `SHOW DATABASES`. Level: `DATABASE`. Allows to execute `SHOW DATABASES`, `SHOW CREATE DATABASE`, `USE ` queries. + - `SHOW TABLES`. Level: `TABLE`. Allows to execute `SHOW TABLES`, `EXISTS
    `, `CHECK
    ` queries. + - `SHOW COLUMNS`. Level: `COLUMN`. Allows to execute `SHOW CREATE TABLE`, `DESCRIBE` queries. + - `SHOW DICTIONARIES`. Level: `DICTIONARY`. Allows to execute `SHOW DICTIONARIES`, `SHOW CREATE DICTIONARY`, `EXISTS ` queries. -**Дополнительно** - -У пользователя есть привилегия `SHOW`, если ему присвоена любая другая привилегия по отношению к определенной таблице, словарю или базе данных. +**Notes** +A user has the `SHOW` privilege if it has any other privilege concerning the specified table, dictionary or database. ### KILL QUERY {#grant-kill-query} -Разрешает выполнять запросы [KILL](misc.md#kill-query-statement) в соответствии со следующей иерархией привилегий: +Allows executing [KILL](../../sql-reference/statements/misc.md#kill-query-statement) queries according to the following hierarchy of privileges: -Уровень: `GLOBAL`. +Privilege level: `GLOBAL`. -**Дополнительно** - -`KILL QUERY` позволяет пользователю останавливать запросы других пользователей. +**Notes** +`KILL QUERY` privilege allows one user to kill queries of other users. ### ACCESS MANAGEMENT {#grant-access-management} -Разрешает пользователю выполнять запросы на управление пользователями, ролями и политиками доступа к строкам. +Allows a user to execute queries that manage users, roles and row policies. -- `ACCESS MANAGEMENT`. Уровень: `GROUP` - - `CREATE USER`. Уровень: `GLOBAL` - - `ALTER USER`. Уровень: `GLOBAL` - - `DROP USER`. Уровень: `GLOBAL` - - `CREATE ROLE`. Уровень: `GLOBAL` - - `ALTER ROLE`. Уровень: `GLOBAL` - - `DROP ROLE`. Уровень: `GLOBAL` - - `ROLE ADMIN`. Уровень: `GLOBAL` - - `CREATE ROW POLICY`. Уровень: `GLOBAL`. Алиасы: `CREATE POLICY` - - `ALTER ROW POLICY`. Уровень: `GLOBAL`. Алиасы: `ALTER POLICY` - - `DROP ROW POLICY`. Уровень: `GLOBAL`. Алиасы: `DROP POLICY` - - `CREATE QUOTA`. Уровень: `GLOBAL` - - `ALTER QUOTA`. Уровень: `GLOBAL` - - `DROP QUOTA`. Уровень: `GLOBAL` - - `CREATE SETTINGS PROFILE`. Уровень: `GLOBAL`. Алиасы: `CREATE PROFILE` - - `ALTER SETTINGS PROFILE`. Уровень: `GLOBAL`. Алиасы: `ALTER PROFILE` - - `DROP SETTINGS PROFILE`. Уровень: `GLOBAL`. Алиасы: `DROP PROFILE` - - `SHOW ACCESS`. Уровень: `GROUP` - - `SHOW_USERS`. Уровень: `GLOBAL`. Алиасы: `SHOW CREATE USER` - - `SHOW_ROLES`. Уровень: `GLOBAL`. Алиасы: `SHOW CREATE ROLE` - - `SHOW_ROW_POLICIES`. Уровень: `GLOBAL`. Алиасы: `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY` - - `SHOW_QUOTAS`. Уровень: `GLOBAL`. Алиасы: `SHOW CREATE QUOTA` - - `SHOW_SETTINGS_PROFILES`. Уровень: `GLOBAL`. Алиасы: `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW CREATE PROFILE` +- `ACCESS MANAGEMENT`. Level: `GROUP` + - `CREATE USER`. Level: `GLOBAL` + - `ALTER USER`. Level: `GLOBAL` + - `DROP USER`. Level: `GLOBAL` + - `CREATE ROLE`. Level: `GLOBAL` + - `ALTER ROLE`. Level: `GLOBAL` + - `DROP ROLE`. Level: `GLOBAL` + - `ROLE ADMIN`. Level: `GLOBAL` + - `CREATE ROW POLICY`. Level: `GLOBAL`. Aliases: `CREATE POLICY` + - `ALTER ROW POLICY`. Level: `GLOBAL`. Aliases: `ALTER POLICY` + - `DROP ROW POLICY`. Level: `GLOBAL`. Aliases: `DROP POLICY` + - `CREATE QUOTA`. Level: `GLOBAL` + - `ALTER QUOTA`. Level: `GLOBAL` + - `DROP QUOTA`. Level: `GLOBAL` + - `CREATE SETTINGS PROFILE`. Level: `GLOBAL`. Aliases: `CREATE PROFILE` + - `ALTER SETTINGS PROFILE`. Level: `GLOBAL`. Aliases: `ALTER PROFILE` + - `DROP SETTINGS PROFILE`. Level: `GLOBAL`. Aliases: `DROP PROFILE` + - `SHOW ACCESS`. Level: `GROUP` + - `SHOW_USERS`. Level: `GLOBAL`. Aliases: `SHOW CREATE USER` + - `SHOW_ROLES`. Level: `GLOBAL`. Aliases: `SHOW CREATE ROLE` + - `SHOW_ROW_POLICIES`. Level: `GLOBAL`. Aliases: `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY` + - `SHOW_QUOTAS`. Level: `GLOBAL`. Aliases: `SHOW CREATE QUOTA` + - `SHOW_SETTINGS_PROFILES`. Level: `GLOBAL`. Aliases: `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW CREATE PROFILE` -Привилегия `ROLE ADMIN` разрешает пользователю назначать и отзывать любые роли, включая те, которые не назначены пользователю с опцией администратора. +The `ROLE ADMIN` privilege allows a user to assign and revoke any roles including those which are not assigned to the user with the admin option. ### SYSTEM {#grant-system} -Разрешает выполнять запросы [SYSTEM](system.md) в соответствии со следующей иерархией привилегий: +Allows a user to execute [SYSTEM](../../sql-reference/statements/system.md) queries according to the following hierarchy of privileges. -- `SYSTEM`. Уровень: `GROUP` - - `SYSTEM SHUTDOWN`. Уровень: `GLOBAL`. Алиасы: `SYSTEM KILL`, `SHUTDOWN` - - `SYSTEM DROP CACHE`. Алиасы: `DROP CACHE` - - `SYSTEM DROP DNS CACHE`. Уровень: `GLOBAL`. Алиасы: `SYSTEM DROP DNS`, `DROP DNS CACHE`, `DROP DNS` - - `SYSTEM DROP MARK CACHE`. Уровень: `GLOBAL`. Алиасы: `SYSTEM DROP MARK`, `DROP MARK CACHE`, `DROP MARKS` - - `SYSTEM DROP UNCOMPRESSED CACHE`. Уровень: `GLOBAL`. Алиасы: `SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, `DROP UNCOMPRESSED` - - `SYSTEM RELOAD`. Уровень: `GROUP` - - `SYSTEM RELOAD CONFIG`. Уровень: `GLOBAL`. Алиасы: `RELOAD CONFIG` - - `SYSTEM RELOAD DICTIONARY`. Уровень: `GLOBAL`. Алиасы: `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARY`, `RELOAD DICTIONARIES` - - `SYSTEM RELOAD EMBEDDED DICTIONARIES`. Уровень: `GLOBAL`. Алиасы: `RELOAD EMBEDDED DICTIONARIES` - - `SYSTEM MERGES`. Уровень: `TABLE`. Алиасы: `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, `START MERGES` - - `SYSTEM TTL MERGES`. Уровень: `TABLE`. Алиасы: `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, `START TTL MERGES` - - `SYSTEM FETCHES`. Уровень: `TABLE`. Алиасы: `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, `START FETCHES` - - `SYSTEM MOVES`. Уровень: `TABLE`. Алиасы: `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, `START MOVES` - - `SYSTEM SENDS`. Уровень: `GROUP`. Алиасы: `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, `START SENDS` - - `SYSTEM DISTRIBUTED SENDS`. Уровень: `TABLE`. Алиасы: `SYSTEM STOP DISTRIBUTED SENDS`, `SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, `START DISTRIBUTED SENDS` - - `SYSTEM REPLICATED SENDS`. Уровень: `TABLE`. Алиасы: `SYSTEM STOP REPLICATED SENDS`, `SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, `START REPLICATED SENDS` - - `SYSTEM REPLICATION QUEUES`. Уровень: `TABLE`. Алиасы: `SYSTEM STOP REPLICATION QUEUES`, `SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, `START REPLICATION QUEUES` - - `SYSTEM SYNC REPLICA`. Уровень: `TABLE`. Алиасы: `SYNC REPLICA` - - `SYSTEM RESTART REPLICA`. Уровень: `TABLE`. Алиасы: `RESTART REPLICA` - - `SYSTEM FLUSH`. Уровень: `GROUP` - - `SYSTEM FLUSH DISTRIBUTED`. Уровень: `TABLE`. Алиасы: `FLUSH DISTRIBUTED` - - `SYSTEM FLUSH LOGS`. Уровень: `GLOBAL`. Алиасы: `FLUSH LOGS` - -Привилегия `SYSTEM RELOAD EMBEDDED DICTIONARIES` имплицитно присваивается привилегией `SYSTEM RELOAD DICTIONARY ON *.*`. +- `SYSTEM`. Level: `GROUP` + - `SYSTEM SHUTDOWN`. Level: `GLOBAL`. Aliases: `SYSTEM KILL`, `SHUTDOWN` + - `SYSTEM DROP CACHE`. Aliases: `DROP CACHE` + - `SYSTEM DROP DNS CACHE`. Level: `GLOBAL`. Aliases: `SYSTEM DROP DNS`, `DROP DNS CACHE`, `DROP DNS` + - `SYSTEM DROP MARK CACHE`. Level: `GLOBAL`. Aliases: `SYSTEM DROP MARK`, `DROP MARK CACHE`, `DROP MARKS` + - `SYSTEM DROP UNCOMPRESSED CACHE`. Level: `GLOBAL`. Aliases: `SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, `DROP UNCOMPRESSED` + - `SYSTEM RELOAD`. Level: `GROUP` + - `SYSTEM RELOAD CONFIG`. Level: `GLOBAL`. Aliases: `RELOAD CONFIG` + - `SYSTEM RELOAD DICTIONARY`. Level: `GLOBAL`. Aliases: `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARY`, `RELOAD DICTIONARIES` + - `SYSTEM RELOAD EMBEDDED DICTIONARIES`. Level: `GLOBAL`. Aliases: R`ELOAD EMBEDDED DICTIONARIES` + - `SYSTEM MERGES`. Level: `TABLE`. Aliases: `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, `START MERGES` + - `SYSTEM TTL MERGES`. Level: `TABLE`. Aliases: `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, `START TTL MERGES` + - `SYSTEM FETCHES`. Level: `TABLE`. Aliases: `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, `START FETCHES` + - `SYSTEM MOVES`. Level: `TABLE`. Aliases: `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, `START MOVES` + - `SYSTEM SENDS`. Level: `GROUP`. Aliases: `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, `START SENDS` + - `SYSTEM DISTRIBUTED SENDS`. Level: `TABLE`. Aliases: `SYSTEM STOP DISTRIBUTED SENDS`, `SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, `START DISTRIBUTED SENDS` + - `SYSTEM REPLICATED SENDS`. Level: `TABLE`. Aliases: `SYSTEM STOP REPLICATED SENDS`, `SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, `START REPLICATED SENDS` + - `SYSTEM REPLICATION QUEUES`. Level: `TABLE`. Aliases: `SYSTEM STOP REPLICATION QUEUES`, `SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, `START REPLICATION QUEUES` + - `SYSTEM SYNC REPLICA`. Level: `TABLE`. Aliases: `SYNC REPLICA` + - `SYSTEM RESTART REPLICA`. Level: `TABLE`. Aliases: `RESTART REPLICA` + - `SYSTEM FLUSH`. Level: `GROUP` + - `SYSTEM FLUSH DISTRIBUTED`. Level: `TABLE`. Aliases: `FLUSH DISTRIBUTED` + - `SYSTEM FLUSH LOGS`. Level: `GLOBAL`. Aliases: `FLUSH LOGS` +The `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilege implicitly granted by the `SYSTEM RELOAD DICTIONARY ON *.*` privilege. ### INTROSPECTION {#grant-introspection} -Разрешает использовать функции [интроспекции](../../operations/optimizing-performance/sampling-query-profiler.md). - -- `INTROSPECTION`. Уровень: `GROUP`. Алиасы: `INTROSPECTION FUNCTIONS` - - `addressToLine`. Уровень: `GLOBAL` - - `addressToSymbol`. Уровень: `GLOBAL` - - `demangle`. Уровень: `GLOBAL` +Allows using [introspection](../../operations/optimizing-performance/sampling-query-profiler.md) functions. +- `INTROSPECTION`. Level: `GROUP`. Aliases: `INTROSPECTION FUNCTIONS` + - `addressToLine`. Level: `GLOBAL` + - `addressToSymbol`. Level: `GLOBAL` + - `demangle`. Level: `GLOBAL` ### SOURCES {#grant-sources} -Разрешает использовать внешние источники данных. Применяется к [движкам таблиц](../../engines/table-engines/index.md) и [табличным функциям](../table-functions/index.md#table-functions). +Allows using external data sources. Applies to [table engines](../../engines/table-engines/index.md) and [table functions](../../sql-reference/table-functions/index.md#table-functions). -- `SOURCES`. Уровень: `GROUP` - - `FILE`. Уровень: `GLOBAL` - - `URL`. Уровень: `GLOBAL` - - `REMOTE`. Уровень: `GLOBAL` - - `YSQL`. Уровень: `GLOBAL` - - `ODBC`. Уровень: `GLOBAL` - - `JDBC`. Уровень: `GLOBAL` - - `HDFS`. Уровень: `GLOBAL` - - `S3`. Уровень: `GLOBAL` +- `SOURCES`. Level: `GROUP` + - `FILE`. Level: `GLOBAL` + - `URL`. Level: `GLOBAL` + - `REMOTE`. Level: `GLOBAL` + - `YSQL`. Level: `GLOBAL` + - `ODBC`. Level: `GLOBAL` + - `JDBC`. Level: `GLOBAL` + - `HDFS`. Level: `GLOBAL` + - `S3`. Level: `GLOBAL` -Привилегия `SOURCES` разрешает использование всех источников. Также вы можете присвоить привилегию для каждого источника отдельно. Для использования источников необходимы дополнительные привилегии. +The `SOURCES` privilege enables use of all the sources. Also you can grant a privilege for each source individually. To use sources, you need additional privileges. -Примеры: +Examples: -- Чтобы создать таблицу с [движком MySQL](../../engines/table-engines/integrations/mysql.md), необходимы привилегии `CREATE TABLE (ON db.table_name)` и `MYSQL`. -- Чтобы использовать [табличную функцию mysql](../table-functions/mysql.md), необходимы привилегии `CREATE TEMPORARY TABLE` и `MYSQL`. +- To create a table with the [MySQL table engine](../../engines/table-engines/integrations/mysql.md), you need `CREATE TABLE (ON db.table_name)` and `MYSQL` privileges. +- To use the [mysql table function](../../sql-reference/table-functions/mysql.md), you need `CREATE TEMPORARY TABLE` and `MYSQL` privileges. ### dictGet {#grant-dictget} -- `dictGet`. Алиасы: `dictHas`, `dictGetHierarchy`, `dictIsIn` +- `dictGet`. Aliases: `dictHas`, `dictGetHierarchy`, `dictIsIn` -Разрешает вызывать функции [dictGet](../functions/ext-dict-functions.md#dictget), [dictHas](../functions/ext-dict-functions.md#dicthas), [dictGetHierarchy](../functions/ext-dict-functions.md#dictgethierarchy), [dictIsIn](../functions/ext-dict-functions.md#dictisin). +Allows a user to execute [dictGet](../../sql-reference/functions/ext-dict-functions.md#dictget), [dictHas](../../sql-reference/functions/ext-dict-functions.md#dicthas), [dictGetHierarchy](../../sql-reference/functions/ext-dict-functions.md#dictgethierarchy), [dictIsIn](../../sql-reference/functions/ext-dict-functions.md#dictisin) functions. -Уровень: `DICTIONARY`. +Privilege level: `DICTIONARY`. -**Примеры** +**Examples** -- `GRANT dictGet ON mydb.mydictionary TO john` -- `GRANT dictGet ON mydictionary TO john` +- `GRANT dictGet ON mydb.mydictionary TO john` +- `GRANT dictGet ON mydictionary TO john` ### ALL {#grant-all} -Присваивает пользователю или роли все привилегии на объект с регулируемым доступом. - +Grants all the privileges on regulated entity to a user account or a role. ### NONE {#grant-none} -Не присваивает никаких привилегий. - +Doesn’t grant any privileges. ### ADMIN OPTION {#admin-option-privilege} -Привилегия `ADMIN OPTION` разрешает пользователю назначать свои роли другому пользователю. +The `ADMIN OPTION` privilege allows a user to grant their role to another user. -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/grant/) +[Original article](https://clickhouse.tech/docs/en/query_language/grant/) diff --git a/docs/tr/sql-reference/statements/grant.md b/docs/tr/sql-reference/statements/grant.md deleted file mode 120000 index f2acbe125b4..00000000000 --- a/docs/tr/sql-reference/statements/grant.md +++ /dev/null @@ -1 +0,0 @@ -../../../en/sql-reference/statements/grant.md \ No newline at end of file diff --git a/docs/tr/sql-reference/statements/grant.md b/docs/tr/sql-reference/statements/grant.md new file mode 100644 index 00000000000..61a40a297fc --- /dev/null +++ b/docs/tr/sql-reference/statements/grant.md @@ -0,0 +1,476 @@ +--- +machine_translated: true +machine_translated_rev: cbd8aa9052361a7ee11c209560cff7175c2b8e42 +toc_priority: 39 +toc_title: GRANT +--- + +# GRANT {#grant} + +- Veriyor [ayrıcalıklar](#grant-privileges) kullanıcı hesaplarını veya rollerini tıklamak için. +- Rolleri kullanıcı hesaplarına veya diğer rollere atar. + +Ayrıcalıkları iptal etmek için [REVOKE](../../sql-reference/statements/revoke.md) deyim. Ayrıca, verilen ayrıcalıkları listeleyebilirsiniz [SHOW GRANTS](../../sql-reference/statements/show.md#show-grants-statement) deyim. + +## Ayrıcalık Sözdizimi Verme {#grant-privigele-syntax} + +``` sql +GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...] ON {db.table|db.*|*.*|table|*} TO {user | role | CURRENT_USER} [,...] [WITH GRANT OPTION] +``` + +- `privilege` — Type of privilege. +- `role` — ClickHouse user role. +- `user` — ClickHouse user account. + +Bu `WITH GRANT OPTION` madde hibeleri `user` veya `role` yürütme izni ile `GRANT` sorgu. Kullanıcılar sahip oldukları ve daha az olan aynı kapsamdaki ayrıcalıkları verebilir. + +## Rol Sözdizimi Atama {#assign-role-syntax} + +``` sql +GRANT [ON CLUSTER cluster_name] role [,...] TO {user | another_role | CURRENT_USER} [,...] [WITH ADMIN OPTION] +``` + +- `role` — ClickHouse user role. +- `user` — ClickHouse user account. + +Bu `WITH ADMIN OPTION` madde hibeleri [ADMIN OPTION](#admin-option-privilege) ayrıcalık `user` veya `role`. + +## Kullanma {#grant-usage} + +Kullanmak `GRANT` hesabınız olmalıdır var `GRANT OPTION` ayrıcalık. Ayrıcalıkları yalnızca hesap ayrıcalıklarınız kapsamında verebilirsiniz. + +Örneğin, yönetici için ayrıcalıklar verdi `john` sorguya göre hesap: + +``` sql +GRANT SELECT(x,y) ON db.table TO john WITH GRANT OPTION +``` + +Demek ki `john` yürütme izni var mı: + +- `SELECT x,y FROM db.table`. +- `SELECT x FROM db.table`. +- `SELECT y FROM db.table`. + +`john` idam edilemiyor `SELECT z FROM db.table`. Bu `SELECT * FROM db.table` ayrıca mevcut değildir. Bu sorguyu işlerken, ClickHouse bile herhangi bir veri döndürmez `x` ve `y`. Tek istisna, bir tablo yalnızca `x` ve `y` sütun. Bu durumda ClickHouse tüm verileri döndürür. + +Ayrıca `john` vardır `GRANT OPTION` ayrıcalık, böylece diğer kullanıcılara aynı veya daha küçük kapsamdaki ayrıcalıklar verebilir. + +Ayrıcalıkları belirtme Yıldız İşareti kullanabilirsiniz (`*`) bir tablo veya veritabanı adı yerine. Örneğin, `GRANT SELECT ON db.* TO john` sorgu sağlar `john` yürütmek için `SELECT` tüm tablolar üzerinde sorgu `db` veritabanı. Ayrıca, veritabanı adı atlayabilirsiniz. Bu durumda, geçerli veritabanı için ayrıcalıklar verilir. Mesela, `GRANT SELECT ON * TO john` geçerli veritabanındaki tüm tablolarda ayrıcalık verir, `GRANT SELECT ON mytable TO john` bu ayrıcalığı verir `mytable` geçerli veritabanındaki tablo. + +Erişim `system` veritabanına her zaman izin verilir (bu veritabanı sorguları işlemek için kullanıldığından). + +Tek bir sorguda birden çok hesaba birden çok ayrıcalık verebilirsiniz. Sorgu `GRANT SELECT, INSERT ON *.* TO john, robin` hesaplara izin verir `john` ve `robin` yürütmek için `INSERT` ve `SELECT` sunucudaki tüm veritabanlarındaki tüm tablolar üzerinden sorgular. + +## Ayrıcalıklar {#grant-privileges} + +Ayrıcalık, belirli bir sorgu türünü yürütme iznidir. + +Ayrıcalıklar hiyerarşik bir yapıya sahiptir. İzin verilen sorgular kümesi ayrıcalık kapsamına bağlıdır. + +Ayrıcalıkların hiyerarşisi: + +- [SELECT](#grant-select) +- [INSERT](#grant-insert) +- [ALTER](#grant-alter) + - `ALTER TABLE` + - `ALTER UPDATE` + - `ALTER DELETE` + - `ALTER COLUMN` + - `ALTER ADD COLUMN` + - `ALTER DROP COLUMN` + - `ALTER MODIFY COLUMN` + - `ALTER COMMENT COLUMN` + - `ALTER CLEAR COLUMN` + - `ALTER RENAME COLUMN` + - `ALTER INDEX` + - `ALTER ORDER BY` + - `ALTER ADD INDEX` + - `ALTER DROP INDEX` + - `ALTER MATERIALIZE INDEX` + - `ALTER CLEAR INDEX` + - `ALTER CONSTRAINT` + - `ALTER ADD CONSTRAINT` + - `ALTER DROP CONSTRAINT` + - `ALTER TTL` + - `ALTER MATERIALIZE TTL` + - `ALTER SETTINGS` + - `ALTER MOVE PARTITION` + - `ALTER FETCH PARTITION` + - `ALTER FREEZE PARTITION` + - `ALTER VIEW` + - `ALTER VIEW REFRESH` + - `ALTER VIEW MODIFY QUERY` +- [CREATE](#grant-create) + - `CREATE DATABASE` + - `CREATE TABLE` + - `CREATE VIEW` + - `CREATE DICTIONARY` + - `CREATE TEMPORARY TABLE` +- [DROP](#grant-drop) + - `DROP DATABASE` + - `DROP TABLE` + - `DROP VIEW` + - `DROP DICTIONARY` +- [TRUNCATE](#grant-truncate) +- [OPTIMIZE](#grant-optimize) +- [SHOW](#grant-show) + - `SHOW DATABASES` + - `SHOW TABLES` + - `SHOW COLUMNS` + - `SHOW DICTIONARIES` +- [KILL QUERY](#grant-kill-query) +- [ACCESS MANAGEMENT](#grant-access-management) + - `CREATE USER` + - `ALTER USER` + - `DROP USER` + - `CREATE ROLE` + - `ALTER ROLE` + - `DROP ROLE` + - `CREATE ROW POLICY` + - `ALTER ROW POLICY` + - `DROP ROW POLICY` + - `CREATE QUOTA` + - `ALTER QUOTA` + - `DROP QUOTA` + - `CREATE SETTINGS PROFILE` + - `ALTER SETTINGS PROFILE` + - `DROP SETTINGS PROFILE` + - `SHOW ACCESS` + - `SHOW_USERS` + - `SHOW_ROLES` + - `SHOW_ROW_POLICIES` + - `SHOW_QUOTAS` + - `SHOW_SETTINGS_PROFILES` + - `ROLE ADMIN` +- [SYSTEM](#grant-system) + - `SYSTEM SHUTDOWN` + - `SYSTEM DROP CACHE` + - `SYSTEM DROP DNS CACHE` + - `SYSTEM DROP MARK CACHE` + - `SYSTEM DROP UNCOMPRESSED CACHE` + - `SYSTEM RELOAD` + - `SYSTEM RELOAD CONFIG` + - `SYSTEM RELOAD DICTIONARY` + - `SYSTEM RELOAD EMBEDDED DICTIONARIES` + - `SYSTEM MERGES` + - `SYSTEM TTL MERGES` + - `SYSTEM FETCHES` + - `SYSTEM MOVES` + - `SYSTEM SENDS` + - `SYSTEM DISTRIBUTED SENDS` + - `SYSTEM REPLICATED SENDS` + - `SYSTEM REPLICATION QUEUES` + - `SYSTEM SYNC REPLICA` + - `SYSTEM RESTART REPLICA` + - `SYSTEM FLUSH` + - `SYSTEM FLUSH DISTRIBUTED` + - `SYSTEM FLUSH LOGS` +- [INTROSPECTION](#grant-introspection) + - `addressToLine` + - `addressToSymbol` + - `demangle` +- [SOURCES](#grant-sources) + - `FILE` + - `URL` + - `REMOTE` + - `YSQL` + - `ODBC` + - `JDBC` + - `HDFS` + - `S3` +- [dictGet](#grant-dictget) + +Bu hiyerarşinin nasıl ele alındığına dair örnekler: + +- Bu `ALTER` ayrıcalık diğer her şeyi içerir `ALTER*` ayrıcalıklar. +- `ALTER CONSTRAINT` içerir `ALTER ADD CONSTRAINT` ve `ALTER DROP CONSTRAINT` ayrıcalıklar. + +Ayrıcalıklar farklı düzeylerde uygulanır. Bir seviyeyi bilmek, ayrıcalık için mevcut sözdizimini önerir. + +Seviyeler (düşükten yükseğe): + +- `COLUMN` — Privilege can be granted for column, table, database, or globally. +- `TABLE` — Privilege can be granted for table, database, or globally. +- `VIEW` — Privilege can be granted for view, database, or globally. +- `DICTIONARY` — Privilege can be granted for dictionary, database, or globally. +- `DATABASE` — Privilege can be granted for database or globally. +- `GLOBAL` — Privilege can be granted only globally. +- `GROUP` — Groups privileges of different levels. When `GROUP`- seviye ayrıcalığı verilir, yalnızca kullanılan sözdizimine karşılık gelen grup ayrıcalıkları verilir. + +İzin verilen sözdizimi örnekleri: + +- `GRANT SELECT(x) ON db.table TO user` +- `GRANT SELECT ON db.* TO user` + +İzin verilmeyen sözdizimi örnekleri: + +- `GRANT CREATE USER(x) ON db.table TO user` +- `GRANT CREATE USER ON db.* TO user` + +Özel ayrıcalık [ALL](#grant-all) bir kullanıcı hesabı veya rol için tüm ayrıcalıkları verir. + +Varsayılan olarak, bir kullanıcı hesabının veya rolün ayrıcalıkları yoktur. + +Eğer bir kullanıcı veya rolü herhangi bir imtiyaz varsa, görüntülenen bu [NONE](#grant-none) ayrıcalık. + +Bazı sorgular uygulamalarıyla bir dizi ayrıcalık gerektirir. Örneğin, yürütmek için [RENAME](../../sql-reference/statements/misc.md#misc_operations-rename) sorgu aşağıdaki ayrıcalıklara ihtiyacınız var: `SELECT`, `CREATE TABLE`, `INSERT` ve `DROP TABLE`. + +### SELECT {#grant-select} + +Çalıştırmaya izin verir [SELECT](../../sql-reference/statements/select/index.md) sorgular. + +Ayrıcalık düzeyi: `COLUMN`. + +**Açıklama** + +Bu ayrıcalığa sahip bir kullanıcı yürütebilir `SELECT` belirtilen tablo ve veritabanındaki sütunların belirtilen bir liste üzerinde sorgular. Kullanıcı başka sütunlar içeriyorsa, belirtilen sorgu herhangi bir veri döndürmez. + +Aşağıdaki ayrıcalığı göz önünde bulundurun: + +``` sql +GRANT SELECT(x,y) ON db.table TO john +``` + +Bu ayrıcalık sağlar `john` herhangi bir yürütmek için `SELECT` veri içeren sorgu `x` ve / veya `y` içindeki sütunlar `db.table`, mesela, `SELECT x FROM db.table`. `john` idam edilemiyor `SELECT z FROM db.table`. Bu `SELECT * FROM db.table` ayrıca mevcut değildir. Bu sorguyu işlerken, ClickHouse bile herhangi bir veri döndürmez `x` ve `y`. Tek istisna, bir tablo yalnızca `x` ve `y` sütunlar, bu durumda ClickHouse tüm verileri döndürür. + +### INSERT {#grant-insert} + +Çalıştırmaya izin verir [INSERT](../../sql-reference/statements/insert-into.md) sorgular. + +Ayrıcalık düzeyi: `COLUMN`. + +**Açıklama** + +Bu ayrıcalığa sahip bir kullanıcı yürütebilir `INSERT` belirtilen tablo ve veritabanındaki sütunların belirtilen bir liste üzerinde sorgular. Kullanıcı başka sütunlar içeriyorsa, belirtilen sorgu herhangi bir veri eklemez. + +**Örnek** + +``` sql +GRANT INSERT(x,y) ON db.table TO john +``` + +Verilen ayrıcalık izin verir `john` veri eklemek için `x` ve / veya `y` içindeki sütunlar `db.table`. + +### ALTER {#grant-alter} + +Çalıştırmaya izin verir [ALTER](../../sql-reference/statements/alter.md) aşağıdaki ayrıcalıklar hiyerarşisine göre sorgular: + +- `ALTER`. Düzey: `COLUMN`. + - `ALTER TABLE`. Düzey: `GROUP` + - `ALTER UPDATE`. Düzey: `COLUMN`. Takma ad: `UPDATE` + - `ALTER DELETE`. Düzey: `COLUMN`. Takma ad: `DELETE` + - `ALTER COLUMN`. Düzey: `GROUP` + - `ALTER ADD COLUMN`. Düzey: `COLUMN`. Takma ad: `ADD COLUMN` + - `ALTER DROP COLUMN`. Düzey: `COLUMN`. Takma ad: `DROP COLUMN` + - `ALTER MODIFY COLUMN`. Düzey: `COLUMN`. Takma ad: `MODIFY COLUMN` + - `ALTER COMMENT COLUMN`. Düzey: `COLUMN`. Takma ad: `COMMENT COLUMN` + - `ALTER CLEAR COLUMN`. Düzey: `COLUMN`. Takma ad: `CLEAR COLUMN` + - `ALTER RENAME COLUMN`. Düzey: `COLUMN`. Takma ad: `RENAME COLUMN` + - `ALTER INDEX`. Düzey: `GROUP`. Takma ad: `INDEX` + - `ALTER ORDER BY`. Düzey: `TABLE`. Takma ad: `ALTER MODIFY ORDER BY`, `MODIFY ORDER BY` + - `ALTER ADD INDEX`. Düzey: `TABLE`. Takma ad: `ADD INDEX` + - `ALTER DROP INDEX`. Düzey: `TABLE`. Takma ad: `DROP INDEX` + - `ALTER MATERIALIZE INDEX`. Düzey: `TABLE`. Takma ad: `MATERIALIZE INDEX` + - `ALTER CLEAR INDEX`. Düzey: `TABLE`. Takma ad: `CLEAR INDEX` + - `ALTER CONSTRAINT`. Düzey: `GROUP`. Takma ad: `CONSTRAINT` + - `ALTER ADD CONSTRAINT`. Düzey: `TABLE`. Takma ad: `ADD CONSTRAINT` + - `ALTER DROP CONSTRAINT`. Düzey: `TABLE`. Takma ad: `DROP CONSTRAINT` + - `ALTER TTL`. Düzey: `TABLE`. Takma ad: `ALTER MODIFY TTL`, `MODIFY TTL` + - `ALTER MATERIALIZE TTL`. Düzey: `TABLE`. Takma ad: `MATERIALIZE TTL` + - `ALTER SETTINGS`. Düzey: `TABLE`. Takma ad: `ALTER SETTING`, `ALTER MODIFY SETTING`, `MODIFY SETTING` + - `ALTER MOVE PARTITION`. Düzey: `TABLE`. Takma ad: `ALTER MOVE PART`, `MOVE PARTITION`, `MOVE PART` + - `ALTER FETCH PARTITION`. Düzey: `TABLE`. Takma ad: `FETCH PARTITION` + - `ALTER FREEZE PARTITION`. Düzey: `TABLE`. Takma ad: `FREEZE PARTITION` + - `ALTER VIEW` Düzey: `GROUP` + - `ALTER VIEW REFRESH`. Düzey: `VIEW`. Takma ad: `ALTER LIVE VIEW REFRESH`, `REFRESH VIEW` + - `ALTER VIEW MODIFY QUERY`. Düzey: `VIEW`. Takma ad: `ALTER TABLE MODIFY QUERY` + +Bu hiyerarşinin nasıl ele alındığına dair örnekler: + +- Bu `ALTER` ayrıcalık diğer her şeyi içerir `ALTER*` ayrıcalıklar. +- `ALTER CONSTRAINT` içerir `ALTER ADD CONSTRAINT` ve `ALTER DROP CONSTRAINT` ayrıcalıklar. + +**Not** + +- Bu `MODIFY SETTING` ayrıcalık, tablo altyapısı ayarlarını değiştirmenize izin verir. Ayarları veya sunucu yapılandırma parametrelerini etkilemez. +- Bu `ATTACH` operasyon ihtiyacı [CREATE](#grant-create) ayrıcalık. +- Bu `DETACH` operasyon ihtiyacı [DROP](#grant-drop) ayrıcalık. +- Mutasyonu durdurmak için [KILL MUTATION](../../sql-reference/statements/misc.md#kill-mutation) sorgu, bu mutasyonu başlatmak için bir ayrıcalığa sahip olmanız gerekir. Örneğin, durdurmak istiyorsanız `ALTER UPDATE` sorgu, ihtiyacınız olan `ALTER UPDATE`, `ALTER TABLE`, veya `ALTER` ayrıcalık. + +### CREATE {#grant-create} + +Çalıştırmaya izin verir [CREATE](../../sql-reference/statements/create.md) ve [ATTACH](../../sql-reference/statements/misc.md#attach) DDL-aşağıdaki ayrıcalıklar hiyerarşisine göre sorgular: + +- `CREATE`. Düzey: `GROUP` + - `CREATE DATABASE`. Düzey: `DATABASE` + - `CREATE TABLE`. Düzey: `TABLE` + - `CREATE VIEW`. Düzey: `VIEW` + - `CREATE DICTIONARY`. Düzey: `DICTIONARY` + - `CREATE TEMPORARY TABLE`. Düzey: `GLOBAL` + +**Not** + +- Oluşturulan tabloyu silmek için bir kullanıcının ihtiyacı vardır [DROP](#grant-drop). + +### DROP {#grant-drop} + +Çalıştırmaya izin verir [DROP](../../sql-reference/statements/misc.md#drop) ve [DETACH](../../sql-reference/statements/misc.md#detach) aşağıdaki ayrıcalıklar hiyerarşisine göre sorgular: + +- `DROP`. Düzey: + - `DROP DATABASE`. Düzey: `DATABASE` + - `DROP TABLE`. Düzey: `TABLE` + - `DROP VIEW`. Düzey: `VIEW` + - `DROP DICTIONARY`. Düzey: `DICTIONARY` + +### TRUNCATE {#grant-truncate} + +Çalıştırmaya izin verir [TRUNCATE](../../sql-reference/statements/misc.md#truncate-statement) sorgular. + +Ayrıcalık düzeyi: `TABLE`. + +### OPTIMIZE {#grant-optimize} + +Çalıştırmaya izin verir [OPTIMIZE TABLE](../../sql-reference/statements/misc.md#misc_operations-optimize) sorgular. + +Ayrıcalık düzeyi: `TABLE`. + +### SHOW {#grant-show} + +Çalıştırmaya izin verir `SHOW`, `DESCRIBE`, `USE`, ve `EXISTS` aşağıdaki ayrıcalıklar hiyerarşisine göre sorgular: + +- `SHOW`. Düzey: `GROUP` + - `SHOW DATABASES`. Düzey: `DATABASE`. Yürütmek için izin verir `SHOW DATABASES`, `SHOW CREATE DATABASE`, `USE ` sorgular. + - `SHOW TABLES`. Düzey: `TABLE`. Yürütmek için izin verir `SHOW TABLES`, `EXISTS
    `, `CHECK
    ` sorgular. + - `SHOW COLUMNS`. Düzey: `COLUMN`. Yürütmek için izin verir `SHOW CREATE TABLE`, `DESCRIBE` sorgular. + - `SHOW DICTIONARIES`. Düzey: `DICTIONARY`. Yürütmek için izin verir `SHOW DICTIONARIES`, `SHOW CREATE DICTIONARY`, `EXISTS ` sorgular. + +**Not** + +Bir kullanıcı aşağıdaki özelliklere sahiptir `SHOW` belirtilen tablo, sözlük veya veritabanı ile ilgili başka bir ayrıcalık varsa ayrıcalık. + +### KILL QUERY {#grant-kill-query} + +Çalıştırmaya izin verir [KILL](../../sql-reference/statements/misc.md#kill-query-statement) aşağıdaki ayrıcalıklar hiyerarşisine göre sorgular: + +Ayrıcalık düzeyi: `GLOBAL`. + +**Not** + +`KILL QUERY` ayrıcalık, bir kullanıcının diğer kullanıcıların sorgularını öldürmesine izin verir. + +### ACCESS MANAGEMENT {#grant-access-management} + +Bir kullanıcının kullanıcıları, rolleri ve satır ilkelerini yöneten sorguları yürütmesine izin verir. + +- `ACCESS MANAGEMENT`. Düzey: `GROUP` + - `CREATE USER`. Düzey: `GLOBAL` + - `ALTER USER`. Düzey: `GLOBAL` + - `DROP USER`. Düzey: `GLOBAL` + - `CREATE ROLE`. Düzey: `GLOBAL` + - `ALTER ROLE`. Düzey: `GLOBAL` + - `DROP ROLE`. Düzey: `GLOBAL` + - `ROLE ADMIN`. Düzey: `GLOBAL` + - `CREATE ROW POLICY`. Düzey: `GLOBAL`. Takma ad: `CREATE POLICY` + - `ALTER ROW POLICY`. Düzey: `GLOBAL`. Takma ad: `ALTER POLICY` + - `DROP ROW POLICY`. Düzey: `GLOBAL`. Takma ad: `DROP POLICY` + - `CREATE QUOTA`. Düzey: `GLOBAL` + - `ALTER QUOTA`. Düzey: `GLOBAL` + - `DROP QUOTA`. Düzey: `GLOBAL` + - `CREATE SETTINGS PROFILE`. Düzey: `GLOBAL`. Takma ad: `CREATE PROFILE` + - `ALTER SETTINGS PROFILE`. Düzey: `GLOBAL`. Takma ad: `ALTER PROFILE` + - `DROP SETTINGS PROFILE`. Düzey: `GLOBAL`. Takma ad: `DROP PROFILE` + - `SHOW ACCESS`. Düzey: `GROUP` + - `SHOW_USERS`. Düzey: `GLOBAL`. Takma ad: `SHOW CREATE USER` + - `SHOW_ROLES`. Düzey: `GLOBAL`. Takma ad: `SHOW CREATE ROLE` + - `SHOW_ROW_POLICIES`. Düzey: `GLOBAL`. Takma ad: `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY` + - `SHOW_QUOTAS`. Düzey: `GLOBAL`. Takma ad: `SHOW CREATE QUOTA` + - `SHOW_SETTINGS_PROFILES`. Düzey: `GLOBAL`. Takma ad: `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW CREATE PROFILE` + +Bu `ROLE ADMIN` ayrıcalık, kullanıcının yönetici seçeneğiyle kullanıcıya atanmayanlar da dahil olmak üzere herhangi bir rol atamasına ve iptal etmesine izin verir. + +### SYSTEM {#grant-system} + +Bir kullanıcının yürütmesine izin verir [SYSTEM](../../sql-reference/statements/system.md) aşağıdaki ayrıcalıklar hiyerarşisine göre sorgular. + +- `SYSTEM`. Düzey: `GROUP` + - `SYSTEM SHUTDOWN`. Düzey: `GLOBAL`. Takma ad: `SYSTEM KILL`, `SHUTDOWN` + - `SYSTEM DROP CACHE`. Takma ad: `DROP CACHE` + - `SYSTEM DROP DNS CACHE`. Düzey: `GLOBAL`. Takma ad: `SYSTEM DROP DNS`, `DROP DNS CACHE`, `DROP DNS` + - `SYSTEM DROP MARK CACHE`. Düzey: `GLOBAL`. Takma ad: `SYSTEM DROP MARK`, `DROP MARK CACHE`, `DROP MARKS` + - `SYSTEM DROP UNCOMPRESSED CACHE`. Düzey: `GLOBAL`. Takma ad: `SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, `DROP UNCOMPRESSED` + - `SYSTEM RELOAD`. Düzey: `GROUP` + - `SYSTEM RELOAD CONFIG`. Düzey: `GLOBAL`. Takma ad: `RELOAD CONFIG` + - `SYSTEM RELOAD DICTIONARY`. Düzey: `GLOBAL`. Takma ad: `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARY`, `RELOAD DICTIONARIES` + - `SYSTEM RELOAD EMBEDDED DICTIONARIES`. Düzey: `GLOBAL`. Takma Adlar: R`ELOAD EMBEDDED DICTIONARIES` + - `SYSTEM MERGES`. Düzey: `TABLE`. Takma ad: `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, `START MERGES` + - `SYSTEM TTL MERGES`. Düzey: `TABLE`. Takma ad: `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, `START TTL MERGES` + - `SYSTEM FETCHES`. Düzey: `TABLE`. Takma ad: `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, `START FETCHES` + - `SYSTEM MOVES`. Düzey: `TABLE`. Takma ad: `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, `START MOVES` + - `SYSTEM SENDS`. Düzey: `GROUP`. Takma ad: `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, `START SENDS` + - `SYSTEM DISTRIBUTED SENDS`. Düzey: `TABLE`. Takma ad: `SYSTEM STOP DISTRIBUTED SENDS`, `SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, `START DISTRIBUTED SENDS` + - `SYSTEM REPLICATED SENDS`. Düzey: `TABLE`. Takma ad: `SYSTEM STOP REPLICATED SENDS`, `SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, `START REPLICATED SENDS` + - `SYSTEM REPLICATION QUEUES`. Düzey: `TABLE`. Takma ad: `SYSTEM STOP REPLICATION QUEUES`, `SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, `START REPLICATION QUEUES` + - `SYSTEM SYNC REPLICA`. Düzey: `TABLE`. Takma ad: `SYNC REPLICA` + - `SYSTEM RESTART REPLICA`. Düzey: `TABLE`. Takma ad: `RESTART REPLICA` + - `SYSTEM FLUSH`. Düzey: `GROUP` + - `SYSTEM FLUSH DISTRIBUTED`. Düzey: `TABLE`. Takma ad: `FLUSH DISTRIBUTED` + - `SYSTEM FLUSH LOGS`. Düzey: `GLOBAL`. Takma ad: `FLUSH LOGS` + +Bu `SYSTEM RELOAD EMBEDDED DICTIONARIES` tarafından örtülü olarak verilen ayrıcalık `SYSTEM RELOAD DICTIONARY ON *.*` ayrıcalık. + +### INTROSPECTION {#grant-introspection} + +Kullanmanıza izin verir [içgözlem](../../operations/optimizing-performance/sampling-query-profiler.md) işlevler. + +- `INTROSPECTION`. Düzey: `GROUP`. Takma ad: `INTROSPECTION FUNCTIONS` + - `addressToLine`. Düzey: `GLOBAL` + - `addressToSymbol`. Düzey: `GLOBAL` + - `demangle`. Düzey: `GLOBAL` + +### SOURCES {#grant-sources} + +Harici veri kaynaklarının kullanılmasına izin verir. İçin geçerlidir [masa motorları](../../engines/table-engines/index.md) ve [tablo işlevleri](../../sql-reference/table-functions/index.md#table-functions). + +- `SOURCES`. Düzey: `GROUP` + - `FILE`. Düzey: `GLOBAL` + - `URL`. Düzey: `GLOBAL` + - `REMOTE`. Düzey: `GLOBAL` + - `YSQL`. Düzey: `GLOBAL` + - `ODBC`. Düzey: `GLOBAL` + - `JDBC`. Düzey: `GLOBAL` + - `HDFS`. Düzey: `GLOBAL` + - `S3`. Düzey: `GLOBAL` + +Bu `SOURCES` ayrıcalık, tüm kaynakların kullanılmasına izin verir. Ayrıca, her kaynak için ayrı ayrı bir ayrıcalık verebilirsiniz. Kaynakları kullanmak için ek ayrıcalıklara ihtiyacınız var. + +Örnekler: + +- İle bir tablo oluşturmak için [MySQL tablo motoru](../../engines/table-engines/integrations/mysql.md) ihtiyacınız `CREATE TABLE (ON db.table_name)` ve `MYSQL` ayrıcalıklar. +- Kullanmak için [mysql tablo işlevi](../../sql-reference/table-functions/mysql.md) ihtiyacınız `CREATE TEMPORARY TABLE` ve `MYSQL` ayrıcalıklar. + +### dictGet {#grant-dictget} + +- `dictGet`. Takma ad: `dictHas`, `dictGetHierarchy`, `dictIsIn` + +Bir kullanıcının yürütmesine izin verir [dictGet](../../sql-reference/functions/ext-dict-functions.md#dictget), [dictHas](../../sql-reference/functions/ext-dict-functions.md#dicthas), [dictGetHierarchy](../../sql-reference/functions/ext-dict-functions.md#dictgethierarchy), [dictİsİn](../../sql-reference/functions/ext-dict-functions.md#dictisin) işlevler. + +Ayrıcalık düzeyi: `DICTIONARY`. + +**Örnekler** + +- `GRANT dictGet ON mydb.mydictionary TO john` +- `GRANT dictGet ON mydictionary TO john` + +### ALL {#grant-all} + +Düzenlenmiş varlık üzerindeki tüm ayrıcalıkları bir kullanıcı hesabına veya bir role verir. + +### NONE {#grant-none} + +Herhangi bir ayrıcalık vermez. + +### ADMIN OPTION {#admin-option-privilege} + +Bu `ADMIN OPTION` ayrıcalık, kullanıcının rolünü başka bir kullanıcıya vermesine izin verir. + +[Orijinal makale](https://clickhouse.tech/docs/en/query_language/grant/) From 8095a4e2bb7674da1c45aec8eb75da734a9f4d29 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 9 Jul 2020 18:46:07 +0300 Subject: [PATCH 196/196] Update skip_list.json --- tests/queries/skip_list.json | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 14aea9715c1..955c67b0b96 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -132,7 +132,6 @@ "00933_ttl_simple", "00753_system_columns_and_system_tables", "01343_min_bytes_to_use_mmap_io", - "01344_min_bytes_to_use_mmap_io_index", - "01213_alter_rename_with_default_zookeeper" + "01344_min_bytes_to_use_mmap_io_index" ] }