From cb4801e3be9d0aea4595f6e9429375091acb79dc Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 14 Sep 2020 14:22:17 +0300 Subject: [PATCH 001/264] allow to read subcolumns of complex types --- src/Core/NamesAndTypes.cpp | 42 +++++++---- src/Core/NamesAndTypes.h | 37 ++++++++-- src/DataTypes/DataTypeArray.cpp | 30 ++++++++ src/DataTypes/DataTypeArray.h | 4 ++ src/DataTypes/DataTypeNullable.cpp | 23 +++++++ src/DataTypes/DataTypeNullable.h | 3 + src/DataTypes/DataTypeTuple.cpp | 10 +++ src/DataTypes/DataTypeTuple.h | 2 + src/DataTypes/IDataType.cpp | 23 ++++++- src/DataTypes/IDataType.h | 10 +++ src/Interpreters/TreeRewriter.cpp | 4 +- src/Storages/ColumnsDescription.cpp | 47 ++++++++++++- src/Storages/ColumnsDescription.h | 2 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- .../MergeTree/IMergedBlockOutputStream.cpp | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 7 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 4 +- .../MergeTree/MergeTreeDataPartCompact.h | 2 +- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 7 +- .../MergeTree/MergeTreeDataPartWide.h | 2 +- .../MergeTree/MergeTreeReaderWide.cpp | 28 ++++---- src/Storages/MergeTree/MergeTreeReaderWide.h | 4 +- src/Storages/StorageInMemoryMetadata.cpp | 7 +- src/Storages/StorageLog.cpp | 69 ++++++++++--------- src/Storages/StorageLog.h | 3 +- src/Storages/StorageTinyLog.cpp | 51 +++++++------- src/Storages/StorageTinyLog.h | 2 +- .../001475_read_subcolumns.reference | 21 ++++++ .../0_stateless/001475_read_subcolumns.sql | 44 ++++++++++++ 31 files changed, 384 insertions(+), 112 deletions(-) create mode 100644 tests/queries/0_stateless/001475_read_subcolumns.reference create mode 100644 tests/queries/0_stateless/001475_read_subcolumns.sql diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 3a55a4328a7..611d3d2371f 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -17,6 +17,29 @@ namespace ErrorCodes extern const int THERE_IS_NO_COLUMN; } +NameAndTypePair::NameAndTypePair( + const String & name_, const String & subcolumn_name_, + const DataTypePtr & storage_type_, const DataTypePtr & type_) + : name(name_ + "." + subcolumn_name_) + , type(type_) + , storage_type(storage_type_) + , subcolumn_delimiter_position(name_.size()) {} + +String NameAndTypePair::getStorageName() const +{ + if (subcolumn_delimiter_position == -1) + return name; + + return name.substr(0, subcolumn_delimiter_position); +} + +String NameAndTypePair::getSubcolumnName() const +{ + if (subcolumn_delimiter_position == -1) + return ""; + + return name.substr(subcolumn_delimiter_position + 1, name.size() - subcolumn_delimiter_position); +} void NamesAndTypesList::readText(ReadBuffer & buf) { @@ -137,25 +160,20 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { - /// NOTE: It's better to make a map in `IStorage` than to create it here every time again. -#if !defined(ARCADIA_BUILD) - google::dense_hash_map types; -#else - google::sparsehash::dense_hash_map types; -#endif - types.set_empty_key(StringRef()); + std::unordered_map self_columns; - for (const NameAndTypePair & column : *this) - types[column.name] = &column.type; + for (const auto & column : *this) + self_columns[column.name] = &column; NamesAndTypesList res; for (const String & name : names) { - auto it = types.find(name); - if (it == types.end()) + auto it = self_columns.find(name); + if (it == self_columns.end()) throw Exception("No column " + name, ErrorCodes::THERE_IS_NO_COLUMN); - res.emplace_back(name, *it->second); + res.emplace_back(*it->second); } + return res; } diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 28567fed3e3..40e3fc1788e 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -15,11 +15,17 @@ namespace DB struct NameAndTypePair { - String name; - DataTypePtr type; - - NameAndTypePair() {} +public: + NameAndTypePair() = default; NameAndTypePair(const String & name_, const DataTypePtr & type_) : name(name_), type(type_) {} + NameAndTypePair(const String & name_, const String & subcolumn_name_, + const DataTypePtr & storage_type_, const DataTypePtr & type_); + + String getStorageName() const; + String getSubcolumnName() const; + + bool isSubcolumn() const { return subcolumn_delimiter_position != -1; } + DataTypePtr getStorageType() const { return storage_type; } bool operator<(const NameAndTypePair & rhs) const { @@ -30,8 +36,24 @@ struct NameAndTypePair { return name == rhs.name && type->equals(*rhs.type); } + + String name; + DataTypePtr type; + +private: + DataTypePtr storage_type; + ssize_t subcolumn_delimiter_position = -1; }; +template +auto get(const NameAndTypePair & name_and_type) +{ + if constexpr (I == 0) + return name_and_type.name; + else if constexpr (I == 1) + return name_and_type.type; +} + using NamesAndTypes = std::vector; class NamesAndTypesList : public std::list @@ -81,3 +103,10 @@ public: }; } + +namespace std +{ + template <> struct tuple_size : std::integral_constant {}; + template <> struct tuple_element<0, DB::NameAndTypePair> { using type = DB::String; }; + template <> struct tuple_element<1, DB::NameAndTypePair> { using type = DB::DataTypePtr; }; +} diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 49666cca428..005e48c89b6 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -15,7 +15,10 @@ #include #include +#include +#include +#include namespace DB { @@ -499,6 +502,33 @@ bool DataTypeArray::equals(const IDataType & rhs) const return typeid(rhs) == typeid(*this) && nested->equals(*static_cast(rhs).nested); } +DataTypePtr DataTypeArray::getSubcolumnType(const String & subcolumn_name) const +{ + ReadBufferFromString buf(subcolumn_name); + size_t dim; + if (checkString("size", buf) && tryReadIntText(dim, buf) && dim < getNumberOfDimensions()) + return std::make_shared(); + + return nullptr; +} + +std::vector DataTypeArray::getSubcolumnNames() const +{ + size_t num_of_dimentions = getNumberOfDimensions(); + std::vector res(num_of_dimentions); + for (size_t i = 0; i < num_of_dimentions; ++i) + res[i] = "size" + std::to_string(i); + + return res; +} + +String DataTypeArray::getEscapedFileName(const NameAndTypePair & column) const +{ + if (column.isSubcolumn()) + return escapeForFileName(column.getStorageName()) + "." + column.getSubcolumnName(); + + return escapeForFileName(column.name); +} size_t DataTypeArray::getNumberOfDimensions() const { diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 1451f27dfbe..2923b9ac676 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -111,6 +111,10 @@ public: return nested->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion(); } + DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; + std::vector getSubcolumnNames() const override; + String getEscapedFileName(const NameAndTypePair & column) const override; + const DataTypePtr & getNestedType() const { return nested; } /// 1 for plain array, 2 for array of arrays and so on. diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 67acf89ef42..b493f225995 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -13,6 +14,7 @@ #include #include #include +#include namespace DB @@ -511,6 +513,27 @@ bool DataTypeNullable::equals(const IDataType & rhs) const return rhs.isNullable() && nested_data_type->equals(*static_cast(rhs).nested_data_type); } +DataTypePtr DataTypeNullable::getSubcolumnType(const String & subcolumn_name) const +{ + if (subcolumn_name == "null") + return std::make_shared(); + + return nullptr; +} + +std::vector DataTypeNullable::getSubcolumnNames() const +{ + return {"null"}; +} + +String DataTypeNullable::getEscapedFileName(const NameAndTypePair & column) const +{ + if (column.isSubcolumn()) + return escapeForFileName(column.getStorageName()) + "." + column.getSubcolumnName(); + + return escapeForFileName(column.name); +} + static DataTypePtr create(const ASTPtr & arguments) { diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 22d403da6c4..d1cb327d6c2 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -97,6 +97,9 @@ public: size_t getSizeOfValueInMemory() const override; bool onlyNull() const override; bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); } + DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; + std::vector getSubcolumnNames() const override; + String getEscapedFileName(const NameAndTypePair & column) const override; const DataTypePtr & getNestedType() const { return nested_data_type; } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index b69c4c31ca4..4564ea39b54 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -529,6 +529,16 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const return res; } +DataTypePtr DataTypeTuple::getSubcolumnType(const String & subcolumn_name) const +{ + return elems[getPositionByName(subcolumn_name)]; +} + +std::vector DataTypeTuple::getSubcolumnNames() const +{ + return names; +} + static DataTypePtr create(const ASTPtr & arguments) { diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index a8d16c28fa5..795eb6cf4a9 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -94,6 +94,8 @@ public: bool haveMaximumSizeOfValue() const override; size_t getMaximumSizeOfValueInMemory() const override; size_t getSizeOfValueInMemory() const override; + DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; + std::vector getSubcolumnNames() const override; const DataTypes & getElements() const { return elems; } const Strings & getElementNames() const { return names; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 561166cbc78..29310f15056 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -10,7 +10,6 @@ #include #include - namespace DB { @@ -93,6 +92,20 @@ size_t IDataType::getSizeOfValueInMemory() const throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR); } +String IDataType::getEscapedFileName(const NameAndTypePair & column) const +{ + return escapeForFileName(column.name); +} + +String IDataType::getFileNameForStream(const NameAndTypePair & column, const IDataType::SubstreamPath & path) +{ + + if (!column.isSubcolumn()) + return getFileNameForStream(column.name, path); + + auto stream_name = column.getStorageType()->getEscapedFileName(column); + return getFileNameForStreamImpl(std::move(stream_name), path); +} String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path) { @@ -105,8 +118,13 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy && path[0].type == IDataType::Substream::ArraySizes && nested_table_name != column_name; + auto stream_name = escapeForFileName(is_sizes_of_nested_type ? nested_table_name : column_name); + return getFileNameForStreamImpl(std::move(stream_name), path); +} + +String IDataType::getFileNameForStreamImpl(String stream_name, const IDataType::SubstreamPath & path) +{ size_t array_level = 0; - String stream_name = escapeForFileName(is_sizes_of_nested_type ? nested_table_name : column_name); for (const Substream & elem : path) { if (elem.type == Substream::NullMap) @@ -126,6 +144,7 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy else if (elem.type == Substream::DictionaryKeys) stream_name += ".dict"; } + return stream_name; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 6adcc0fda90..ef615a4ea9c 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -27,6 +27,8 @@ using DataTypes = std::vector; class ProtobufReader; class ProtobufWriter; +struct NameAndTypePair; + /** Properties of data type. * Contains methods for serialization/deserialization. @@ -227,6 +229,9 @@ public: virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const = 0; virtual void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const = 0; + virtual DataTypePtr getSubcolumnType(const String & /* subcolumn_path */) const { return nullptr; } + virtual std::vector getSubcolumnNames() const { return {}; } + /** Text serialization with escaping but without quoting. */ void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const; @@ -437,9 +442,12 @@ public: /// Strings, Numbers, Date, DateTime, Nullable virtual bool canBeInsideLowCardinality() const { return false; } + virtual String getEscapedFileName(const NameAndTypePair & column) const; + /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); + static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path); static String getFileNameForStream(const String & column_name, const SubstreamPath & path); private: @@ -451,6 +459,8 @@ private: mutable DataTypeCustomNamePtr custom_name; mutable DataTypeCustomTextSerializationPtr custom_text_serialization; + static String getFileNameForStreamImpl(String stream_name, const SubstreamPath & path); + public: const IDataTypeCustomName * getCustomName() const { return custom_name.get(); } }; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 0b2f8ac3eb7..784e5c7bd0a 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -373,7 +373,9 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) { const ColumnsDescription & columns = metadata_snapshot->getColumns(); - auto columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical(); + UNUSED(add_special); + // auto columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical(); + auto columns_from_storage = columns.getAllWithSubcolumns(); if (source_columns.empty()) source_columns.swap(columns_from_storage); else diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6e4bc4dc80c..0633f40c846 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -322,6 +323,19 @@ NamesAndTypesList ColumnsDescription::getAll() const return ret; } +NamesAndTypesList ColumnsDescription::getAllWithSubcolumns() const +{ + NamesAndTypesList ret; + for (const auto & col : columns) + { + ret.emplace_back(col.name, col.type); + for (const auto & subcolumn : col.type->getSubcolumnNames()) + ret.emplace_back(col.name, subcolumn, col.type, col.type->getSubcolumnType(subcolumn)); + } + + return ret; +} + bool ColumnsDescription::has(const String & column_name) const { @@ -371,13 +385,44 @@ NameAndTypePair ColumnsDescription::getPhysical(const String & column_name) cons return NameAndTypePair(it->name, it->type); } +NameAndTypePair ColumnsDescription::getPhysicalOrSubcolumn(const String & column_name) const +{ + auto it = columns.get<1>().find(column_name); + if (it != columns.get<1>().end() && it->default_desc.kind != ColumnDefaultKind::Alias) + return NameAndTypePair(it->name, it->type); + + std::optional res; + for (const auto & storage_column : columns) + { + if (startsWith(column_name, storage_column.name)) + { + ReadBufferFromString buf(column_name); + if (checkString(storage_column.name, buf) && checkChar('.', buf)) + { + String subcolumn_name; + readString(subcolumn_name, buf); + auto subcolumn_type = storage_column.type->getSubcolumnType(subcolumn_name); + if (subcolumn_type) + { + res.emplace(storage_column.name, subcolumn_name, storage_column.type, subcolumn_type); + break; + } + } + } + } + + if (!res) + throw Exception("There is no physical column or subcolumn " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + + return *res; +} + bool ColumnsDescription::hasPhysical(const String & column_name) const { auto it = columns.get<1>().find(column_name); return it != columns.get<1>().end() && it->default_desc.kind != ColumnDefaultKind::Alias; } - ColumnDefaults ColumnsDescription::getDefaults() const { ColumnDefaults ret; diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 6e2d3299091..0e4f82b7e73 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -77,6 +77,7 @@ public: NamesAndTypesList getAliases() const; NamesAndTypesList getAllPhysical() const; /// ordinary + materialized. NamesAndTypesList getAll() const; /// ordinary + materialized + aliases + NamesAndTypesList getAllWithSubcolumns() const; using ColumnTTLs = std::unordered_map; ColumnTTLs getColumnTTLs() const; @@ -106,6 +107,7 @@ public: Names getNamesOfPhysical() const; bool hasPhysical(const String & column_name) const; NameAndTypePair getPhysical(const String & column_name) const; + NameAndTypePair getPhysicalOrSubcolumn(const String & column_name) const; ColumnDefaults getDefaults() const; /// TODO: remove bool hasDefault(const String & column_name) const; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 872e34adb83..b38edaddb42 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -366,7 +366,7 @@ String IMergeTreeDataPart::getColumnNameWithMinumumCompressedSize(const StorageM if (alter_conversions.isColumnRenamed(column.name)) column_name = alter_conversions.getColumnOldName(column.name); - if (!hasColumnFiles(column_name, *column_type)) + if (!hasColumnFiles(column)) continue; const auto size = getColumnSize(column_name, *column_type).data_compressed; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7df0468dc13..3a38bd57a7f 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -324,7 +324,7 @@ public: /// NOTE: Doesn't take column renames into account, if some column renames /// take place, you must take original name of column for this part from /// storage and pass it to this method. - virtual bool hasColumnFiles(const String & /* column */, const IDataType & /* type */) const{ return false; } + virtual bool hasColumnFiles(const NameAndTypePair & /* column */) const { return false; } /// Calculate the total size of the entire directory with all the files static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk_, const String & from); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 4e98caf066a..7ec735bd5c1 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -53,7 +53,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( column.type->enumerateStreams( [&](const IDataType::SubstreamPath & substream_path) { - ++stream_counts[IDataType::getFileNameForStream(column.name, substream_path)]; + ++stream_counts[IDataType::getFileNameForStream(column, substream_path)]; }, {}); } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 03235742a68..3308e76b6d2 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -17,7 +17,6 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetada { NameSet required_columns{std::begin(columns), std::end(columns)}; NameSet injected_columns; - auto all_column_files_missing = true; const auto & storage_columns = metadata_snapshot->getColumns(); @@ -30,8 +29,10 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetada if (alter_conversions.isColumnRenamed(column_name_in_part)) column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); + auto column_in_storage = storage_columns.getPhysicalOrSubcolumn(column_name_in_part); + /// column has files and hence does not require evaluation - if (part->hasColumnFiles(column_name_in_part, *storage_columns.getPhysical(columns[i]).type)) + if (part->hasColumnFiles(column_in_storage)) { all_column_files_missing = false; continue; @@ -256,7 +257,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (check_columns) { - const NamesAndTypesList & physical_columns = metadata_snapshot->getColumns().getAllPhysical(); + const NamesAndTypesList & physical_columns = metadata_snapshot->getColumns().getAllWithSubcolumns(); result.pre_columns = physical_columns.addTypes(pre_column_names); result.columns = physical_columns.addTypes(column_names); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index c42caa2d4d4..fa5669ce0a2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -121,9 +121,9 @@ void MergeTreeDataPartCompact::loadIndexGranularity() index_granularity.setInitialized(); } -bool MergeTreeDataPartCompact::hasColumnFiles(const String & column_name, const IDataType &) const +bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const { - if (!getColumnPosition(column_name)) + if (!getColumnPosition(column.name)) return false; auto bin_checksum = checksums.files.find(DATA_FILE_NAME_WITH_EXTENSION); diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 2f2a2f537aa..2c0c4020bb0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -55,7 +55,7 @@ public: bool isStoredOnDisk() const override { return true; } - bool hasColumnFiles(const String & column_name, const IDataType & type) const override; + bool hasColumnFiles(const NameAndTypePair & column) const override; String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 1fceb47cba8..4596fe92b96 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -41,7 +41,7 @@ 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); } + bool hasColumnFiles(const NameAndTypePair & column) 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 StorageMetadataPtr & metadata_snapshot) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index c53362c847d..b1a09abb2b5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -201,13 +202,13 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const } } -bool MergeTreeDataPartWide::hasColumnFiles(const String & column_name, const IDataType & type) const +bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const { bool res = true; - type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) { - String file_name = IDataType::getFileNameForStream(column_name, substream_path); + String file_name = IDataType::getFileNameForStream(column, substream_path); auto bin_checksum = checksums.files.find(file_name + ".bin"); auto mrk_checksum = checksums.files.find(file_name + index_granularity_info.marks_file_extension); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index aa8c3aedea7..bb1a0b4790f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -54,7 +54,7 @@ public: ~MergeTreeDataPartWide() override; - bool hasColumnFiles(const String & column, const IDataType & type) const override; + bool hasColumnFiles(const NameAndTypePair & column) const override; private: void checkConsistency(bool require_part_metadata) const override; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 352d1f93589..a6780722af8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -9,7 +9,6 @@ #include #include - namespace DB { @@ -50,7 +49,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( for (const NameAndTypePair & column : columns) { auto column_from_part = getColumnFromPart(column); - addStreams(column_from_part.name, *column_from_part.type, profile_callback_, clock_type_); + addStreams(column_from_part, profile_callback_, clock_type_); } } catch (...) @@ -77,7 +76,8 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { - auto [name, type] = getColumnFromPart(*name_and_type); + auto column_from_part = getColumnFromPart(*name_and_type); + const auto & [name, type] = column_from_part; /// The column is already present in the block so we will append the values to the end. bool append = res_columns[pos] != nullptr; @@ -114,7 +114,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si { size_t column_size_before_reading = column->size(); - readData(name, *type, *column, from_mark, continue_reading, max_rows_to_read, read_offsets); + readData(column_from_part, *column, from_mark, continue_reading, max_rows_to_read, read_offsets); /// For elements of Nested, column_size_before_reading may be greater than column size /// if offsets are not empty and were already read, but elements are empty. @@ -159,12 +159,12 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si return read_rows; } -void MergeTreeReaderWide::addStreams(const String & name, const IDataType & type, +void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path) { - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(name_and_type, substream_path); if (streams.count(stream_name)) return; @@ -186,12 +186,12 @@ void MergeTreeReaderWide::addStreams(const String & name, const IDataType & type }; IDataType::SubstreamPath substream_path; - type.enumerateStreams(callback, substream_path); + name_and_type.type->enumerateStreams(callback, substream_path); } void MergeTreeReaderWide::readData( - const String & name, const IDataType & type, IColumn & column, + const NameAndTypePair & name_and_type, IColumn & column, size_t from_mark, bool continue_reading, size_t max_rows_to_read, bool with_offsets) { @@ -203,7 +203,7 @@ void MergeTreeReaderWide::readData( if (!with_offsets && substream_path.size() == 1 && substream_path[0].type == IDataType::Substream::ArraySizes) return nullptr; - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(name_and_type, substream_path); auto it = streams.find(stream_name); if (it == streams.end()) @@ -223,20 +223,20 @@ void MergeTreeReaderWide::readData( }; }; - double & avg_value_size_hint = avg_value_size_hints[name]; + double & avg_value_size_hint = avg_value_size_hints[name_and_type.name]; IDataType::DeserializeBinaryBulkSettings deserialize_settings; deserialize_settings.avg_value_size_hint = avg_value_size_hint; - if (deserialize_binary_bulk_state_map.count(name) == 0) + if (deserialize_binary_bulk_state_map.count(name_and_type.name) == 0) { deserialize_settings.getter = get_stream_getter(true); - type.deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name]); + name_and_type.type->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name_and_type.name]); } deserialize_settings.getter = get_stream_getter(false); deserialize_settings.continuous_reading = continue_reading; - auto & deserialize_state = deserialize_binary_bulk_state_map[name]; - type.deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, deserialize_settings, deserialize_state); + auto & deserialize_state = deserialize_binary_bulk_state_map[name_and_type.name]; + name_and_type.type->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, deserialize_settings, deserialize_state); IDataType::updateAvgValueSizeHint(column, avg_value_size_hint); } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 69652d1e954..8a384578685 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -37,11 +37,11 @@ private: FileStreams streams; - void addStreams(const String & name, const IDataType & type, + void addStreams(const NameAndTypePair & name_and_type, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void readData( - const String & name, const IDataType & type, IColumn & column, + const NameAndTypePair & name_and_type, IColumn & column, size_t from_mark, bool continue_reading, size_t max_rows_to_read, bool with_offsets = true); }; diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index b7f4565a55a..e4dbf27d8db 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -3,7 +3,10 @@ #include #include #include +#include #include +#include +#include namespace DB @@ -256,7 +259,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( std::unordered_map columns_map; - NamesAndTypesList all_columns = getColumns().getAll(); + NamesAndTypesList all_columns = getColumns().getAllWithSubcolumns(); for (const auto & elem : all_columns) columns_map.emplace(elem.name, elem.type); @@ -445,7 +448,7 @@ namespace void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const { - NamesAndTypesList available_columns = getColumns().getAllPhysical(); + NamesAndTypesList available_columns = getColumns().getAllWithSubcolumns(); available_columns.insert(available_columns.end(), virtuals.begin(), virtuals.end()); const String list_of_columns = listOfColumns(available_columns); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 45ab3293723..4c07aa93d82 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -107,7 +107,7 @@ private: using DeserializeStates = std::map; DeserializeStates deserialize_states; - void readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read); + void readData(const NameAndTypePair & name_and_type, IColumn & column, size_t max_rows_to_read); }; @@ -185,9 +185,9 @@ private: using SerializeStates = std::map; SerializeStates serialize_states; - IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams); + IDataType::OutputStreamGetter createStreamGetter(const NameAndTypePair & name_and_type, WrittenStreams & written_streams); - void writeData(const String & name, const IDataType & type, const IColumn & column, + void writeData(const NameAndTypePair & name_and_type, const IColumn & column, MarksForColumns & out_marks, WrittenStreams & written_streams); @@ -214,7 +214,7 @@ Chunk LogSource::generate() try { - readData(name_type.name, *name_type.type, *column, max_rows_to_read); + readData(name_type, *column, max_rows_to_read); } catch (Exception & e) { @@ -244,15 +244,16 @@ Chunk LogSource::generate() } -void LogSource::readData(const String & name, const IDataType & type, IColumn & column, size_t max_rows_to_read) +void LogSource::readData(const NameAndTypePair & name_and_type, IColumn & column, size_t max_rows_to_read) { IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. + const auto & [name, type] = name_and_type; auto create_string_getter = [&](bool stream_for_prefix) { return [&, stream_for_prefix] (const IDataType::SubstreamPath & path) -> ReadBuffer * { - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(name_and_type, path); const auto & file_it = storage.files.find(stream_name); if (storage.files.end() == file_it) @@ -271,11 +272,11 @@ void LogSource::readData(const String & name, const IDataType & type, IColumn & if (deserialize_states.count(name) == 0) { settings.getter = create_string_getter(true); - type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + type->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); } settings.getter = create_string_getter(false); - type.deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name]); + type->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name]); } @@ -292,7 +293,7 @@ void LogBlockOutputStream::write(const Block & block) for (size_t i = 0; i < block.columns(); ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); - writeData(column.name, *column.type, *column.column, marks, written_streams); + writeData(NameAndTypePair(column.name, column.type), *column.column, marks, written_streams); } writeMarks(std::move(marks)); @@ -311,7 +312,7 @@ void LogBlockOutputStream::writeSuffix() auto it = serialize_states.find(column.name); if (it != serialize_states.end()) { - settings.getter = createStreamGetter(column.name, written_streams); + settings.getter = createStreamGetter(NameAndTypePair(column.name, column.type), written_streams); column.type->serializeBinaryBulkStateSuffix(settings, it->second); } } @@ -337,12 +338,12 @@ void LogBlockOutputStream::writeSuffix() } -IDataType::OutputStreamGetter LogBlockOutputStream::createStreamGetter(const String & name, +IDataType::OutputStreamGetter LogBlockOutputStream::createStreamGetter(const NameAndTypePair & name_and_type, WrittenStreams & written_streams) { return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * { - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(name_and_type, path); if (written_streams.count(stream_name)) return nullptr; @@ -355,14 +356,15 @@ IDataType::OutputStreamGetter LogBlockOutputStream::createStreamGetter(const Str } -void LogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, +void LogBlockOutputStream::writeData(const NameAndTypePair & name_and_type, const IColumn & column, MarksForColumns & out_marks, WrittenStreams & written_streams) { IDataType::SerializeBinaryBulkSettings settings; + const auto & [name, type] = name_and_type; - type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + type->enumerateStreams([&] (const IDataType::SubstreamPath & path) { - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(name_and_type, path); if (written_streams.count(stream_name)) return; @@ -371,18 +373,18 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type stream_name, storage.disk, storage.files[stream_name].data_file_path, - columns.getCodecOrDefault(name), + columns.getCodecOrDefault(name_and_type.name), storage.max_compress_block_size); }, settings.path); - settings.getter = createStreamGetter(name, written_streams); + settings.getter = createStreamGetter(name_and_type, written_streams); if (serialize_states.count(name) == 0) - type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]); + type->serializeBinaryBulkStatePrefix(settings, serialize_states[name]); - type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + type->enumerateStreams([&] (const IDataType::SubstreamPath & path) { - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(name_and_type, path); if (written_streams.count(stream_name)) return; @@ -396,11 +398,11 @@ void LogBlockOutputStream::writeData(const String & name, const IDataType & type out_marks.emplace_back(file.column_index, mark); }, settings.path); - type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); + type->serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); - type.enumerateStreams([&] (const IDataType::SubstreamPath & path) + type->enumerateStreams([&] (const IDataType::SubstreamPath & path) { - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(name_and_type, path); if (!written_streams.emplace(stream_name).second) return; @@ -469,7 +471,7 @@ StorageLog::StorageLog( } for (const auto & column : storage_metadata.getColumns().getAllPhysical()) - addFiles(column.name, *column.type); + addFiles(column); marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; @@ -479,15 +481,15 @@ StorageLog::StorageLog( } -void StorageLog::addFiles(const String & column_name, const IDataType & type) +void StorageLog::addFiles(const NameAndTypePair & column) { - if (files.end() != files.find(column_name)) - throw Exception("Duplicate column with name " + column_name + " in constructor of StorageLog.", + if (files.end() != files.find(column.name)) + throw Exception("Duplicate column with name " + column.name + " in constructor of StorageLog.", ErrorCodes::DUPLICATE_COLUMN); IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path) { - String stream_name = IDataType::getFileNameForStream(column_name, substream_path); + String stream_name = IDataType::getFileNameForStream(column, substream_path); if (!files.count(stream_name)) { @@ -501,7 +503,7 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) }; IDataType::SubstreamPath substream_path; - type.enumerateStreams(stream_callback, substream_path); + column.type->enumerateStreams(stream_callback, substream_path); } @@ -573,7 +575,7 @@ void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn disk->clearDirectory(table_path); for (const auto & column : metadata_snapshot->getColumns().getAllPhysical()) - addFiles(column.name, *column.type); + addFiles(column); file_checker = FileChecker{disk, table_path + "sizes.json"}; marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME; @@ -583,8 +585,7 @@ void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_sn const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMetadataPtr & metadata_snapshot) const { /// There should be at least one physical column - const String column_name = metadata_snapshot->getColumns().getAllPhysical().begin()->name; - const auto column_type = metadata_snapshot->getColumns().getAllPhysical().begin()->type; + auto column = *metadata_snapshot->getColumns().getAllPhysical().begin(); String filename; /** We take marks from first column. @@ -592,10 +593,10 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta * (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays). */ IDataType::SubstreamPath substream_root_path; - column_type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) { if (filename.empty()) - filename = IDataType::getFileNameForStream(column_name, substream_path); + filename = IDataType::getFileNameForStream(column, substream_path); }, substream_root_path); Files::const_iterator it = files.find(filename); diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 49fc9a576c5..990a5204261 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -92,7 +93,7 @@ private: String marks_file_path; /// The order of adding files should not change: it corresponds to the order of the columns in the marks file. - void addFiles(const String & column_name, const IDataType & type); + void addFiles(const NameAndTypePair & column); bool loaded_marks = false; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index e7fe7e2d5f9..c22f09826da 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -102,7 +102,7 @@ private: using DeserializeStates = std::map; DeserializeStates deserialize_states; - void readData(const String & name, const IDataType & type, IColumn & column, UInt64 limit); + void readData(const NameAndTypePair & name_and_type, IColumn & column, UInt64 limit); }; @@ -169,8 +169,8 @@ private: using WrittenStreams = std::set; - IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenStreams & written_streams); - void writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams); + IDataType::OutputStreamGetter createStreamGetter(const NameAndTypePair & column, WrittenStreams & written_streams); + void writeData(const NameAndTypePair & name_and_type, const IColumn & column, WrittenStreams & written_streams); }; @@ -199,7 +199,7 @@ Chunk TinyLogSource::generate() try { - readData(name_type.name, *name_type.type, *column, block_size); + readData(name_type, *column, block_size); } catch (Exception & e) { @@ -222,12 +222,13 @@ Chunk TinyLogSource::generate() } -void TinyLogSource::readData(const String & name, const IDataType & type, IColumn & column, UInt64 limit) +void TinyLogSource::readData(const NameAndTypePair & name_and_type, IColumn & column, UInt64 limit) { IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. + const auto & [name, type] = name_and_type; settings.getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * { - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(name_and_type, path); if (!streams.count(stream_name)) streams[stream_name] = std::make_unique(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size); @@ -236,19 +237,19 @@ void TinyLogSource::readData(const String & name, const IDataType & type, IColum }; if (deserialize_states.count(name) == 0) - type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + type->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); - type.deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]); + type->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]); } IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter( - const String & name, + const NameAndTypePair & column, WrittenStreams & written_streams) { return [&] (const IDataType::SubstreamPath & path) -> WriteBuffer * { - String stream_name = IDataType::getFileNameForStream(name, path); + String stream_name = IDataType::getFileNameForStream(column, path); if (!written_streams.insert(stream_name).second) return nullptr; @@ -258,7 +259,7 @@ IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter( streams[stream_name] = std::make_unique( storage.disk, storage.files[stream_name].data_file_path, - columns.getCodecOrDefault(name), + columns.getCodecOrDefault(column.name), storage.max_compress_block_size); return &streams[stream_name]->compressed; @@ -266,15 +267,16 @@ IDataType::OutputStreamGetter TinyLogBlockOutputStream::createStreamGetter( } -void TinyLogBlockOutputStream::writeData(const String & name, const IDataType & type, const IColumn & column, WrittenStreams & written_streams) +void TinyLogBlockOutputStream::writeData(const NameAndTypePair & name_and_type, const IColumn & column, WrittenStreams & written_streams) { IDataType::SerializeBinaryBulkSettings settings; - settings.getter = createStreamGetter(name, written_streams); + const auto & [name, type] = name_and_type; + settings.getter = createStreamGetter(name_and_type, written_streams); if (serialize_states.count(name) == 0) - type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]); + type->serializeBinaryBulkStatePrefix(settings, serialize_states[name]); - type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); + type->serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); } @@ -297,7 +299,7 @@ void TinyLogBlockOutputStream::writeSuffix() auto it = serialize_states.find(column.name); if (it != serialize_states.end()) { - settings.getter = createStreamGetter(column.name, written_streams); + settings.getter = createStreamGetter(NameAndTypePair(column.name, column.type), written_streams); column.type->serializeBinaryBulkStateSuffix(settings, it->second); } } @@ -329,7 +331,7 @@ void TinyLogBlockOutputStream::write(const Block & block) for (size_t i = 0; i < block.columns(); ++i) { const ColumnWithTypeAndName & column = block.safeGetByPosition(i); - writeData(column.name, *column.type, *column.column, written_streams); + writeData(NameAndTypePair(column.name, column.type), *column.column, written_streams); } } @@ -375,7 +377,7 @@ StorageTinyLog::StorageTinyLog( } for (const auto & col : storage_metadata.getColumns().getAllPhysical()) - addFiles(col.name, *col.type); + addFiles(col); if (!attach) for (const auto & file : files) @@ -383,15 +385,16 @@ StorageTinyLog::StorageTinyLog( } -void StorageTinyLog::addFiles(const String & column_name, const IDataType & type) +void StorageTinyLog::addFiles(const NameAndTypePair & column) { - if (files.end() != files.find(column_name)) - throw Exception("Duplicate column with name " + column_name + " in constructor of StorageTinyLog.", + const auto & [name, type] = column; + if (files.end() != files.find(name)) + throw Exception("Duplicate column with name " + name + " in constructor of StorageTinyLog.", ErrorCodes::DUPLICATE_COLUMN); IDataType::StreamCallback stream_callback = [&] (const IDataType::SubstreamPath & substream_path) { - String stream_name = IDataType::getFileNameForStream(column_name, substream_path); + String stream_name = IDataType::getFileNameForStream(column, substream_path); if (!files.count(stream_name)) { ColumnData column_data; @@ -401,7 +404,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type }; IDataType::SubstreamPath substream_path; - type.enumerateStreams(stream_callback, substream_path); + type->enumerateStreams(stream_callback, substream_path); } @@ -461,7 +464,7 @@ void StorageTinyLog::truncate( file_checker = FileChecker{disk, table_path + "sizes.json"}; for (const auto & column : metadata_snapshot->getColumns().getAllPhysical()) - addFiles(column.name, *column.type); + addFiles(column); } void StorageTinyLog::drop() diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index dc6ff101503..091cda93d7a 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -74,7 +74,7 @@ private: Poco::Logger * log; - void addFiles(const String & column_name, const IDataType & type); + void addFiles(const NameAndTypePair & column); }; } diff --git a/tests/queries/0_stateless/001475_read_subcolumns.reference b/tests/queries/0_stateless/001475_read_subcolumns.reference new file mode 100644 index 00000000000..336c11fe775 --- /dev/null +++ b/tests/queries/0_stateless/001475_read_subcolumns.reference @@ -0,0 +1,21 @@ +====array==== +1 +0 +3 +2 +2 +====tuple==== +foo +bar +baz +1 +2 +42 +2 +2 +====nullable==== +0 +1 +0 +1 +2 diff --git a/tests/queries/0_stateless/001475_read_subcolumns.sql b/tests/queries/0_stateless/001475_read_subcolumns.sql new file mode 100644 index 00000000000..7c405c521d2 --- /dev/null +++ b/tests/queries/0_stateless/001475_read_subcolumns.sql @@ -0,0 +1,44 @@ +SELECT '====array===='; +DROP TABLE IF EXISTS t_arr; +CREATE TABLE t_arr (a Array(UInt32)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t_arr VALUES ([1]) ([]) ([1, 2, 3]) ([1, 2]); + +SYSTEM DROP MARK CACHE; +SELECT a.size0 FROM t_arr; + +SYSTEM FLUSH LOGS; +SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] +FROM system.query_log +WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT a.size0 FROM %t_arr%')) + AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + +SELECT '====tuple===='; +DROP TABLE IF EXISTS t_tup; +CREATE TABLE t_tup (t Tuple(s String, u UInt32)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t_tup VALUES (('foo', 1)) (('bar', 2)) (('baz', 42)); + +SYSTEM DROP MARK CACHE; +SELECT t.s FROM t_tup; + +SYSTEM DROP MARK CACHE; +SELECT t.u FROM t_tup; + +SYSTEM FLUSH LOGS; +SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] +FROM system.query_log +WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT t._ FROM %t_tup%')) + AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + +SELECT '====nullable===='; +DROP TABLE IF EXISTS t_nul; +CREATE TABLE t_nul (n Nullable(UInt32)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO t_nul VALUES (1) (NULL) (2) (NULL); + +SYSTEM DROP MARK CACHE; +SELECT n.null FROM t_nul; + +SYSTEM FLUSH LOGS; +SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] +FROM system.query_log +WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT n.null FROM %t_nul%')) + AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); From 13917969ab5ff5901b2acd7550d8a25ed2351ad0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 18 Sep 2020 20:37:08 +0300 Subject: [PATCH 002/264] better storing info about subcolumns --- src/Storages/ColumnsDescription.cpp | 67 +++++++++---------- src/Storages/ColumnsDescription.h | 5 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 4 +- 3 files changed, 38 insertions(+), 38 deletions(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 2679357291b..df62bfdc67c 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -185,6 +185,7 @@ void ColumnsDescription::add(ColumnDescription column, const String & after_colu insert_it = range.second; } + addSubcolumns(NameAndTypePair(column.name, column.type)); columns.get<0>().insert(insert_it, std::move(column)); } @@ -339,7 +340,8 @@ NamesAndTypesList ColumnsDescription::getAllWithSubcolumns() const bool ColumnsDescription::has(const String & column_name) const { - return columns.get<1>().find(column_name) != columns.get<1>().end(); + return columns.get<1>().find(column_name) != columns.get<1>().end() + || subcolumns.find(column_name) != subcolumns.end(); } bool ColumnsDescription::hasNested(const String & column_name) const @@ -385,42 +387,21 @@ NameAndTypePair ColumnsDescription::getPhysical(const String & column_name) cons return NameAndTypePair(it->name, it->type); } -std::optional ColumnsDescription::tryGetPhysicalOrSubcolumn(const String & column_name) const -{ - auto it = columns.get<1>().find(column_name); - if (it != columns.get<1>().end() && it->default_desc.kind != ColumnDefaultKind::Alias) - return NameAndTypePair(it->name, it->type); - - std::optional res; - for (const auto & storage_column : columns) - { - if (startsWith(column_name, storage_column.name)) - { - ReadBufferFromString buf(column_name); - if (checkString(storage_column.name, buf) && checkChar('.', buf)) - { - String subcolumn_name; - readString(subcolumn_name, buf); - auto subcolumn_type = storage_column.type->getSubcolumnType(subcolumn_name); - if (subcolumn_type) - { - res.emplace(storage_column.name, subcolumn_name, storage_column.type, subcolumn_type); - break; - } - } - } - } - - return res; -} - NameAndTypePair ColumnsDescription::getPhysicalOrSubcolumn(const String & column_name) const { - auto res = tryGetPhysicalOrSubcolumn(column_name); - if (!res) - throw Exception("There is no physical column or subcolumn " + column_name + " in table.", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + if (auto it = columns.get<1>().find(column_name); it != columns.get<1>().end() + && it->default_desc.kind != ColumnDefaultKind::Alias) + { + return NameAndTypePair(it->name, it->type); + } - return *res; + if (auto it = subcolumns.find(column_name); it != subcolumns.end()) + { + return it->second; + } + + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no physical column or subcolumn {} in table.", column_name); } bool ColumnsDescription::hasPhysical(const String & column_name) const @@ -431,7 +412,7 @@ bool ColumnsDescription::hasPhysical(const String & column_name) const bool ColumnsDescription::hasPhysicalOrSubcolumn(const String & column_name) const { - return tryGetPhysicalOrSubcolumn(column_name) != std::nullopt; + return hasPhysical(column_name) || subcolumns.find(column_name) != subcolumns.end(); } ColumnDefaults ColumnsDescription::getDefaults() const @@ -521,13 +502,27 @@ ColumnsDescription ColumnsDescription::parse(const String & str) ColumnDescription column; column.readText(buf); buf.ignore(1); /// ignore new line - result.add(std::move(column)); + result.add(column); } assertEOF(buf); return result; } +void ColumnsDescription::addSubcolumns(NameAndTypePair storage_column) +{ + for (const auto & subcolumn_name : storage_column.type->getSubcolumnNames()) + { + auto subcolumn = NameAndTypePair(storage_column.name, subcolumn_name, + storage_column.type, storage_column.type->getSubcolumnType(subcolumn_name)); + + if (has(subcolumn.name)) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Cannot add subcolumn {}: column with this name already exists", subcolumn.name); + + subcolumns[subcolumn.name] = subcolumn; + } +} Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, const Context & context) { diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 803d62e256c..4d5c9fc799b 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -142,8 +142,11 @@ public: private: Container columns; + using SubcolumnsContainer = std::unordered_map; + SubcolumnsContainer subcolumns; + void modifyColumnOrder(const String & column_name, const String & after_column, bool first); - std::optional tryGetPhysicalOrSubcolumn(const String & column_name) const; + void addSubcolumns(NameAndTypePair storage_column); }; /// Validate default expressions and corresponding types compatibility, i.e. diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index f45428873c1..d6bb68660c0 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -37,8 +37,10 @@ bool injectRequiredColumnsRecursively( if (alter_conversions.isColumnRenamed(column_name_in_part)) column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); + auto storage_column = storage_columns.getPhysicalOrSubcolumn(column_name_in_part); + /// column has files and hence does not require evaluation - if (storage_columns.hasPhysical(column_name) && part->hasColumnFiles(column_name_in_part, *storage_columns.getPhysical(column_name).type)) + if (storage_columns.hasPhysicalOrSubcolumn(column_name) && part->hasColumnFiles(storage_column)) { /// ensure each column is added only once if (required_columns.count(column_name) == 0) From cbe12a532e222052978d59cf0f9ef141233dc4b1 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Oct 2020 15:46:17 +0300 Subject: [PATCH 003/264] allow to extract subcolumns from column --- src/DataTypes/DataTypeArray.cpp | 22 ++-- src/DataTypes/DataTypeArray.h | 6 +- src/DataTypes/DataTypeLowCardinality.h | 1 + src/DataTypes/DataTypeNullable.cpp | 11 +- src/DataTypes/DataTypeNullable.h | 2 +- src/DataTypes/DataTypeTuple.cpp | 44 +++++++- src/DataTypes/DataTypeTuple.h | 4 +- src/DataTypes/IDataType.cpp | 104 +++++++++++++----- src/DataTypes/IDataType.h | 14 ++- src/Storages/ColumnsDescription.cpp | 11 +- src/Storages/ColumnsDescription.h | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 + src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 39 +++++-- .../MergeTree/MergeTreeReaderCompact.h | 2 +- .../0_stateless/001475_read_subcolumns_2.sql | 48 ++++++++ .../001475_read_subcolumns_storages.sh | 23 ++++ 18 files changed, 272 insertions(+), 69 deletions(-) create mode 100644 tests/queries/0_stateless/001475_read_subcolumns_2.sql create mode 100755 tests/queries/0_stateless/001475_read_subcolumns_storages.sh diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 530d02ba68f..0c94aa693be 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -509,17 +509,25 @@ DataTypePtr DataTypeArray::getSubcolumnType(const String & subcolumn_name) const if (checkString("size", buf) && tryReadIntText(dim, buf) && dim < getNumberOfDimensions()) return std::make_shared(); - return nullptr; + return std::make_shared(nested->getSubcolumnType(subcolumn_name)); } -std::vector DataTypeArray::getSubcolumnNames() const +MutableColumnPtr DataTypeArray::getSubcolumn(const String & subcolumn_name, IColumn & column) const { - size_t num_of_dimentions = getNumberOfDimensions(); - std::vector res(num_of_dimentions); - for (size_t i = 0; i < num_of_dimentions; ++i) - res[i] = "size" + std::to_string(i); + return getSubcolumnImpl(subcolumn_name, column, 0); +} - return res; +MutableColumnPtr DataTypeArray::getSubcolumnImpl(const String & subcolumn_name, IColumn & column, size_t level) const +{ + auto & column_array = assert_cast(column); + if (subcolumn_name == "size" + std::to_string(level)) + return column_array.getOffsetsPtr()->assumeMutable(); + + if (const auto * nested_array = typeid_cast(nested.get())) + return nested_array->getSubcolumnImpl(subcolumn_name, column, level + 1); + + auto subcolumn = nested->getSubcolumn(subcolumn_name, column_array.getData()); + return ColumnArray::create(std::move(subcolumn), column_array.getOffsetsPtr()->assumeMutable()); } String DataTypeArray::getEscapedFileName(const NameAndTypePair & column) const diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 2923b9ac676..b10c8b32b5e 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -112,13 +112,17 @@ public: } DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; - std::vector getSubcolumnNames() const override; + MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; + String getEscapedFileName(const NameAndTypePair & column) const override; const DataTypePtr & getNestedType() const { return nested; } /// 1 for plain array, 2 for array of arrays and so on. size_t getNumberOfDimensions() const; + +private: + MutableColumnPtr getSubcolumnImpl(const String & subcolumn_name, IColumn & column, size_t level) const; }; } diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index f8c314909b8..0c6c1c4b7cb 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -23,6 +23,7 @@ public: TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + DataTypePtr getSubcolumnType(const String & /* subcolumn_name */) const override { return shared_from_this(); } void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 8120fea1106..ffbd918c538 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -529,15 +529,20 @@ bool DataTypeNullable::equals(const IDataType & rhs) const DataTypePtr DataTypeNullable::getSubcolumnType(const String & subcolumn_name) const { + std::cerr << "(DataTypeNullable::getSubcolumnType) subcolumn_name: " << subcolumn_name << "\n"; if (subcolumn_name == "null") return std::make_shared(); - return nullptr; + return nested_data_type->getSubcolumnType(subcolumn_name); } -std::vector DataTypeNullable::getSubcolumnNames() const +MutableColumnPtr DataTypeNullable::getSubcolumn(const String & subcolumn_name, IColumn & column) const { - return {"null"}; + auto & column_nullable = assert_cast(column); + if (subcolumn_name == "null") + return column_nullable.getNullMapColumnPtr()->assumeMutable(); + + return nested_data_type->getSubcolumn(subcolumn_name, column_nullable.getNestedColumn()); } String DataTypeNullable::getEscapedFileName(const NameAndTypePair & column) const diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index c62f9eb64c8..a2d5fcc53bf 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -98,7 +98,7 @@ public: bool onlyNull() const override; bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); } DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; - std::vector getSubcolumnNames() const override; + MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; String getEscapedFileName(const NameAndTypePair & column) const override; const DataTypePtr & getNestedType() const { return nested_data_type; } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 4564ea39b54..f7cfdb903ca 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -31,6 +32,7 @@ namespace ErrorCodes extern const int DUPLICATE_COLUMN; extern const int BAD_ARGUMENTS; extern const int NOT_FOUND_COLUMN_IN_BLOCK; + extern const int ILLEGAL_COLUMN; } @@ -419,6 +421,14 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } +String DataTypeTuple::getEscapedFileName(const NameAndTypePair & column) const +{ + if (column.isSubcolumn()) + return escapeForFileName(column.getStorageName()) + "%2E" + column.getSubcolumnName(); + + return escapeForFileName(column.name); +} + void DataTypeTuple::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { for (; value_index < elems.size(); ++value_index) @@ -531,12 +541,40 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const DataTypePtr DataTypeTuple::getSubcolumnType(const String & subcolumn_name) const { - return elems[getPositionByName(subcolumn_name)]; + for (size_t i = 0; i < names.size(); ++i) + { + if (startsWith(subcolumn_name, names[i])) + { + size_t name_length = names[i].size(); + if (subcolumn_name.size() == name_length) + return elems[i]; + + if (subcolumn_name[name_length] == '.') + return elems[i]->getSubcolumnType(subcolumn_name.substr(name_length + 1)); + } + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } -std::vector DataTypeTuple::getSubcolumnNames() const +MutableColumnPtr DataTypeTuple::getSubcolumn(const String & subcolumn_name, IColumn & column) const { - return names; + for (size_t i = 0; i < names.size(); ++i) + { + if (startsWith(subcolumn_name, names[i])) + { + size_t name_length = names[i].size(); + auto & subcolumn = extractElementColumn(column, i); + + if (subcolumn_name.size() == name_length) + return subcolumn.assumeMutable(); + + if (subcolumn_name[name_length] == '.') + return elems[i]->getSubcolumn(subcolumn_name.substr(name_length + 1), subcolumn); + } + } + + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 795eb6cf4a9..6ae3804f2e9 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -94,8 +94,10 @@ public: bool haveMaximumSizeOfValue() const override; size_t getMaximumSizeOfValueInMemory() const override; size_t getSizeOfValueInMemory() const override; + DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; - std::vector getSubcolumnNames() const override; + MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; + String getEscapedFileName(const NameAndTypePair & column) const override; const DataTypes & getElements() const { return elems; } const Strings & getElementNames() const { return names; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 7eb7f3a346e..05936b28f62 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes extern const int MULTIPLE_STREAMS_REQUIRED; extern const int LOGICAL_ERROR; extern const int DATA_TYPE_CANNOT_BE_PROMOTED; + extern const int ILLEGAL_COLUMN; } IDataType::IDataType() : custom_name(nullptr), custom_text_serialization(nullptr) @@ -92,19 +93,83 @@ size_t IDataType::getSizeOfValueInMemory() const throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR); } +DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const +{ + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); +} + +MutableColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, IColumn &) const +{ + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); +} + +std::vector IDataType::getSubcolumnNames() const +{ + std::vector res; + enumerateStreams([&res](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + { + auto subcolumn_name = IDataType::getSubcolumnNameForStream("", substream_path); + if (!subcolumn_name.empty()) + res.push_back(subcolumn_name.substr(1)); // It starts with a dot. + }); + + return res; +} + String IDataType::getEscapedFileName(const NameAndTypePair & column) const { return escapeForFileName(column.name); } +static String getNameForSubstreamPath( + String stream_name, + const IDataType::SubstreamPath & path, + const String & tuple_element_delimeter = ".") +{ + size_t array_level = 0; + for (const auto & elem : path) + { + if (elem.type == IDataType::Substream::NullMap) + stream_name += ".null"; + else if (elem.type == IDataType::Substream::ArraySizes) + stream_name += ".size" + toString(array_level); + else if (elem.type == IDataType::Substream::ArrayElements) + ++array_level; + else if (elem.type == IDataType::Substream::TupleElement) + stream_name += tuple_element_delimeter + escapeForFileName(elem.tuple_element_name); + else if (elem.type == IDataType::Substream::DictionaryKeys) + stream_name += ".dict"; + } + + return stream_name; +} + + +/// FIXME: rewrite it. String IDataType::getFileNameForStream(const NameAndTypePair & column, const IDataType::SubstreamPath & path) { - if (!column.isSubcolumn()) return getFileNameForStream(column.name, path); - auto stream_name = column.getStorageType()->getEscapedFileName(column); - return getFileNameForStreamImpl(std::move(stream_name), path); + String storage_name = column.getStorageName(); + String nested_table_name = Nested::extractTableName(storage_name); + + bool is_sizes_of_nested_type = + (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes + && nested_table_name != storage_name) || column.getSubcolumnName() == "size0"; + + String stream_name; + if (is_sizes_of_nested_type) + { + if (column.getSubcolumnName() == "size0") + return escapeForFileName(nested_table_name) + ".size0"; + + stream_name = escapeForFileName(Nested::extractTableName(storage_name)); + } + else + stream_name = column.getStorageType()->getEscapedFileName(column); + + return getNameForSubstreamPath(std::move(stream_name), path, "%2E"); } String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path) @@ -119,36 +184,19 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy && nested_table_name != column_name; auto stream_name = escapeForFileName(is_sizes_of_nested_type ? nested_table_name : column_name); - return getFileNameForStreamImpl(std::move(stream_name), path); + + /// For compatibility reasons, we use %2E instead of dot. + /// Because nested data may be represented not by Array of Tuple, + /// but by separate Array columns with names in a form of a.b, + /// and name is encoded as a whole. + return getNameForSubstreamPath(std::move(stream_name), path, "%2E"); } -String IDataType::getFileNameForStreamImpl(String stream_name, const IDataType::SubstreamPath & path) +String IDataType::getSubcolumnNameForStream(String stream_name, const SubstreamPath & path) { - size_t array_level = 0; - for (const Substream & elem : path) - { - if (elem.type == Substream::NullMap) - stream_name += ".null"; - else if (elem.type == Substream::ArraySizes) - stream_name += ".size" + toString(array_level); - else if (elem.type == Substream::ArrayElements) - ++array_level; - else if (elem.type == Substream::TupleElement) - { - /// For compatibility reasons, we use %2E instead of dot. - /// Because nested data may be represented not by Array of Tuple, - /// but by separate Array columns with names in a form of a.b, - /// and name is encoded as a whole. - stream_name += "%2E" + escapeForFileName(elem.tuple_element_name); - } - else if (elem.type == Substream::DictionaryKeys) - stream_name += ".dict"; - } - - return stream_name; + return getNameForSubstreamPath(std::move(stream_name), path); } - bool IDataType::isSpecialCompressionAllowed(const SubstreamPath & path) { for (const Substream & elem : path) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 8c81bf86261..0a88886e348 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -37,7 +37,7 @@ struct NameAndTypePair; * * DataType is totally immutable object. You can always share them. */ -class IDataType : private boost::noncopyable +class IDataType : private boost::noncopyable, public std::enable_shared_from_this { public: IDataType(); @@ -115,6 +115,10 @@ public: void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } + virtual DataTypePtr getSubcolumnType(const String & subcolumn_name) const; + virtual MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const; + std::vector getSubcolumnNames() const; + using OutputStreamGetter = std::function; using InputStreamGetter = std::function; @@ -152,6 +156,8 @@ public: bool position_independent_encoding = true; /// If not zero, may be used to avoid reallocations while reading column of String type. double avg_value_size_hint = 0; + + std::vector temporary_column_holders; }; /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. @@ -230,9 +236,6 @@ public: virtual void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const = 0; virtual void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const = 0; - virtual DataTypePtr getSubcolumnType(const String & /* subcolumn_path */) const { return nullptr; } - virtual std::vector getSubcolumnNames() const { return {}; } - /** Text serialization with escaping but without quoting. */ void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const; @@ -450,6 +453,7 @@ public: static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path); static String getFileNameForStream(const String & column_name, const SubstreamPath & path); + static String getSubcolumnNameForStream(String stream_name, const SubstreamPath & path); /// Substream path supports special compression methods like codec Delta. /// For all other substreams (like ArraySizes, NullMasks, etc.) we use only @@ -464,8 +468,6 @@ private: mutable DataTypeCustomNamePtr custom_name; mutable DataTypeCustomTextSerializationPtr custom_text_serialization; - static String getFileNameForStreamImpl(String stream_name, const SubstreamPath & path); - public: const IDataTypeCustomName * getCustomName() const { return custom_name.get(); } }; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index a638f15f1ba..bf10dc654dc 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -185,7 +185,7 @@ void ColumnsDescription::add(ColumnDescription column, const String & after_colu insert_it = range.second; } - addSubcolumns(NameAndTypePair(column.name, column.type)); + addSubcolumns(column.name, column.type); columns.get<0>().insert(insert_it, std::move(column)); } @@ -519,12 +519,13 @@ ColumnsDescription ColumnsDescription::parse(const String & str) return result; } -void ColumnsDescription::addSubcolumns(NameAndTypePair storage_column) +void ColumnsDescription::addSubcolumns(const String & storage_name, const DataTypePtr & storage_type) { - for (const auto & subcolumn_name : storage_column.type->getSubcolumnNames()) + for (const auto & subcolumn_name : storage_type->getSubcolumnNames()) { - auto subcolumn = NameAndTypePair(storage_column.name, subcolumn_name, - storage_column.type, storage_column.type->getSubcolumnType(subcolumn_name)); + std::cerr << "storage_name: " << storage_name << ", subcolumn_name: " << subcolumn_name << "\n"; + auto subcolumn = NameAndTypePair(storage_name, subcolumn_name, + storage_type, storage_type->getSubcolumnType(subcolumn_name)); if (has(subcolumn.name)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index c3988da5ff8..507d234f6bc 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -147,7 +147,7 @@ private: SubcolumnsContainer subcolumns; void modifyColumnOrder(const String & column_name, const String & after_column, bool first); - void addSubcolumns(NameAndTypePair storage_column); + void addSubcolumns(const String & storage_name, const DataTypePtr & storage_type); }; /// Validate default expressions and corresponding types compatibility, i.e. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index a60194bb05b..f6db3ac70d3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -191,6 +191,11 @@ std::optional IMergeTreeDataPart::getColumnPosition(const String & colum return it->second; } +std::optional IMergeTreeDataPart::getColumnPosition(const NameAndTypePair & column) const +{ + return getColumnPosition(column.getStorageName()); +} + DayNum IMergeTreeDataPart::getMinDate() const { if (storage.minmax_idx_date_column_pos != -1 && minmax_idx.initialized) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index c4936bc263a..51c5999f7c4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -142,6 +142,7 @@ public: /// take place, you must take original name of column for this part from /// storage and pass it to this method. std::optional getColumnPosition(const String & column_name) const; + std::optional getColumnPosition(const NameAndTypePair & column) const; /// Returns the name of a column with minimum compressed size (as returned by getColumnSize()). /// If no checksums are present returns the name of the first physically existing column. diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 5740eef6810..75b216c989f 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -265,7 +265,7 @@ IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const St { if (typeid_cast(part_column.type.get())) { - auto position = data_part->getColumnPosition(part_column.name); + auto position = data_part->getColumnPosition(part_column); if (position && Nested::extractTableName(part_column.name) == table_name) return position; } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 87b3f0a4329..41af0040fc5 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -53,14 +53,14 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( auto name_and_type = columns.begin(); for (size_t i = 0; i < columns_num; ++i, ++name_and_type) { - const auto & [name, type] = getColumnFromPart(*name_and_type); - auto position = data_part->getColumnPosition(name); + auto column_from_part = getColumnFromPart(*name_and_type); + auto position = data_part->getColumnPosition(column_from_part); - if (!position && typeid_cast(type.get())) + if (!position && typeid_cast(column_from_part.type.get())) { /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. - position = findColumnForOffsets(name); + position = findColumnForOffsets(column_from_part.name); read_only_offsets[i] = (position != std::nullopt); } @@ -149,14 +149,14 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, if (!res_columns[pos]) continue; - auto [name, type] = getColumnFromPart(*name_and_type); + auto column_from_part = getColumnFromPart(*name_and_type); auto & column = mutable_columns[pos]; try { size_t column_size_before_reading = column->size(); - readData(name, *column, *type, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); + readData(column_from_part, *column, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); size_t read_rows_in_column = column->size() - column_size_before_reading; @@ -170,7 +170,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, storage.reportBrokenPart(data_part->name); /// Better diagnostics. - e.addMessage("(while reading column " + name + ")"); + e.addMessage("(while reading column " + column_from_part.name + ")"); throw; } catch (...) @@ -199,9 +199,11 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, } void MergeTreeReaderCompact::readData( - const String & name, IColumn & column, const IDataType & type, + const NameAndTypePair & name_and_type, IColumn & column, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) { + const auto & [name, type] = name_and_type; + if (!isContinuousReading(from_mark, column_position)) seekToMark(from_mark, column_position); @@ -213,14 +215,29 @@ void MergeTreeReaderCompact::readData( return data_buffer; }; + IDataType::DeserializeBinaryBulkStatePtr state; IDataType::DeserializeBinaryBulkSettings deserialize_settings; deserialize_settings.getter = buffer_getter; deserialize_settings.avg_value_size_hint = avg_value_size_hints[name]; deserialize_settings.position_independent_encoding = true; - IDataType::DeserializeBinaryBulkStatePtr state; - type.deserializeBinaryBulkStatePrefix(deserialize_settings, state); - type.deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state); + if (name_and_type.isSubcolumn()) + { + const auto & storage_type = name_and_type.getStorageType(); + auto temp_column = storage_type->createColumn(); + + storage_type->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + storage_type->deserializeBinaryBulkWithMultipleStreams(*temp_column, rows_to_read, deserialize_settings, state); + + auto subcolumn = storage_type->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column); + column.insertRangeFrom(*subcolumn, 0, subcolumn->size()); + } + else + { + deserialize_settings.position_independent_encoding = true; + type->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + type->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state); + } /// The buffer is left in inconsistent state after reading single offsets if (only_offsets) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 9ef88716579..27cabaa2a2f 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -56,7 +56,7 @@ private: void seekToMark(size_t row_index, size_t column_index); - void readData(const String & name, IColumn & column, const IDataType & type, + void readData(const NameAndTypePair & name_and_type, IColumn & column, size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets = false); /// Returns maximal value of granule size in compressed file from @mark_ranges. diff --git a/tests/queries/0_stateless/001475_read_subcolumns_2.sql b/tests/queries/0_stateless/001475_read_subcolumns_2.sql new file mode 100644 index 00000000000..18868a3863a --- /dev/null +++ b/tests/queries/0_stateless/001475_read_subcolumns_2.sql @@ -0,0 +1,48 @@ +DROP TABLE IF EXISTS subcolumns; + +CREATE TABLE subcolumns +( + t Tuple + ( + a Array(Nullable(UInt32)), + u UInt32, + s Nullable(String) + ), + arr Array(Nullable(String)), + arr2 Array(Array(LowCardinality(Nullable(String)))), + lc LowCardinality(String), + nested Nested(col1 String, col2 Nullable(UInt32)) +) +ENGINE = MergeTree order by tuple() SETTINGS min_bytes_for_wide_part = '10M'; + +INSERT INTO subcolumns VALUES (([1, NULL], 2, 'a'), ['foo', NULL, 'bar'], [['123'], ['456', '789']], 'qqqq', ['zzz', 'xxx'], [42, 43]); +SELECT * FROM subcolumns; +SELECT t.a, t.u, t.s, nested.col1, nested.col2, lc FROM subcolumns; +SELECT t.a.size0, t.a.null, t.u, t.s, t.s.null FROM subcolumns; +-- SELECT arr2, arr2.size0, arr2.size1, arr2.null FROM subcolumns; +-- SELECT nested.col1, nested.col2, nested.col1.size0, nested.col2.size0, nested.col2.null FROM subcolumns; +SELECT sumArray(arr.null), sum(arr.size0) FROM subcolumns; +DROP TABLE IF EXISTS subcolumns; + +CREATE TABLE subcolumns +( + t Tuple + ( + a Array(Nullable(UInt32)), + u UInt32, + s Nullable(String) + ), + arr Array(Nullable(String)), + arr2 Array(Array(LowCardinality(Nullable(String)))), + lc LowCardinality(String), + nested Nested(col1 String, col2 Nullable(UInt32)) +) +ENGINE = MergeTree order by tuple() SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO subcolumns VALUES (([1, NULL], 2, 'a'), ['foo', NULL, 'bar'], [['123'], ['456', '789']], 'qqqq', ['zzz', 'xxx'], [42, 43]); +SELECT * FROM subcolumns; +SELECT t.a, t.u, t.s, nested.col1, nested.col2, lc FROM subcolumns; +SELECT t.a.size0, t.a.null, t.u, t.s, t.s.null FROM subcolumns; +-- SELECT arr2, arr2.size0, arr2.size1, arr2.null FROM subcolumns; +-- SELECT nested.col1, nested.col2, nested.col1.size0, nested.col2.size0, nested.col2.null FROM subcolumns; +SELECT sumArray(arr.null), sum(arr.size0) FROM subcolumns; diff --git a/tests/queries/0_stateless/001475_read_subcolumns_storages.sh b/tests/queries/0_stateless/001475_read_subcolumns_storages.sh new file mode 100755 index 00000000000..2304726bf6f --- /dev/null +++ b/tests/queries/0_stateless/001475_read_subcolumns_storages.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +set -e + +create_query="CREATE TABLE subcolumns(n Nullable(UInt32), a1 Array(UInt32),\ + a2 Array(Array(Array(UInt32))), a3 Array(Nullable(UInt32)), t Tuple(s String, v UInt32))" + +declare -a ENGINES=("Log" "StripeLog" "TinyLog" "Memory" \ + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_compact_part='10M'" + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part='10M'" + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part=0") + +for engine in "${ENGINES[@]}"; do + echo $engine + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS subcolumns" + $CLICKHOUSE_CLIENT --query "$create_query ENGINE = $engine" + $CLICKHOUSE_CLIENT --query "INSERT INTO subcolumns VALUES (100, [1, 2, 3], [[[1, 2], [], [4]], [[5, 6], [7, 8]], [[]]], [1, NULL, 2], ('foo', 200))" + $CLICKHOUSE_CLIENT --query "SELECT * FROM subcolumns" + $CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v FROM subcolumns" +done From c75262120a8149a406e0674fa67c7df96464252f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 14 Oct 2020 20:47:14 +0300 Subject: [PATCH 004/264] better subcolumns for arrays --- src/Core/NamesAndTypes.h | 2 +- src/DataTypes/DataTypeArray.cpp | 56 +++++++++---- src/DataTypes/DataTypeArray.h | 11 ++- src/DataTypes/DataTypeLowCardinality.h | 1 - src/DataTypes/DataTypeNullable.cpp | 13 +-- src/DataTypes/DataTypeNullable.h | 3 +- src/DataTypes/DataTypeTuple.cpp | 14 +--- src/DataTypes/DataTypeTuple.h | 3 +- src/DataTypes/IDataType.cpp | 84 ++++++++++++++----- src/DataTypes/IDataType.h | 9 +- src/Storages/ColumnsDescription.cpp | 1 - ...erence => 01475_read_subcolumns.reference} | 0 ...bcolumns.sql => 01475_read_subcolumns.sql} | 6 +- .../01475_read_subcolumns_2.reference | 10 +++ ...umns_2.sql => 01475_read_subcolumns_2.sql} | 13 +-- .../01475_read_subcolumns_storages.sh | 23 +++++ 16 files changed, 164 insertions(+), 85 deletions(-) rename tests/queries/0_stateless/{001475_read_subcolumns.reference => 01475_read_subcolumns.reference} (100%) rename tests/queries/0_stateless/{001475_read_subcolumns.sql => 01475_read_subcolumns.sql} (91%) create mode 100644 tests/queries/0_stateless/01475_read_subcolumns_2.reference rename tests/queries/0_stateless/{001475_read_subcolumns_2.sql => 01475_read_subcolumns_2.sql} (81%) create mode 100755 tests/queries/0_stateless/01475_read_subcolumns_storages.sh diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 40e3fc1788e..26378a2a0f0 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -46,7 +46,7 @@ private: }; template -auto get(const NameAndTypePair & name_and_type) +decltype(auto) get(const NameAndTypePair & name_and_type) { if constexpr (I == 0) return name_and_type.name; diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 0c94aa693be..04513ed204a 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -32,8 +32,8 @@ namespace ErrorCodes } -DataTypeArray::DataTypeArray(const DataTypePtr & nested_) - : nested{nested_} +DataTypeArray::DataTypeArray(const DataTypePtr & nested_, size_t nested_level_) + : nested{nested_}, nested_level{nested_level_} { } @@ -148,6 +148,23 @@ namespace offset_values.resize(i); } + + MutableColumnPtr getArraySizesPositionIndependent(ColumnArray & column_array) + { + ColumnArray::Offsets & offset_values = column_array.getOffsets(); + MutableColumnPtr new_offsets = column_array.getOffsetsColumn().cloneEmpty(); + + if (offset_values.empty()) + return new_offsets; + + auto & new_offsets_values = assert_cast &>(*new_offsets).getData(); + new_offsets_values.reserve(offset_values.size()); + new_offsets_values.push_back(offset_values[0]); + for (size_t i = 1; i < offset_values.size(); ++i) + new_offsets_values.push_back(offset_values[i] - offset_values[i - 1]); + + return new_offsets; + } } @@ -502,14 +519,23 @@ bool DataTypeArray::equals(const IDataType & rhs) const return typeid(rhs) == typeid(*this) && nested->equals(*static_cast(rhs).nested); } -DataTypePtr DataTypeArray::getSubcolumnType(const String & subcolumn_name) const +DataTypePtr DataTypeArray::tryGetSubcolumnType(const String & subcolumn_name) const { - ReadBufferFromString buf(subcolumn_name); - size_t dim; - if (checkString("size", buf) && tryReadIntText(dim, buf) && dim < getNumberOfDimensions()) + return tryGetSubcolumnTypeImpl(subcolumn_name, 0); +} + +DataTypePtr DataTypeArray::tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const +{ + if (subcolumn_name == "size" + std::to_string(level)) return std::make_shared(); - return std::make_shared(nested->getSubcolumnType(subcolumn_name)); + DataTypePtr subcolumn; + if (const auto * nested_array = typeid_cast(nested.get())) + subcolumn = nested_array->tryGetSubcolumnTypeImpl(subcolumn_name, level + 1); + else + subcolumn = nested->tryGetSubcolumnType(subcolumn_name); + + return (subcolumn ? std::make_shared(std::move(subcolumn), nested_level + 1) : subcolumn); } MutableColumnPtr DataTypeArray::getSubcolumn(const String & subcolumn_name, IColumn & column) const @@ -521,23 +547,17 @@ MutableColumnPtr DataTypeArray::getSubcolumnImpl(const String & subcolumn_name, { auto & column_array = assert_cast(column); if (subcolumn_name == "size" + std::to_string(level)) - return column_array.getOffsetsPtr()->assumeMutable(); + return getArraySizesPositionIndependent(column_array); + MutableColumnPtr subcolumn; if (const auto * nested_array = typeid_cast(nested.get())) - return nested_array->getSubcolumnImpl(subcolumn_name, column, level + 1); + subcolumn = nested_array->getSubcolumnImpl(subcolumn_name, column_array.getData(), level + 1); + else + subcolumn = nested->getSubcolumn(subcolumn_name, column_array.getData()); - auto subcolumn = nested->getSubcolumn(subcolumn_name, column_array.getData()); return ColumnArray::create(std::move(subcolumn), column_array.getOffsetsPtr()->assumeMutable()); } -String DataTypeArray::getEscapedFileName(const NameAndTypePair & column) const -{ - if (column.isSubcolumn()) - return escapeForFileName(column.getStorageName()) + "." + column.getSubcolumnName(); - - return escapeForFileName(column.name); -} - size_t DataTypeArray::getNumberOfDimensions() const { const DataTypeArray * nested_array = typeid_cast(nested.get()); diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index b10c8b32b5e..c03e34aa848 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -13,10 +13,12 @@ private: /// The type of array elements. DataTypePtr nested; + size_t nested_level = 0; + public: static constexpr bool is_parametric = true; - DataTypeArray(const DataTypePtr & nested_); + DataTypeArray(const DataTypePtr & nested_, size_t nested_level_ = 0); TypeIndex getTypeId() const override { return TypeIndex::Array; } @@ -35,6 +37,8 @@ public: return false; } + size_t getNestedLevel() const override { return nested_level; } + void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; @@ -111,11 +115,9 @@ public: return nested->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion(); } - DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; + DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; - String getEscapedFileName(const NameAndTypePair & column) const override; - const DataTypePtr & getNestedType() const { return nested; } /// 1 for plain array, 2 for array of arrays and so on. @@ -123,6 +125,7 @@ public: private: MutableColumnPtr getSubcolumnImpl(const String & subcolumn_name, IColumn & column, size_t level) const; + DataTypePtr tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const; }; } diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 0c6c1c4b7cb..f8c314909b8 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -23,7 +23,6 @@ public: TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; - DataTypePtr getSubcolumnType(const String & /* subcolumn_name */) const override { return shared_from_this(); } void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index ffbd918c538..806083a2a73 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -527,13 +527,12 @@ bool DataTypeNullable::equals(const IDataType & rhs) const return rhs.isNullable() && nested_data_type->equals(*static_cast(rhs).nested_data_type); } -DataTypePtr DataTypeNullable::getSubcolumnType(const String & subcolumn_name) const +DataTypePtr DataTypeNullable::tryGetSubcolumnType(const String & subcolumn_name) const { - std::cerr << "(DataTypeNullable::getSubcolumnType) subcolumn_name: " << subcolumn_name << "\n"; if (subcolumn_name == "null") return std::make_shared(); - return nested_data_type->getSubcolumnType(subcolumn_name); + return nested_data_type->tryGetSubcolumnType(subcolumn_name); } MutableColumnPtr DataTypeNullable::getSubcolumn(const String & subcolumn_name, IColumn & column) const @@ -545,14 +544,6 @@ MutableColumnPtr DataTypeNullable::getSubcolumn(const String & subcolumn_name, I return nested_data_type->getSubcolumn(subcolumn_name, column_nullable.getNestedColumn()); } -String DataTypeNullable::getEscapedFileName(const NameAndTypePair & column) const -{ - if (column.isSubcolumn()) - return escapeForFileName(column.getStorageName()) + "." + column.getSubcolumnName(); - - return escapeForFileName(column.name); -} - static DataTypePtr create(const ASTPtr & arguments) { diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index a2d5fcc53bf..0bb5ace2f2a 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -97,9 +97,8 @@ public: size_t getSizeOfValueInMemory() const override; bool onlyNull() const override; bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); } - DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; + DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; - String getEscapedFileName(const NameAndTypePair & column) const override; const DataTypePtr & getNestedType() const { return nested_data_type; } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index f7cfdb903ca..01b6f0ccf0b 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -421,14 +421,6 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } -String DataTypeTuple::getEscapedFileName(const NameAndTypePair & column) const -{ - if (column.isSubcolumn()) - return escapeForFileName(column.getStorageName()) + "%2E" + column.getSubcolumnName(); - - return escapeForFileName(column.name); -} - void DataTypeTuple::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const { for (; value_index < elems.size(); ++value_index) @@ -539,7 +531,7 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const return res; } -DataTypePtr DataTypeTuple::getSubcolumnType(const String & subcolumn_name) const +DataTypePtr DataTypeTuple::tryGetSubcolumnType(const String & subcolumn_name) const { for (size_t i = 0; i < names.size(); ++i) { @@ -550,11 +542,11 @@ DataTypePtr DataTypeTuple::getSubcolumnType(const String & subcolumn_name) const return elems[i]; if (subcolumn_name[name_length] == '.') - return elems[i]->getSubcolumnType(subcolumn_name.substr(name_length + 1)); + return elems[i]->tryGetSubcolumnType(subcolumn_name.substr(name_length + 1)); } } - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); + return nullptr; } MutableColumnPtr DataTypeTuple::getSubcolumn(const String & subcolumn_name, IColumn & column) const diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 6ae3804f2e9..1c5f956ee8f 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -95,9 +95,8 @@ public: size_t getMaximumSizeOfValueInMemory() const override; size_t getSizeOfValueInMemory() const override; - DataTypePtr getSubcolumnType(const String & subcolumn_name) const override; + DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; - String getEscapedFileName(const NameAndTypePair & column) const override; const DataTypes & getElements() const { return elems; } const Strings & getElementNames() const { return names; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 05936b28f62..6ffd88460fd 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { @@ -95,6 +96,10 @@ size_t IDataType::getSizeOfValueInMemory() const DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const { + auto subcolumn_type = tryGetSubcolumnType(subcolumn_name); + if (subcolumn_type) + return subcolumn_type; + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } @@ -106,21 +111,21 @@ MutableColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, IColumn std::vector IDataType::getSubcolumnNames() const { std::vector res; - enumerateStreams([&res](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + enumerateStreams([&res, this](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { auto subcolumn_name = IDataType::getSubcolumnNameForStream("", substream_path); if (!subcolumn_name.empty()) - res.push_back(subcolumn_name.substr(1)); // It starts with a dot. + { + subcolumn_name = subcolumn_name.substr(1); // It starts with a dot. + /// Not all of substreams have its subcolumn. + if (tryGetSubcolumnType(subcolumn_name)) + res.push_back(subcolumn_name); + } }); return res; } -String IDataType::getEscapedFileName(const NameAndTypePair & column) const -{ - return escapeForFileName(column.name); -} - static String getNameForSubstreamPath( String stream_name, const IDataType::SubstreamPath & path, @@ -144,30 +149,65 @@ static String getNameForSubstreamPath( return stream_name; } +static bool isOldStyleNestedSizes(const NameAndTypePair & column, const IDataType::SubstreamPath & path) +{ + auto storage_name = column.getStorageName(); + auto nested_storage_name = Nested::extractTableName(column.getStorageName()); + + if (storage_name == nested_storage_name) + return false; + + return (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes) || column.getSubcolumnName() == "size0"; +} + +static String getDelimiterForSubcolumnPart(const String & subcolumn_part) +{ + if (subcolumn_part == "null" || startsWith(subcolumn_part, "size")) + return "."; + + return "%2E"; +} /// FIXME: rewrite it. String IDataType::getFileNameForStream(const NameAndTypePair & column, const IDataType::SubstreamPath & path) { - if (!column.isSubcolumn()) - return getFileNameForStream(column.name, path); + auto storage_name = column.getStorageName(); + if (isOldStyleNestedSizes(column, path)) + storage_name = Nested::extractTableName(storage_name); - String storage_name = column.getStorageName(); - String nested_table_name = Nested::extractTableName(storage_name); + auto stream_name = escapeForFileName(storage_name); + auto subcolumn_name = column.getSubcolumnName(); - bool is_sizes_of_nested_type = - (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes - && nested_table_name != storage_name) || column.getSubcolumnName() == "size0"; - - String stream_name; - if (is_sizes_of_nested_type) + if (!subcolumn_name.empty()) { - if (column.getSubcolumnName() == "size0") - return escapeForFileName(nested_table_name) + ".size0"; + std::vector subcolumn_parts; + boost::split(subcolumn_parts, subcolumn_name, [](char c) { return c == '.'; }); - stream_name = escapeForFileName(Nested::extractTableName(storage_name)); + size_t current_nested_level = 0; + for (const auto & elem : path) + { + if (elem.type == Substream::ArrayElements && elem.is_part_of_nested) + { + ++current_nested_level; + } + else if (elem.type == Substream::ArraySizes) + { + size_t nested_level = column.type->getNestedLevel(); + + for (size_t i = 0; i < nested_level - current_nested_level; ++i) + { + if (subcolumn_parts.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get substream name for column {}." + " Not enough subcolumn parts. Needed: {}", column.name, nested_level - current_nested_level); + + subcolumn_parts.pop_back(); + } + } + } + + for (const auto & subcolumn_part : subcolumn_parts) + stream_name += getDelimiterForSubcolumnPart(subcolumn_part) + escapeForFileName(subcolumn_part); } - else - stream_name = column.getStorageType()->getEscapedFileName(column); return getNameForSubstreamPath(std::move(stream_name), path, "%2E"); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 0a88886e348..84d6995a7bd 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -37,7 +37,7 @@ struct NameAndTypePair; * * DataType is totally immutable object. You can always share them. */ -class IDataType : private boost::noncopyable, public std::enable_shared_from_this +class IDataType : private boost::noncopyable { public: IDataType(); @@ -101,6 +101,8 @@ public: /// Index of tuple element, starting at 1. String tuple_element_name; + bool is_part_of_nested = false; + Substream(Type type_) : type(type_) {} }; @@ -115,7 +117,8 @@ public: void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } - virtual DataTypePtr getSubcolumnType(const String & subcolumn_name) const; + virtual DataTypePtr tryGetSubcolumnType(const String & /* subcolumn_name */) const { return nullptr; } + DataTypePtr getSubcolumnType(const String & subcolumn_name) const; virtual MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const; std::vector getSubcolumnNames() const; @@ -446,7 +449,7 @@ public: /// Strings, Numbers, Date, DateTime, Nullable virtual bool canBeInsideLowCardinality() const { return false; } - virtual String getEscapedFileName(const NameAndTypePair & column) const; + virtual size_t getNestedLevel() const { return 0; } /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index bf10dc654dc..241adde685a 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -523,7 +523,6 @@ void ColumnsDescription::addSubcolumns(const String & storage_name, const DataTy { for (const auto & subcolumn_name : storage_type->getSubcolumnNames()) { - std::cerr << "storage_name: " << storage_name << ", subcolumn_name: " << subcolumn_name << "\n"; auto subcolumn = NameAndTypePair(storage_name, subcolumn_name, storage_type, storage_type->getSubcolumnType(subcolumn_name)); diff --git a/tests/queries/0_stateless/001475_read_subcolumns.reference b/tests/queries/0_stateless/01475_read_subcolumns.reference similarity index 100% rename from tests/queries/0_stateless/001475_read_subcolumns.reference rename to tests/queries/0_stateless/01475_read_subcolumns.reference diff --git a/tests/queries/0_stateless/001475_read_subcolumns.sql b/tests/queries/0_stateless/01475_read_subcolumns.sql similarity index 91% rename from tests/queries/0_stateless/001475_read_subcolumns.sql rename to tests/queries/0_stateless/01475_read_subcolumns.sql index 7c405c521d2..06b2d3795de 100644 --- a/tests/queries/0_stateless/001475_read_subcolumns.sql +++ b/tests/queries/0_stateless/01475_read_subcolumns.sql @@ -1,6 +1,6 @@ SELECT '====array===='; DROP TABLE IF EXISTS t_arr; -CREATE TABLE t_arr (a Array(UInt32)) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE t_arr (a Array(UInt32)) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO t_arr VALUES ([1]) ([]) ([1, 2, 3]) ([1, 2]); SYSTEM DROP MARK CACHE; @@ -14,7 +14,7 @@ WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT a.size0 FROM % SELECT '====tuple===='; DROP TABLE IF EXISTS t_tup; -CREATE TABLE t_tup (t Tuple(s String, u UInt32)) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE t_tup (t Tuple(s String, u UInt32)) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO t_tup VALUES (('foo', 1)) (('bar', 2)) (('baz', 42)); SYSTEM DROP MARK CACHE; @@ -31,7 +31,7 @@ WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT t._ FROM %t_tu SELECT '====nullable===='; DROP TABLE IF EXISTS t_nul; -CREATE TABLE t_nul (n Nullable(UInt32)) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE t_nul (n Nullable(UInt32)) ENGINE = MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0; INSERT INTO t_nul VALUES (1) (NULL) (2) (NULL); SYSTEM DROP MARK CACHE; diff --git a/tests/queries/0_stateless/01475_read_subcolumns_2.reference b/tests/queries/0_stateless/01475_read_subcolumns_2.reference new file mode 100644 index 00000000000..31c929a9133 --- /dev/null +++ b/tests/queries/0_stateless/01475_read_subcolumns_2.reference @@ -0,0 +1,10 @@ +([1,NULL],2,'a') ['foo',NULL,'bar'] [['123'],['456','789']] qqqq ['zzz','xxx'] [42,43] +[1,NULL] 2 a ['zzz','xxx'] [42,43] qqqq +2 [0,1] 2 a 0 +1 3 +[['123'],['456','789']] 2 [1,2] [[0],[0,0]] +([1,NULL],2,'a') ['foo',NULL,'bar'] [['123'],['456','789']] qqqq ['zzz','xxx'] [42,43] +[1,NULL] 2 a ['zzz','xxx'] [42,43] qqqq +2 [0,1] 2 a 0 +1 3 +[['123'],['456','789']] 2 [1,2] [[0],[0,0]] diff --git a/tests/queries/0_stateless/001475_read_subcolumns_2.sql b/tests/queries/0_stateless/01475_read_subcolumns_2.sql similarity index 81% rename from tests/queries/0_stateless/001475_read_subcolumns_2.sql rename to tests/queries/0_stateless/01475_read_subcolumns_2.sql index 18868a3863a..b8959cf27f7 100644 --- a/tests/queries/0_stateless/001475_read_subcolumns_2.sql +++ b/tests/queries/0_stateless/01475_read_subcolumns_2.sql @@ -9,7 +9,7 @@ CREATE TABLE subcolumns s Nullable(String) ), arr Array(Nullable(String)), - arr2 Array(Array(LowCardinality(Nullable(String)))), + arr2 Array(Array(Nullable(String))), lc LowCardinality(String), nested Nested(col1 String, col2 Nullable(UInt32)) ) @@ -19,9 +19,10 @@ INSERT INTO subcolumns VALUES (([1, NULL], 2, 'a'), ['foo', NULL, 'bar'], [['123 SELECT * FROM subcolumns; SELECT t.a, t.u, t.s, nested.col1, nested.col2, lc FROM subcolumns; SELECT t.a.size0, t.a.null, t.u, t.s, t.s.null FROM subcolumns; --- SELECT arr2, arr2.size0, arr2.size1, arr2.null FROM subcolumns; --- SELECT nested.col1, nested.col2, nested.col1.size0, nested.col2.size0, nested.col2.null FROM subcolumns; SELECT sumArray(arr.null), sum(arr.size0) FROM subcolumns; +SELECT arr2, arr2.size0, arr2.size1, arr2.null FROM subcolumns; +-- SELECT nested.col1, nested.col2, nested.col1.size0, nested.col2.size0, nested.col2.null FROM subcolumns; + DROP TABLE IF EXISTS subcolumns; CREATE TABLE subcolumns @@ -33,7 +34,7 @@ CREATE TABLE subcolumns s Nullable(String) ), arr Array(Nullable(String)), - arr2 Array(Array(LowCardinality(Nullable(String)))), + arr2 Array(Array(Nullable(String))), lc LowCardinality(String), nested Nested(col1 String, col2 Nullable(UInt32)) ) @@ -43,6 +44,6 @@ INSERT INTO subcolumns VALUES (([1, NULL], 2, 'a'), ['foo', NULL, 'bar'], [['123 SELECT * FROM subcolumns; SELECT t.a, t.u, t.s, nested.col1, nested.col2, lc FROM subcolumns; SELECT t.a.size0, t.a.null, t.u, t.s, t.s.null FROM subcolumns; --- SELECT arr2, arr2.size0, arr2.size1, arr2.null FROM subcolumns; --- SELECT nested.col1, nested.col2, nested.col1.size0, nested.col2.size0, nested.col2.null FROM subcolumns; SELECT sumArray(arr.null), sum(arr.size0) FROM subcolumns; +SELECT arr2, arr2.size0, arr2.size1, arr2.null FROM subcolumns; +-- SELECT nested.col1, nested.col2, nested.size0, nested.size0, nested.col2.null FROM subcolumns; diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh new file mode 100755 index 00000000000..2304726bf6f --- /dev/null +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +set -e + +create_query="CREATE TABLE subcolumns(n Nullable(UInt32), a1 Array(UInt32),\ + a2 Array(Array(Array(UInt32))), a3 Array(Nullable(UInt32)), t Tuple(s String, v UInt32))" + +declare -a ENGINES=("Log" "StripeLog" "TinyLog" "Memory" \ + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_compact_part='10M'" + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part='10M'" + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part=0") + +for engine in "${ENGINES[@]}"; do + echo $engine + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS subcolumns" + $CLICKHOUSE_CLIENT --query "$create_query ENGINE = $engine" + $CLICKHOUSE_CLIENT --query "INSERT INTO subcolumns VALUES (100, [1, 2, 3], [[[1, 2], [], [4]], [[5, 6], [7, 8]], [[]]], [1, NULL, 2], ('foo', 200))" + $CLICKHOUSE_CLIENT --query "SELECT * FROM subcolumns" + $CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v FROM subcolumns" +done From 06dc0155e5c65c3b8214ff4c814c864d85113d08 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Oct 2020 02:02:20 +0300 Subject: [PATCH 005/264] allow to read subhcolumns from other storages --- src/Core/NamesAndTypes.cpp | 2 +- src/Core/NamesAndTypes.h | 2 +- src/DataTypes/DataTypeArray.cpp | 4 +- src/DataTypes/IDataType.cpp | 21 ------ src/DataTypes/IDataType.h | 1 - .../MergeTree/IMergedBlockOutputStream.cpp | 11 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 8 +-- .../MergeTree/MergeTreeDataPartWide.cpp | 14 ++-- .../MergeTree/MergeTreeDataPartWide.h | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 10 +-- .../MergeTreeDataPartWriterCompact.h | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 71 +++++++++---------- .../MergeTree/MergeTreeDataPartWriterWide.h | 17 ++--- .../MergeTree/MergeTreeReaderInMemory.cpp | 27 +++++-- src/Storages/MergeTree/checkDataPart.cpp | 2 +- src/Storages/StorageLog.cpp | 23 +++--- src/Storages/StorageMemory.cpp | 16 +++-- src/Storages/StorageTinyLog.cpp | 21 +++--- .../01475_read_subcolumns_storages.reference | 18 +++++ .../01475_read_subcolumns_storages.sh | 7 +- 20 files changed, 146 insertions(+), 133 deletions(-) create mode 100644 tests/queries/0_stateless/01475_read_subcolumns_storages.reference diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 611d3d2371f..ab44665fd3b 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -160,7 +160,7 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const { - std::unordered_map self_columns; + std::unordered_map self_columns; for (const auto & column : *this) self_columns[column.name] = &column; diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 26378a2a0f0..0731580a64a 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -17,7 +17,7 @@ struct NameAndTypePair { public: NameAndTypePair() = default; - NameAndTypePair(const String & name_, const DataTypePtr & type_) : name(name_), type(type_) {} + NameAndTypePair(const String & name_, const DataTypePtr & type_) : name(name_), type(type_), storage_type(type_) {} NameAndTypePair(const String & name_, const String & subcolumn_name_, const DataTypePtr & storage_type_, const DataTypePtr & type_); diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 04513ed204a..7587a8f2250 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -149,9 +149,9 @@ namespace offset_values.resize(i); } - MutableColumnPtr getArraySizesPositionIndependent(ColumnArray & column_array) + MutableColumnPtr getArraySizesPositionIndependent(const ColumnArray & column_array) { - ColumnArray::Offsets & offset_values = column_array.getOffsets(); + const auto & offset_values = column_array.getOffsets(); MutableColumnPtr new_offsets = column_array.getOffsetsColumn().cloneEmpty(); if (offset_values.empty()) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 6ffd88460fd..cdf1b02c1b3 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -193,7 +193,6 @@ String IDataType::getFileNameForStream(const NameAndTypePair & column, const IDa else if (elem.type == Substream::ArraySizes) { size_t nested_level = column.type->getNestedLevel(); - for (size_t i = 0; i < nested_level - current_nested_level; ++i) { if (subcolumn_parts.empty()) @@ -212,26 +211,6 @@ String IDataType::getFileNameForStream(const NameAndTypePair & column, const IDa return getNameForSubstreamPath(std::move(stream_name), path, "%2E"); } -String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path) -{ - /// Sizes of arrays (elements of Nested type) are shared (all reside in single file). - String nested_table_name = Nested::extractTableName(column_name); - - bool is_sizes_of_nested_type = - path.size() == 1 /// Nested structure may have arrays as nested elements (so effectively we have multidimensional arrays). - /// Sizes of arrays are shared only at first level. - && path[0].type == IDataType::Substream::ArraySizes - && nested_table_name != column_name; - - auto stream_name = escapeForFileName(is_sizes_of_nested_type ? nested_table_name : column_name); - - /// For compatibility reasons, we use %2E instead of dot. - /// Because nested data may be represented not by Array of Tuple, - /// but by separate Array columns with names in a form of a.b, - /// and name is encoded as a whole. - return getNameForSubstreamPath(std::move(stream_name), path, "%2E"); -} - String IDataType::getSubcolumnNameForStream(String stream_name, const SubstreamPath & path) { return getNameForSubstreamPath(std::move(stream_name), path); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 84d6995a7bd..a89d2c5ac70 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -455,7 +455,6 @@ public: static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path); - static String getFileNameForStream(const String & column_name, const SubstreamPath & path); static String getSubcolumnNameForStream(String stream_name, const SubstreamPath & path); /// Substream path supports special compression methods like codec Delta. diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 074f8ac3be3..908f10736ff 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -62,9 +62,13 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( const String mrk_extension = data_part->getMarksFileExtension(); for (const auto & column_name : empty_columns) { + auto column_with_type = columns.tryGetByName(column_name); + if (!column_with_type) + continue; + IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_path */) { - String stream_name = IDataType::getFileNameForStream(column_name, substream_path); + String stream_name = IDataType::getFileNameForStream(*column_with_type, substream_path); /// Delete files if they are no longer shared with another column. if (--stream_counts[stream_name] == 0) { @@ -72,10 +76,9 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( remove_files.emplace(stream_name + mrk_extension); } }; + IDataType::SubstreamPath stream_path; - auto column_with_type = columns.tryGetByName(column_name); - if (column_with_type) - column_with_type->type->enumerateStreams(callback, stream_path); + column_with_type->type->enumerateStreams(callback, stream_path); } /// Remove files on disk and checksums diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 73b3fe698cc..9216d87d484 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1454,7 +1454,7 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames( column.type->enumerateStreams( [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - ++stream_counts[IDataType::getFileNameForStream(column.name, substream_path)]; + ++stream_counts[IDataType::getFileNameForStream(column, substream_path)]; }, {}); } @@ -1472,7 +1472,7 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames( { IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - String stream_name = IDataType::getFileNameForStream(command.column_name, substream_path); + String stream_name = IDataType::getFileNameForStream({command.column_name, command.data_type}, substream_path); /// Delete files if they are no longer shared with another column. if (--stream_counts[stream_name] == 0) { @@ -1493,7 +1493,7 @@ NameToNameVector MergeTreeDataMergerMutator::collectFilesForRenames( IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - String stream_from = IDataType::getFileNameForStream(command.column_name, substream_path); + String stream_from = IDataType::getFileNameForStream({command.column_name, command.data_type}, substream_path); String stream_to = boost::replace_first_copy(stream_from, escaped_name_from, escaped_name_to); @@ -1526,7 +1526,7 @@ NameSet MergeTreeDataMergerMutator::collectFilesToSkip( { IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - String stream_name = IDataType::getFileNameForStream(entry.name, substream_path); + String stream_name = IDataType::getFileNameForStream({entry.name, entry.type}, substream_path); files_to_skip.insert(stream_name + ".bin"); files_to_skip.insert(stream_name + mrk_extension); }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index c888a76b013..3d20a307b42 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -72,15 +72,15 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( /// Takes into account the fact that several columns can e.g. share their .size substreams. /// When calculating totals these should be counted only once. ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( - const String & column_name, const IDataType & type, std::unordered_set * processed_substreams) const + const NameAndTypePair & column, std::unordered_set * processed_substreams) const { ColumnSize size; if (checksums.empty()) return size; - type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - String file_name = IDataType::getFileNameForStream(column_name, substream_path); + String file_name = IDataType::getFileNameForStream(column, substream_path); if (processed_substreams && !processed_substreams->insert(file_name).second) return; @@ -158,7 +158,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const IDataType::SubstreamPath stream_path; name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - String file_name = IDataType::getFileNameForStream(name_type.name, substream_path); + String file_name = IDataType::getFileNameForStream(name_type, substream_path); String mrk_file_name = file_name + index_granularity_info.marks_file_extension; String bin_file_name = file_name + ".bin"; if (!checksums.files.count(mrk_file_name)) @@ -180,7 +180,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const { name_type.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - auto file_path = path + IDataType::getFileNameForStream(name_type.name, substream_path) + index_granularity_info.marks_file_extension; + auto file_path = path + IDataType::getFileNameForStream(name_type, substream_path) + index_granularity_info.marks_file_extension; /// Missing file is Ok for case when new column was added. if (volume->getDisk()->exists(file_path)) @@ -226,7 +226,7 @@ String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & colum column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { if (filename.empty()) - filename = IDataType::getFileNameForStream(column.name, substream_path); + filename = IDataType::getFileNameForStream(column, substream_path); }); return filename; } @@ -236,7 +236,7 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col std::unordered_set processed_substreams; for (const NameAndTypePair & column : columns) { - ColumnSize size = getColumnSizeImpl(column.name, *column.type, &processed_substreams); + ColumnSize size = getColumnSizeImpl(column, &processed_substreams); each_columns_size[column.name] = size; total_size.add(size); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index bb1a0b4790f..30d3021d003 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -62,7 +62,7 @@ private: /// Loads marks index granularity into memory void loadIndexGranularity() override; - ColumnSize getColumnSizeImpl(const String & name, const IDataType & type, std::unordered_set * processed_substreams) const; + ColumnSize getColumnSizeImpl(const NameAndTypePair & column, std::unordered_set * processed_substreams) const; void calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const override; }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index c81894ee36d..17a8a9c56c2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -31,14 +31,14 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( { const auto & storage_columns = metadata_snapshot->getColumns(); for (const auto & column : columns_list) - addStreams(column.name, *column.type, storage_columns.getCodecDescOrDefault(column.name, default_codec)); + addStreams(column, storage_columns.getCodecDescOrDefault(column.name, default_codec)); } -void MergeTreeDataPartWriterCompact::addStreams(const String & name, const IDataType & type, const ASTPtr & effective_codec_desc) +void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) { - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(column, substream_path); /// Shared offsets for Nested type. if (compressed_streams.count(stream_name)) @@ -61,7 +61,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const String & name, const IData }; IDataType::SubstreamPath stream_path; - type.enumerateStreams(callback, stream_path); + column.type->enumerateStreams(callback, stream_path); } void MergeTreeDataPartWriterCompact::write( @@ -141,7 +141,7 @@ void MergeTreeDataPartWriterCompact::writeBlock(const Block & block) CompressedStreamPtr prev_stream; auto stream_getter = [&, this](const IDataType::SubstreamPath & substream_path) -> WriteBuffer * { - String stream_name = IDataType::getFileNameForStream(name_and_type->name, substream_path); + String stream_name = IDataType::getFileNameForStream(*name_and_type, substream_path); auto & result_stream = compressed_streams[stream_name]; /// Write one compressed block per column in granule for more optimal reading. diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index a5a1a859e7a..5b8f093f5ce 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -30,7 +30,7 @@ private: void addToChecksums(MergeTreeDataPartChecksums & checksums); - void addStreams(const String & name, const IDataType & type, const ASTPtr & effective_codec_desc); + void addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc); Block header; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index c15c39e7b7f..85ef787c6d7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -29,18 +29,17 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) - addStreams(it.name, *it.type, columns.getCodecDescOrDefault(it.name, default_codec), settings.estimated_size); + addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec), settings.estimated_size); } void MergeTreeDataPartWriterWide::addStreams( - const String & name, - const IDataType & type, + const NameAndTypePair & column, const ASTPtr & effective_codec_desc, size_t estimated_size) { IDataType::StreamCallback callback = [&] (const IDataType::SubstreamPath & substream_path, const IDataType & substream_type) { - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(column, substream_path); /// Shared offsets for Nested type. if (column_streams.count(stream_name)) return; @@ -64,18 +63,18 @@ void MergeTreeDataPartWriterWide::addStreams( }; IDataType::SubstreamPath stream_path; - type.enumerateStreams(callback, stream_path); + column.type->enumerateStreams(callback, stream_path); } IDataType::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGetter( - const String & name, WrittenOffsetColumns & offset_columns) + const NameAndTypePair & column, WrittenOffsetColumns & offset_columns) { return [&, this] (const IDataType::SubstreamPath & substream_path) -> WriteBuffer * { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(column, substream_path); /// Don't write offsets more than one time for Nested type. if (is_offsets && offset_columns.count(stream_name)) @@ -104,45 +103,45 @@ void MergeTreeDataPartWriterWide::write(const Block & block, for (size_t i = 0; i < columns_list.size(); ++i, ++it) { const ColumnWithTypeAndName & column = block.getByName(it->name); + auto name_and_type = NameAndTypePair(column.name, column.type); if (permutation) { if (primary_key_block.has(it->name)) { const auto & primary_column = *primary_key_block.getByName(it->name).column; - writeColumn(column.name, *column.type, primary_column, offset_columns); + writeColumn(name_and_type, primary_column, offset_columns); } else if (skip_indexes_block.has(it->name)) { const auto & index_column = *skip_indexes_block.getByName(it->name).column; - writeColumn(column.name, *column.type, index_column, offset_columns); + writeColumn(name_and_type, index_column, offset_columns); } else { /// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM. ColumnPtr permuted_column = column.column->permute(*permutation, 0); - writeColumn(column.name, *column.type, *permuted_column, offset_columns); + writeColumn(name_and_type, *permuted_column, offset_columns); } } else { - writeColumn(column.name, *column.type, *column.column, offset_columns); + writeColumn(name_and_type, *column.column, offset_columns); } } } void MergeTreeDataPartWriterWide::writeSingleMark( - const String & name, - const IDataType & type, + const NameAndTypePair & column, WrittenOffsetColumns & offset_columns, size_t number_of_rows, DB::IDataType::SubstreamPath & path) { - type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + column.type->enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(column, substream_path); /// Don't write offsets more than one time for Nested type. if (is_offsets && offset_columns.count(stream_name)) @@ -162,8 +161,7 @@ void MergeTreeDataPartWriterWide::writeSingleMark( } size_t MergeTreeDataPartWriterWide::writeSingleGranule( - const String & name, - const IDataType & type, + const NameAndTypePair & name_and_type, const IColumn & column, WrittenOffsetColumns & offset_columns, IDataType::SerializeBinaryBulkStatePtr & serialization_state, @@ -173,16 +171,16 @@ size_t MergeTreeDataPartWriterWide::writeSingleGranule( bool write_marks) { if (write_marks) - writeSingleMark(name, type, offset_columns, number_of_rows, serialize_settings.path); + writeSingleMark(name_and_type, offset_columns, number_of_rows, serialize_settings.path); - type.serializeBinaryBulkWithMultipleStreams(column, from_row, number_of_rows, serialize_settings, serialization_state); + name_and_type.type->serializeBinaryBulkWithMultipleStreams(column, from_row, number_of_rows, serialize_settings, serialization_state); /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. - type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + name_and_type.type->enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(name_and_type, substream_path); /// Don't write offsets more than one time for Nested type. if (is_offsets && offset_columns.count(stream_name)) @@ -196,22 +194,21 @@ size_t MergeTreeDataPartWriterWide::writeSingleGranule( /// Column must not be empty. (column.size() !== 0) void MergeTreeDataPartWriterWide::writeColumn( - const String & name, - const IDataType & type, + const NameAndTypePair & name_and_type, const IColumn & column, WrittenOffsetColumns & offset_columns) { - auto [it, inserted] = serialization_states.emplace(name, nullptr); + auto [it, inserted] = serialization_states.emplace(name_and_type.name, nullptr); if (inserted) { IDataType::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.getter = createStreamGetter(name, offset_columns); - type.serializeBinaryBulkStatePrefix(serialize_settings, it->second); + serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); + name_and_type.type->serializeBinaryBulkStatePrefix(serialize_settings, it->second); } const auto & global_settings = storage.global_context.getSettingsRef(); IDataType::SerializeBinaryBulkSettings serialize_settings; - serialize_settings.getter = createStreamGetter(name, offset_columns); + serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); serialize_settings.low_cardinality_max_dictionary_size = global_settings.low_cardinality_max_dictionary_size; serialize_settings.low_cardinality_use_single_dictionary_for_part = global_settings.low_cardinality_use_single_dictionary_for_part != 0; @@ -244,8 +241,7 @@ void MergeTreeDataPartWriterWide::writeColumn( data_written = true; current_row = writeSingleGranule( - name, - type, + name_and_type, column, offset_columns, it->second, @@ -259,12 +255,12 @@ void MergeTreeDataPartWriterWide::writeColumn( current_column_mark++; } - type.enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + name_and_type.type->enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; if (is_offsets) { - String stream_name = IDataType::getFileNameForStream(name, substream_path); + String stream_name = IDataType::getFileNameForStream(name_and_type, substream_path); offset_columns.insert(stream_name); } }, serialize_settings.path); @@ -289,13 +285,13 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch { if (!serialization_states.empty()) { - serialize_settings.getter = createStreamGetter(it->name, written_offset_columns ? *written_offset_columns : offset_columns); + serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns); it->type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); } if (write_final_mark) { - writeFinalMark(it->name, it->type, offset_columns, serialize_settings.path); + writeFinalMark(*it, offset_columns, serialize_settings.path); } } } @@ -313,19 +309,18 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch } void MergeTreeDataPartWriterWide::writeFinalMark( - const std::string & column_name, - const DataTypePtr column_type, + const NameAndTypePair & column, WrittenOffsetColumns & offset_columns, DB::IDataType::SubstreamPath & path) { - writeSingleMark(column_name, *column_type, offset_columns, 0, path); + writeSingleMark(column, offset_columns, 0, path); /// Memoize information about offsets - column_type->enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + column.type->enumerateStreams([&] (const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { bool is_offsets = !substream_path.empty() && substream_path.back().type == IDataType::Substream::ArraySizes; if (is_offsets) { - String stream_name = IDataType::getFileNameForStream(column_name, substream_path); + String stream_name = IDataType::getFileNameForStream(column, substream_path); offset_columns.insert(stream_name); } }, path); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 6dd3104c5b4..a3b148ad87c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -25,22 +25,20 @@ public: void finishDataSerialization(IMergeTreeDataPart::Checksums & checksums, bool sync) override; - IDataType::OutputStreamGetter createStreamGetter(const String & name, WrittenOffsetColumns & offset_columns); + IDataType::OutputStreamGetter createStreamGetter(const NameAndTypePair & column, WrittenOffsetColumns & offset_columns); private: /// Write data of one column. /// Return how many marks were written and /// how many rows were written for last mark void writeColumn( - const String & name, - const IDataType & type, + const NameAndTypePair & name_and_type, const IColumn & column, WrittenOffsetColumns & offset_columns); /// Write single granule of one column (rows between 2 marks) size_t writeSingleGranule( - const String & name, - const IDataType & type, + const NameAndTypePair & name_and_type, const IColumn & column, WrittenOffsetColumns & offset_columns, IDataType::SerializeBinaryBulkStatePtr & serialization_state, @@ -51,21 +49,18 @@ private: /// Write mark for column void writeSingleMark( - const String & name, - const IDataType & type, + const NameAndTypePair & column, WrittenOffsetColumns & offset_columns, size_t number_of_rows, DB::IDataType::SubstreamPath & path); void writeFinalMark( - const std::string & column_name, - const DataTypePtr column_type, + const NameAndTypePair & column, WrittenOffsetColumns & offset_columns, DB::IDataType::SubstreamPath & path); void addStreams( - const String & name, - const IDataType & type, + const NameAndTypePair & column, const ASTPtr & effective_codec_desc, size_t estimated_size); diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index e684205658a..4de3c08eddd 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -38,6 +38,19 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( } } +static ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & name_and_type) +{ + auto storage_name = name_and_type.getStorageName(); + if (!block.has(storage_name)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found column '{}' in block", storage_name); + + const auto & column = block.getByName(storage_name).column; + if (name_and_type.isSubcolumn()) + return name_and_type.getStorageType()->getSubcolumn(name_and_type.getSubcolumnName(), *column->assumeMutable()); + + return column; +} + size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { if (!continue_reading) @@ -60,17 +73,17 @@ size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading auto column_it = columns.begin(); for (size_t i = 0; i < num_columns; ++i, ++column_it) { - auto [name, type] = getColumnFromPart(*column_it); + 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()) + auto offsets_it = positions_for_offsets.find(name_type.name); + if (offsets_it != positions_for_offsets.end() && !name_type.isSubcolumn()) { 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(); + res_columns[i] = name_type.type->createColumn(); auto mutable_column = res_columns[i]->assumeMutable(); auto & res_offstes = assert_cast(*mutable_column).getOffsets(); @@ -80,9 +93,9 @@ size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading res_columns[i] = std::move(mutable_column); } - else if (part_in_memory->block.has(name)) + else if (part_in_memory->hasColumnFiles(name_type)) { - const auto & block_column = part_in_memory->block.getByName(name).column; + auto block_column = getColumnFromBlock(part_in_memory->block, name_type); if (rows_to_read == part_rows) { res_columns[i] = block_column; @@ -90,7 +103,7 @@ size_t MergeTreeReaderInMemory::readRows(size_t from_mark, bool continue_reading else { if (res_columns[i] == nullptr) - res_columns[i] = type->createColumn(); + res_columns[i] = name_type.type->createColumn(); auto mutable_column = res_columns[i]->assumeMutable(); mutable_column->insertRangeFrom(*block_column, total_rows_read, rows_to_read); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 2838c8eb881..c9da156dc97 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -122,7 +122,7 @@ IMergeTreeDataPart::Checksums checkDataPart( { column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - String file_name = IDataType::getFileNameForStream(column.name, substream_path) + ".bin"; + String file_name = IDataType::getFileNameForStream(column, substream_path) + ".bin"; checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name); }, {}); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 33899e66755..343fd3baa75 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include @@ -46,7 +46,6 @@ namespace ErrorCodes extern const int INCORRECT_FILE_NAME; } - class LogSource final : public SourceWithProgress { public: @@ -91,15 +90,13 @@ private: struct Stream { Stream(const DiskPtr & disk, const String & data_path, size_t offset, size_t max_read_buffer_size_) - : plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))), - compressed(*plain) + : compressed(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))) { if (offset) - plain->seek(offset, SEEK_SET); + compressed.seek(offset, 0); } - std::unique_ptr plain; - CompressedReadBuffer compressed; + CompressedReadBufferFromFile compressed; }; using FileStreams = std::map; @@ -251,7 +248,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, IColumn & column IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto create_string_getter = [&](bool stream_for_prefix) + auto create_stream_getter = [&](bool stream_for_prefix) { return [&, stream_for_prefix] (const IDataType::SubstreamPath & path) -> ReadBuffer * { @@ -267,17 +264,21 @@ void LogSource::readData(const NameAndTypePair & name_and_type, IColumn & column auto & data_file_path = file_it->second.data_file_path; auto it = streams.try_emplace(stream_name, storage.disk, data_file_path, offset, max_read_buffer_size).first; + + /// FIXME: avoid double reading of subcolumns + it->second.compressed.seek(0, 0); + return &it->second.compressed; }; }; if (deserialize_states.count(name) == 0) { - settings.getter = create_string_getter(true); + settings.getter = create_stream_getter(true); type->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); } - settings.getter = create_string_getter(false); + settings.getter = create_stream_getter(false); type->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name]); } @@ -623,7 +624,7 @@ Pipe StorageLog::read( metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); loadMarks(); - NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)); + NamesAndTypesList all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); std::shared_lock lock(rwlock); diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index 25e232dc4ad..cbdc8e07064 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -32,7 +32,7 @@ public: const StorageMemory & storage, const StorageMetadataPtr & metadata_snapshot) : SourceWithProgress(metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) - , column_names(std::move(column_names_)) + , column_names_and_types(metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(std::move(column_names_))) , current_it(first_) , num_blocks(num_blocks_) { @@ -71,11 +71,17 @@ protected: { const Block & src = *current_it; Columns columns; - columns.reserve(column_names.size()); + columns.reserve(columns.size()); /// Add only required columns to `res`. - for (const auto & name : column_names) - columns.emplace_back(src.getByName(name).column); + for (const auto & elem : column_names_and_types) + { + auto current_column = src.getByName(elem.getStorageName()).column; + if (elem.isSubcolumn()) + columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column->assumeMutable())); + else + columns.emplace_back(std::move(current_column)); + } ++current_block_idx; @@ -88,7 +94,7 @@ protected: } } private: - Names column_names; + NamesAndTypesList column_names_and_types; BlocksList::iterator current_it; size_t current_block_idx = 0; size_t num_blocks; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 962a0bcd4fc..b65ff6f5390 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include @@ -87,13 +87,11 @@ private: struct Stream { Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_) - : plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))), - compressed(*plain) + : compressed(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))) { } - std::unique_ptr plain; - CompressedReadBuffer compressed; + CompressedReadBufferFromFile compressed; }; using FileStreams = std::map>; @@ -231,10 +229,14 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, IColumn & co { String stream_name = IDataType::getFileNameForStream(name_and_type, path); - if (!streams.count(stream_name)) - streams[stream_name] = std::make_unique(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size); + auto & stream = streams[stream_name]; + if (!stream) + stream = std::make_unique(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size); - return &streams[stream_name]->compressed; + /// FIXME: avoid double reading of subcolumns + stream->compressed.seek(0, 0); + + return &stream->compressed; }; if (deserialize_states.count(name) == 0) @@ -441,7 +443,8 @@ Pipe StorageTinyLog::read( // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. return Pipe(std::make_shared( - max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); + max_block_size, metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names), + *this, context.getSettingsRef().max_read_buffer_size)); } diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.reference b/tests/queries/0_stateless/01475_read_subcolumns_storages.reference new file mode 100644 index 00000000000..f848977a55d --- /dev/null +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.reference @@ -0,0 +1,18 @@ +Log +100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 +TinyLog +100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 +Memory +100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 +MergeTree ORDER BY tuple() SETTINGS min_bytes_for_compact_part='10M' +100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 +MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part='10M' +100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 +MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part=0 +100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh index 2304726bf6f..33f757dfe01 100755 --- a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh @@ -8,9 +8,10 @@ set -e create_query="CREATE TABLE subcolumns(n Nullable(UInt32), a1 Array(UInt32),\ a2 Array(Array(Array(UInt32))), a3 Array(Nullable(UInt32)), t Tuple(s String, v UInt32))" -declare -a ENGINES=("Log" "StripeLog" "TinyLog" "Memory" \ - "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_compact_part='10M'" - "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part='10M'" +# "StripeLog" +declare -a ENGINES=("Log" "TinyLog" "Memory" \ + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_compact_part='10M'" \ + "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part='10M'" \ "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part=0") for engine in "${ENGINES[@]}"; do From 600dbc938756c7a135ea403829d1db610b55f914 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Oct 2020 20:57:17 +0300 Subject: [PATCH 006/264] fix rename of columns --- src/Core/NamesAndTypes.cpp | 10 +++--- src/Core/NamesAndTypes.h | 4 +-- .../MergeTree/MergeTreeBlockReadUtils.cpp | 34 ++++++++++++------- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.h | 3 +- .../001475_read_subcolumns_storages.sh | 23 ------------- 6 files changed, 31 insertions(+), 45 deletions(-) delete mode 100755 tests/queries/0_stateless/001475_read_subcolumns_storages.sh diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index ab44665fd3b..4ee4aee923e 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -18,12 +18,12 @@ namespace ErrorCodes } NameAndTypePair::NameAndTypePair( - const String & name_, const String & subcolumn_name_, - const DataTypePtr & storage_type_, const DataTypePtr & type_) - : name(name_ + "." + subcolumn_name_) - , type(type_) + const String & storage_name_, const String & subcolumn_name_, + const DataTypePtr & storage_type_, const DataTypePtr & subcolumn_type_) + : name(storage_name_ + "." + subcolumn_name_) + , type(subcolumn_type_) , storage_type(storage_type_) - , subcolumn_delimiter_position(name_.size()) {} + , subcolumn_delimiter_position(storage_name_.size()) {} String NameAndTypePair::getStorageName() const { diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 0731580a64a..65f4caf0111 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -18,8 +18,8 @@ struct NameAndTypePair public: NameAndTypePair() = default; NameAndTypePair(const String & name_, const DataTypePtr & type_) : name(name_), type(type_), storage_type(type_) {} - NameAndTypePair(const String & name_, const String & subcolumn_name_, - const DataTypePtr & storage_type_, const DataTypePtr & type_); + NameAndTypePair(const String & storage_name_, const String & subcolumn_name_, + const DataTypePtr & storage_type_, const DataTypePtr & subcolumn_type_); String getStorageName() const; String getSubcolumnName() const; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index d6bb68660c0..264f2b7326e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -33,23 +34,30 @@ bool injectRequiredColumnsRecursively( /// huge AST which for some reason was not validated on parsing/interpreter /// stages. checkStackSize(); - String column_name_in_part = column_name; - if (alter_conversions.isColumnRenamed(column_name_in_part)) - column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); - auto storage_column = storage_columns.getPhysicalOrSubcolumn(column_name_in_part); - - /// column has files and hence does not require evaluation - if (storage_columns.hasPhysicalOrSubcolumn(column_name) && part->hasColumnFiles(storage_column)) + if (storage_columns.hasPhysicalOrSubcolumn(column_name)) { - /// ensure each column is added only once - if (required_columns.count(column_name) == 0) + auto column_in_storage = storage_columns.getPhysicalOrSubcolumn(column_name); + auto column_name_in_part = column_in_storage.getStorageName(); + if (alter_conversions.isColumnRenamed(column_name_in_part)) + column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); + + auto column_in_part = NameAndTypePair( + column_name_in_part, column_in_storage.getSubcolumnName(), + column_in_storage.getStorageType(), column_in_storage.type); + + /// column has files and hence does not require evaluation + if (part->hasColumnFiles(column_in_part)) { - columns.emplace_back(column_name); - required_columns.emplace(column_name); - injected_columns.emplace(column_name); + /// ensure each column is added only once + if (required_columns.count(column_name) == 0) + { + columns.emplace_back(column_name); + required_columns.emplace(column_name); + injected_columns.emplace(column_name); + } + return true; } - return true; } /// Column doesn't have default value and don't exist in part diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index fa5669ce0a2..e857139cce9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -123,7 +123,7 @@ void MergeTreeDataPartCompact::loadIndexGranularity() bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const { - if (!getColumnPosition(column.name)) + if (!getColumnPosition(column)) return false; auto bin_checksum = checksums.files.find(DATA_FILE_NAME_WITH_EXTENSION); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 4596fe92b96..ce0bff9e2ba 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -32,6 +32,7 @@ public: 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, @@ -41,7 +42,7 @@ public: const MergeTreeIndexGranularity & computed_index_granularity) const override; bool isStoredOnDisk() const override { return false; } - bool hasColumnFiles(const NameAndTypePair & column) const override { return !!getColumnPosition(column.name); } + bool hasColumnFiles(const NameAndTypePair & column) const override { return !!getColumnPosition(column); } 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 StorageMetadataPtr & metadata_snapshot) const override; diff --git a/tests/queries/0_stateless/001475_read_subcolumns_storages.sh b/tests/queries/0_stateless/001475_read_subcolumns_storages.sh deleted file mode 100755 index 2304726bf6f..00000000000 --- a/tests/queries/0_stateless/001475_read_subcolumns_storages.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CURDIR"/../shell_config.sh - -set -e - -create_query="CREATE TABLE subcolumns(n Nullable(UInt32), a1 Array(UInt32),\ - a2 Array(Array(Array(UInt32))), a3 Array(Nullable(UInt32)), t Tuple(s String, v UInt32))" - -declare -a ENGINES=("Log" "StripeLog" "TinyLog" "Memory" \ - "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_compact_part='10M'" - "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part='10M'" - "MergeTree ORDER BY tuple() SETTINGS min_bytes_for_wide_part=0") - -for engine in "${ENGINES[@]}"; do - echo $engine - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS subcolumns" - $CLICKHOUSE_CLIENT --query "$create_query ENGINE = $engine" - $CLICKHOUSE_CLIENT --query "INSERT INTO subcolumns VALUES (100, [1, 2, 3], [[[1, 2], [], [4]], [[5, 6], [7, 8]], [[]]], [1, NULL, 2], ('foo', 200))" - $CLICKHOUSE_CLIENT --query "SELECT * FROM subcolumns" - $CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v FROM subcolumns" -done From bf62dc97f2d8ccba410a485bcee9e8486ead0b22 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 26 Oct 2020 20:24:15 +0300 Subject: [PATCH 007/264] fix StorageLog --- src/Storages/StorageLog.cpp | 35 ++++++++++++++++++++------------- src/Storages/StorageTinyLog.cpp | 33 +++++++++++++++++++------------ 2 files changed, 41 insertions(+), 27 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 343fd3baa75..85e5ef2c58a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include #include @@ -90,13 +90,15 @@ private: struct Stream { Stream(const DiskPtr & disk, const String & data_path, size_t offset, size_t max_read_buffer_size_) - : compressed(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))) + : plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))) + , compressed(*plain) { if (offset) - compressed.seek(offset, 0); + plain->seek(offset, 0); } - CompressedReadBufferFromFile compressed; + std::unique_ptr plain; + CompressedReadBuffer compressed; }; using FileStreams = std::map; @@ -209,16 +211,24 @@ Chunk LogSource::generate() for (const auto & name_type : columns) { - MutableColumnPtr column = name_type.type->createColumn(); - - try + MutableColumnPtr column; + if (name_type.isSubcolumn() && res.has(name_type.getStorageName())) { - readData(name_type, *column, max_rows_to_read); + auto column_in_block = res.getByName(name_type.getStorageName()).column; + column = name_type.getStorageType()->getSubcolumn(name_type.getSubcolumnName(), *column_in_block->assumeMutable()); } - catch (Exception & e) + else { - e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); - throw; + try + { + column = name_type.type->createColumn(); + readData(name_type, *column, max_rows_to_read); + } + catch (Exception & e) + { + e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); + throw; + } } if (!column->empty()) @@ -265,9 +275,6 @@ void LogSource::readData(const NameAndTypePair & name_and_type, IColumn & column auto & data_file_path = file_it->second.data_file_path; auto it = streams.try_emplace(stream_name, storage.disk, data_file_path, offset, max_read_buffer_size).first; - /// FIXME: avoid double reading of subcolumns - it->second.compressed.seek(0, 0); - return &it->second.compressed; }; }; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index b65ff6f5390..39592fab3f2 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include @@ -87,11 +87,13 @@ private: struct Stream { Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_) - : compressed(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))) + : plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))) + , compressed(*plain) { } - CompressedReadBufferFromFile compressed; + std::unique_ptr plain; + CompressedReadBuffer compressed; }; using FileStreams = std::map>; @@ -194,16 +196,24 @@ Chunk TinyLogSource::generate() for (const auto & name_type : columns) { - MutableColumnPtr column = name_type.type->createColumn(); - - try + MutableColumnPtr column; + if (name_type.isSubcolumn() && res.has(name_type.getStorageName())) { - readData(name_type, *column, block_size); + auto column_in_block = res.getByName(name_type.getStorageName()).column; + column = name_type.getStorageType()->getSubcolumn(name_type.getSubcolumnName(), *column_in_block->assumeMutable()); } - catch (Exception & e) + else { - e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); - throw; + try + { + column = name_type.type->createColumn(); + readData(name_type, *column, block_size); + } + catch (Exception & e) + { + e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); + throw; + } } if (!column->empty()) @@ -233,9 +243,6 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, IColumn & co if (!stream) stream = std::make_unique(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size); - /// FIXME: avoid double reading of subcolumns - stream->compressed.seek(0, 0); - return &stream->compressed; }; From 55d867a66080a7a9e5e39f201f7bf9d8090db6e6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 26 Oct 2020 21:55:52 +0300 Subject: [PATCH 008/264] avoid double reading of subcolumns --- src/Storages/MergeTree/IMergeTreeReader.cpp | 37 +++++++++++++++++++ src/Storages/MergeTree/IMergeTreeReader.h | 4 ++ .../MergeTree/MergeTreeReaderCompact.cpp | 5 ++- .../MergeTree/MergeTreeReaderWide.cpp | 3 ++ 4 files changed, 47 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index d21bb583908..c6f038642b0 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -44,6 +44,27 @@ IMergeTreeReader::IMergeTreeReader( { for (const NameAndTypePair & column_from_part : data_part->getColumns()) columns_from_part[column_from_part.name] = column_from_part.type; + + std::unordered_map positions; + size_t pos = 0; + for (const auto & column : columns) + { + if (!column.isSubcolumn()) + positions[column.name] = pos; + ++pos; + } + + pos = 0; + for (const auto & column : columns) + { + if (column.isSubcolumn()) + { + auto it = positions.find(column.getStorageName()); + if (it != positions.end()) + duplicated_subcolumns[column.name] = it->second; + } + ++pos; + } } IMergeTreeReader::~IMergeTreeReader() = default; @@ -74,6 +95,20 @@ static bool arrayHasNoElementsRead(const IColumn & column) return last_offset != 0; } +void IMergeTreeReader::fillDuplicatedSubcolumns(Columns & res_columns) +{ + auto requested_column = columns.begin(); + for (size_t i = 0; i < columns.size(); ++i, ++requested_column) + { + if (res_columns[i]) + continue; + + auto it = duplicated_subcolumns.find(requested_column->name); + if (it != duplicated_subcolumns.end() && res_columns[it->second]) + res_columns[i] = requested_column->getStorageType()->getSubcolumn( + requested_column->getSubcolumnName(), *res_columns[it->second]->assumeMutable()); + } +} void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) { @@ -86,6 +121,8 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e "Expected " + toString(num_columns) + ", " "got " + toString(res_columns.size()), ErrorCodes::LOGICAL_ERROR); + fillDuplicatedSubcolumns(res_columns); + /// For a missing column of a nested data structure we must create not a column of empty /// arrays, but a column of arrays of correct length. diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index d192339432f..4948784ab8c 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -82,12 +82,16 @@ protected: StorageMetadataPtr metadata_snapshot; MarkRanges all_mark_ranges; + std::unordered_map duplicated_subcolumns; + using ColumnPosition = std::optional; ColumnPosition findColumnForOffsets(const String & column_name) const; friend class MergeTreeRangeReader::DelayedStream; private: + void fillDuplicatedSubcolumns(Columns & res_columns); + /// Alter conversions, which must be applied on fly if required MergeTreeData::AlterConversions alter_conversions; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 41af0040fc5..53ba113fce8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -146,10 +146,11 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { - if (!res_columns[pos]) + auto column_from_part = getColumnFromPart(*name_and_type); + + if (!res_columns[pos] || duplicated_subcolumns.count(column_from_part.name)) continue; - auto column_from_part = getColumnFromPart(*name_and_type); auto & column = mutable_columns[pos]; try diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index ea644c55bd5..30fe6fa57d2 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -79,6 +79,9 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si auto column_from_part = getColumnFromPart(*name_and_type); const auto & [name, type] = column_from_part; + if (duplicated_subcolumns.count(name)) + continue; + /// The column is already present in the block so we will append the values to the end. bool append = res_columns[pos] != nullptr; if (!append) From 734b50386417ec8513cb78c8a80726ecc9aa4c3d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 27 Oct 2020 14:27:14 +0300 Subject: [PATCH 009/264] fix alters of nested --- src/Storages/ColumnsDescription.cpp | 13 ++++++++++++- src/Storages/ColumnsDescription.h | 1 + 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6f933e9a39c..1224d34f8f1 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -182,10 +182,12 @@ void ColumnsDescription::add(ColumnDescription column, const String & after_colu throw Exception("Wrong column name. Cannot find column " + after_column + " to insert after", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); + for (auto it = range.first; it != range.second; ++it) + addSubcolumns(it->name, it->type); + insert_it = range.second; } - addSubcolumns(column.name, column.type); columns.get<0>().insert(insert_it, std::move(column)); } @@ -197,7 +199,10 @@ void ColumnsDescription::remove(const String & column_name) ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); for (auto list_it = range.first; list_it != range.second;) + { + removeSubcolumns(list_it->name, list_it->type); list_it = columns.get<0>().erase(list_it); + } } void ColumnsDescription::rename(const String & column_from, const String & column_to) @@ -542,6 +547,12 @@ void ColumnsDescription::addSubcolumns(const String & storage_name, const DataTy } } +void ColumnsDescription::removeSubcolumns(const String & storage_name, const DataTypePtr & storage_type) +{ + for (const auto & subcolumn_name : storage_type->getSubcolumnNames()) + subcolumns.erase(storage_name + "." + subcolumn_name); +} + Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, const Context & context) { for (const auto & child : default_expr_list->children) diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index deb4f668e63..d79c7a09484 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -149,6 +149,7 @@ private: void modifyColumnOrder(const String & column_name, const String & after_column, bool first); void addSubcolumns(const String & storage_name, const DataTypePtr & storage_type); + void removeSubcolumns(const String & storage_name, const DataTypePtr & storage_type); }; /// Validate default expressions and corresponding types compatibility, i.e. From e32349ea566366e25407bad66f11ce551abaa0e7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Oct 2020 17:14:23 +0300 Subject: [PATCH 010/264] fixes related to Nested --- src/Storages/ColumnsDescription.cpp | 6 +++--- src/Storages/StorageLog.cpp | 12 ++++++++++-- src/Storages/StorageTinyLog.cpp | 12 ++++++++++-- .../01475_read_subcolumns_storages.reference | 4 ++-- .../0_stateless/01475_read_subcolumns_storages.sh | 8 +++++++- 5 files changed, 32 insertions(+), 10 deletions(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 1224d34f8f1..4b8678354b7 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -182,12 +182,10 @@ void ColumnsDescription::add(ColumnDescription column, const String & after_colu throw Exception("Wrong column name. Cannot find column " + after_column + " to insert after", ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - for (auto it = range.first; it != range.second; ++it) - addSubcolumns(it->name, it->type); - insert_it = range.second; } + addSubcolumns(column.name, column.type); columns.get<0>().insert(insert_it, std::move(column)); } @@ -275,6 +273,7 @@ void ColumnsDescription::flattenNested() } ColumnDescription column = std::move(*it); + removeSubcolumns(column.name, column.type); it = columns.get<0>().erase(it); const DataTypes & elements = type_tuple->getElements(); @@ -288,6 +287,7 @@ void ColumnsDescription::flattenNested() nested_column.name = Nested::concatenateName(column.name, names[i]); nested_column.type = std::make_shared(elements[i]); + addSubcolumns(nested_column.name, nested_column.type); columns.get<0>().insert(it, std::move(nested_column)); } } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 85e5ef2c58a..69dec91fab9 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; + extern const int NOT_IMPLEMENTED; } class LogSource final : public SourceWithProgress @@ -94,7 +95,7 @@ private: , compressed(*plain) { if (offset) - plain->seek(offset, 0); + plain->seek(offset, SEEK_SET); } std::unique_ptr plain; @@ -631,7 +632,14 @@ Pipe StorageLog::read( metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); loadMarks(); - NamesAndTypesList all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); + auto all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); + + /// TODO: implement DataType Nested and support them + for (const auto & column : all_columns) + if (column.isSubcolumn() && startsWith(column.getSubcolumnName(), "size")) + throw Exception("Subcolumns of arrays are not supported in StorageLog", ErrorCodes::NOT_IMPLEMENTED); + + all_columns = Nested::collect(all_columns); std::shared_lock lock(rwlock); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 39592fab3f2..144a8695c64 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -49,6 +49,7 @@ namespace ErrorCodes extern const int DUPLICATE_COLUMN; extern const int INCORRECT_FILE_NAME; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int NOT_IMPLEMENTED; } @@ -92,7 +93,7 @@ private: { } - std::unique_ptr plain; + std::unique_ptr plain; CompressedReadBuffer compressed; }; @@ -447,10 +448,17 @@ Pipe StorageTinyLog::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); + auto all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); + + /// TODO: implement DataType Nested and support them + for (const auto & column : all_columns) + if (column.isSubcolumn() && startsWith(column.getSubcolumnName(), "size")) + throw Exception("Subcolumns of arrays are not supported in StorageLog", ErrorCodes::NOT_IMPLEMENTED); + // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. return Pipe(std::make_shared( - max_block_size, metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names), + max_block_size, Nested::collect(all_columns), *this, context.getSettingsRef().max_read_buffer_size)); } diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.reference b/tests/queries/0_stateless/01475_read_subcolumns_storages.reference index f848977a55d..dc4f34b330c 100644 --- a/tests/queries/0_stateless/01475_read_subcolumns_storages.reference +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.reference @@ -1,9 +1,9 @@ Log 100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) -100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 +100 0 ('foo',200) foo 200 TinyLog 100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) -100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 +100 0 ('foo',200) foo 200 Memory 100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) 100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh index 33f757dfe01..b0cfc9b29a7 100755 --- a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh @@ -20,5 +20,11 @@ for engine in "${ENGINES[@]}"; do $CLICKHOUSE_CLIENT --query "$create_query ENGINE = $engine" $CLICKHOUSE_CLIENT --query "INSERT INTO subcolumns VALUES (100, [1, 2, 3], [[[1, 2], [], [4]], [[5, 6], [7, 8]], [[]]], [1, NULL, 2], ('foo', 200))" $CLICKHOUSE_CLIENT --query "SELECT * FROM subcolumns" - $CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v FROM subcolumns" + + # Some subcolumns are not supported in + if [[ $engine == "Log" || $engine = "TinyLog" ]]; then + $CLICKHOUSE_CLIENT --query "SELECT n, n.null, t, t.s, t.v FROM subcolumns" + else + $CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v FROM subcolumns" + fi done From 01a04e06ef2ce2804c7d355fa2794f1b61ea6665 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 5 Nov 2020 16:24:31 +0300 Subject: [PATCH 011/264] implement nested with multiple nesting --- src/DataTypes/DataTypeArray.cpp | 10 +- src/DataTypes/DataTypeArray.h | 6 +- src/DataTypes/DataTypeFactory.cpp | 1 + src/DataTypes/DataTypeFactory.h | 1 + src/DataTypes/DataTypeNested.cpp | 68 ++++++++++ src/DataTypes/DataTypeNested.h | 32 +++++ src/DataTypes/DataTypeNullable.cpp | 3 +- src/DataTypes/DataTypeOneElementTuple.cpp | 121 ++++++++++++++++++ src/DataTypes/DataTypeOneElementTuple.h | 103 +++++++++++++++ src/DataTypes/DataTypeTuple.cpp | 21 ++- src/DataTypes/IDataType.cpp | 104 +++++---------- src/DataTypes/IDataType.h | 9 +- src/DataTypes/ya.make | 2 + src/Parsers/ASTNameTypePair.h | 2 + src/Parsers/ParserDataType.cpp | 2 +- src/Storages/ColumnsDescription.cpp | 7 + .../MergeTree/MergeTreeReaderCompact.cpp | 8 +- 17 files changed, 398 insertions(+), 102 deletions(-) create mode 100644 src/DataTypes/DataTypeNested.cpp create mode 100644 src/DataTypes/DataTypeNested.h create mode 100644 src/DataTypes/DataTypeOneElementTuple.cpp create mode 100644 src/DataTypes/DataTypeOneElementTuple.h diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 7587a8f2250..2c978f58fe6 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -10,12 +10,12 @@ #include #include #include +#include #include #include #include -#include #include #include @@ -32,8 +32,8 @@ namespace ErrorCodes } -DataTypeArray::DataTypeArray(const DataTypePtr & nested_, size_t nested_level_) - : nested{nested_}, nested_level{nested_level_} +DataTypeArray::DataTypeArray(const DataTypePtr & nested_) + : nested{nested_} { } @@ -527,7 +527,7 @@ DataTypePtr DataTypeArray::tryGetSubcolumnType(const String & subcolumn_name) co DataTypePtr DataTypeArray::tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const { if (subcolumn_name == "size" + std::to_string(level)) - return std::make_shared(); + return std::make_shared(std::make_shared(), subcolumn_name, false); DataTypePtr subcolumn; if (const auto * nested_array = typeid_cast(nested.get())) @@ -535,7 +535,7 @@ DataTypePtr DataTypeArray::tryGetSubcolumnTypeImpl(const String & subcolumn_name else subcolumn = nested->tryGetSubcolumnType(subcolumn_name); - return (subcolumn ? std::make_shared(std::move(subcolumn), nested_level + 1) : subcolumn); + return (subcolumn ? std::make_shared(std::move(subcolumn)) : subcolumn); } MutableColumnPtr DataTypeArray::getSubcolumn(const String & subcolumn_name, IColumn & column) const diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index c03e34aa848..ea01a1f81a9 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -13,12 +13,10 @@ private: /// The type of array elements. DataTypePtr nested; - size_t nested_level = 0; - public: static constexpr bool is_parametric = true; - DataTypeArray(const DataTypePtr & nested_, size_t nested_level_ = 0); + DataTypeArray(const DataTypePtr & nested_); TypeIndex getTypeId() const override { return TypeIndex::Array; } @@ -37,8 +35,6 @@ public: return false; } - size_t getNestedLevel() const override { return nested_level; } - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 9386f4b39f1..c9d948defa6 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -180,6 +180,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDomainIPv4AndIPv6(*this); registerDataTypeDomainSimpleAggregateFunction(*this); registerDataTypeDomainGeo(*this); + registerDataTypeOneElementTuple(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index ea77c50170c..eab20482f04 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -83,5 +83,6 @@ void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); +void registerDataTypeOneElementTuple(DataTypeFactory & factory); } diff --git a/src/DataTypes/DataTypeNested.cpp b/src/DataTypes/DataTypeNested.cpp new file mode 100644 index 00000000000..6dfccb57b1f --- /dev/null +++ b/src/DataTypes/DataTypeNested.cpp @@ -0,0 +1,68 @@ +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int EMPTY_DATA_PASSED; + extern const int BAD_ARGUMENTS; +} + +String DataTypeNestedCustomName::getName() const +{ + WriteBufferFromOwnString s; + s << "Nested("; + for (size_t i = 0; i < elems.size(); ++i) + { + if (i != 0) + s << ", "; + + s << backQuoteIfNeed(names[i]) << ' '; + s << elems[i]->getName(); + } + s << ")"; + + return s.str(); +} + +static std::pair create(const ASTPtr & arguments) +{ + if (!arguments || arguments->children.empty()) + throw Exception("Nested cannot be empty", ErrorCodes::EMPTY_DATA_PASSED); + + DataTypes nested_types; + Strings nested_names; + nested_types.reserve(arguments->children.size()); + nested_names.reserve(arguments->children.size()); + + for (const auto & child : arguments->children) + { + const auto * name_type = child->as(); + if (!name_type) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Data type Nested accepts only pairs with name and type"); + + auto nested_type = DataTypeFactory::instance().get(name_type->type); + nested_types.push_back(std::move(nested_type)); + nested_names.push_back(name_type->name); + } + + auto data_type = std::make_shared(std::make_shared(nested_types, nested_names)); + auto custom_name = std::make_unique(nested_types, nested_names); + + return std::make_pair(std::move(data_type), std::make_unique(std::move(custom_name), nullptr)); +} + +void registerDataTypeNested(DataTypeFactory & factory) +{ + return factory.registerDataTypeCustom("Nested", create); +} + +} diff --git a/src/DataTypes/DataTypeNested.h b/src/DataTypes/DataTypeNested.h new file mode 100644 index 00000000000..50649fbcddc --- /dev/null +++ b/src/DataTypes/DataTypeNested.h @@ -0,0 +1,32 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class DataTypeNestedCustomName final : public IDataTypeCustomName +{ +private: + DataTypes elems; + Strings names; + +public: + DataTypeNestedCustomName(const DataTypes & elems_, const Strings & names_) + : elems(elems_), names(names_) + { + } + + String getName() const override; +}; + +template +inline bool isNested(const DataType & data_type) +{ + return isArray(data_type) && typeid_cast(data_type->getCustomName()); +} + +} + diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 806083a2a73..da8bbb632ff 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -530,7 +531,7 @@ bool DataTypeNullable::equals(const IDataType & rhs) const DataTypePtr DataTypeNullable::tryGetSubcolumnType(const String & subcolumn_name) const { if (subcolumn_name == "null") - return std::make_shared(); + return std::make_shared(std::make_shared(), subcolumn_name, false); return nested_data_type->tryGetSubcolumnType(subcolumn_name); } diff --git a/src/DataTypes/DataTypeOneElementTuple.cpp b/src/DataTypes/DataTypeOneElementTuple.cpp new file mode 100644 index 00000000000..d14eddb0070 --- /dev/null +++ b/src/DataTypes/DataTypeOneElementTuple.cpp @@ -0,0 +1,121 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void DataTypeOneElementTuple::addToPath(SubstreamPath & path) const +{ + path.push_back(Substream::TupleElement); + path.back().tuple_element_name = name; + path.back().escape_tuple_delimiter = escape_delimiter; +} + +std::string DataTypeOneElementTuple::doGetName() const +{ + WriteBufferFromOwnString s; + s << TYPE_NAME << "(" << backQuoteIfNeed(name) << " " << element->getName() << ")"; + return s.str(); +} + +bool DataTypeOneElementTuple::equals(const IDataType & rhs) const +{ + const auto * rhs_tuple = typeid_cast(&rhs); + if (!rhs_tuple) + return false; + + return element->equals(*rhs_tuple->element) + && name == rhs_tuple->name + && escape_delimiter == rhs_tuple->escape_delimiter; +} + +void DataTypeOneElementTuple::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +{ + addToPath(path); + element->enumerateStreams(callback, path); + path.pop_back(); +} + +void DataTypeOneElementTuple::DataTypeOneElementTuple::serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + addToPath(settings.path); + element->serializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); +} + +void DataTypeOneElementTuple::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + addToPath(settings.path); + element->serializeBinaryBulkStateSuffix(settings, state); + settings.path.pop_back(); +} + +void DataTypeOneElementTuple::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + addToPath(settings.path); + element->deserializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); +} + +void DataTypeOneElementTuple::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + addToPath(settings.path); + element->serializeBinaryBulkWithMultipleStreams(column, offset, limit, settings, state); + settings.path.pop_back(); +} + +void DataTypeOneElementTuple::deserializeBinaryBulkWithMultipleStreams( + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + addToPath(settings.path); + element->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state); + settings.path.pop_back(); +} + +static DataTypePtr create(const ASTPtr & arguments) +{ + if (!arguments || arguments->children.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Data type {} takes only 1 argument", DataTypeOneElementTuple::TYPE_NAME); + + const auto * name_type = arguments->children[0]->as(); + if (!name_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Data type {} takes only pair with name and type", DataTypeOneElementTuple::TYPE_NAME); + + auto nested_type = DataTypeFactory::instance().get(name_type->type); + return std::make_shared(std::move(nested_type), name_type->name); +} + +void registerDataTypeOneElementTuple(DataTypeFactory & factory) +{ + factory.registerDataType(DataTypeOneElementTuple::TYPE_NAME, create); +} + +} diff --git a/src/DataTypes/DataTypeOneElementTuple.h b/src/DataTypes/DataTypeOneElementTuple.h new file mode 100644 index 00000000000..49580433e5a --- /dev/null +++ b/src/DataTypes/DataTypeOneElementTuple.h @@ -0,0 +1,103 @@ +#pragma once + +#include + +namespace DB +{ + +class DataTypeOneElementTuple final : public DataTypeWithSimpleSerialization +{ +private: + DataTypePtr element; + String name; + bool escape_delimiter; + +public: + static constexpr bool is_parametric = true; + static constexpr auto TYPE_NAME = "__OneElementTuple"; + + DataTypeOneElementTuple(const DataTypePtr & element_, const String & name_, bool escape_delimiter_ = true) + : element(element_), name(name_), escape_delimiter(escape_delimiter_) {} + + /// Customized methods. + const char * getFamilyName() const override { return TYPE_NAME; } + std::string doGetName() const override; + + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + + void serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const override; + + void deserializeBinaryBulkWithMultipleStreams( + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const override; + + bool equals(const IDataType & rhs) const override; + + bool isParametric() const override { return true; } + bool haveSubtypes() const override { return true; } + + /// Non-customized methods. + TypeIndex getTypeId() const override { return element->getTypeId(); } + + void serializeBinary(const Field & field, WriteBuffer & ostr) const override { element->serializeBinary(field, ostr); } + void deserializeBinary(Field & field, ReadBuffer & istr) const override { element->deserializeBinary(field, istr); } + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { element->serializeBinary(column, row_num, ostr); } + void deserializeBinary(IColumn & column, ReadBuffer & istr) const override { element->deserializeBinary(column, istr); } + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override + { + element->serializeAsText(column, row_num, ostr, settings); + } + + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override + { + element->deserializeAsWholeText(column, istr, settings); + } + + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override + { + element->serializeProtobuf(column, row_num, protobuf, value_index); + } + + void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override + { + element->deserializeProtobuf(column, protobuf, allow_add_row, row_added); + } + + bool canBeInsideNullable() const override { return element->canBeInsideNullable(); } + MutableColumnPtr createColumn() const override { return element->createColumn(); } + Field getDefault() const override { return element->getDefault(); } + void insertDefaultInto(IColumn & column) const override { element->insertDefaultInto(column); } + bool isComparable() const override { return element->isComparable(); } + bool textCanContainOnlyValidUTF8() const override { return element->textCanContainOnlyValidUTF8(); } + bool haveMaximumSizeOfValue() const override { return element->haveMaximumSizeOfValue(); } + size_t getMaximumSizeOfValueInMemory() const override { return element->getMaximumSizeOfValueInMemory(); } + size_t getSizeOfValueInMemory() const override { return element->getSizeOfValueInMemory(); } + + DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override { return element->tryGetSubcolumnType(subcolumn_name); } + MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override { return element->getSubcolumn(subcolumn_name, column); } + +private: + void addToPath(SubstreamPath & path) const; +}; + +} diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 01b6f0ccf0b..c980e4382b6 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include #include #include @@ -538,11 +540,14 @@ DataTypePtr DataTypeTuple::tryGetSubcolumnType(const String & subcolumn_name) co if (startsWith(subcolumn_name, names[i])) { size_t name_length = names[i].size(); + DataTypePtr subcolumn_type; if (subcolumn_name.size() == name_length) - return elems[i]; + subcolumn_type = elems[i]; + else if (subcolumn_name[name_length] == '.') + subcolumn_type = elems[i]->tryGetSubcolumnType(subcolumn_name.substr(name_length + 1)); - if (subcolumn_name[name_length] == '.') - return elems[i]->tryGetSubcolumnType(subcolumn_name.substr(name_length + 1)); + if (subcolumn_type) + return std::make_shared(std::move(subcolumn_type), names[i]); } } @@ -604,15 +609,7 @@ static DataTypePtr create(const ASTPtr & arguments) void registerDataTypeTuple(DataTypeFactory & factory) { factory.registerDataType("Tuple", create); -} - -void registerDataTypeNested(DataTypeFactory & factory) -{ - /// Nested(...) data type is just a sugar for Array(Tuple(...)) - factory.registerDataType("Nested", [&factory](const ASTPtr & arguments) - { - return std::make_shared(factory.get("Tuple", arguments)); - }); + // factory.registerDataTypeCustom(DATA_TYPE_ONE_ELEMENT_TUPLE_NAME, createOneElementTuple); } } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index cdf1b02c1b3..4a298c570d0 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -108,28 +108,33 @@ MutableColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, IColumn throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } -std::vector IDataType::getSubcolumnNames() const +Names IDataType::getSubcolumnNames() const { - std::vector res; - enumerateStreams([&res, this](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) + NameSet res; + enumerateStreams([&res, this](const SubstreamPath & substream_path, const IDataType & /* substream_type */) { - auto subcolumn_name = IDataType::getSubcolumnNameForStream("", substream_path); - if (!subcolumn_name.empty()) + SubstreamPath new_path; + /// Iterate over path to try to get intermediate subcolumns for complex nested types. + for (const auto & elem : substream_path) { - subcolumn_name = subcolumn_name.substr(1); // It starts with a dot. - /// Not all of substreams have its subcolumn. - if (tryGetSubcolumnType(subcolumn_name)) - res.push_back(subcolumn_name); + new_path.push_back(elem); + auto subcolumn_name = getSubcolumnNameForStream(new_path); + if (!subcolumn_name.empty() && tryGetSubcolumnType(subcolumn_name)) + { + /// Not all of substreams have its subcolumn. + if (tryGetSubcolumnType(subcolumn_name)) + res.insert(subcolumn_name); + } } }); - return res; + return Names(std::make_move_iterator(res.begin()), std::make_move_iterator(res.end())); } static String getNameForSubstreamPath( String stream_name, const IDataType::SubstreamPath & path, - const String & tuple_element_delimeter = ".") + bool escape_tuple_delimiter) { size_t array_level = 0; for (const auto & elem : path) @@ -140,80 +145,41 @@ static String getNameForSubstreamPath( stream_name += ".size" + toString(array_level); else if (elem.type == IDataType::Substream::ArrayElements) ++array_level; - else if (elem.type == IDataType::Substream::TupleElement) - stream_name += tuple_element_delimeter + escapeForFileName(elem.tuple_element_name); else if (elem.type == IDataType::Substream::DictionaryKeys) stream_name += ".dict"; + else if (elem.type == IDataType::Substream::TupleElement) + { + /// For compatibility reasons, we use %2E (escaped dot) instead of dot. + /// Because nested data may be represented not by Array of Tuple, + /// but by separate Array columns with names in a form of a.b, + /// and name is encoded as a whole. + stream_name += (escape_tuple_delimiter && elem.escape_tuple_delimiter ? + escapeForFileName(".") : ".") + escapeForFileName(elem.tuple_element_name); + } } return stream_name; } -static bool isOldStyleNestedSizes(const NameAndTypePair & column, const IDataType::SubstreamPath & path) +String IDataType::getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path) { auto storage_name = column.getStorageName(); auto nested_storage_name = Nested::extractTableName(column.getStorageName()); - if (storage_name == nested_storage_name) - return false; - - return (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes) || column.getSubcolumnName() == "size0"; -} - -static String getDelimiterForSubcolumnPart(const String & subcolumn_part) -{ - if (subcolumn_part == "null" || startsWith(subcolumn_part, "size")) - return "."; - - return "%2E"; -} - -/// FIXME: rewrite it. -String IDataType::getFileNameForStream(const NameAndTypePair & column, const IDataType::SubstreamPath & path) -{ - auto storage_name = column.getStorageName(); - if (isOldStyleNestedSizes(column, path)) - storage_name = Nested::extractTableName(storage_name); + if (storage_name != nested_storage_name && (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes)) + storage_name = nested_storage_name; auto stream_name = escapeForFileName(storage_name); - auto subcolumn_name = column.getSubcolumnName(); - - if (!subcolumn_name.empty()) - { - std::vector subcolumn_parts; - boost::split(subcolumn_parts, subcolumn_name, [](char c) { return c == '.'; }); - - size_t current_nested_level = 0; - for (const auto & elem : path) - { - if (elem.type == Substream::ArrayElements && elem.is_part_of_nested) - { - ++current_nested_level; - } - else if (elem.type == Substream::ArraySizes) - { - size_t nested_level = column.type->getNestedLevel(); - for (size_t i = 0; i < nested_level - current_nested_level; ++i) - { - if (subcolumn_parts.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get substream name for column {}." - " Not enough subcolumn parts. Needed: {}", column.name, nested_level - current_nested_level); - - subcolumn_parts.pop_back(); - } - } - } - - for (const auto & subcolumn_part : subcolumn_parts) - stream_name += getDelimiterForSubcolumnPart(subcolumn_part) + escapeForFileName(subcolumn_part); - } - - return getNameForSubstreamPath(std::move(stream_name), path, "%2E"); + return getNameForSubstreamPath(std::move(stream_name), path, true); } -String IDataType::getSubcolumnNameForStream(String stream_name, const SubstreamPath & path) +String IDataType::getSubcolumnNameForStream(const SubstreamPath & path) { - return getNameForSubstreamPath(std::move(stream_name), path); + auto subcolumn_name = getNameForSubstreamPath("", path, false); + if (!subcolumn_name.empty()) + subcolumn_name = subcolumn_name.substr(1); // It starts with a dot. + + return subcolumn_name; } bool IDataType::isSpecialCompressionAllowed(const SubstreamPath & path) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index a89d2c5ac70..f4dd24ff3e5 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -101,7 +102,7 @@ public: /// Index of tuple element, starting at 1. String tuple_element_name; - bool is_part_of_nested = false; + bool escape_tuple_delimiter = true; Substream(Type type_) : type(type_) {} }; @@ -120,7 +121,7 @@ public: virtual DataTypePtr tryGetSubcolumnType(const String & /* subcolumn_name */) const { return nullptr; } DataTypePtr getSubcolumnType(const String & subcolumn_name) const; virtual MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const; - std::vector getSubcolumnNames() const; + Names getSubcolumnNames() const; using OutputStreamGetter = std::function; using InputStreamGetter = std::function; @@ -449,13 +450,11 @@ public: /// Strings, Numbers, Date, DateTime, Nullable virtual bool canBeInsideLowCardinality() const { return false; } - virtual size_t getNestedLevel() const { return 0; } - /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path); - static String getSubcolumnNameForStream(String stream_name, const SubstreamPath & path); + static String getSubcolumnNameForStream(const SubstreamPath & path); /// Substream path supports special compression methods like codec Delta. /// For all other substreams (like ArraySizes, NullMasks, etc.) we use only diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index 20a63bb7727..2c7ea8fa76d 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -27,9 +27,11 @@ SRCS( DataTypeInterval.cpp DataTypeLowCardinality.cpp DataTypeLowCardinalityHelpers.cpp + DataTypeNested.cpp DataTypeNothing.cpp DataTypeNullable.cpp DataTypeNumberBase.cpp + DataTypeOneElementTuple.cpp DataTypesDecimal.cpp DataTypesNumber.cpp DataTypeString.cpp diff --git a/src/Parsers/ASTNameTypePair.h b/src/Parsers/ASTNameTypePair.h index 638e980cbdc..1f0df8fa485 100644 --- a/src/Parsers/ASTNameTypePair.h +++ b/src/Parsers/ASTNameTypePair.h @@ -16,6 +16,8 @@ public: /// type ASTPtr type; + size_t level_of_nesting = 0; + /** Get the text that identifies this element. */ String getID(char delim) const override { return "NameTypePair" + (delim + name); } ASTPtr clone() const override; diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 9b10111db06..ec2f169b525 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -82,7 +82,7 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// Parse optional parameters ASTPtr expr_list_args; - ParserList args_parser_nested(std::make_unique(), std::make_unique(TokenType::Comma), false); + ParserList args_parser_nested(std::make_unique(), std::make_unique(TokenType::Comma), false); if (args_parser_nested.parse(pos, expr_list_args, expected)) { if (pos->type != TokenType::ClosingRoundBracket) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 4b8678354b7..e76c394b46d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -252,6 +253,12 @@ void ColumnsDescription::flattenNested() { for (auto it = columns.begin(); it != columns.end();) { + if (isNested(it->type)) + { + ++it; + continue; + } + const auto * type_arr = typeid_cast(it->type.get()); if (!type_arr) { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 53ba113fce8..79607fcf169 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -54,8 +54,10 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( for (size_t i = 0; i < columns_num; ++i, ++name_and_type) { auto column_from_part = getColumnFromPart(*name_and_type); - auto position = data_part->getColumnPosition(column_from_part); + if (duplicated_subcolumns.count(column_from_part.name)) + continue; + auto position = data_part->getColumnPosition(column_from_part); if (!position && typeid_cast(column_from_part.type.get())) { /// If array of Nested column is missing in part, @@ -148,11 +150,10 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, { auto column_from_part = getColumnFromPart(*name_and_type); - if (!res_columns[pos] || duplicated_subcolumns.count(column_from_part.name)) + if (!res_columns[pos]) continue; auto & column = mutable_columns[pos]; - try { size_t column_size_before_reading = column->size(); @@ -160,7 +161,6 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, readData(column_from_part, *column, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); size_t read_rows_in_column = column->size() - column_size_before_reading; - if (read_rows_in_column < rows_to_read) throw Exception("Cannot read all data in MergeTreeReaderCompact. Rows read: " + toString(read_rows_in_column) + ". Rows expected: " + toString(rows_to_read) + ".", ErrorCodes::CANNOT_READ_ALL_DATA); From 829c7c99ea662bf3f2a9c81a9e9f884d9427d6a6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 9 Nov 2020 15:26:24 +0300 Subject: [PATCH 012/264] implement nested with multiple nesting --- src/DataTypes/DataTypeNullable.cpp | 2 -- src/DataTypes/DataTypeTuple.cpp | 2 -- src/DataTypes/IDataType.cpp | 1 - src/DataTypes/IDataType.h | 2 -- src/Parsers/ASTNameTypePair.h | 2 -- src/Storages/ColumnsDescription.cpp | 1 - 6 files changed, 10 deletions(-) diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index da8bbb632ff..27443e4df49 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include #include @@ -15,7 +14,6 @@ #include #include #include -#include namespace DB diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index e3cd6ac6b3b..f17479137c4 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include #include #include @@ -620,7 +619,6 @@ static DataTypePtr create(const ASTPtr & arguments) void registerDataTypeTuple(DataTypeFactory & factory) { factory.registerDataType("Tuple", create); - // factory.registerDataTypeCustom(DATA_TYPE_ONE_ELEMENT_TUPLE_NAME, createOneElementTuple); } } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 4a298c570d0..d672f0e8a88 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -9,7 +9,6 @@ #include #include #include -#include namespace DB { diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index f4dd24ff3e5..a3142f0c3c1 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -160,8 +160,6 @@ public: bool position_independent_encoding = true; /// If not zero, may be used to avoid reallocations while reading column of String type. double avg_value_size_hint = 0; - - std::vector temporary_column_holders; }; /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. diff --git a/src/Parsers/ASTNameTypePair.h b/src/Parsers/ASTNameTypePair.h index 1f0df8fa485..638e980cbdc 100644 --- a/src/Parsers/ASTNameTypePair.h +++ b/src/Parsers/ASTNameTypePair.h @@ -16,8 +16,6 @@ public: /// type ASTPtr type; - size_t level_of_nesting = 0; - /** Get the text that identifies this element. */ String getID(char delim) const override { return "NameTypePair" + (delim + name); } ASTPtr clone() const override; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index e76c394b46d..dfae95d8925 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include From e9333e37ba535dfb7eb2e79715f60cf27e15fd48 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 10 Nov 2020 15:13:33 +0300 Subject: [PATCH 013/264] implement nested with multiple nesting --- src/DataTypes/DataTypeArray.cpp | 14 +- src/DataTypes/DataTypeArray.h | 12 +- src/DataTypes/DataTypeCustom.h | 48 +++++- src/DataTypes/DataTypeCustom_fwd.h | 18 +++ src/DataTypes/DataTypeFactory.cpp | 11 +- src/DataTypes/DataTypeFactory.h | 2 + src/DataTypes/DataTypeLowCardinality.cpp | 12 +- src/DataTypes/DataTypeLowCardinality.h | 12 +- src/DataTypes/DataTypeNullable.cpp | 14 +- src/DataTypes/DataTypeNullable.h | 12 +- src/DataTypes/DataTypeOneElementTuple.cpp | 186 ++++++++++------------ src/DataTypes/DataTypeOneElementTuple.h | 97 +---------- src/DataTypes/DataTypeTuple.cpp | 12 +- src/DataTypes/DataTypeTuple.h | 12 +- src/DataTypes/IDataType.cpp | 67 ++++++++ src/DataTypes/IDataType.h | 82 +++++++--- 16 files changed, 337 insertions(+), 274 deletions(-) create mode 100644 src/DataTypes/DataTypeCustom_fwd.h diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 2c978f58fe6..d35aba4f410 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -168,7 +168,7 @@ namespace } -void DataTypeArray::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeArray::enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::ArraySizes); callback(path, *this); @@ -178,7 +178,7 @@ void DataTypeArray::enumerateStreams(const StreamCallback & callback, SubstreamP } -void DataTypeArray::serializeBinaryBulkStatePrefix( +void DataTypeArray::serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -188,7 +188,7 @@ void DataTypeArray::serializeBinaryBulkStatePrefix( } -void DataTypeArray::serializeBinaryBulkStateSuffix( +void DataTypeArray::serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -198,7 +198,7 @@ void DataTypeArray::serializeBinaryBulkStateSuffix( } -void DataTypeArray::deserializeBinaryBulkStatePrefix( +void DataTypeArray::deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { @@ -208,7 +208,7 @@ void DataTypeArray::deserializeBinaryBulkStatePrefix( } -void DataTypeArray::serializeBinaryBulkWithMultipleStreams( +void DataTypeArray::serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, @@ -255,7 +255,7 @@ void DataTypeArray::serializeBinaryBulkWithMultipleStreams( } -void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( +void DataTypeArray::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, @@ -527,7 +527,7 @@ DataTypePtr DataTypeArray::tryGetSubcolumnType(const String & subcolumn_name) co DataTypePtr DataTypeArray::tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const { if (subcolumn_name == "size" + std::to_string(level)) - return std::make_shared(std::make_shared(), subcolumn_name, false); + return createOneElementTuple(std::make_shared(), subcolumn_name, false); DataTypePtr subcolumn; if (const auto * nested_array = typeid_cast(nested.get())) diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index ea01a1f81a9..e8ba6f73e5f 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -57,28 +57,28 @@ public: * This is necessary, because when implementing nested structures, several arrays can have common sizes. */ - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const override; - void serializeBinaryBulkStatePrefix( + void serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkStateSuffix( + void serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkStatePrefix( + void deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkWithMultipleStreams( + void serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkWithMultipleStreams( + void deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/DataTypeCustom.h b/src/DataTypes/DataTypeCustom.h index c4f846d0259..64695a01f11 100644 --- a/src/DataTypes/DataTypeCustom.h +++ b/src/DataTypes/DataTypeCustom.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -62,8 +63,45 @@ public: virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const = 0; }; +class IDataTypeCustomStreams +{ +public: + virtual ~IDataTypeCustomStreams() = default; + + virtual void enumerateStreams( + const IDataType::StreamCallback & callback, + IDataType::SubstreamPath & path) const = 0; + + virtual void serializeBinaryBulkStatePrefix( + IDataType::SerializeBinaryBulkSettings & settings, + IDataType::SerializeBinaryBulkStatePtr & state) const = 0; + + virtual void serializeBinaryBulkStateSuffix( + IDataType::SerializeBinaryBulkSettings & settings, + IDataType::SerializeBinaryBulkStatePtr & state) const = 0; + + virtual void deserializeBinaryBulkStatePrefix( + IDataType::DeserializeBinaryBulkSettings & settings, + IDataType::DeserializeBinaryBulkStatePtr & state) const = 0; + + virtual void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + IDataType::SerializeBinaryBulkSettings & settings, + IDataType::SerializeBinaryBulkStatePtr & state) const = 0; + + virtual void deserializeBinaryBulkWithMultipleStreams( + IColumn & column, + size_t limit, + IDataType::DeserializeBinaryBulkSettings & settings, + IDataType::DeserializeBinaryBulkStatePtr & state) const = 0; +}; + using DataTypeCustomNamePtr = std::unique_ptr; using DataTypeCustomTextSerializationPtr = std::unique_ptr; +using DataTypeCustomStreamsPtr = std::unique_ptr; + /** Describe a data type customization */ @@ -71,9 +109,15 @@ struct DataTypeCustomDesc { DataTypeCustomNamePtr name; DataTypeCustomTextSerializationPtr text_serialization; + DataTypeCustomStreamsPtr streams; - DataTypeCustomDesc(DataTypeCustomNamePtr name_, DataTypeCustomTextSerializationPtr text_serialization_) - : name(std::move(name_)), text_serialization(std::move(text_serialization_)) {} + DataTypeCustomDesc( + DataTypeCustomNamePtr name_, + DataTypeCustomTextSerializationPtr text_serialization_ = nullptr, + DataTypeCustomStreamsPtr streams_ = nullptr) + : name(std::move(name_)) + , text_serialization(std::move(text_serialization_)) + , streams(std::move(streams_)) {} }; using DataTypeCustomDescPtr = std::unique_ptr; diff --git a/src/DataTypes/DataTypeCustom_fwd.h b/src/DataTypes/DataTypeCustom_fwd.h new file mode 100644 index 00000000000..99c8eee9748 --- /dev/null +++ b/src/DataTypes/DataTypeCustom_fwd.h @@ -0,0 +1,18 @@ +#pragma once + +#include + +namespace DB +{ + +class IDataTypeCustomName; +class IDataTypeCustomTextSerialization; +class IDataTypeCustomStreams; +struct DataTypeCustomDesc; + +using DataTypeCustomNamePtr = std::unique_ptr; +using DataTypeCustomTextSerializationPtr = std::unique_ptr; +using DataTypeCustomStreamsPtr = std::unique_ptr; +using DataTypeCustomDescPtr = std::unique_ptr; + +} diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 1a9e9eeaf49..7c380722099 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -79,6 +79,16 @@ DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr return findCreatorByName(family_name)(parameters); } +DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) const +{ + if (!customization->name) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create custom type without name"); + + auto type = get(customization->name->getName()); + type->setCustomization(std::move(customization)); + return type; +} + void DataTypeFactory::registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness) { @@ -180,7 +190,6 @@ DataTypeFactory::DataTypeFactory() registerDataTypeDomainIPv4AndIPv6(*this); registerDataTypeDomainSimpleAggregateFunction(*this); registerDataTypeDomainGeo(*this); - registerDataTypeOneElementTuple(*this); } DataTypeFactory & DataTypeFactory::instance() diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index eab20482f04..45247060552 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -33,6 +34,7 @@ public: DataTypePtr get(const String & full_name) const; DataTypePtr get(const String & family_name, const ASTPtr & parameters) const; DataTypePtr get(const ASTPtr & ast) const; + DataTypePtr getCustom(DataTypeCustomDescPtr customization) const; /// Register a type family by its name. void registerDataType(const String & family_name, Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index 6b1166dc26a..cb558aa9999 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -49,7 +49,7 @@ DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_) + dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } -void DataTypeLowCardinality::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeLowCardinality::enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::DictionaryKeys); dictionary_type->enumerateStreams(callback, path); @@ -242,7 +242,7 @@ static DeserializeStateLowCardinality * checkAndGetLowCardinalityDeserializeStat return low_cardinality_state; } -void DataTypeLowCardinality::serializeBinaryBulkStatePrefix( +void DataTypeLowCardinality::serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -262,7 +262,7 @@ void DataTypeLowCardinality::serializeBinaryBulkStatePrefix( state = std::make_shared(key_version); } -void DataTypeLowCardinality::serializeBinaryBulkStateSuffix( +void DataTypeLowCardinality::serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -288,7 +288,7 @@ void DataTypeLowCardinality::serializeBinaryBulkStateSuffix( } } -void DataTypeLowCardinality::deserializeBinaryBulkStatePrefix( +void DataTypeLowCardinality::deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { @@ -481,7 +481,7 @@ namespace } } -void DataTypeLowCardinality::serializeBinaryBulkWithMultipleStreams( +void DataTypeLowCardinality::serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, @@ -578,7 +578,7 @@ void DataTypeLowCardinality::serializeBinaryBulkWithMultipleStreams( index_version.getDataType()->serializeBinaryBulk(*positions, *indexes_stream, 0, num_rows); } -void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams( +void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index f8c314909b8..88e1e48fdbe 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -22,28 +22,28 @@ public: const char * getFamilyName() const override { return "LowCardinality"; } TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; } - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const override; - void serializeBinaryBulkStatePrefix( + void serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkStateSuffix( + void serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkStatePrefix( + void deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkWithMultipleStreams( + void serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkWithMultipleStreams( + void deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 27443e4df49..8a58512dd8a 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -42,7 +42,7 @@ bool DataTypeNullable::onlyNull() const } -void DataTypeNullable::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeNullable::enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::NullMap); callback(path, *this); @@ -52,7 +52,7 @@ void DataTypeNullable::enumerateStreams(const StreamCallback & callback, Substre } -void DataTypeNullable::serializeBinaryBulkStatePrefix( +void DataTypeNullable::serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -62,7 +62,7 @@ void DataTypeNullable::serializeBinaryBulkStatePrefix( } -void DataTypeNullable::serializeBinaryBulkStateSuffix( +void DataTypeNullable::serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -72,7 +72,7 @@ void DataTypeNullable::serializeBinaryBulkStateSuffix( } -void DataTypeNullable::deserializeBinaryBulkStatePrefix( +void DataTypeNullable::deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { @@ -82,7 +82,7 @@ void DataTypeNullable::deserializeBinaryBulkStatePrefix( } -void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( +void DataTypeNullable::serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, @@ -104,7 +104,7 @@ void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( } -void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( +void DataTypeNullable::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, @@ -529,7 +529,7 @@ bool DataTypeNullable::equals(const IDataType & rhs) const DataTypePtr DataTypeNullable::tryGetSubcolumnType(const String & subcolumn_name) const { if (subcolumn_name == "null") - return std::make_shared(std::make_shared(), subcolumn_name, false); + return createOneElementTuple(std::make_shared(), subcolumn_name, false); return nested_data_type->tryGetSubcolumnType(subcolumn_name); } diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index 0bb5ace2f2a..c4f8bff0e69 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -18,28 +18,28 @@ public: const char * getFamilyName() const override { return "Nullable"; } TypeIndex getTypeId() const override { return TypeIndex::Nullable; } - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const override; - void serializeBinaryBulkStatePrefix( + void serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkStateSuffix( + void serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkStatePrefix( + void deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkWithMultipleStreams( + void serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkWithMultipleStreams( + void deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/DataTypeOneElementTuple.cpp b/src/DataTypes/DataTypeOneElementTuple.cpp index d14eddb0070..0654a0b09b1 100644 --- a/src/DataTypes/DataTypeOneElementTuple.cpp +++ b/src/DataTypes/DataTypeOneElementTuple.cpp @@ -1,121 +1,107 @@ -#include -#include #include #include +#include #include #include #include -#include -#include #include namespace DB { -namespace ErrorCodes +namespace { - extern const int LOGICAL_ERROR; -} -void DataTypeOneElementTuple::addToPath(SubstreamPath & path) const +class DataTypeOneElementTupleStreams : public IDataTypeCustomStreams { - path.push_back(Substream::TupleElement); - path.back().tuple_element_name = name; - path.back().escape_tuple_delimiter = escape_delimiter; +private: + DataTypePtr nested; + String name; + bool escape_delimiter; + +public: + DataTypeOneElementTupleStreams(const DataTypePtr & nested_, const String & name_, bool escape_delimiter_ = true) + : nested(nested_), name(name_), escape_delimiter(escape_delimiter_) {} + + void enumerateStreams( + const IDataType::StreamCallback & callback, + IDataType::SubstreamPath & path) const override + { + addToPath(path); + nested->enumerateStreams(callback, path); + path.pop_back(); + } + + void serializeBinaryBulkStatePrefix( + IDataType:: SerializeBinaryBulkSettings & settings, + IDataType::SerializeBinaryBulkStatePtr & state) const override + { + addToPath(settings.path); + nested->serializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); + } + + void serializeBinaryBulkStateSuffix( + IDataType::SerializeBinaryBulkSettings & settings, + IDataType::SerializeBinaryBulkStatePtr & state) const override + { + addToPath(settings.path); + nested->serializeBinaryBulkStateSuffix(settings, state); + settings.path.pop_back(); + } + + void deserializeBinaryBulkStatePrefix( + IDataType::DeserializeBinaryBulkSettings & settings, + IDataType::DeserializeBinaryBulkStatePtr & state) const override + { + addToPath(settings.path); + nested->deserializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); + } + + void serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + IDataType::SerializeBinaryBulkSettings & settings, + IDataType::SerializeBinaryBulkStatePtr & state) const override + { + addToPath(settings.path); + nested->serializeBinaryBulkWithMultipleStreams(column, offset, limit, settings, state); + settings.path.pop_back(); + } + + void deserializeBinaryBulkWithMultipleStreams( + IColumn & column, + size_t limit, + IDataType::DeserializeBinaryBulkSettings & settings, + IDataType::DeserializeBinaryBulkStatePtr & state) const override + { + addToPath(settings.path); + nested->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state); + settings.path.pop_back(); + } + +private: + void addToPath(IDataType::SubstreamPath & path) const + { + path.push_back(IDataType::Substream::TupleElement); + path.back().tuple_element_name = name; + path.back().escape_tuple_delimiter = escape_delimiter; + } +}; + } -std::string DataTypeOneElementTuple::doGetName() const +DataTypePtr createOneElementTuple(const DataTypePtr & type, const String & name, bool escape_delimiter) { - WriteBufferFromOwnString s; - s << TYPE_NAME << "(" << backQuoteIfNeed(name) << " " << element->getName() << ")"; - return s.str(); -} + auto custom_desc = std::make_unique( + std::make_unique(type->getName()), + nullptr, + std::make_unique(type, name, escape_delimiter)); -bool DataTypeOneElementTuple::equals(const IDataType & rhs) const -{ - const auto * rhs_tuple = typeid_cast(&rhs); - if (!rhs_tuple) - return false; - - return element->equals(*rhs_tuple->element) - && name == rhs_tuple->name - && escape_delimiter == rhs_tuple->escape_delimiter; -} - -void DataTypeOneElementTuple::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const -{ - addToPath(path); - element->enumerateStreams(callback, path); - path.pop_back(); -} - -void DataTypeOneElementTuple::DataTypeOneElementTuple::serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - addToPath(settings.path); - element->serializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); -} - -void DataTypeOneElementTuple::serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - addToPath(settings.path); - element->serializeBinaryBulkStateSuffix(settings, state); - settings.path.pop_back(); -} - -void DataTypeOneElementTuple::deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const -{ - addToPath(settings.path); - element->deserializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); -} - -void DataTypeOneElementTuple::serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - size_t offset, - size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - addToPath(settings.path); - element->serializeBinaryBulkWithMultipleStreams(column, offset, limit, settings, state); - settings.path.pop_back(); -} - -void DataTypeOneElementTuple::deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const -{ - addToPath(settings.path); - element->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state); - settings.path.pop_back(); -} - -static DataTypePtr create(const ASTPtr & arguments) -{ - if (!arguments || arguments->children.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Data type {} takes only 1 argument", DataTypeOneElementTuple::TYPE_NAME); - - const auto * name_type = arguments->children[0]->as(); - if (!name_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Data type {} takes only pair with name and type", DataTypeOneElementTuple::TYPE_NAME); - - auto nested_type = DataTypeFactory::instance().get(name_type->type); - return std::make_shared(std::move(nested_type), name_type->name); -} - -void registerDataTypeOneElementTuple(DataTypeFactory & factory) -{ - factory.registerDataType(DataTypeOneElementTuple::TYPE_NAME, create); + return DataTypeFactory::instance().getCustom(std::move(custom_desc)); } } diff --git a/src/DataTypes/DataTypeOneElementTuple.h b/src/DataTypes/DataTypeOneElementTuple.h index 49580433e5a..03b0511ef4a 100644 --- a/src/DataTypes/DataTypeOneElementTuple.h +++ b/src/DataTypes/DataTypeOneElementTuple.h @@ -1,103 +1,10 @@ #pragma once -#include +#include namespace DB { -class DataTypeOneElementTuple final : public DataTypeWithSimpleSerialization -{ -private: - DataTypePtr element; - String name; - bool escape_delimiter; - -public: - static constexpr bool is_parametric = true; - static constexpr auto TYPE_NAME = "__OneElementTuple"; - - DataTypeOneElementTuple(const DataTypePtr & element_, const String & name_, bool escape_delimiter_ = true) - : element(element_), name(name_), escape_delimiter(escape_delimiter_) {} - - /// Customized methods. - const char * getFamilyName() const override { return TYPE_NAME; } - std::string doGetName() const override; - - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; - - void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; - - void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - size_t offset, - size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; - - bool equals(const IDataType & rhs) const override; - - bool isParametric() const override { return true; } - bool haveSubtypes() const override { return true; } - - /// Non-customized methods. - TypeIndex getTypeId() const override { return element->getTypeId(); } - - void serializeBinary(const Field & field, WriteBuffer & ostr) const override { element->serializeBinary(field, ostr); } - void deserializeBinary(Field & field, ReadBuffer & istr) const override { element->deserializeBinary(field, istr); } - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override { element->serializeBinary(column, row_num, ostr); } - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override { element->deserializeBinary(column, istr); } - - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - element->serializeAsText(column, row_num, ostr, settings); - } - - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - element->deserializeAsWholeText(column, istr, settings); - } - - void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override - { - element->serializeProtobuf(column, row_num, protobuf, value_index); - } - - void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override - { - element->deserializeProtobuf(column, protobuf, allow_add_row, row_added); - } - - bool canBeInsideNullable() const override { return element->canBeInsideNullable(); } - MutableColumnPtr createColumn() const override { return element->createColumn(); } - Field getDefault() const override { return element->getDefault(); } - void insertDefaultInto(IColumn & column) const override { element->insertDefaultInto(column); } - bool isComparable() const override { return element->isComparable(); } - bool textCanContainOnlyValidUTF8() const override { return element->textCanContainOnlyValidUTF8(); } - bool haveMaximumSizeOfValue() const override { return element->haveMaximumSizeOfValue(); } - size_t getMaximumSizeOfValueInMemory() const override { return element->getMaximumSizeOfValueInMemory(); } - size_t getSizeOfValueInMemory() const override { return element->getSizeOfValueInMemory(); } - - DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override { return element->tryGetSubcolumnType(subcolumn_name); } - MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override { return element->getSubcolumn(subcolumn_name, column); } - -private: - void addToPath(SubstreamPath & path) const; -}; +DataTypePtr createOneElementTuple(const DataTypePtr & type, const String & name, bool escape_delimiter = true); } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index f17479137c4..d111601188c 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -289,7 +289,7 @@ void DataTypeTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons }); } -void DataTypeTuple::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeTuple::enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const { path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) @@ -344,7 +344,7 @@ static DeserializeBinaryBulkStateTuple * checkAndGetTupleDeserializeState(IDataT return tuple_state; } -void DataTypeTuple::serializeBinaryBulkStatePrefix( +void DataTypeTuple::serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -362,7 +362,7 @@ void DataTypeTuple::serializeBinaryBulkStatePrefix( state = std::move(tuple_state); } -void DataTypeTuple::serializeBinaryBulkStateSuffix( +void DataTypeTuple::serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -377,7 +377,7 @@ void DataTypeTuple::serializeBinaryBulkStateSuffix( settings.path.pop_back(); } -void DataTypeTuple::deserializeBinaryBulkStatePrefix( +void DataTypeTuple::deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const { @@ -395,7 +395,7 @@ void DataTypeTuple::deserializeBinaryBulkStatePrefix( state = std::move(tuple_state); } -void DataTypeTuple::serializeBinaryBulkWithMultipleStreams( +void DataTypeTuple::serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, @@ -414,7 +414,7 @@ void DataTypeTuple::serializeBinaryBulkWithMultipleStreams( settings.path.pop_back(); } -void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( +void DataTypeTuple::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 764f1e23a76..cb2a91795f4 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -53,28 +53,28 @@ public: /** Each sub-column in a tuple is serialized in separate stream. */ - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; + void enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const override; - void serializeBinaryBulkStatePrefix( + void serializeBinaryBulkStatePrefixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkStateSuffix( + void serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkStatePrefix( + void deserializeBinaryBulkStatePrefixImpl( DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; - void serializeBinaryBulkWithMultipleStreams( + void serializeBinaryBulkWithMultipleStreamsImpl( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; - void deserializeBinaryBulkWithMultipleStreams( + void deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index d672f0e8a88..cef36fec4a4 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { @@ -198,6 +199,69 @@ void IDataType::insertDefaultInto(IColumn & column) const column.insertDefault(); } +void IDataType::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +{ + if (custom_streams) + custom_streams->enumerateStreams(callback, path); + else + enumerateStreamsImpl(callback, path); +} + +void IDataType::serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + if (custom_streams) + custom_streams->serializeBinaryBulkStatePrefix(settings, state); + else + serializeBinaryBulkStatePrefixImpl(settings, state); +} + +void IDataType::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + if (custom_streams) + custom_streams->serializeBinaryBulkStateSuffix(settings, state); + else + serializeBinaryBulkStateSuffixImpl(settings, state); +} + +void IDataType::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + if (custom_streams) + custom_streams->deserializeBinaryBulkStatePrefix(settings, state); + else + deserializeBinaryBulkStatePrefixImpl(settings, state); +} + +void IDataType::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + if (custom_streams) + custom_streams->serializeBinaryBulkWithMultipleStreams(column, offset, limit, settings, state); + else + serializeBinaryBulkWithMultipleStreamsImpl(column, offset, limit, settings, state); +} + +void IDataType::deserializeBinaryBulkWithMultipleStreams( + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + if (custom_streams) + custom_streams->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state); + else + deserializeBinaryBulkWithMultipleStreamsImpl(column, limit, settings, state); +} + void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { if (custom_text_serialization) @@ -294,6 +358,9 @@ void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const if (custom_desc_->text_serialization) custom_text_serialization = std::move(custom_desc_->text_serialization); + + if (custom_desc_->streams) + custom_streams = std::move(custom_desc_->streams); } } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index a3142f0c3c1..d197711fdfc 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -3,8 +3,9 @@ #include #include #include -#include #include +#include +#include namespace DB @@ -111,10 +112,7 @@ public: using StreamCallback = std::function; - virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const - { - callback(path, *this); - } + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const; void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } @@ -163,19 +161,19 @@ public: }; /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. - virtual void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & /*settings*/, - SerializeBinaryBulkStatePtr & /*state*/) const {} + void serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const; /// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization. - virtual void serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & /*settings*/, - SerializeBinaryBulkStatePtr & /*state*/) const {} + void serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const; /// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr. - virtual void deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & /*settings*/, - DeserializeBinaryBulkStatePtr & /*state*/) const {} + void deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const; /** 'offset' and 'limit' are used to specify range. * limit = 0 - means no limit. @@ -183,27 +181,19 @@ public: * offset + limit could be greater than size of column * - in that case, column is serialized till the end. */ - virtual void serializeBinaryBulkWithMultipleStreams( + void serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, size_t limit, SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & /*state*/) const - { - if (WriteBuffer * stream = settings.getter(settings.path)) - serializeBinaryBulk(column, *stream, offset, limit); - } + SerializeBinaryBulkStatePtr & state) const; /// Read no more than limit values and append them into column. - virtual void deserializeBinaryBulkWithMultipleStreams( + void deserializeBinaryBulkWithMultipleStreams( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & /*state*/) const - { - if (ReadBuffer * stream = settings.getter(settings.path)) - deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint); - } + DeserializeBinaryBulkStatePtr & state) const; /** Override these methods for data types that require just single stream (most of data types). */ @@ -276,6 +266,44 @@ public: protected: virtual String doGetName() const; + virtual void enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const + { + callback(path, *this); + } + + virtual void serializeBinaryBulkStatePrefixImpl( + SerializeBinaryBulkSettings & /*settings*/, + SerializeBinaryBulkStatePtr & /*state*/) const {} + + virtual void serializeBinaryBulkStateSuffixImpl( + SerializeBinaryBulkSettings & /*settings*/, + SerializeBinaryBulkStatePtr & /*state*/) const {} + + virtual void deserializeBinaryBulkStatePrefixImpl( + DeserializeBinaryBulkSettings & /*settings*/, + DeserializeBinaryBulkStatePtr & /*state*/) const {} + + virtual void serializeBinaryBulkWithMultipleStreamsImpl( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & /*state*/) const + { + if (WriteBuffer * stream = settings.getter(settings.path)) + serializeBinaryBulk(column, *stream, offset, limit); + } + + virtual void deserializeBinaryBulkWithMultipleStreamsImpl( + IColumn & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & /*state*/) const + { + if (ReadBuffer * stream = settings.getter(settings.path)) + deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint); + } + /// Default implementations of text serialization in case of 'custom_text_serialization' is not set. virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0; @@ -466,9 +494,11 @@ private: /// This is mutable to allow setting custom name and serialization on `const IDataType` post construction. mutable DataTypeCustomNamePtr custom_name; mutable DataTypeCustomTextSerializationPtr custom_text_serialization; + mutable DataTypeCustomStreamsPtr custom_streams; public: const IDataTypeCustomName * getCustomName() const { return custom_name.get(); } + const IDataTypeCustomStreams * getCustomStreams() const { return custom_streams.get(); } }; From 1f65478b49d701ddc26de5412c2188e680c02e9d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 10 Nov 2020 20:32:00 +0300 Subject: [PATCH 014/264] implement nested with multiple nesting --- src/Columns/ColumnNullable.h | 2 + src/Columns/ColumnTuple.h | 1 + src/Core/Settings.h | 1 + src/DataStreams/NativeBlockInputStream.cpp | 10 +- src/DataStreams/NativeBlockInputStream.h | 2 +- src/DataTypes/DataTypeArray.cpp | 15 ++- src/DataTypes/DataTypeArray.h | 3 +- src/DataTypes/DataTypeCustom.h | 5 +- src/DataTypes/DataTypeLowCardinality.cpp | 4 +- src/DataTypes/DataTypeLowCardinality.h | 3 +- src/DataTypes/DataTypeNested.cpp | 8 ++ src/DataTypes/DataTypeNested.h | 4 +- src/DataTypes/DataTypeNullable.cpp | 14 ++- src/DataTypes/DataTypeNullable.h | 3 +- src/DataTypes/DataTypeOneElementTuple.cpp | 16 ++- src/DataTypes/DataTypeOneElementTuple.h | 1 + src/DataTypes/DataTypeTuple.cpp | 10 +- src/DataTypes/DataTypeTuple.h | 3 +- src/DataTypes/IDataType.cpp | 116 +++++++++++++++++- src/DataTypes/IDataType.h | 29 +++-- src/DataTypes/NestedUtils.cpp | 35 ++++-- src/DataTypes/ya.make | 2 - src/Interpreters/InterpreterCreateQuery.cpp | 3 +- src/Parsers/ParserDataType.cpp | 22 +--- src/Storages/AlterCommands.cpp | 3 +- src/Storages/ColumnsDescription.cpp | 6 - src/Storages/MergeTree/IMergeTreeReader.cpp | 38 ------ src/Storages/MergeTree/IMergeTreeReader.h | 4 - .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 3 +- .../MergeTree/MergeTreeDataPartWide.cpp | 3 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 9 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 6 +- .../MergeTree/MergeTreeReaderCompact.cpp | 42 +++---- .../MergeTree/MergeTreeReaderCompact.h | 4 +- .../MergeTree/MergeTreeReaderWide.cpp | 48 ++------ src/Storages/MergeTree/MergeTreeReaderWide.h | 4 +- src/Storages/StorageInMemoryMetadata.cpp | 2 +- src/Storages/StorageLog.cpp | 45 +++---- src/Storages/StorageTinyLog.cpp | 46 +++---- .../01475_read_subcolumns_storages.reference | 4 +- .../01475_read_subcolumns_storages.sh | 8 +- .../01533_multiple_nested.reference | 40 ++++++ .../0_stateless/01533_multiple_nested.sql | 66 ++++++++++ 44 files changed, 422 insertions(+), 274 deletions(-) create mode 100644 tests/queries/0_stateless/01533_multiple_nested.reference create mode 100644 tests/queries/0_stateless/01533_multiple_nested.sql diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 47b0103eab4..d6f19a5739c 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -142,9 +142,11 @@ public: const IColumn & getNestedColumn() const { return *nested_column; } const ColumnPtr & getNestedColumnPtr() const { return nested_column; } + ColumnPtr & getNestedColumnPtr() { return nested_column; } /// Return the column that represents the byte map. const ColumnPtr & getNullMapColumnPtr() const { return null_map; } + ColumnPtr & getNullMapColumnPtr() { return null_map; } ColumnUInt8 & getNullMapColumn() { return assert_cast(*null_map); } const ColumnUInt8 & getNullMapColumn() const { return assert_cast(*null_map); } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 0bee3463f2f..bff82c5eb82 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -98,6 +98,7 @@ public: Columns getColumnsCopy() const { return {columns.begin(), columns.end()}; } const ColumnPtr & getColumnPtr(size_t idx) const { return columns[idx]; } + ColumnPtr & getColumnPtr(size_t idx) { return columns[idx]; } private: int compareAtImpl(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator * collator=nullptr) const; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 580756361b1..e31b4cd8cf0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -396,6 +396,7 @@ class IColumn; M(Bool, allow_non_metadata_alters, true, "Allow to execute alters which affects not only tables metadata, but also data on disk", 0) \ M(Bool, enable_global_with_statement, false, "Propagate WITH statements to UNION queries and all subqueries", 0) \ M(Bool, aggregate_functions_null_for_empty, false, "Rewrite all aggregate functions in a query, adding -OrNull suffix to them", 0) \ + M(Bool, flatten_nested, true, "If true, columns of type Nested will be flatten to separate array columns instead of one array of tuples", 0) \ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index ae2d6886fa8..575acb2a793 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -71,7 +71,7 @@ void NativeBlockInputStream::resetParser() is_killed.store(false); } -void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) +void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) { IDataType::DeserializeBinaryBulkSettings settings; settings.getter = [&](IDataType::SubstreamPath) -> ReadBuffer * { return &istr; }; @@ -82,8 +82,8 @@ void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, type.deserializeBinaryBulkStatePrefix(settings, state); type.deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state); - if (column.size() != rows) - throw Exception("Cannot read all data in NativeBlockInputStream. Rows read: " + toString(column.size()) + ". Rows expected: " + toString(rows) + ".", + if (column->size() != rows) + throw Exception("Cannot read all data in NativeBlockInputStream. Rows read: " + toString(column->size()) + ". Rows expected: " + toString(rows) + ".", ErrorCodes::CANNOT_READ_ALL_DATA); } @@ -158,11 +158,11 @@ Block NativeBlockInputStream::readImpl() } /// Data - MutableColumnPtr read_column = column.type->createColumn(); + ColumnPtr read_column = column.type->createColumn(); double avg_value_size_hint = avg_value_size_hints.empty() ? 0 : avg_value_size_hints[i]; if (rows) /// If no rows, nothing to read. - readData(*column.type, *read_column, istr, rows, avg_value_size_hint); + readData(*column.type, read_column, istr, rows, avg_value_size_hint); column.column = std::move(read_column); diff --git a/src/DataStreams/NativeBlockInputStream.h b/src/DataStreams/NativeBlockInputStream.h index 774a1cfa1cd..8f3d2843e0f 100644 --- a/src/DataStreams/NativeBlockInputStream.h +++ b/src/DataStreams/NativeBlockInputStream.h @@ -74,7 +74,7 @@ public: String getName() const override { return "Native"; } - static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); + static void readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); Block getHeader() const override; diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index d35aba4f410..3434edd4747 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -259,17 +259,25 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const { ColumnArray & column_array = typeid_cast(column); settings.path.push_back(Substream::ArraySizes); - if (auto * stream = settings.getter(settings.path)) + + if (auto cached_column = getFromSubstreamsCache(cache, settings.path)) + { + column_array.getOffsetsPtr() = cached_column; + } + else if (auto * stream = settings.getter(settings.path)) { if (settings.position_independent_encoding) deserializeArraySizesPositionIndependent(column, *stream, limit); else DataTypeNumber().deserializeBinaryBulk(column_array.getOffsetsColumn(), *stream, limit, 0); + + addToSubstreamsCache(cache, settings.path, column_array.getOffsetsPtr()); } settings.path.back() = Substream::ArrayElements; @@ -286,7 +294,8 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreamsImpl( /// Adjust value size hint. Divide it to the average array size. settings.avg_value_size_hint = nested_limit ? settings.avg_value_size_hint / nested_limit * offset_values.size() : 0; - nested->deserializeBinaryBulkWithMultipleStreams(nested_column, nested_limit, settings, state); + nested->deserializeBinaryBulkWithMultipleStreams(column_array.getDataPtr(), nested_limit, settings, state, cache); + settings.path.pop_back(); /// Check consistency between offsets and elements subcolumns. diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index e8ba6f73e5f..ae55fa252ea 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -82,7 +82,8 @@ public: IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; void serializeProtobuf(const IColumn & column, size_t row_num, diff --git a/src/DataTypes/DataTypeCustom.h b/src/DataTypes/DataTypeCustom.h index 64695a01f11..ec6093cf1e4 100644 --- a/src/DataTypes/DataTypeCustom.h +++ b/src/DataTypes/DataTypeCustom.h @@ -92,10 +92,11 @@ public: IDataType::SerializeBinaryBulkStatePtr & state) const = 0; virtual void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, + ColumnPtr & column, size_t limit, IDataType::DeserializeBinaryBulkSettings & settings, - IDataType::DeserializeBinaryBulkStatePtr & state) const = 0; + IDataType::DeserializeBinaryBulkStatePtr & state, + IDataType::SubstreamsCache * cache) const = 0; }; using DataTypeCustomNamePtr = std::unique_ptr; diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index cb558aa9999..569a135f8bf 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -582,8 +582,10 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const { + UNUSED(cache); ColumnLowCardinality & low_cardinality_column = typeid_cast(column); settings.path.push_back(Substream::DictionaryKeys); diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 88e1e48fdbe..6ed2b792ce3 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -47,7 +47,8 @@ public: IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; diff --git a/src/DataTypes/DataTypeNested.cpp b/src/DataTypes/DataTypeNested.cpp index 6dfccb57b1f..cfbfb4c1750 100644 --- a/src/DataTypes/DataTypeNested.cpp +++ b/src/DataTypes/DataTypeNested.cpp @@ -65,4 +65,12 @@ void registerDataTypeNested(DataTypeFactory & factory) return factory.registerDataTypeCustom("Nested", create); } +DataTypePtr createNested(const DataTypes & types, const Names & names) +{ + auto custom_desc = std::make_unique( + std::make_unique(types, names)); + + return DataTypeFactory::instance().getCustom(std::move(custom_desc)); +} + } diff --git a/src/DataTypes/DataTypeNested.h b/src/DataTypes/DataTypeNested.h index 50649fbcddc..9fb12ad4924 100644 --- a/src/DataTypes/DataTypeNested.h +++ b/src/DataTypes/DataTypeNested.h @@ -22,10 +22,12 @@ public: String getName() const override; }; +DataTypePtr createNested(const DataTypes & types, const Names & names); + template inline bool isNested(const DataType & data_type) { - return isArray(data_type) && typeid_cast(data_type->getCustomName()); + return typeid_cast(data_type->getCustomName()) != nullptr; } } diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index 8a58512dd8a..d26c5af8378 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -108,16 +108,24 @@ void DataTypeNullable::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const { ColumnNullable & col = assert_cast(column); settings.path.push_back(Substream::NullMap); - if (auto * stream = settings.getter(settings.path)) + if (auto cached_column = getFromSubstreamsCache(cache, settings.path)) + { + col.getNullMapColumnPtr() = cached_column; + } + else if (auto * stream = settings.getter(settings.path)) + { DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); + addToSubstreamsCache(cache, settings.path, col.getNullMapColumnPtr()); + } settings.path.back() = Substream::NullableElements; - nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), limit, settings, state); + nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumnPtr(), limit, settings, state, cache); settings.path.pop_back(); } diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index c4f8bff0e69..f5e8f74ab8c 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -43,7 +43,8 @@ public: IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; diff --git a/src/DataTypes/DataTypeOneElementTuple.cpp b/src/DataTypes/DataTypeOneElementTuple.cpp index 0654a0b09b1..679b34773e9 100644 --- a/src/DataTypes/DataTypeOneElementTuple.cpp +++ b/src/DataTypes/DataTypeOneElementTuple.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -73,13 +74,14 @@ public: } void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, + ColumnPtr & column, size_t limit, IDataType::DeserializeBinaryBulkSettings & settings, - IDataType::DeserializeBinaryBulkStatePtr & state) const override + IDataType::DeserializeBinaryBulkStatePtr & state, + IDataType::SubstreamsCache * cache) const override { addToPath(settings.path); - nested->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state); + nested->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state, cache); settings.path.pop_back(); } @@ -97,11 +99,15 @@ private: DataTypePtr createOneElementTuple(const DataTypePtr & type, const String & name, bool escape_delimiter) { auto custom_desc = std::make_unique( - std::make_unique(type->getName()), - nullptr, + std::make_unique(type->getName()), nullptr, std::make_unique(type, name, escape_delimiter)); return DataTypeFactory::instance().getCustom(std::move(custom_desc)); } +bool isOneElementTuple(const DataTypePtr & type) +{ + return typeid_cast(type->getCustomStreams()) != nullptr; +} + } diff --git a/src/DataTypes/DataTypeOneElementTuple.h b/src/DataTypes/DataTypeOneElementTuple.h index 03b0511ef4a..a452a547364 100644 --- a/src/DataTypes/DataTypeOneElementTuple.h +++ b/src/DataTypes/DataTypeOneElementTuple.h @@ -6,5 +6,6 @@ namespace DB { DataTypePtr createOneElementTuple(const DataTypePtr & type, const String & name, bool escape_delimiter = true); +bool isOneElementTuple(const DataTypePtr & type); } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index d111601188c..b0004c45520 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -418,17 +417,18 @@ void DataTypeTuple::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const { auto * tuple_state = checkAndGetTupleDeserializeState(state); + auto & column_tuple = assert_cast(column); settings.path.push_back(Substream::TupleElement); settings.avg_value_size_hint = 0; for (const auto i : ext::range(0, ext::size(elems))) { settings.path.back().tuple_element_name = names[i]; - auto & element_col = extractElementColumn(column, i); - elems[i]->deserializeBinaryBulkWithMultipleStreams(element_col, limit, settings, tuple_state->states[i]); + elems[i]->deserializeBinaryBulkWithMultipleStreams(column_tuple.getColumnPtr(i), limit, settings, tuple_state->states[i], cache); } settings.path.pop_back(); } @@ -557,7 +557,7 @@ DataTypePtr DataTypeTuple::tryGetSubcolumnType(const String & subcolumn_name) co subcolumn_type = elems[i]->tryGetSubcolumnType(subcolumn_name.substr(name_length + 1)); if (subcolumn_type) - return std::make_shared(std::move(subcolumn_type), names[i]); + return createOneElementTuple(std::move(subcolumn_type), names[i]); } } diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index cb2a91795f4..476b3316ed2 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -78,7 +78,8 @@ public: IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override; void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index cef36fec4a4..f84cc844853 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -3,8 +3,10 @@ #include #include +#include #include +#include #include #include @@ -22,7 +24,58 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; } -IDataType::IDataType() : custom_name(nullptr), custom_text_serialization(nullptr) +String IDataType::Substream::toString() const +{ + switch (type) + { + case ArrayElements: + return "ArrayElements"; + case ArraySizes: + return "ArraySizes"; + case NullableElements: + return "NullableElements"; + case NullMap: + return "NullMap"; + case TupleElement: + return "TupleElement(" + tuple_element_name + ", " + + std::to_string(escape_tuple_delimiter) + ")"; + case DictionaryKeys: + return "DictionaryKeys"; + case DictionaryIndexes: + return "DictionaryIndexes"; + } + + __builtin_unreachable(); +} + +size_t IDataType::SubstreamPath::getHash() const +{ + SipHash hash; + for (const auto & elem : *this) + { + hash.update(elem.type); + hash.update(elem.tuple_element_name); + hash.update(elem.escape_tuple_delimiter); + } + + return hash.get64(); +} + +String IDataType::SubstreamPath::toString() const +{ + WriteBufferFromOwnString wb; + wb << "{"; + for (size_t i = 0; i < size(); ++i) + { + if (i != 0) + wb << ", "; + wb << at(i).toString(); + } + wb << "}"; + return wb.str(); +} + +IDataType::IDataType() : custom_name(nullptr), custom_text_serialization(nullptr), custom_streams(nullptr) { } @@ -250,16 +303,49 @@ void IDataType::serializeBinaryBulkWithMultipleStreams( serializeBinaryBulkWithMultipleStreamsImpl(column, offset, limit, settings, state); } -void IDataType::deserializeBinaryBulkWithMultipleStreams( +void IDataType::deserializeBinaryBulkWithMultipleStreamsImpl( IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + DeserializeBinaryBulkStatePtr & /* state */, + SubstreamsCache * /* cache */) const +{ + if (ReadBuffer * stream = settings.getter(settings.path)) + deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint); +} + + +void IDataType::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const { if (custom_streams) - custom_streams->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state); - else - deserializeBinaryBulkWithMultipleStreamsImpl(column, limit, settings, state); + { + custom_streams->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state, cache); + return; + } + + /// Do not cache complex type, because they can be constructed + /// their subcolumns, which are in cache. + if (!haveSubtypes()) + { + auto cached_column = getFromSubstreamsCache(cache, settings.path); + if (cached_column) + { + column = cached_column; + return; + } + } + + auto mutable_column = IColumn::mutate(std::move(column)); + deserializeBinaryBulkWithMultipleStreamsImpl(*mutable_column, limit, settings, state, cache); + column = std::move(mutable_column); + + if (!haveSubtypes()) + addToSubstreamsCache(cache, settings.path, column); } void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -363,4 +449,22 @@ void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const custom_streams = std::move(custom_desc_->streams); } +void IDataType::addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column) +{ + if (cache && !path.empty()) + cache->emplace(getSubcolumnNameForStream(path), column); +} + +ColumnPtr IDataType::getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path) +{ + if (!cache || path.empty()) + return nullptr; + + auto it = cache->find(getSubcolumnNameForStream(path)); + if (it == cache->end()) + return nullptr; + + return it->second; +} + } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index d197711fdfc..68c6b65f7c0 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -106,9 +106,17 @@ public: bool escape_tuple_delimiter = true; Substream(Type type_) : type(type_) {} + + String toString() const; }; - using SubstreamPath = std::vector; + struct SubstreamPath : public std::vector + { + UInt64 getHash() const; + String toString() const; + }; + + using SubstreamsCache = std::unordered_map; using StreamCallback = std::function; @@ -190,10 +198,11 @@ public: /// Read no more than limit values and append them into column. void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, + ColumnPtr & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const; + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache = nullptr) const; /** Override these methods for data types that require just single stream (most of data types). */ @@ -272,8 +281,8 @@ protected: } virtual void serializeBinaryBulkStatePrefixImpl( - SerializeBinaryBulkSettings & /*settings*/, - SerializeBinaryBulkStatePtr & /*state*/) const {} + SerializeBinaryBulkSettings & /*settings*/, + SerializeBinaryBulkStatePtr & /*state*/) const {} virtual void serializeBinaryBulkStateSuffixImpl( SerializeBinaryBulkSettings & /*settings*/, @@ -298,11 +307,8 @@ protected: IColumn & column, size_t limit, DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & /*state*/) const - { - if (ReadBuffer * stream = settings.getter(settings.path)) - deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint); - } + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const; /// Default implementations of text serialization in case of 'custom_text_serialization' is not set. @@ -322,6 +328,9 @@ protected: } public: + static void addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column); + static ColumnPtr getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path); + /** Create empty column for corresponding type. */ virtual MutableColumnPtr createColumn() const = 0; diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 0537fa5cdc1..1d080d1032a 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -84,7 +85,8 @@ Block flatten(const Block & block) for (const auto & elem : block) { - if (const DataTypeArray * type_arr = typeid_cast(elem.type.get())) + const DataTypeArray * type_arr = typeid_cast(elem.type.get()); + if (!isNested(elem.type) && type_arr) { const DataTypeTuple * type_tuple = typeid_cast(type_arr->getNestedType().get()); if (type_tuple && type_tuple->haveExplicitNames()) @@ -131,29 +133,36 @@ Block flatten(const Block & block) NamesAndTypesList collect(const NamesAndTypesList & names_and_types) { - NamesAndTypesList res; + NamesAndTypesList res = names_and_types; std::map nested; for (const auto & name_type : names_and_types) { - bool collected = false; - if (const DataTypeArray * type_arr = typeid_cast(name_type.type.get())) + const DataTypeArray * type_arr = typeid_cast(name_type.type.get()); + + /// Ignore true Nested type, but try to unite flatten arrays to Nested type. + if (!isNested(name_type.type) && type_arr) { auto split = splitName(name_type.name); if (!split.second.empty()) - { nested[split.first].emplace_back(split.second, type_arr->getNestedType()); - collected = true; - } } - - if (!collected) - res.push_back(name_type); } - for (const auto & name_elems : nested) - res.emplace_back(name_elems.first, std::make_shared( - std::make_shared(name_elems.second.getTypes(), name_elems.second.getNames()))); + std::unordered_map nested_types; + for (const auto & [name, elems] : nested) + nested_types.emplace(name, createNested(elems.getTypes(), elems.getNames())); + + for (auto & name_type : res) + { + auto split = splitName(name_type.name); + if (name_type.isSubcolumn() || split.second.empty()) + continue; + + auto it = nested_types.find(split.first); + if (it != nested_types.end()) + name_type = NameAndTypePair{split.first, split.second, it->second, it->second->getSubcolumnType(split.second)}; + } return res; } diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index 87717f08a6b..a3d56e0ee3a 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -30,8 +30,6 @@ SRCS( DataTypeNullable.cpp DataTypeNumberBase.cpp DataTypeOneElementTuple.cpp - DataTypesDecimal.cpp - DataTypesNumber.cpp DataTypeString.cpp DataTypeTuple.cpp DataTypeUUID.cpp diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 286d5269a64..a9e79a9a6f8 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -430,7 +430,8 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( res.add(std::move(column)); } - res.flattenNested(); + if (context.getSettingsRef().flatten_nested) + res.flattenNested(); if (res.getAllPhysical().empty()) throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED}; diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index 5f761be94c8..ee746329bff 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -103,24 +103,14 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ++pos; /// Parse optional parameters + ParserList args_parser(std::make_unique(), std::make_unique(TokenType::Comma)); ASTPtr expr_list_args; - ParserList args_parser_nested(std::make_unique(), std::make_unique(TokenType::Comma), false); - if (args_parser_nested.parse(pos, expr_list_args, expected)) - { - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - } - else - { - ParserList args_parser_expr(std::make_unique(), std::make_unique(TokenType::Comma)); - if (!args_parser_expr.parse(pos, expr_list_args, expected)) - return false; - if (pos->type != TokenType::ClosingRoundBracket) - return false; - ++pos; - } + if (!args_parser.parse(pos, expr_list_args, expected)) + return false; + if (pos->type != TokenType::ClosingRoundBracket) + return false; + ++pos; function_node->arguments = expr_list_args; function_node->children.push_back(function_node->arguments); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 559121d6911..323e331b7ee 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -320,7 +320,8 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con metadata.columns.add(column, after_column, first); /// Slow, because each time a list is copied - metadata.columns.flattenNested(); + if (context.getSettingsRef().flatten_nested) + metadata.columns.flattenNested(); } else if (type == DROP_COLUMN) { diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index dfae95d8925..2649117d7ff 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -252,12 +252,6 @@ void ColumnsDescription::flattenNested() { for (auto it = columns.begin(); it != columns.end();) { - if (isNested(it->type)) - { - ++it; - continue; - } - const auto * type_arr = typeid_cast(it->type.get()); if (!type_arr) { diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index c6f038642b0..2680ca96d5b 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -44,27 +44,6 @@ IMergeTreeReader::IMergeTreeReader( { for (const NameAndTypePair & column_from_part : data_part->getColumns()) columns_from_part[column_from_part.name] = column_from_part.type; - - std::unordered_map positions; - size_t pos = 0; - for (const auto & column : columns) - { - if (!column.isSubcolumn()) - positions[column.name] = pos; - ++pos; - } - - pos = 0; - for (const auto & column : columns) - { - if (column.isSubcolumn()) - { - auto it = positions.find(column.getStorageName()); - if (it != positions.end()) - duplicated_subcolumns[column.name] = it->second; - } - ++pos; - } } IMergeTreeReader::~IMergeTreeReader() = default; @@ -95,21 +74,6 @@ static bool arrayHasNoElementsRead(const IColumn & column) return last_offset != 0; } -void IMergeTreeReader::fillDuplicatedSubcolumns(Columns & res_columns) -{ - auto requested_column = columns.begin(); - for (size_t i = 0; i < columns.size(); ++i, ++requested_column) - { - if (res_columns[i]) - continue; - - auto it = duplicated_subcolumns.find(requested_column->name); - if (it != duplicated_subcolumns.end() && res_columns[it->second]) - res_columns[i] = requested_column->getStorageType()->getSubcolumn( - requested_column->getSubcolumnName(), *res_columns[it->second]->assumeMutable()); - } -} - void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_evaluate_missing_defaults, size_t num_rows) { try @@ -121,8 +85,6 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e "Expected " + toString(num_columns) + ", " "got " + toString(res_columns.size()), ErrorCodes::LOGICAL_ERROR); - fillDuplicatedSubcolumns(res_columns); - /// For a missing column of a nested data structure we must create not a column of empty /// arrays, but a column of arrays of correct length. diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 4948784ab8c..d192339432f 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -82,16 +82,12 @@ protected: StorageMetadataPtr metadata_snapshot; MarkRanges all_mark_ranges; - std::unordered_map duplicated_subcolumns; - using ColumnPosition = std::optional; ColumnPosition findColumnForOffsets(const String & column_name) const; friend class MergeTreeRangeReader::DelayedStream; private: - void fillDuplicatedSubcolumns(Columns & res_columns); - /// Alter conversions, which must be applied on fly if required MergeTreeData::AlterConversions alter_conversions; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index e857139cce9..f469f656deb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -1,4 +1,5 @@ #include "MergeTreeDataPartCompact.h" +#include #include #include #include @@ -47,7 +48,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, metadata_snapshot, uncompressed_cache, + ptr, Nested::collect(columns_to_read), metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index ce56efe744f..f94a7bdaad4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -50,7 +51,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings); + ptr, Nested::collect(columns_to_read), metadata_snapshot, mark_ranges, reader_settings); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 3d20a307b42..aea53c36bde 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -49,7 +50,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, metadata_snapshot, uncompressed_cache, + ptr, Nested::collect(columns_to_read), metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 85ef787c6d7..dc213e99061 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -103,30 +103,29 @@ void MergeTreeDataPartWriterWide::write(const Block & block, for (size_t i = 0; i < columns_list.size(); ++i, ++it) { const ColumnWithTypeAndName & column = block.getByName(it->name); - auto name_and_type = NameAndTypePair(column.name, column.type); if (permutation) { if (primary_key_block.has(it->name)) { const auto & primary_column = *primary_key_block.getByName(it->name).column; - writeColumn(name_and_type, primary_column, offset_columns); + writeColumn(*it, primary_column, offset_columns); } else if (skip_indexes_block.has(it->name)) { const auto & index_column = *skip_indexes_block.getByName(it->name).column; - writeColumn(name_and_type, index_column, offset_columns); + writeColumn(*it, index_column, offset_columns); } else { /// We rearrange the columns that are not included in the primary key here; Then the result is released - to save RAM. ColumnPtr permuted_column = column.column->permute(*permutation, 0); - writeColumn(name_and_type, *permuted_column, offset_columns); + writeColumn(*it, *permuted_column, offset_columns); } } else { - writeColumn(name_and_type, *column.column, offset_columns); + writeColumn(*it, *column.column, offset_columns); } } } diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 9aaf894a0cb..b6706367bfa 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -93,7 +93,7 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr) { const auto & column = index_sample_block.getByPosition(i); const auto & type = column.type; - auto new_column = type->createColumn(); + ColumnPtr new_column = type->createColumn(); IDataType::DeserializeBinaryBulkSettings settings; settings.getter = [&](IDataType::SubstreamPath) -> ReadBuffer * { return &istr; }; @@ -101,9 +101,9 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr) IDataType::DeserializeBinaryBulkStatePtr state; type->deserializeBinaryBulkStatePrefix(settings, state); - type->deserializeBinaryBulkWithMultipleStreams(*new_column, rows_to_read, settings, state); + type->deserializeBinaryBulkWithMultipleStreams(new_column, rows_to_read, settings, state); - block.insert(ColumnWithTypeAndName(new_column->getPtr(), type, column.name)); + block.insert(ColumnWithTypeAndName(new_column, type, column.name)); } } diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 79607fcf169..949ed9ed6d3 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -54,8 +54,6 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( for (size_t i = 0; i < columns_num; ++i, ++name_and_type) { auto column_from_part = getColumnFromPart(*name_and_type); - if (duplicated_subcolumns.count(column_from_part.name)) - continue; auto position = data_part->getColumnPosition(column_from_part); if (!position && typeid_cast(column_from_part.type.get())) @@ -135,30 +133,29 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, if (!column_positions[i]) continue; - bool append = res_columns[i] != nullptr; - if (!append) + if (res_columns[i] == nullptr) res_columns[i] = getColumnFromPart(*column_it).type->createColumn(); - mutable_columns[i] = res_columns[i]->assumeMutable(); } while (read_rows < max_rows_to_read) { size_t rows_to_read = data_part->index_granularity.getMarkRows(from_mark); + std::unordered_map caches; auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { auto column_from_part = getColumnFromPart(*name_and_type); - if (!res_columns[pos]) continue; - auto & column = mutable_columns[pos]; try { + auto & column = res_columns[pos]; size_t column_size_before_reading = column->size(); - readData(column_from_part, *column, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); + readData(column_from_part, column, from_mark, *column_positions[pos], + rows_to_read, read_only_offsets[pos], caches[column_from_part.getStorageName()]); size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column < rows_to_read) @@ -185,23 +182,15 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, read_rows += rows_to_read; } - for (size_t i = 0; i < num_columns; ++i) - { - auto & column = mutable_columns[i]; - if (column && !column->empty()) - res_columns[i] = std::move(column); - else - res_columns[i] = nullptr; - } - next_mark = from_mark; return read_rows; } void MergeTreeReaderCompact::readData( - const NameAndTypePair & name_and_type, IColumn & column, - size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) + const NameAndTypePair & name_and_type, ColumnPtr & column, + size_t from_mark, size_t column_position, size_t rows_to_read, + bool only_offsets, IDataType::SubstreamsCache & cache) { const auto & [name, type] = name_and_type; @@ -210,6 +199,9 @@ void MergeTreeReaderCompact::readData( auto buffer_getter = [&](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { + // if (cache.count(IDataType::getSubcolumnNameForStream(substream_path))) + // return nullptr; + if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != IDataType::Substream::ArraySizes)) return nullptr; @@ -220,26 +212,24 @@ void MergeTreeReaderCompact::readData( IDataType::DeserializeBinaryBulkSettings deserialize_settings; deserialize_settings.getter = buffer_getter; deserialize_settings.avg_value_size_hint = avg_value_size_hints[name]; - deserialize_settings.position_independent_encoding = true; if (name_and_type.isSubcolumn()) { const auto & storage_type = name_and_type.getStorageType(); - auto temp_column = storage_type->createColumn(); + ColumnPtr temp_column = storage_type->createColumn(); storage_type->deserializeBinaryBulkStatePrefix(deserialize_settings, state); - storage_type->deserializeBinaryBulkWithMultipleStreams(*temp_column, rows_to_read, deserialize_settings, state); - - auto subcolumn = storage_type->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column); - column.insertRangeFrom(*subcolumn, 0, subcolumn->size()); + storage_type->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state); + column = storage_type->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column->assumeMutable()); } else { - deserialize_settings.position_independent_encoding = true; type->deserializeBinaryBulkStatePrefix(deserialize_settings, state); type->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state); } + UNUSED(cache); + /// The buffer is left in inconsistent state after reading single offsets if (only_offsets) last_read_granule.reset(); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index 27cabaa2a2f..a37ac304fe9 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -56,8 +56,8 @@ private: void seekToMark(size_t row_index, size_t column_index); - void readData(const NameAndTypePair & name_and_type, IColumn & column, - size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets = false); + void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark, + size_t column_position, size_t rows_to_read, bool only_offsets, IDataType::SubstreamsCache & cache); /// Returns maximal value of granule size in compressed file from @mark_ranges. /// This value is used as size of read buffer. diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 30fe6fa57d2..7cebb13172a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -72,6 +72,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si /// If append is true, then the value will be equal to nullptr and will be used only to /// check that the offsets column has been already read. OffsetColumns offset_columns; + std::unordered_map caches; auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) @@ -79,45 +80,18 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si auto column_from_part = getColumnFromPart(*name_and_type); const auto & [name, type] = column_from_part; - if (duplicated_subcolumns.count(name)) - continue; - /// The column is already present in the block so we will append the values to the end. bool append = res_columns[pos] != nullptr; if (!append) res_columns[pos] = type->createColumn(); - /// To keep offsets shared. TODO Very dangerous. Get rid of this. - MutableColumnPtr column = res_columns[pos]->assumeMutable(); - - bool read_offsets = true; - - /// For nested data structures collect pointers to offset columns. - if (const auto * type_arr = typeid_cast(type.get())) - { - String table_name = Nested::extractTableName(name); - - auto it_inserted = offset_columns.emplace(table_name, nullptr); - - /// offsets have already been read on the previous iteration and we don't need to read it again - if (!it_inserted.second) - read_offsets = false; - - /// need to create new offsets - if (it_inserted.second && !append) - it_inserted.first->second = ColumnArray::ColumnOffsets::create(); - - /// share offsets in all elements of nested structure - if (!append) - column = ColumnArray::create(type_arr->getNestedType()->createColumn(), - it_inserted.first->second)->assumeMutable(); - } - + auto & column = res_columns[pos]; try { size_t column_size_before_reading = column->size(); + auto & cache = caches[column_from_part.getStorageName()]; - readData(column_from_part, *column, from_mark, continue_reading, max_rows_to_read, read_offsets); + readData(column_from_part, column, from_mark, continue_reading, max_rows_to_read, cache); /// For elements of Nested, column_size_before_reading may be greater than column size /// if offsets are not empty and were already read, but elements are empty. @@ -133,8 +107,6 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si if (column->empty()) res_columns[pos] = nullptr; - else - res_columns[pos] = std::move(column); } /// NOTE: positions for all streams must be kept in sync. @@ -194,16 +166,16 @@ void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type, void MergeTreeReaderWide::readData( - const NameAndTypePair & name_and_type, IColumn & column, + const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark, bool continue_reading, size_t max_rows_to_read, - bool with_offsets) + IDataType::SubstreamsCache & cache) { auto get_stream_getter = [&](bool stream_for_prefix) -> IDataType::InputStreamGetter { return [&, stream_for_prefix](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { - /// If offsets for arrays have already been read. - if (!with_offsets && substream_path.size() == 1 && substream_path[0].type == IDataType::Substream::ArraySizes) + /// If offsets for arrays have already been read. TODO + if (cache.count(IDataType::getSubcolumnNameForStream(substream_path))) return nullptr; String stream_name = IDataType::getFileNameForStream(name_and_type, substream_path); @@ -239,8 +211,8 @@ void MergeTreeReaderWide::readData( deserialize_settings.getter = get_stream_getter(false); deserialize_settings.continuous_reading = continue_reading; auto & deserialize_state = deserialize_binary_bulk_state_map[name_and_type.name]; - name_and_type.type->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, deserialize_settings, deserialize_state); - IDataType::updateAvgValueSizeHint(column, avg_value_size_hint); + name_and_type.type->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, deserialize_settings, deserialize_state, &cache); + IDataType::updateAvgValueSizeHint(*column, avg_value_size_hint); } } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 8a384578685..bf9e97035d0 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -41,9 +41,9 @@ private: const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); void readData( - const NameAndTypePair & name_and_type, IColumn & column, + const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark, bool continue_reading, size_t max_rows_to_read, - bool with_offsets = true); + IDataType::SubstreamsCache & cache); }; } diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index ef46f3282d7..7d91b18fc20 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -272,7 +272,7 @@ Block StorageInMemoryMetadata::getSampleBlockForColumns( std::unordered_map columns_map; - NamesAndTypesList all_columns = getColumns().getAllWithSubcolumns(); + auto all_columns = getColumns().getAllWithSubcolumns(); for (const auto & elem : all_columns) columns_map.emplace(elem.name, elem.type); diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 1273b2e9edb..b67b5d2291a 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -44,7 +44,6 @@ namespace ErrorCodes extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; - extern const int NOT_IMPLEMENTED; } class LogSource final : public SourceWithProgress @@ -58,7 +57,8 @@ public: for (const auto & name_type : columns) res.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); - return Nested::flatten(res); + // return Nested::flatten(res); + return res; } LogSource( @@ -109,7 +109,7 @@ private: using DeserializeStates = std::map; DeserializeStates deserialize_states; - void readData(const NameAndTypePair & name_and_type, IColumn & column, size_t max_rows_to_read); + void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t max_rows_to_read, IDataType::SubstreamsCache & cache); }; @@ -209,27 +209,20 @@ Chunk LogSource::generate() /// How many rows to read for the next block. size_t max_rows_to_read = std::min(block_size, rows_limit - rows_read); + std::unordered_map caches; for (const auto & name_type : columns) { - MutableColumnPtr column; - if (name_type.isSubcolumn() && res.has(name_type.getStorageName())) + ColumnPtr column; + try { - auto column_in_block = res.getByName(name_type.getStorageName()).column; - column = name_type.getStorageType()->getSubcolumn(name_type.getSubcolumnName(), *column_in_block->assumeMutable()); + column = name_type.type->createColumn(); + readData(name_type, column, max_rows_to_read, caches[name_type.getStorageName()]); } - else + catch (Exception & e) { - try - { - column = name_type.type->createColumn(); - readData(name_type, *column, max_rows_to_read); - } - catch (Exception & e) - { - e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); - throw; - } + e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); + throw; } if (!column->empty()) @@ -248,13 +241,13 @@ Chunk LogSource::generate() streams.clear(); } - res = Nested::flatten(res); UInt64 num_rows = res.rows(); return Chunk(res.getColumns(), num_rows); } -void LogSource::readData(const NameAndTypePair & name_and_type, IColumn & column, size_t max_rows_to_read) +void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & column, + size_t max_rows_to_read, IDataType::SubstreamsCache & cache) { IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; @@ -263,8 +256,10 @@ void LogSource::readData(const NameAndTypePair & name_and_type, IColumn & column { return [&, stream_for_prefix] (const IDataType::SubstreamPath & path) -> ReadBuffer * { - String stream_name = IDataType::getFileNameForStream(name_and_type, path); + if (cache.count(IDataType::getSubcolumnNameForStream(path))) + return nullptr; + String stream_name = IDataType::getFileNameForStream(name_and_type, path); const auto & file_it = storage.files.find(stream_name); if (storage.files.end() == file_it) throw Exception("Logical error: no information about file " + stream_name + " in StorageLog", ErrorCodes::LOGICAL_ERROR); @@ -287,7 +282,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, IColumn & column } settings.getter = create_stream_getter(false); - type->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name]); + type->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, settings, deserialize_states[name], &cache); } @@ -633,12 +628,6 @@ Pipe StorageLog::read( loadMarks(); auto all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); - - /// TODO: implement DataType Nested and support them - for (const auto & column : all_columns) - if (column.isSubcolumn() && startsWith(column.getSubcolumnName(), "size")) - throw Exception("Subcolumns of arrays are not supported in StorageLog", ErrorCodes::NOT_IMPLEMENTED); - all_columns = Nested::collect(all_columns); std::shared_lock lock(rwlock); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 16f0ba2699e..e12ed3d274b 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -49,7 +49,6 @@ namespace ErrorCodes extern const int DUPLICATE_COLUMN; extern const int INCORRECT_FILE_NAME; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int NOT_IMPLEMENTED; } @@ -64,7 +63,7 @@ public: for (const auto & name_type : columns) res.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); - return Nested::flatten(res); + return res; } TinyLogSource(size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, size_t max_read_buffer_size_) @@ -104,7 +103,7 @@ private: using DeserializeStates = std::map; DeserializeStates deserialize_states; - void readData(const NameAndTypePair & name_and_type, IColumn & column, UInt64 limit); + void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, UInt64 limit, IDataType::SubstreamsCache & cache); }; @@ -195,26 +194,19 @@ Chunk TinyLogSource::generate() if (storage.disk->isDirectoryEmpty(storage.table_path)) return {}; + std::unordered_map caches; for (const auto & name_type : columns) { - MutableColumnPtr column; - if (name_type.isSubcolumn() && res.has(name_type.getStorageName())) + ColumnPtr column; + try { - auto column_in_block = res.getByName(name_type.getStorageName()).column; - column = name_type.getStorageType()->getSubcolumn(name_type.getSubcolumnName(), *column_in_block->assumeMutable()); + column = name_type.type->createColumn(); + readData(name_type, column, block_size, caches[name_type.getStorageName()]); } - else + catch (Exception & e) { - try - { - column = name_type.type->createColumn(); - readData(name_type, *column, block_size); - } - catch (Exception & e) - { - e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); - throw; - } + e.addMessage("while reading column " + name_type.name + " at " + fullPath(storage.disk, storage.table_path)); + throw; } if (!column->empty()) @@ -227,12 +219,13 @@ Chunk TinyLogSource::generate() streams.clear(); } - auto flatten = Nested::flatten(res); - return Chunk(flatten.getColumns(), flatten.rows()); + // auto flatten = Nested::flatten(res); + return Chunk(res.getColumns(), res.rows()); } -void TinyLogSource::readData(const NameAndTypePair & name_and_type, IColumn & column, UInt64 limit) +void TinyLogSource::readData(const NameAndTypePair & name_and_type, + ColumnPtr & column, UInt64 limit, IDataType::SubstreamsCache & cache) { IDataType::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; @@ -250,7 +243,7 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, IColumn & co if (deserialize_states.count(name) == 0) type->deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); - type->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]); + type->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name], &cache); } @@ -448,17 +441,10 @@ Pipe StorageTinyLog::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - auto all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); - - /// TODO: implement DataType Nested and support them - for (const auto & column : all_columns) - if (column.isSubcolumn() && startsWith(column.getSubcolumnName(), "size")) - throw Exception("Subcolumns of arrays are not supported in StorageLog", ErrorCodes::NOT_IMPLEMENTED); - // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. return Pipe(std::make_shared( - max_block_size, Nested::collect(all_columns), + max_block_size, metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names), *this, context.getSettingsRef().max_read_buffer_size)); } diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.reference b/tests/queries/0_stateless/01475_read_subcolumns_storages.reference index dc4f34b330c..f848977a55d 100644 --- a/tests/queries/0_stateless/01475_read_subcolumns_storages.reference +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.reference @@ -1,9 +1,9 @@ Log 100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) -100 0 ('foo',200) foo 200 +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 TinyLog 100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) -100 0 ('foo',200) foo 200 +100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 Memory 100 [1,2,3] [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] [1,NULL,2] ('foo',200) 100 0 [1,2,3] 3 [[[1,2],[],[4]],[[5,6],[7,8]],[[]]] 3 [3,2,1] [[2,0,1],[2,2],[0]] [1,NULL,2] 3 [0,1,0] ('foo',200) foo 200 diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh index b0cfc9b29a7..33f757dfe01 100755 --- a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh @@ -20,11 +20,5 @@ for engine in "${ENGINES[@]}"; do $CLICKHOUSE_CLIENT --query "$create_query ENGINE = $engine" $CLICKHOUSE_CLIENT --query "INSERT INTO subcolumns VALUES (100, [1, 2, 3], [[[1, 2], [], [4]], [[5, 6], [7, 8]], [[]]], [1, NULL, 2], ('foo', 200))" $CLICKHOUSE_CLIENT --query "SELECT * FROM subcolumns" - - # Some subcolumns are not supported in - if [[ $engine == "Log" || $engine = "TinyLog" ]]; then - $CLICKHOUSE_CLIENT --query "SELECT n, n.null, t, t.s, t.v FROM subcolumns" - else - $CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v FROM subcolumns" - fi + $CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v FROM subcolumns" done diff --git a/tests/queries/0_stateless/01533_multiple_nested.reference b/tests/queries/0_stateless/01533_multiple_nested.reference new file mode 100644 index 00000000000..ba37ce1c32c --- /dev/null +++ b/tests/queries/0_stateless/01533_multiple_nested.reference @@ -0,0 +1,40 @@ +all +[(1,'q'),(2,'w'),(3,'e')] [(4,[('a',5),('s',6),('d',7)])] [([(8,9),(10,11)],[('z','x'),('c','v')])] +[(12,'qq')] [(4,[]),(5,[('b',6),('n',7)])] [([],[]),([(44,55),(66,77)],[])] +col1 +[1,2,3] ['q','w','e'] +[12] ['qq'] +col2 +[4] [[('a',5),('s',6),('d',7)]] [['a','s','d']] [[5,6,7]] +[4,5] [[],[('b',6),('n',7)]] [[],['b','n']] [[],[6,7]] +col3 +[[(8,9),(10,11)]] [[('z','x'),('c','v')]] [[8,10]] [[9,11]] [['z','c']] [['x','v']] +[[],[(44,55),(66,77)]] [[],[]] [[],[44,66]] [[],[55,77]] [[],[]] [[],[]] +read files +4 +6 +0 899984 7199412 +1 899987 7199877 +2 899990 7200255 +3 899993 7199883 +4 899996 7199798 +5 899999 7200306 +6 900002 7200064 +7 900005 7199429 +8 900008 7200067 +9 899992 7199993 +0 [] +0 [0] +1 [0,2] +3 [0,2,8] +6 [0,2,8,18] +2 3 +0 2 +2 3 +4 4 +0 0 +0 1 +0 2 +0 3 +0 1 +1 2 diff --git a/tests/queries/0_stateless/01533_multiple_nested.sql b/tests/queries/0_stateless/01533_multiple_nested.sql new file mode 100644 index 00000000000..6374d6fca21 --- /dev/null +++ b/tests/queries/0_stateless/01533_multiple_nested.sql @@ -0,0 +1,66 @@ +DROP TABLE IF EXISTS nested; + +SET flatten_nested = 0; + +CREATE TABLE nested +( + col1 Nested(a UInt32, s String), + col2 Nested(a UInt32, n Nested(s String, b UInt32)), + col3 Nested(n1 Nested(a UInt32, b UInt32), n2 Nested(s String, t String)) +) +ENGINE = MergeTree +ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO nested VALUES ([(1, 'q'), (2, 'w'), (3, 'e')], [(4, [('a', 5), ('s', 6), ('d', 7)])], [([(8, 9), (10, 11)], [('z', 'x'), ('c', 'v')])]); +INSERT INTO nested VALUES ([(12, 'qq')], [(4, []), (5, [('b', 6), ('n', 7)])], [([], []), ([(44, 55), (66, 77)], [])]); + +OPTIMIZE TABLE nested FINAL; + +SELECT 'all'; +SELECT * FROM nested; +SELECT 'col1'; +SELECT col1.a, col1.s FROM nested; +SELECT 'col2'; +SELECT col2.a, col2.n, col2.n.s, col2.n.b FROM nested; +SELECT 'col3'; +SELECT col3.n1, col3.n2, col3.n1.a, col3.n1.b, col3.n2.s, col3.n2.t FROM nested; + +SELECT 'read files'; + +SYSTEM DROP MARK CACHE; +SELECT col1.a FROM nested FORMAT Null; + +-- 4 files: (col1.size0, col1.a) x2 +SYSTEM FLUSH LOGS; +SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] +FROM system.query_log +WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col1.a FROM %nested%')) + AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + +SYSTEM DROP MARK CACHE; +SELECT col3.n2.s FROM nested FORMAT Null; + +-- 6 files: (col3.size0, col3.n2.size1, col3.n2.s) x2 +SYSTEM FLUSH LOGS; +SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')] +FROM system.query_log +WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT col3.n2.s FROM %nested%')) + AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase(); + +DROP TABLE nested; + +CREATE TABLE nested +( + id UInt32, + col1 Nested(a UInt32, n Nested(s String, b UInt32)) +) +ENGINE = MergeTree +ORDER BY id +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO nested SELECT number, arrayMap(x -> (x, arrayMap(y -> (toString(y * x), y + x), range(number % 17))), range(number % 19)) FROM numbers(1000000); +SELECT id % 10, sum(length(col1)), sumArray(arrayMap(x -> length(x), col1.n.b)) FROM nested GROUP BY id % 10; + +SELECT arraySum(col1.a), arrayMap(x -> x * x * 2, col1.a) FROM nested ORDER BY id LIMIT 5; +SELECT untuple(arrayJoin(arrayJoin(col1.n))) FROM nested ORDER BY id LIMIT 10 OFFSET 10; From 52ace243b4957474667236d3c4ac000d996a87c5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 23 Nov 2020 19:26:39 +0300 Subject: [PATCH 015/264] fix array deserialization --- src/DataTypes/DataTypeArray.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 3434edd4747..a6c2e6dc063 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -283,25 +283,25 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreamsImpl( settings.path.back() = Substream::ArrayElements; ColumnArray::Offsets & offset_values = column_array.getOffsets(); - IColumn & nested_column = column_array.getData(); + ColumnPtr & nested_column = column_array.getDataPtr(); /// Number of values corresponding with `offset_values` must be read. size_t last_offset = offset_values.back(); - if (last_offset < nested_column.size()) + if (last_offset < nested_column->size()) throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR); - size_t nested_limit = last_offset - nested_column.size(); + size_t nested_limit = last_offset - nested_column->size(); /// Adjust value size hint. Divide it to the average array size. settings.avg_value_size_hint = nested_limit ? settings.avg_value_size_hint / nested_limit * offset_values.size() : 0; - nested->deserializeBinaryBulkWithMultipleStreams(column_array.getDataPtr(), nested_limit, settings, state, cache); + nested->deserializeBinaryBulkWithMultipleStreams(nested_column, nested_limit, settings, state, cache); settings.path.pop_back(); /// Check consistency between offsets and elements subcolumns. /// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER. - if (!nested_column.empty() && nested_column.size() != last_offset) - throw Exception("Cannot read all array values: read just " + toString(nested_column.size()) + " of " + toString(last_offset), + if (!nested_column->empty() && nested_column->size() != last_offset) + throw Exception("Cannot read all array values: read just " + toString(nested_column->size()) + " of " + toString(last_offset), ErrorCodes::CANNOT_READ_ALL_DATA); } From be493ed5614c5ebe6fc7973ac54db487aedfbb20 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 24 Nov 2020 17:14:47 +0300 Subject: [PATCH 016/264] fix nested and subcolumns --- src/Interpreters/TreeRewriter.cpp | 4 +--- src/Storages/ColumnsDescription.cpp | 37 ++++++++++++++++++----------- src/Storages/ColumnsDescription.h | 1 + src/Storages/StorageTinyLog.cpp | 7 +++++- 4 files changed, 31 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 1b86f3fcf03..d14137b66e1 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -395,9 +395,7 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) { const ColumnsDescription & columns = metadata_snapshot->getColumns(); - UNUSED(add_special); - // auto columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical(); - auto columns_from_storage = columns.getAllWithSubcolumns(); + auto columns_from_storage = add_special ? columns.getAllWithSubcolumns() : columns.getAllPhysicalWithSubcolumns(); if (source_columns.empty()) source_columns.swap(columns_from_storage); else diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 6fb8d789188..c187dbc8485 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -329,20 +329,6 @@ NamesAndTypesList ColumnsDescription::getAll() const return ret; } -NamesAndTypesList ColumnsDescription::getAllWithSubcolumns() const -{ - NamesAndTypesList ret; - for (const auto & col : columns) - { - ret.emplace_back(col.name, col.type); - for (const auto & subcolumn : col.type->getSubcolumnNames()) - ret.emplace_back(col.name, subcolumn, col.type, col.type->getSubcolumnType(subcolumn)); - } - - return ret; -} - - bool ColumnsDescription::has(const String & column_name) const { return columns.get<1>().find(column_name) != columns.get<1>().end() @@ -420,6 +406,29 @@ bool ColumnsDescription::hasPhysicalOrSubcolumn(const String & column_name) cons return hasPhysical(column_name) || subcolumns.find(column_name) != subcolumns.end(); } +static NamesAndTypesList getWithSubcolumns(NamesAndTypesList && source_list) +{ + NamesAndTypesList ret; + for (const auto & col : source_list) + { + ret.emplace_back(col.name, col.type); + for (const auto & subcolumn : col.type->getSubcolumnNames()) + ret.emplace_back(col.name, subcolumn, col.type, col.type->getSubcolumnType(subcolumn)); + } + + return ret; +} + +NamesAndTypesList ColumnsDescription::getAllWithSubcolumns() const +{ + return getWithSubcolumns(getAll()); +} + +NamesAndTypesList ColumnsDescription::getAllPhysicalWithSubcolumns() const +{ + return getWithSubcolumns(getAllPhysical()); +} + bool ColumnsDescription::hasDefaults() const { for (const auto & column : columns) diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index d79c7a09484..40ebdb7ec5b 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -78,6 +78,7 @@ public: NamesAndTypesList getAllPhysical() const; /// ordinary + materialized. NamesAndTypesList getAll() const; /// ordinary + materialized + aliases NamesAndTypesList getAllWithSubcolumns() const; + NamesAndTypesList getAllPhysicalWithSubcolumns() const; using ColumnTTLs = std::unordered_map; ColumnTTLs getColumnTTLs() const; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index e12ed3d274b..6a08f4f301f 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -231,6 +231,9 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, const auto & [name, type] = name_and_type; settings.getter = [&] (const IDataType::SubstreamPath & path) -> ReadBuffer * { + if (cache.count(IDataType::getSubcolumnNameForStream(path))) + return nullptr; + String stream_name = IDataType::getFileNameForStream(name_and_type, path); auto & stream = streams[stream_name]; @@ -441,10 +444,12 @@ Pipe StorageTinyLog::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); + auto all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); + // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. return Pipe(std::make_shared( - max_block_size, metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names), + max_block_size, Nested::collect(all_columns), *this, context.getSettingsRef().max_read_buffer_size)); } From 66e0add2bae1f316695275f1efb147fbd276f13d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 27 Nov 2020 14:00:33 +0300 Subject: [PATCH 017/264] fix nested --- src/Core/NamesAndTypes.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 +++++-- src/Storages/MergeTree/IMergeTreeReader.cpp | 26 +++++++++++++------ .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- 4 files changed, 28 insertions(+), 12 deletions(-) diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 4ee4aee923e..9c92e818c14 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes NameAndTypePair::NameAndTypePair( const String & storage_name_, const String & subcolumn_name_, const DataTypePtr & storage_type_, const DataTypePtr & subcolumn_type_) - : name(storage_name_ + "." + subcolumn_name_) + : name(storage_name_ + (subcolumn_name_.empty() ? "" : "." + subcolumn_name_)) , type(subcolumn_type_) , storage_type(storage_type_) , subcolumn_delimiter_position(storage_name_.size()) {} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 45ad75c3b1c..3c7576db4b1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { @@ -194,7 +195,7 @@ std::optional IMergeTreeDataPart::getColumnPosition(const String & colum std::optional IMergeTreeDataPart::getColumnPosition(const NameAndTypePair & column) const { - return getColumnPosition(column.getStorageName()); + return getColumnPosition(column.name); } DayNum IMergeTreeDataPart::getMinDate() const @@ -238,7 +239,12 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) column_name_to_position.reserve(new_columns.size()); size_t pos = 0; for (const auto & column : columns) - column_name_to_position.emplace(column.name, pos++); + { + column_name_to_position.emplace(column.name, pos); + for (const auto & subcolumn : column.type->getSubcolumnNames()) + column_name_to_position.emplace(Nested::concatenateName(column.name, subcolumn), pos); + ++pos; + } } IMergeTreeDataPart::~IMergeTreeDataPart() = default; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 2680ca96d5b..9a5ba335b38 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -196,19 +196,29 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & required_column) const { - if (alter_conversions.isColumnRenamed(required_column.name)) + auto name_in_storage = required_column.getStorageName(); + + decltype(columns_from_part.begin()) it; + if (alter_conversions.isColumnRenamed(name_in_storage)) { - String old_name = alter_conversions.getColumnOldName(required_column.name); - auto it = columns_from_part.find(old_name); - if (it != columns_from_part.end()) - return {it->first, it->second}; + String old_name = alter_conversions.getColumnOldName(name_in_storage); + it = columns_from_part.find(old_name); } - else if (auto it = columns_from_part.find(required_column.name); it != columns_from_part.end()) + else { - return {it->first, it->second}; + it = columns_from_part.find(name_in_storage); } - return required_column; + if (it == columns_from_part.end()) + return required_column; + + if (required_column.isSubcolumn()) + { + auto subcolumn_name = required_column.getSubcolumnName(); + return {it->first, subcolumn_name, it->second, it->second->getSubcolumnType(subcolumn_name)}; + } + + return {it->first, it->second}; } void IMergeTreeReader::performRequiredConversions(Columns & res_columns) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index f469f656deb..234f904ac80 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -48,7 +48,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, Nested::collect(columns_to_read), metadata_snapshot, uncompressed_cache, + ptr, columns_to_read, metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } From 06d5b87bc9fe90688bf819f7d4399947651abb7e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 7 Dec 2020 22:02:26 +0300 Subject: [PATCH 018/264] fix nested and subcolumns --- src/DataTypes/NestedUtils.cpp | 40 ++++++++++++++++--- src/DataTypes/NestedUtils.h | 3 ++ .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 2 +- src/Storages/StorageBuffer.cpp | 20 ++++++---- src/Storages/StorageLog.cpp | 3 +- src/Storages/StorageTinyLog.cpp | 3 +- 7 files changed, 54 insertions(+), 19 deletions(-) diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 1d080d1032a..6c13eea0a1b 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -86,7 +86,7 @@ Block flatten(const Block & block) for (const auto & elem : block) { const DataTypeArray * type_arr = typeid_cast(elem.type.get()); - if (!isNested(elem.type) && type_arr) + if (type_arr) { const DataTypeTuple * type_tuple = typeid_cast(type_arr->getNestedType().get()); if (type_tuple && type_tuple->haveExplicitNames()) @@ -130,12 +130,14 @@ Block flatten(const Block & block) return res; } - -NamesAndTypesList collect(const NamesAndTypesList & names_and_types) +namespace { - NamesAndTypesList res = names_and_types; - std::map nested; +using NameToDataType = std::map; + +NameToDataType getSubcolumnsOfNested(const NamesAndTypesList & names_and_types) +{ + std::unordered_map nested; for (const auto & name_type : names_and_types) { const DataTypeArray * type_arr = typeid_cast(name_type.type.get()); @@ -149,10 +151,36 @@ NamesAndTypesList collect(const NamesAndTypesList & names_and_types) } } - std::unordered_map nested_types; + std::map nested_types; + for (const auto & [name, elems] : nested) nested_types.emplace(name, createNested(elems.getTypes(), elems.getNames())); + return nested_types; +} + +} + +NamesAndTypesList collect(const NamesAndTypesList & names_and_types) +{ + NamesAndTypesList res; + auto nested_types = getSubcolumnsOfNested(names_and_types); + + for (const auto & name_type : names_and_types) + if (!nested_types.count(splitName(name_type.name).first)) + res.push_back(name_type); + + for (const auto & name_type : nested_types) + res.emplace_back(name_type.first, name_type.second); + + return res; +} + +NamesAndTypesList convertToSubcolumns(const NamesAndTypesList & names_and_types) +{ + auto nested_types = getSubcolumnsOfNested(names_and_types); + auto res = names_and_types; + for (auto & name_type : res) { auto split = splitName(name_type.name); diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index 3039fd7f118..b8428b96d3e 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -23,6 +23,9 @@ namespace Nested /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. NamesAndTypesList collect(const NamesAndTypesList & names_and_types); + /// Convert old-style nested (single arrays with same prefix, `n.a`, `n.b`...) to subcolumns of data type Nested. + NamesAndTypesList convertToSubcolumns(const NamesAndTypesList & names_and_types); + /// Check that sizes of arrays - elements of nested data structures - are equal. void validateArraySizes(const Block & block); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index eb191219acc..96fa411339c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -51,7 +51,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartInMemory::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, Nested::collect(columns_to_read), metadata_snapshot, mark_ranges, reader_settings); + ptr, columns_to_read, metadata_snapshot, mark_ranges, reader_settings); } IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index aea53c36bde..a38df9dc89c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -50,7 +50,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, Nested::collect(columns_to_read), metadata_snapshot, uncompressed_cache, + ptr, Nested::convertToSubcolumns(columns_to_read), metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 549caf427ea..752cef78d71 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -95,7 +95,7 @@ public: BufferSource(const Names & column_names_, StorageBuffer::Buffer & buffer_, const StorageBuffer & storage, const StorageMetadataPtr & metadata_snapshot) : SourceWithProgress( metadata_snapshot->getSampleBlockForColumns(column_names_, storage.getVirtuals(), storage.getStorageID())) - , column_names(column_names_.begin(), column_names_.end()) + , column_names_and_types(metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names_)) , buffer(buffer_) {} String getName() const override { return "Buffer"; } @@ -115,10 +115,16 @@ protected: return res; Columns columns; - columns.reserve(column_names.size()); + columns.reserve(column_names_and_types.size()); - for (const auto & name : column_names) - columns.push_back(buffer.data.getByName(name).column); + for (const auto & elem : column_names_and_types) + { + const auto & current_column = buffer.data.getByName(elem.getStorageName()).column; + if (elem.isSubcolumn()) + columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column->assumeMutable())); + else + columns.emplace_back(std::move(current_column)); + } UInt64 size = columns.at(0)->size(); res.setColumns(std::move(columns), size); @@ -127,7 +133,7 @@ protected: } private: - Names column_names; + NamesAndTypesList column_names_and_types; StorageBuffer::Buffer & buffer; bool has_been_read = false; }; @@ -188,8 +194,8 @@ void StorageBuffer::read( { const auto & dest_columns = destination_metadata_snapshot->getColumns(); const auto & our_columns = metadata_snapshot->getColumns(); - return dest_columns.hasPhysical(column_name) && - dest_columns.get(column_name).type->equals(*our_columns.get(column_name).type); + return dest_columns.hasPhysicalOrSubcolumn(column_name) && + dest_columns.getPhysicalOrSubcolumn(column_name).type->equals(*our_columns.getPhysicalOrSubcolumn(column_name).type); }); if (dst_has_same_structure) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index b67b5d2291a..6b2845702aa 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -57,7 +57,6 @@ public: for (const auto & name_type : columns) res.insert({ name_type.type->createColumn(), name_type.type, name_type.name }); - // return Nested::flatten(res); return res; } @@ -628,7 +627,7 @@ Pipe StorageLog::read( loadMarks(); auto all_columns = metadata_snapshot->getColumns().getAllWithSubcolumns().addTypes(column_names); - all_columns = Nested::collect(all_columns); + all_columns = Nested::convertToSubcolumns(all_columns); std::shared_lock lock(rwlock); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 6a08f4f301f..38705bdf0ec 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -219,7 +219,6 @@ Chunk TinyLogSource::generate() streams.clear(); } - // auto flatten = Nested::flatten(res); return Chunk(res.getColumns(), res.rows()); } @@ -449,7 +448,7 @@ Pipe StorageTinyLog::read( // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. return Pipe(std::make_shared( - max_block_size, Nested::collect(all_columns), + max_block_size, Nested::convertToSubcolumns(all_columns), *this, context.getSettingsRef().max_read_buffer_size)); } From 6de8b05b7c06ab1f114fe7d4a5aa5476c72d9cc0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 17 Dec 2020 20:10:33 +0300 Subject: [PATCH 019/264] improvements in subcolumns reading --- src/DataTypes/DataTypeArray.cpp | 60 ++++++++++++------- src/DataTypes/DataTypeArray.h | 4 +- src/DataTypes/DataTypeNullable.cpp | 4 +- src/DataTypes/DataTypeNullable.h | 2 +- src/DataTypes/DataTypeOneElementTuple.cpp | 4 +- src/DataTypes/DataTypeTuple.cpp | 4 +- src/DataTypes/DataTypeTuple.h | 2 +- src/DataTypes/IDataType.cpp | 12 ++-- src/DataTypes/IDataType.h | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 2 +- .../MergeTree/MergeTreeReaderInMemory.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageMemory.cpp | 2 +- 13 files changed, 59 insertions(+), 43 deletions(-) diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index a6c2e6dc063..3dfe9f526da 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -149,21 +149,42 @@ namespace offset_values.resize(i); } - MutableColumnPtr getArraySizesPositionIndependent(const ColumnArray & column_array) + ColumnPtr arrayOffsetsToSizes(const IColumn & column) { - const auto & offset_values = column_array.getOffsets(); - MutableColumnPtr new_offsets = column_array.getOffsetsColumn().cloneEmpty(); + const auto & column_offsets = assert_cast(column); + MutableColumnPtr column_sizes = column_offsets.cloneEmpty(); - if (offset_values.empty()) - return new_offsets; + if (column_offsets.empty()) + return column_sizes; - auto & new_offsets_values = assert_cast &>(*new_offsets).getData(); - new_offsets_values.reserve(offset_values.size()); - new_offsets_values.push_back(offset_values[0]); - for (size_t i = 1; i < offset_values.size(); ++i) - new_offsets_values.push_back(offset_values[i] - offset_values[i - 1]); + const auto & offsets_data = column_offsets.getData(); + auto & sizes_data = assert_cast(*column_sizes).getData(); - return new_offsets; + sizes_data.resize(offsets_data.size()); + sizes_data[0] = offsets_data[0]; + for (size_t i = 1; i < offsets_data.size(); ++i) + sizes_data[i] = offsets_data[i] - offsets_data[i - 1]; + + return column_sizes; + } + + ColumnPtr arraySizesToOffsets(const IColumn & column) + { + const auto & column_sizes = assert_cast(column); + MutableColumnPtr column_offsets = column_sizes.cloneEmpty(); + + if (column_sizes.empty()) + return column_offsets; + + const auto & sizes_data = column_sizes.getData(); + auto & offsets_data = assert_cast(*column_offsets).getData(); + + offsets_data.resize(sizes_data.size()); + offsets_data[0] = sizes_data[0]; + for (size_t i = 0; i < sizes_data.size(); ++i) + offsets_data[i] = offsets_data[i - 1] + sizes_data[i]; + + return column_offsets; } } @@ -263,12 +284,11 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreamsImpl( SubstreamsCache * cache) const { ColumnArray & column_array = typeid_cast(column); - settings.path.push_back(Substream::ArraySizes); if (auto cached_column = getFromSubstreamsCache(cache, settings.path)) { - column_array.getOffsetsPtr() = cached_column; + column_array.getOffsetsPtr() = arraySizesToOffsets(*cached_column); } else if (auto * stream = settings.getter(settings.path)) { @@ -277,7 +297,7 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreamsImpl( else DataTypeNumber().deserializeBinaryBulk(column_array.getOffsetsColumn(), *stream, limit, 0); - addToSubstreamsCache(cache, settings.path, column_array.getOffsetsPtr()); + addToSubstreamsCache(cache, settings.path, arrayOffsetsToSizes(column_array.getOffsetsColumn())); } settings.path.back() = Substream::ArrayElements; @@ -547,24 +567,24 @@ DataTypePtr DataTypeArray::tryGetSubcolumnTypeImpl(const String & subcolumn_name return (subcolumn ? std::make_shared(std::move(subcolumn)) : subcolumn); } -MutableColumnPtr DataTypeArray::getSubcolumn(const String & subcolumn_name, IColumn & column) const +ColumnPtr DataTypeArray::getSubcolumn(const String & subcolumn_name, const IColumn & column) const { return getSubcolumnImpl(subcolumn_name, column, 0); } -MutableColumnPtr DataTypeArray::getSubcolumnImpl(const String & subcolumn_name, IColumn & column, size_t level) const +ColumnPtr DataTypeArray::getSubcolumnImpl(const String & subcolumn_name, const IColumn & column, size_t level) const { - auto & column_array = assert_cast(column); + const auto & column_array = assert_cast(column); if (subcolumn_name == "size" + std::to_string(level)) - return getArraySizesPositionIndependent(column_array); + return arrayOffsetsToSizes(column_array.getOffsetsColumn()); - MutableColumnPtr subcolumn; + ColumnPtr subcolumn; if (const auto * nested_array = typeid_cast(nested.get())) subcolumn = nested_array->getSubcolumnImpl(subcolumn_name, column_array.getData(), level + 1); else subcolumn = nested->getSubcolumn(subcolumn_name, column_array.getData()); - return ColumnArray::create(std::move(subcolumn), column_array.getOffsetsPtr()->assumeMutable()); + return ColumnArray::create(subcolumn, column_array.getOffsetsPtr()); } size_t DataTypeArray::getNumberOfDimensions() const diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index ae55fa252ea..ba19ad021be 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -113,7 +113,7 @@ public: } DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; - MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; + ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; const DataTypePtr & getNestedType() const { return nested; } @@ -121,7 +121,7 @@ public: size_t getNumberOfDimensions() const; private: - MutableColumnPtr getSubcolumnImpl(const String & subcolumn_name, IColumn & column, size_t level) const; + ColumnPtr getSubcolumnImpl(const String & subcolumn_name, const IColumn & column, size_t level) const; DataTypePtr tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const; }; diff --git a/src/DataTypes/DataTypeNullable.cpp b/src/DataTypes/DataTypeNullable.cpp index d26c5af8378..9693133dce3 100644 --- a/src/DataTypes/DataTypeNullable.cpp +++ b/src/DataTypes/DataTypeNullable.cpp @@ -542,9 +542,9 @@ DataTypePtr DataTypeNullable::tryGetSubcolumnType(const String & subcolumn_name) return nested_data_type->tryGetSubcolumnType(subcolumn_name); } -MutableColumnPtr DataTypeNullable::getSubcolumn(const String & subcolumn_name, IColumn & column) const +ColumnPtr DataTypeNullable::getSubcolumn(const String & subcolumn_name, const IColumn & column) const { - auto & column_nullable = assert_cast(column); + const auto & column_nullable = assert_cast(column); if (subcolumn_name == "null") return column_nullable.getNullMapColumnPtr()->assumeMutable(); diff --git a/src/DataTypes/DataTypeNullable.h b/src/DataTypes/DataTypeNullable.h index f5e8f74ab8c..db641faf0af 100644 --- a/src/DataTypes/DataTypeNullable.h +++ b/src/DataTypes/DataTypeNullable.h @@ -99,7 +99,7 @@ public: bool onlyNull() const override; bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); } DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; - MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; + ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; const DataTypePtr & getNestedType() const { return nested_data_type; } diff --git a/src/DataTypes/DataTypeOneElementTuple.cpp b/src/DataTypes/DataTypeOneElementTuple.cpp index 679b34773e9..bb3cf37569b 100644 --- a/src/DataTypes/DataTypeOneElementTuple.cpp +++ b/src/DataTypes/DataTypeOneElementTuple.cpp @@ -22,7 +22,7 @@ private: bool escape_delimiter; public: - DataTypeOneElementTupleStreams(const DataTypePtr & nested_, const String & name_, bool escape_delimiter_ = true) + DataTypeOneElementTupleStreams(const DataTypePtr & nested_, const String & name_, bool escape_delimiter_) : nested(nested_), name(name_), escape_delimiter(escape_delimiter_) {} void enumerateStreams( @@ -99,7 +99,7 @@ private: DataTypePtr createOneElementTuple(const DataTypePtr & type, const String & name, bool escape_delimiter) { auto custom_desc = std::make_unique( - std::make_unique(type->getName()), nullptr, + std::make_unique(type->getName()),nullptr, std::make_unique(type, name, escape_delimiter)); return DataTypeFactory::instance().getCustom(std::move(custom_desc)); diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 15b5a593a73..c62aa1c1187 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -635,14 +635,14 @@ DataTypePtr DataTypeTuple::tryGetSubcolumnType(const String & subcolumn_name) co return nullptr; } -MutableColumnPtr DataTypeTuple::getSubcolumn(const String & subcolumn_name, IColumn & column) const +ColumnPtr DataTypeTuple::getSubcolumn(const String & subcolumn_name, const IColumn & column) const { for (size_t i = 0; i < names.size(); ++i) { if (startsWith(subcolumn_name, names[i])) { size_t name_length = names[i].size(); - auto & subcolumn = extractElementColumn(column, i); + const auto & subcolumn = extractElementColumn(column, i); if (subcolumn_name.size() == name_length) return subcolumn.assumeMutable(); diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 476b3316ed2..0b28ebe5a63 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -100,7 +100,7 @@ public: size_t getSizeOfValueInMemory() const override; DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; - MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const override; + ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; const DataTypes & getElements() const { return elems; } const Strings & getElementNames() const { return names; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index f84cc844853..90f4b4de408 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -156,7 +156,7 @@ DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } -MutableColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, IColumn &) const +ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const IColumn &) const { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } @@ -173,11 +173,7 @@ Names IDataType::getSubcolumnNames() const new_path.push_back(elem); auto subcolumn_name = getSubcolumnNameForStream(new_path); if (!subcolumn_name.empty() && tryGetSubcolumnType(subcolumn_name)) - { - /// Not all of substreams have its subcolumn. - if (tryGetSubcolumnType(subcolumn_name)) - res.insert(subcolumn_name); - } + res.insert(subcolumn_name); } }); @@ -329,7 +325,7 @@ void IDataType::deserializeBinaryBulkWithMultipleStreams( } /// Do not cache complex type, because they can be constructed - /// their subcolumns, which are in cache. + /// from their subcolumns, which are in cache. if (!haveSubtypes()) { auto cached_column = getFromSubstreamsCache(cache, settings.path); @@ -340,7 +336,7 @@ void IDataType::deserializeBinaryBulkWithMultipleStreams( } } - auto mutable_column = IColumn::mutate(std::move(column)); + auto mutable_column = column->assumeMutable(); deserializeBinaryBulkWithMultipleStreamsImpl(*mutable_column, limit, settings, state, cache); column = std::move(mutable_column); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index fd3b5b3f9a9..b51caaddcb0 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -126,7 +126,7 @@ public: virtual DataTypePtr tryGetSubcolumnType(const String & /* subcolumn_name */) const { return nullptr; } DataTypePtr getSubcolumnType(const String & subcolumn_name) const; - virtual MutableColumnPtr getSubcolumn(const String & subcolumn_name, IColumn & column) const; + virtual ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const; Names getSubcolumnNames() const; using OutputStreamGetter = std::function; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 949ed9ed6d3..2fccff4f443 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -220,7 +220,7 @@ void MergeTreeReaderCompact::readData( storage_type->deserializeBinaryBulkStatePrefix(deserialize_settings, state); storage_type->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state); - column = storage_type->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column->assumeMutable()); + column = storage_type->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column); } else { diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index f5dd820146d..114fa118eee 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -47,7 +47,7 @@ static ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & const auto & column = block.getByName(storage_name).column; if (name_and_type.isSubcolumn()) - return name_and_type.getStorageType()->getSubcolumn(name_and_type.getSubcolumnName(), *column->assumeMutable()); + return name_and_type.getStorageType()->getSubcolumn(name_and_type.getSubcolumnName(), *column); return column; } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 752cef78d71..6a91dcfe084 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -121,7 +121,7 @@ protected: { const auto & current_column = buffer.data.getByName(elem.getStorageName()).column; if (elem.isSubcolumn()) - columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column->assumeMutable())); + columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column)); else columns.emplace_back(std::move(current_column)); } diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index ff11a5bfc9e..60bb0c2b06c 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -69,7 +69,7 @@ protected: { auto current_column = src.getByName(elem.getStorageName()).column; if (elem.isSubcolumn()) - columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column->assumeMutable())); + columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column)); else columns.emplace_back(std::move(current_column)); } From 31e6956ac899e8bbeb3b75ef3ceaa99f1113f106 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 18 Dec 2020 15:27:15 +0300 Subject: [PATCH 020/264] fix alters of nested --- src/DataTypes/IDataType.cpp | 13 ------------- src/DataTypes/IDataType.h | 1 - src/Storages/MergeTree/IMergeTreeReader.cpp | 9 ++++++++- src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 5 ++++- src/Storages/MergeTree/MergeTreeIOSettings.h | 1 + 5 files changed, 13 insertions(+), 16 deletions(-) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 90f4b4de408..3bad80ee131 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -48,19 +48,6 @@ String IDataType::Substream::toString() const __builtin_unreachable(); } -size_t IDataType::SubstreamPath::getHash() const -{ - SipHash hash; - for (const auto & elem : *this) - { - hash.update(elem.type); - hash.update(elem.tuple_element_name); - hash.update(elem.escape_tuple_delimiter); - } - - return hash.get64(); -} - String IDataType::SubstreamPath::toString() const { WriteBufferFromOwnString wb; diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 98c5d927b10..e741747fc92 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -112,7 +112,6 @@ public: struct SubstreamPath : public std::vector { - UInt64 getHash() const; String toString() const; }; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 9a5ba335b38..fcc45859764 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -42,7 +42,14 @@ IMergeTreeReader::IMergeTreeReader( , all_mark_ranges(all_mark_ranges_) , alter_conversions(storage.getAlterConversionsForPart(data_part)) { - for (const NameAndTypePair & column_from_part : data_part->getColumns()) + auto part_columns = data_part->getColumns(); + if (settings.convert_nested_to_subcolumns) + { + columns = Nested::convertToSubcolumns(columns); + part_columns = Nested::collect(part_columns); + } + + for (const NameAndTypePair & column_from_part : part_columns) columns_from_part[column_from_part.name] = column_from_part.type; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index a38df9dc89c..202451c6801 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -48,9 +48,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( const ValueSizeMap & avg_value_size_hints, const ReadBufferFromFileBase::ProfileCallback & profile_callback) const { + auto new_settings = reader_settings; + new_settings.convert_nested_to_subcolumns = true; + auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, Nested::convertToSubcolumns(columns_to_read), metadata_snapshot, uncompressed_cache, + ptr, columns_to_read, metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index 41d9be07c70..570fc46467e 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -14,6 +14,7 @@ struct MergeTreeReaderSettings /// If save_marks_in_cache is false, then, if marks are not in cache, /// we will load them but won't save in the cache, to avoid evicting other data. bool save_marks_in_cache = false; + bool convert_nested_to_subcolumns = false; }; struct MergeTreeWriterSettings From 25ddd78976f398945f1e7e5639134cd1c5c794d0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 18 Dec 2020 23:09:34 +0300 Subject: [PATCH 021/264] fix nested --- src/Storages/MergeTree/IMergeTreeReader.cpp | 6 +++++- src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index fcc45859764..b563073ff84 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -222,7 +222,11 @@ NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & requ if (required_column.isSubcolumn()) { auto subcolumn_name = required_column.getSubcolumnName(); - return {it->first, subcolumn_name, it->second, it->second->getSubcolumnType(subcolumn_name)}; + auto subcolumn_type = it->second->tryGetSubcolumnType(subcolumn_name); + if (!subcolumn_type) + subcolumn_type = required_column.type; + + return {it->first, subcolumn_name, it->second, subcolumn_type}; } return {it->first, it->second}; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 202451c6801..5dd8c26f224 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -54,7 +54,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( ptr, columns_to_read, metadata_snapshot, uncompressed_cache, - mark_cache, mark_ranges, reader_settings, + mark_cache, mark_ranges, new_settings, avg_value_size_hints, profile_callback); } From 5a6a6991f11a1e9f96f65b4d1f80dc28c7e06f4c Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 20 Dec 2020 07:32:44 +0000 Subject: [PATCH 022/264] support insert into cluster function --- src/TableFunctions/TableFunctionRemote.cpp | 13 ++++++++++--- src/TableFunctions/TableFunctionRemote.h | 1 + ...602_insert_into_table_function_cluster.reference | 0 .../01602_insert_into_table_function_cluster.sql | 7 +++++++ 4 files changed, 18 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference create mode 100644 tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index a031490b88b..96c3d3815eb 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -37,7 +37,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont ASTs & args = args_func.at(0)->children; - const size_t max_args = is_cluster_function ? 3 : 5; + const size_t max_args = is_cluster_function ? 4 : 5; if (args.size() < 2 || args.size() > max_args) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -110,6 +110,13 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont } } + /// Cluster function may have sharding key for insert + if (is_cluster_function && arg_num < args.size()) + { + sharding_key = args[arg_num]; + ++arg_num; + } + /// Username and password parameters are prohibited in cluster version of the function if (!is_cluster_function) { @@ -208,7 +215,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, con remote_table_function_ptr, String{}, context, - ASTPtr{}, + sharding_key, String{}, String{}, false, @@ -221,7 +228,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & /*ast_function*/, con remote_table_id.table_name, String{}, context, - ASTPtr{}, + sharding_key, String{}, String{}, false, diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index 6ec591a34ac..c859a3a4118 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -40,6 +40,7 @@ private: ClusterPtr cluster; StorageID remote_table_id = StorageID::createEmpty(); ASTPtr remote_table_function_ptr; + ASTPtr sharding_key{}; }; } diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql new file mode 100644 index 00000000000..09dbb526473 --- /dev/null +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS default.x; + +CREATE TABLE default.x ON CLUSTER test_cluster_two_shards_localhost AS system.numbers ENGINE = Log; + +INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', default, x, rand()) SELECT * FROM numbers(10); + +DROP TABLE default.x ON CLUSTER test_cluster_two_shards_localhost; From 61a520debd8bc97ab3ef744635ecf715dd04f121 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 20 Dec 2020 07:48:55 +0000 Subject: [PATCH 023/264] update document --- docs/en/sql-reference/table-functions/cluster.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/en/sql-reference/table-functions/cluster.md b/docs/en/sql-reference/table-functions/cluster.md index 84f534c02dd..ebe76298963 100644 --- a/docs/en/sql-reference/table-functions/cluster.md +++ b/docs/en/sql-reference/table-functions/cluster.md @@ -16,12 +16,18 @@ Signatures: ``` sql cluster('cluster_name', db.table) cluster('cluster_name', db, table) +cluster('cluster_name', db.table, sharding_key) +cluster('cluster_name', db, table, sharding_key) clusterAllReplicas('cluster_name', db.table) clusterAllReplicas('cluster_name', db, table) +clusterAllReplicas('cluster_name', db.table, sharding_key) +clusterAllReplicas('cluster_name', db, table, sharding_key) ``` `cluster_name` – Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. +`sharding_key` - When insert into cluster function with more than one shard, sharding_key need to be provided. + 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. The `cluster` and `clusterAllReplicas` table functions can be useful in the following cases: From ec7202939c667fba476036d7eb6363e62b0e0d0e Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 20 Dec 2020 09:12:29 +0000 Subject: [PATCH 024/264] fix test --- ...sert_into_table_function_cluster.reference | 20 +++++++++++++++++++ ...602_insert_into_table_function_cluster.sql | 10 +++++++--- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference index e69de29bb2d..eda4c04d605 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference @@ -0,0 +1,20 @@ +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/01602_insert_into_table_function_cluster.sql b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql index 09dbb526473..18c70c98433 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql @@ -1,7 +1,11 @@ DROP TABLE IF EXISTS default.x; -CREATE TABLE default.x ON CLUSTER test_cluster_two_shards_localhost AS system.numbers ENGINE = Log; +CREATE TABLE default.x ON CLUSTER test_shard_localhost AS system.numbers ENGINE = Log; -INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', default, x, rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION cluster('test_shard_localhost', default, x) SELECT * FROM numbers(10); +-- In fact, in this case(just one shard), sharding key is not required +INSERT INTO FUNCTION cluster('test_shard_localhost', default, x, rand()) SELECT * FROM numbers(10); -DROP TABLE default.x ON CLUSTER test_cluster_two_shards_localhost; +SELECT * FROM default.x ORDER BY number; + +DROP TABLE default.x ON CLUSTER test_shard_localhost; From 8ec695f024b703201d84e10f96a3e4a1425297bc Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 20 Dec 2020 14:59:21 +0000 Subject: [PATCH 025/264] add sharding_key for remote table function update docs fix --- .../sql-reference/table-functions/cluster.md | 12 ++---- .../sql-reference/table-functions/remote.md | 9 +++-- src/TableFunctions/TableFunctionRemote.cpp | 40 ++++++++++++++----- ...sert_into_table_function_cluster.reference | 20 ++++++++++ ...602_insert_into_table_function_cluster.sql | 8 +++- 5 files changed, 66 insertions(+), 23 deletions(-) diff --git a/docs/en/sql-reference/table-functions/cluster.md b/docs/en/sql-reference/table-functions/cluster.md index ebe76298963..b85542d784f 100644 --- a/docs/en/sql-reference/table-functions/cluster.md +++ b/docs/en/sql-reference/table-functions/cluster.md @@ -14,14 +14,10 @@ Allows to access all shards in an existing cluster which configured in `remote_s Signatures: ``` sql -cluster('cluster_name', db.table) -cluster('cluster_name', db, table) -cluster('cluster_name', db.table, sharding_key) -cluster('cluster_name', db, table, sharding_key) -clusterAllReplicas('cluster_name', db.table) -clusterAllReplicas('cluster_name', db, table) -clusterAllReplicas('cluster_name', db.table, sharding_key) -clusterAllReplicas('cluster_name', db, table, sharding_key) +cluster('cluster_name', db.table[, sharding_key]) +cluster('cluster_name', db, table[, sharding_key]) +clusterAllReplicas('cluster_name', db.table[, sharding_key]) +clusterAllReplicas('cluster_name', db, table[, sharding_key]) ``` `cluster_name` – Name of a cluster that is used to build a set of addresses and connection parameters to remote and local servers. diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index 6da28362e23..bf72aea69a1 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -10,13 +10,14 @@ Allows you to access remote servers without creating a `Distributed` table. Signatures: ``` sql -remote('addresses_expr', db, table[, 'user'[, 'password']]) -remote('addresses_expr', db.table[, 'user'[, 'password']]) -remoteSecure('addresses_expr', db, table[, 'user'[, 'password']]) -remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) +remote('addresses_expr', db, table[, 'user'[, 'password'], sharding_key]) +remote('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) +remoteSecure('addresses_expr', db, table[, 'user'[, 'password'], sharding_key]) +remoteSecure('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) ``` `addresses_expr` – An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port`, or just `host`. The host can be specified as the server name, or as the IPv4 or IPv6 address. An IPv6 address is specified in square brackets. The port is the TCP port on the remote server. If the port is omitted, it uses `tcp_port` from the server’s config file (by default, 9000). +`sharding_key` - We can specify sharding key to support support distributing data across nodes. For example: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. !!! important "Important" The port is required for an IPv6 address. diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 96c3d3815eb..6f368da132e 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -37,7 +37,7 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont ASTs & args = args_func.at(0)->children; - const size_t max_args = is_cluster_function ? 4 : 5; + const size_t max_args = is_cluster_function ? 4 : 6; if (args.size() < 2 || args.size() > max_args) throw Exception(help_message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -122,15 +122,34 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont { if (arg_num < args.size()) { - username = get_string_literal(*args[arg_num], "Username"); + try + { + username = get_string_literal(*args[arg_num], "Username"); + } + catch (const Exception & _ [[maybe_unused]]) + { + username = "default"; + sharding_key = args[arg_num]; + } ++arg_num; } - else - username = "default"; - if (arg_num < args.size()) + if (arg_num < args.size() && !sharding_key) { - password = get_string_literal(*args[arg_num], "Password"); + try + { + password = get_string_literal(*args[arg_num], "Password"); + } + catch (const Exception & _ [[maybe_unused]]) + { + sharding_key = args[arg_num]; + } + ++arg_num; + } + + if (arg_num < args.size() && !sharding_key) + { + sharding_key = args[arg_num]; ++arg_num; } } @@ -248,9 +267,12 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_ : name{name_}, secure{secure_} { is_cluster_function = (name == "cluster" || name == "clusterAllReplicas"); - help_message = fmt::format("Table function '{}' requires from 2 to {} parameters: " - ", , {}", - name, is_cluster_function ? 3 : 5, is_cluster_function ? "" : ", [username, [password]]."); + help_message = fmt::format( + "Table function '{}' requires from 2 to {} parameters: " + ", , {}", + name, + is_cluster_function ? 4 : 6, + is_cluster_function ? ", [sharding_key]" : ", [username, [password], sharding_key]"); } diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference index eda4c04d605..abec951f640 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference @@ -1,20 +1,40 @@ 0 0 +0 +0 1 1 +1 +1 +2 +2 2 2 3 3 +3 +3 +4 +4 4 4 5 5 +5 +5 +6 +6 6 6 7 7 +7 +7 +8 +8 8 8 9 9 +9 +9 diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql index 18c70c98433..bb61c568033 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql @@ -1,11 +1,15 @@ DROP TABLE IF EXISTS default.x; -CREATE TABLE default.x ON CLUSTER test_shard_localhost AS system.numbers ENGINE = Log; +CREATE TABLE default.x AS system.numbers ENGINE = Log; INSERT INTO FUNCTION cluster('test_shard_localhost', default, x) SELECT * FROM numbers(10); -- In fact, in this case(just one shard), sharding key is not required INSERT INTO FUNCTION cluster('test_shard_localhost', default, x, rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION remote('localhost:59000', default, x, rand()) SELECT * FROM numbers(10); + +INSERT INTO FUNCTION remote('localhost:59000', default, x, 'default', rand()) SELECT * FROM numbers(10); + SELECT * FROM default.x ORDER BY number; -DROP TABLE default.x ON CLUSTER test_shard_localhost; +DROP TABLE default.x; From 8f17b3cb88e9b82a0b4d48ef9ac853bc114fa4f8 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 20 Dec 2020 15:49:37 +0000 Subject: [PATCH 026/264] fix --- docs/en/sql-reference/table-functions/remote.md | 2 +- src/TableFunctions/TableFunctionRemote.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index bf72aea69a1..71b1006fc5d 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -17,7 +17,7 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) ``` `addresses_expr` – An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port`, or just `host`. The host can be specified as the server name, or as the IPv4 or IPv6 address. An IPv6 address is specified in square brackets. The port is the TCP port on the remote server. If the port is omitted, it uses `tcp_port` from the server’s config file (by default, 9000). -`sharding_key` - We can specify sharding key to support support distributing data across nodes. For example: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. +`sharding_key` - We can specify sharding key to support distributing data across nodes. For example: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. !!! important "Important" The port is required for an IPv6 address. diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 6f368da132e..9760e2f2e83 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -272,7 +272,7 @@ TableFunctionRemote::TableFunctionRemote(const std::string & name_, bool secure_ ", , {}", name, is_cluster_function ? 4 : 6, - is_cluster_function ? ", [sharding_key]" : ", [username, [password], sharding_key]"); + is_cluster_function ? " [, sharding_key]" : " [, username[, password], sharding_key]"); } From b7d434a0eafaa33200c6ba07a6ed6a0e9fb2374a Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 21 Dec 2020 03:19:05 +0000 Subject: [PATCH 027/264] try fix fix fix fix --- src/TableFunctions/TableFunctionRemote.cpp | 28 ++++++++----------- src/TableFunctions/TableFunctionRemote.h | 2 +- ...602_insert_into_table_function_cluster.sql | 4 +-- 3 files changed, 14 insertions(+), 20 deletions(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 9760e2f2e83..f27fe54dbf1 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -50,16 +50,17 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont size_t arg_num = 0; - auto get_string_literal = [](const IAST & node, const char * description) + auto get_string_literal = [](const IAST & node, String & res) { const auto * lit = node.as(); if (!lit) - throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS); + return false; if (lit->value.getType() != Field::Types::String) - throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS); + return false; - return safeGet(lit->value); + res = safeGet(lit->value); + return true; }; if (is_cluster_function) @@ -70,7 +71,8 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont else { if (!tryGetIdentifierNameInto(args[arg_num], cluster_name)) - cluster_description = get_string_literal(*args[arg_num], "Hosts pattern"); + if (!get_string_literal(*args[arg_num], cluster_description)) + throw Exception("Hosts pattern must be string literal (in single quotes).", ErrorCodes::BAD_ARGUMENTS); } ++arg_num; @@ -122,29 +124,21 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont { if (arg_num < args.size()) { - try - { - username = get_string_literal(*args[arg_num], "Username"); - } - catch (const Exception & _ [[maybe_unused]]) + if (!get_string_literal(*args[arg_num], username)) { username = "default"; sharding_key = args[arg_num]; + ++arg_num; } - ++arg_num; } if (arg_num < args.size() && !sharding_key) { - try - { - password = get_string_literal(*args[arg_num], "Password"); - } - catch (const Exception & _ [[maybe_unused]]) + if (!get_string_literal(*args[arg_num], password)) { sharding_key = args[arg_num]; + ++arg_num; } - ++arg_num; } if (arg_num < args.size() && !sharding_key) diff --git a/src/TableFunctions/TableFunctionRemote.h b/src/TableFunctions/TableFunctionRemote.h index c859a3a4118..d485440d604 100644 --- a/src/TableFunctions/TableFunctionRemote.h +++ b/src/TableFunctions/TableFunctionRemote.h @@ -40,7 +40,7 @@ private: ClusterPtr cluster; StorageID remote_table_id = StorageID::createEmpty(); ASTPtr remote_table_function_ptr; - ASTPtr sharding_key{}; + ASTPtr sharding_key = nullptr; }; } diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql index bb61c568033..cde551bc320 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql @@ -6,9 +6,9 @@ INSERT INTO FUNCTION cluster('test_shard_localhost', default, x) SELECT * FROM n -- In fact, in this case(just one shard), sharding key is not required INSERT INTO FUNCTION cluster('test_shard_localhost', default, x, rand()) SELECT * FROM numbers(10); -INSERT INTO FUNCTION remote('localhost:59000', default, x, rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION remote('127.0.0.1', default, x, rand()) SELECT * FROM numbers(10); -INSERT INTO FUNCTION remote('localhost:59000', default, x, 'default', rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION remote('127.0.0.1', default, x, 'default', rand()) SELECT * FROM numbers(10); SELECT * FROM default.x ORDER BY number; From 257ca664ed3063776b338979e1a13e0aab7edf44 Mon Sep 17 00:00:00 2001 From: feng lv Date: Mon, 21 Dec 2020 11:32:44 +0000 Subject: [PATCH 028/264] fix --- src/TableFunctions/TableFunctionRemote.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index f27fe54dbf1..56ef85b8758 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -71,8 +71,10 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont else { if (!tryGetIdentifierNameInto(args[arg_num], cluster_name)) + { if (!get_string_literal(*args[arg_num], cluster_description)) throw Exception("Hosts pattern must be string literal (in single quotes).", ErrorCodes::BAD_ARGUMENTS); + } } ++arg_num; @@ -128,8 +130,8 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont { username = "default"; sharding_key = args[arg_num]; - ++arg_num; } + ++arg_num; } if (arg_num < args.size() && !sharding_key) @@ -137,8 +139,8 @@ void TableFunctionRemote::parseArguments(const ASTPtr & ast_function, const Cont if (!get_string_literal(*args[arg_num], password)) { sharding_key = args[arg_num]; - ++arg_num; } + ++arg_num; } if (arg_num < args.size() && !sharding_key) From a6429cc416737ec9fac5dc74f3ef1f9817933548 Mon Sep 17 00:00:00 2001 From: feng lv Date: Tue, 22 Dec 2020 03:17:05 +0000 Subject: [PATCH 029/264] update test update test fix fix fix --- ...sert_into_table_function_cluster.reference | 44 ++++++++++++------- ...602_insert_into_table_function_cluster.sql | 15 +++++-- 2 files changed, 38 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference index abec951f640..3dca59e7ef8 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.reference @@ -1,40 +1,50 @@ 0 0 0 -0 1 1 1 +2 +2 +2 +3 +3 +3 +4 +4 +4 +5 +5 +5 +6 +6 +6 +7 +7 +7 +8 +8 +8 +9 +9 +9 +0 +0 +1 1 2 2 -2 -2 -3 -3 3 3 4 4 -4 -4 -5 -5 5 5 6 6 -6 -6 -7 -7 7 7 8 8 -8 -8 -9 -9 9 9 diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql index cde551bc320..53a33676ef8 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql @@ -1,15 +1,22 @@ DROP TABLE IF EXISTS default.x; -CREATE TABLE default.x AS system.numbers ENGINE = Log; +CREATE TABLE default.x AS system.numbers ENGINE = MergeTree ORDER BY number; +CREATE TABLE default.y AS system.numbers ENGINE = MergeTree ORDER BY number; +-- Just one shard, sharding key isn't necessary INSERT INTO FUNCTION cluster('test_shard_localhost', default, x) SELECT * FROM numbers(10); --- In fact, in this case(just one shard), sharding key is not required INSERT INTO FUNCTION cluster('test_shard_localhost', default, x, rand()) SELECT * FROM numbers(10); -INSERT INTO FUNCTION remote('127.0.0.1', default, x, rand()) SELECT * FROM numbers(10); +-- More than one shard, sharding key is necessary +INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', default, x) SELECT * FROM numbers(10); --{ serverError 55 } +INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', default, x, rand()) SELECT * FROM numbers(10); -INSERT INTO FUNCTION remote('127.0.0.1', default, x, 'default', rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION remote('127.0.0.{1,2}', default, y, 'default') SELECT * FROM numbers(10); -- { serverError 55 } +INSERT INTO FUNCTION remote('127.0.0.{1,2}', default, y, 'default', rand()) SELECT * FROM numbers(10); SELECT * FROM default.x ORDER BY number; +SELECT * FROM remote('127.0.0.{1,2}', default, y) ORDER BY number; + DROP TABLE default.x; +DROP TABLE default.y; From 40b5ffe3b52476d43648a28c6850b1beea4aba29 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 22 Dec 2020 18:03:48 +0300 Subject: [PATCH 030/264] minor renames --- src/Core/NamesAndTypes.cpp | 20 ++++++++--------- src/Core/NamesAndTypes.h | 22 +++++++++++-------- src/DataTypes/IDataType.cpp | 10 ++++----- src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 4 ++-- .../MergeTree/MergeTreeReaderCompact.cpp | 12 +++++----- .../MergeTree/MergeTreeReaderInMemory.cpp | 4 ++-- .../MergeTree/MergeTreeReaderWide.cpp | 2 +- src/Storages/StorageBuffer.cpp | 4 ++-- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageMemory.cpp | 4 ++-- src/Storages/StorageTinyLog.cpp | 2 +- 12 files changed, 46 insertions(+), 42 deletions(-) diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 9c92e818c14..e96ce1824d2 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -18,27 +18,27 @@ namespace ErrorCodes } NameAndTypePair::NameAndTypePair( - const String & storage_name_, const String & subcolumn_name_, - const DataTypePtr & storage_type_, const DataTypePtr & subcolumn_type_) - : name(storage_name_ + (subcolumn_name_.empty() ? "" : "." + subcolumn_name_)) + const String & name_in_storage_, const String & subcolumn_name_, + const DataTypePtr & type_in_storage_, const DataTypePtr & subcolumn_type_) + : name(name_in_storage_ + (subcolumn_name_.empty() ? "" : "." + subcolumn_name_)) , type(subcolumn_type_) - , storage_type(storage_type_) - , subcolumn_delimiter_position(storage_name_.size()) {} + , type_in_storage(type_in_storage_) + , subcolumn_delimiter_position(name_in_storage_.size()) {} -String NameAndTypePair::getStorageName() const +String NameAndTypePair::getNameInStorage() const { - if (subcolumn_delimiter_position == -1) + if (!subcolumn_delimiter_position) return name; - return name.substr(0, subcolumn_delimiter_position); + return name.substr(0, *subcolumn_delimiter_position); } String NameAndTypePair::getSubcolumnName() const { - if (subcolumn_delimiter_position == -1) + if (!subcolumn_delimiter_position) return ""; - return name.substr(subcolumn_delimiter_position + 1, name.size() - subcolumn_delimiter_position); + return name.substr(*subcolumn_delimiter_position + 1, name.size() - *subcolumn_delimiter_position); } void NamesAndTypesList::readText(ReadBuffer & buf) diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 65f4caf0111..dad031a543c 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -17,15 +17,17 @@ struct NameAndTypePair { public: NameAndTypePair() = default; - NameAndTypePair(const String & name_, const DataTypePtr & type_) : name(name_), type(type_), storage_type(type_) {} - NameAndTypePair(const String & storage_name_, const String & subcolumn_name_, - const DataTypePtr & storage_type_, const DataTypePtr & subcolumn_type_); + NameAndTypePair(const String & name_, const DataTypePtr & type_) + : name(name_), type(type_), type_in_storage(type_) {} - String getStorageName() const; + NameAndTypePair(const String & name_in_storage_, const String & subcolumn_name_, + const DataTypePtr & type_in_storage_, const DataTypePtr & subcolumn_type_); + + String getNameInStorage() const; String getSubcolumnName() const; - bool isSubcolumn() const { return subcolumn_delimiter_position != -1; } - DataTypePtr getStorageType() const { return storage_type; } + bool isSubcolumn() const { return subcolumn_delimiter_position != std::nullopt; } + DataTypePtr getTypeInStorage() const { return type_in_storage; } bool operator<(const NameAndTypePair & rhs) const { @@ -41,11 +43,13 @@ public: DataTypePtr type; private: - DataTypePtr storage_type; - ssize_t subcolumn_delimiter_position = -1; + DataTypePtr type_in_storage; + std::optional subcolumn_delimiter_position; }; -template +/// This needed to use structured bindings for NameAndTypePair +/// const auto & [name, type] = name_and_type +template decltype(auto) get(const NameAndTypePair & name_and_type) { if constexpr (I == 0) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 3bad80ee131..26fccb24fe4 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -199,13 +199,13 @@ static String getNameForSubstreamPath( String IDataType::getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path) { - auto storage_name = column.getStorageName(); - auto nested_storage_name = Nested::extractTableName(column.getStorageName()); + auto name_in_storage = column.getNameInStorage(); + auto nested_storage_name = Nested::extractTableName(name_in_storage); - if (storage_name != nested_storage_name && (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes)) - storage_name = nested_storage_name; + if (name_in_storage != nested_storage_name && (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes)) + name_in_storage = nested_storage_name; - auto stream_name = escapeForFileName(storage_name); + auto stream_name = escapeForFileName(name_in_storage); return getNameForSubstreamPath(std::move(stream_name), path, true); } diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index b563073ff84..356bef68530 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -203,7 +203,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & required_column) const { - auto name_in_storage = required_column.getStorageName(); + auto name_in_storage = required_column.getNameInStorage(); decltype(columns_from_part.begin()) it; if (alter_conversions.isColumnRenamed(name_in_storage)) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 7d863cefd56..f8b5e0a9c0a 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -38,13 +38,13 @@ bool injectRequiredColumnsRecursively( if (storage_columns.hasPhysicalOrSubcolumn(column_name)) { auto column_in_storage = storage_columns.getPhysicalOrSubcolumn(column_name); - auto column_name_in_part = column_in_storage.getStorageName(); + auto column_name_in_part = column_in_storage.getNameInStorage(); if (alter_conversions.isColumnRenamed(column_name_in_part)) column_name_in_part = alter_conversions.getColumnOldName(column_name_in_part); auto column_in_part = NameAndTypePair( column_name_in_part, column_in_storage.getSubcolumnName(), - column_in_storage.getStorageType(), column_in_storage.type); + column_in_storage.getTypeInStorage(), column_in_storage.type); /// column has files and hence does not require evaluation if (part->hasColumnFiles(column_in_part)) diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 2fccff4f443..d6bfac4a5b6 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -155,7 +155,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, size_t column_size_before_reading = column->size(); readData(column_from_part, column, from_mark, *column_positions[pos], - rows_to_read, read_only_offsets[pos], caches[column_from_part.getStorageName()]); + rows_to_read, read_only_offsets[pos], caches[column_from_part.getNameInStorage()]); size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column < rows_to_read) @@ -215,12 +215,12 @@ void MergeTreeReaderCompact::readData( if (name_and_type.isSubcolumn()) { - const auto & storage_type = name_and_type.getStorageType(); - ColumnPtr temp_column = storage_type->createColumn(); + auto type_in_storage = name_and_type.getTypeInStorage(); + ColumnPtr temp_column = type_in_storage->createColumn(); - storage_type->deserializeBinaryBulkStatePrefix(deserialize_settings, state); - storage_type->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state); - column = storage_type->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column); + type_in_storage->deserializeBinaryBulkStatePrefix(deserialize_settings, state); + type_in_storage->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state); + column = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column); } else { diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index 114fa118eee..5ee4aa555e6 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -41,13 +41,13 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( static ColumnPtr getColumnFromBlock(const Block & block, const NameAndTypePair & name_and_type) { - auto storage_name = name_and_type.getStorageName(); + auto storage_name = name_and_type.getNameInStorage(); if (!block.has(storage_name)) throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found column '{}' in block", storage_name); const auto & column = block.getByName(storage_name).column; if (name_and_type.isSubcolumn()) - return name_and_type.getStorageType()->getSubcolumn(name_and_type.getSubcolumnName(), *column); + return name_and_type.getTypeInStorage()->getSubcolumn(name_and_type.getSubcolumnName(), *column); return column; } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 7cebb13172a..42798ee4dd0 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -89,7 +89,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si try { size_t column_size_before_reading = column->size(); - auto & cache = caches[column_from_part.getStorageName()]; + auto & cache = caches[column_from_part.getNameInStorage()]; readData(column_from_part, column, from_mark, continue_reading, max_rows_to_read, cache); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 6a91dcfe084..ef0209ebabf 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -119,9 +119,9 @@ protected: for (const auto & elem : column_names_and_types) { - const auto & current_column = buffer.data.getByName(elem.getStorageName()).column; + const auto & current_column = buffer.data.getByName(elem.getNameInStorage()).column; if (elem.isSubcolumn()) - columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column)); + columns.emplace_back(elem.getTypeInStorage()->getSubcolumn(elem.getSubcolumnName(), *current_column)); else columns.emplace_back(std::move(current_column)); } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 6b2845702aa..6a0639b324b 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -216,7 +216,7 @@ Chunk LogSource::generate() try { column = name_type.type->createColumn(); - readData(name_type, column, max_rows_to_read, caches[name_type.getStorageName()]); + readData(name_type, column, max_rows_to_read, caches[name_type.getNameInStorage()]); } catch (Exception & e) { diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index f8842b887a7..3ab14f0dbdc 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -70,9 +70,9 @@ protected: /// Add only required columns to `res`. for (const auto & elem : column_names_and_types) { - auto current_column = src.getByName(elem.getStorageName()).column; + auto current_column = src.getByName(elem.getNameInStorage()).column; if (elem.isSubcolumn()) - columns.emplace_back(elem.getStorageType()->getSubcolumn(elem.getSubcolumnName(), *current_column)); + columns.emplace_back(elem.getTypeInStorage()->getSubcolumn(elem.getSubcolumnName(), *current_column)); else columns.emplace_back(std::move(current_column)); } diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 38705bdf0ec..bcb806816ce 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -201,7 +201,7 @@ Chunk TinyLogSource::generate() try { column = name_type.type->createColumn(); - readData(name_type, column, block_size, caches[name_type.getStorageName()]); + readData(name_type, column, block_size, caches[name_type.getNameInStorage()]); } catch (Exception & e) { From 1be39fddacf2689b023079fbb82b07819c26d12e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 22 Dec 2020 19:40:53 +0300 Subject: [PATCH 031/264] fix subcolumns with some storages --- src/Interpreters/TreeRewriter.cpp | 7 ++++++- src/Storages/IStorage.h | 3 +++ src/Storages/MergeTree/MergeTreeData.h | 2 ++ src/Storages/StorageBuffer.h | 2 ++ src/Storages/StorageLog.h | 1 + src/Storages/StorageMaterializedView.h | 1 + src/Storages/StorageMemory.h | 2 ++ src/Storages/StorageMerge.h | 1 + src/Storages/StorageTinyLog.h | 1 + src/Storages/StorageView.h | 1 + 10 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index e294228c5c8..aa59696ec76 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -468,7 +468,12 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) { const ColumnsDescription & columns = metadata_snapshot->getColumns(); - auto columns_from_storage = add_special ? columns.getAllWithSubcolumns() : columns.getAllPhysicalWithSubcolumns(); + NamesAndTypesList columns_from_storage; + if (storage->supportsSubcolumns()) + columns_from_storage = add_special ? columns.getAllWithSubcolumns() : columns.getAllPhysicalWithSubcolumns(); + else + columns_from_storage = add_special ? columns.getAll() : columns.getAllPhysical(); + if (source_columns.empty()) source_columns.swap(columns_from_storage); else diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 89241f461f6..c0225f4bb09 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -132,6 +132,9 @@ public: /// Example is StorageSystemNumbers. virtual bool hasEvenlyDistributedRead() const { return false; } + /// Returns true if the storage supports reading of subcolumns of complex types. + virtual bool supportsSubcolumns() const { return false; } + /// Optional size information of each physical column. /// Currently it's only used by the MergeTree family for query optimizations. diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e5ffe8c025b..470c4f56aaf 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -357,6 +357,8 @@ public: || merging_params.mode == MergingParams::VersionedCollapsing; } + bool supportsSubcolumns() const override { return true; } + bool supportsSettings() const override { return true; } NamesAndTypesList getVirtuals() const override; diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h index 38596531a1a..c07d1f987f3 100644 --- a/src/Storages/StorageBuffer.h +++ b/src/Storages/StorageBuffer.h @@ -76,6 +76,8 @@ public: bool supportsParallelInsert() const override { return true; } + bool supportsSubcolumns() const override { return true; } + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; void startup() override; diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 0e7b9842737..540681fe72a 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -44,6 +44,7 @@ public: bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } + bool supportsSubcolumns() const override { return true; } protected: /** Attach the table with the appropriate name, along the appropriate path (with / at the end), diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index e1dd73e8580..fab9e28afe3 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -26,6 +26,7 @@ public: bool supportsFinal() const override { return getTargetTable()->supportsFinal(); } bool supportsIndexForIn() const override { return getTargetTable()->supportsIndexForIn(); } bool supportsParallelInsert() const override { return getTargetTable()->supportsParallelInsert(); } + bool supportsSubcolumns() const override { return getTargetTable()->supportsSubcolumns(); } bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & /* metadata_snapshot */) const override { auto target_table = getTargetTable(); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index 6453e6a53e2..702cb265ea9 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -41,6 +41,8 @@ public: bool supportsParallelInsert() const override { return true; } + bool supportsSubcolumns() const override { return true; } + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, const Context & context) override; void drop() override; diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 223383a6975..3ac251fbe52 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -25,6 +25,7 @@ public: bool supportsPrewhere() const override { return true; } bool supportsFinal() const override { return true; } bool supportsIndexForIn() const override { return true; } + bool supportsSubcolumns() const override { return true; } QueryProcessingStage::Enum getQueryProcessingStage(const Context &, QueryProcessingStage::Enum /*to_stage*/, SelectQueryInfo &) const override; diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index bddc4351e51..8050f5965f0 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -41,6 +41,7 @@ public: bool storesDataOnDisk() const override { return true; } Strings getDataPaths() const override { return {DB::fullPath(disk, table_path)}; } + bool supportsSubcolumns() const override { return true; } void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index 6f894ce2775..dca9cf6ed08 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -20,6 +20,7 @@ public: /// It is passed inside the query and solved at its level. bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } + bool supportsSubcolumns() const override { return true; } Pipe read( const Names & column_names, From cc6df719b8ac4d51c783342fb9125362987756d7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 22 Dec 2020 21:03:30 +0300 Subject: [PATCH 032/264] fix optimize deduplicate --- src/Storages/StorageInMemoryMetadata.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 35b60ac8aeb..2f488ce36c6 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -462,7 +462,7 @@ namespace void StorageInMemoryMetadata::check(const Names & column_names, const NamesAndTypesList & virtuals, const StorageID & storage_id) const { - NamesAndTypesList available_columns = getColumns().getAllWithSubcolumns(); + NamesAndTypesList available_columns = getColumns().getAllPhysicalWithSubcolumns(); available_columns.insert(available_columns.end(), virtuals.begin(), virtuals.end()); const String list_of_columns = listOfColumns(available_columns); From 8b5714b2ac63dd0d143eec85265b07759046414b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 23 Dec 2020 01:51:09 +0800 Subject: [PATCH 033/264] Fix 2-arg functions with constant in PK analysis --- src/Storages/MergeTree/KeyCondition.cpp | 80 ++++++++++++++++++- ...1615_two_args_function_index_fix.reference | 1 + .../01615_two_args_function_index_fix.sql | 9 +++ 3 files changed, 86 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/01615_two_args_function_index_fix.reference create mode 100644 tests/queries/0_stateless/01615_two_args_function_index_fix.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index ead93a6e1d9..735d764eb8e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -873,6 +873,70 @@ bool KeyCondition::tryPrepareSetIndex( return true; } +class FunctionWithOptionalConstArg : public IFunctionBase +{ +public: + enum Kind + { + NO_CONST = 0, + LEFT_CONST, + RIGHT_CONST, + }; + + FunctionWithOptionalConstArg(const FunctionBasePtr & func_) : func(func_) { } + FunctionWithOptionalConstArg(const FunctionBasePtr & func_, const ColumnWithTypeAndName & const_arg_, Kind kind_) + : func(func_), const_arg(const_arg_), kind(kind_) + { + } + + String getName() const override { return func->getName(); } + + const DataTypes & getArgumentTypes() const override { return func->getArgumentTypes(); } + + const DataTypePtr & getResultType() const override { return func->getResultType(); } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & arguments) const override { return func->prepare(arguments); } + + ColumnPtr + execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run = false) + const override + { + if (kind == Kind::LEFT_CONST) + { + ColumnsWithTypeAndName new_arguments; + new_arguments.reserve(arguments.size() + 1); + new_arguments.push_back(const_arg); + for (const auto & arg : arguments) + new_arguments.push_back(arg); + return func->prepare(new_arguments)->execute(new_arguments, result_type, input_rows_count, dry_run); + } + else if (kind == Kind::RIGHT_CONST) + { + auto new_arguments = arguments; + new_arguments.push_back(const_arg); + return func->prepare(new_arguments)->execute(new_arguments, result_type, input_rows_count, dry_run); + } + else + return func->prepare(arguments)->execute(arguments, result_type, input_rows_count, dry_run); + } + + bool isDeterministic() const override { return func->isDeterministic(); } + + bool isDeterministicInScopeOfQuery() const override { return func->isDeterministicInScopeOfQuery(); } + + bool hasInformationAboutMonotonicity() const override { return func->hasInformationAboutMonotonicity(); } + + IFunctionBase::Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override + { + return func->getMonotonicityForRange(type, left, right); + } + +private: + FunctionBasePtr func; + ColumnWithTypeAndName const_arg; + Kind kind = Kind::NO_CONST; +}; + bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( const ASTPtr & node, @@ -892,19 +956,25 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( const auto & args = (*it)->arguments->children; auto func_builder = FunctionFactory::instance().tryGet((*it)->name, context); ColumnsWithTypeAndName arguments; + ColumnWithTypeAndName const_arg; + FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; if (args.size() == 2) { if (const auto * arg_left = args[0]->as()) { auto left_arg_type = applyVisitor(FieldToDataType(), arg_left->value); - arguments.push_back({ left_arg_type->createColumnConst(0, arg_left->value), left_arg_type, "" }); + const_arg = { left_arg_type->createColumnConst(0, arg_left->value), left_arg_type, "" }; + arguments.push_back(const_arg); arguments.push_back({ nullptr, key_column_type, "" }); + kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; } else if (const auto * arg_right = args[1]->as()) { arguments.push_back({ nullptr, key_column_type, "" }); auto right_arg_type = applyVisitor(FieldToDataType(), arg_right->value); - arguments.push_back({ right_arg_type->createColumnConst(0, arg_right->value), right_arg_type, "" }); + const_arg = { right_arg_type->createColumnConst(0, arg_right->value), right_arg_type, "" }; + arguments.push_back(const_arg); + kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; } } else @@ -916,7 +986,10 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( return false; key_column_type = func->getResultType(); - out_functions_chain.push_back(func); + if (kind == FunctionWithOptionalConstArg::Kind::NO_CONST) + out_functions_chain.push_back(func); + else + out_functions_chain.push_back(std::make_shared(func, const_arg, kind)); } out_key_res_column_type = key_column_type; @@ -1395,7 +1468,6 @@ BoolMask KeyCondition::checkInRange( }); } - std::optional KeyCondition::applyMonotonicFunctionsChainToRange( Range key_range, const MonotonicFunctionsChain & functions, diff --git a/tests/queries/0_stateless/01615_two_args_function_index_fix.reference b/tests/queries/0_stateless/01615_two_args_function_index_fix.reference new file mode 100644 index 00000000000..6439de16e24 --- /dev/null +++ b/tests/queries/0_stateless/01615_two_args_function_index_fix.reference @@ -0,0 +1 @@ +2020-12-19 2020-12-19 22:59:52 2020-12-19 22:59:52 1 diff --git a/tests/queries/0_stateless/01615_two_args_function_index_fix.sql b/tests/queries/0_stateless/01615_two_args_function_index_fix.sql new file mode 100644 index 00000000000..258a07d25f2 --- /dev/null +++ b/tests/queries/0_stateless/01615_two_args_function_index_fix.sql @@ -0,0 +1,9 @@ +drop table if exists bad_date_time; + +create table bad_date_time (time Datetime, count UInt16) Engine = MergeTree() ORDER BY (time); + +insert into bad_date_time values('2020-12-20 20:59:52', 1), ('2020-12-20 21:59:52', 1), ('2020-12-20 01:59:52', 1); + +select toDate(time, 'UTC') dt, min(toDateTime(time, 'UTC')), max(toDateTime(time, 'UTC')), sum(count) from bad_date_time where toDate(time, 'UTC') = '2020-12-19' group by dt; + +drop table if exists bad_date_time; From 5f9c47787a70359f0602067b1d0b0013a1c55ad2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 23 Dec 2020 15:16:26 +0300 Subject: [PATCH 034/264] fix subcolumns with some storages --- src/Storages/StorageView.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageView.h b/src/Storages/StorageView.h index dca9cf6ed08..6f894ce2775 100644 --- a/src/Storages/StorageView.h +++ b/src/Storages/StorageView.h @@ -20,7 +20,6 @@ public: /// It is passed inside the query and solved at its level. bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } - bool supportsSubcolumns() const override { return true; } Pipe read( const Names & column_names, From b413b450f5d7a6746a90fb4737c293174720c51f Mon Sep 17 00:00:00 2001 From: nauta <870284156@qq.com> Date: Sat, 26 Dec 2020 16:00:35 +0800 Subject: [PATCH 035/264] add decode XML function --- contrib/openssl | 1 + contrib/ryu | 1 + src/Functions/decodeXMLComponent.cpp | 191 ++++++++++++++++++ src/Functions/registerFunctionsString.cpp | 3 + src/Functions/ya.make | 1 + .../0_stateless/01621_decode_XML.reference | 4 + .../queries/0_stateless/01621_decode_XML.sql | 4 + 7 files changed, 205 insertions(+) create mode 160000 contrib/openssl create mode 160000 contrib/ryu create mode 100644 src/Functions/decodeXMLComponent.cpp create mode 100644 tests/queries/0_stateless/01621_decode_XML.reference create mode 100644 tests/queries/0_stateless/01621_decode_XML.sql diff --git a/contrib/openssl b/contrib/openssl new file mode 160000 index 00000000000..237260dd6a4 --- /dev/null +++ b/contrib/openssl @@ -0,0 +1 @@ +Subproject commit 237260dd6a4bca5cb5a321d366a8a9c807957455 diff --git a/contrib/ryu b/contrib/ryu new file mode 160000 index 00000000000..5b4a853534b --- /dev/null +++ b/contrib/ryu @@ -0,0 +1 @@ +Subproject commit 5b4a853534b47438b4d97935370f6b2397137c2b diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp new file mode 100644 index 00000000000..e546a8ba075 --- /dev/null +++ b/src/Functions/decodeXMLComponent.cpp @@ -0,0 +1,191 @@ +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + struct DecodeXMLComponentName + { + static constexpr auto name = "decodeXMLComponent"; + }; + + class FunctionDecodeXMLComponentImpl + { + public: + static void vector( + const ColumnString::Chars & data, + const ColumnString::Offsets & offsets, + ColumnString::Chars & res_data, + ColumnString::Offsets & res_offsets) + { + res_data.resize(data.size() * 6); + size_t size = offsets.size(); + res_offsets.resize(size); + + size_t prev_offset = 0; + size_t res_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + const char * src_data = reinterpret_cast(&data[prev_offset]); + size_t src_size = offsets[i] - prev_offset; + size_t dst_size = execute(src_data, src_size, reinterpret_cast(res_data.data() + res_offset)); + + res_offset += dst_size; + res_offsets[i] = res_offset; + prev_offset = offsets[i]; + } + + res_data.resize(res_offset); + } + + [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &) + { + throw Exception("Function decodeXMLComponent cannot work with FixedString argument", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + private: + static size_t execute(const char * src, size_t src_size, char * dst) + { + const char * src_prev_pos = src; + const char * src_curr_pos = src; + const char * src_next_pos = src; + const char * src_end = src + src_size; + char * dst_pos = dst; + + while (true) + { + src_curr_pos = find_first_symbols<'&'>(src_curr_pos, src_end); + + if (src_curr_pos == src_end) + { + break; + } + else if (*src_curr_pos == '&') + { + src_next_pos = find_first_symbols<';'>(src_curr_pos, src_end); + if (src_next_pos == src_end || src_next_pos - src_curr_pos < 3) + { + src_curr_pos = src_end; + break; + } + else if (src_next_pos - src_curr_pos == 3) + { + if (strncmp(src_curr_pos, "<", 3) == 0) + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + *dst_pos = '<'; + ++dst_pos; + src_prev_pos = src_curr_pos + 4; + } + else if (strncmp(src_curr_pos, ">", 3) == 0) + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + *dst_pos = '>'; + ++dst_pos; + src_prev_pos = src_curr_pos + 4; + } + else + { + src_curr_pos = src_next_pos + 1; + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_prev_pos = src_curr_pos; + } + src_curr_pos += 4; + } + else if (src_next_pos - src_curr_pos == 4) + { + if (strncmp(src_curr_pos, "&", 4) == 0) + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + *dst_pos = '&'; + ++dst_pos; + src_prev_pos = src_curr_pos + 5; + } + else + { + src_curr_pos = src_next_pos + 1; + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_prev_pos = src_curr_pos; + } + src_curr_pos += 5; + } + else if (src_next_pos - src_curr_pos == 5) + { + if (strncmp(src_curr_pos, """, 5) == 0) + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + *dst_pos = '"'; + ++dst_pos; + src_prev_pos = src_curr_pos + 6; + } + else if (strncmp(src_curr_pos, "&apos", 5) == 0) + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + *dst_pos = '\''; + ++dst_pos; + src_prev_pos = src_curr_pos + 6; + } + else + { + src_curr_pos = src_next_pos + 1; + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_prev_pos = src_curr_pos; + } + src_curr_pos += 6; + } + else + { + src_curr_pos = src_next_pos + 1; + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_prev_pos = src_curr_pos; + } + } + } + + if (src_prev_pos < src_curr_pos) + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + } + + return dst_pos - dst; + } + }; + + using FunctionDecodeXMLComponent = FunctionStringToString; + +} // namespace + +void registerFunctionDecodeXMLComponent(FunctionFactory & factory) +{ + factory.registerFunction(); +} +} // namespace DB diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp index 426cc8f8d56..5cf30dd83a6 100644 --- a/src/Functions/registerFunctionsString.cpp +++ b/src/Functions/registerFunctionsString.cpp @@ -34,6 +34,8 @@ void registerFunctionNormalizeQuery(FunctionFactory &); void registerFunctionNormalizedQueryHash(FunctionFactory &); void registerFunctionCountMatches(FunctionFactory &); void registerFunctionEncodeXMLComponent(FunctionFactory & factory); +void registerFunctionDecodeXMLComponent(FunctionFactory & factory); + #if USE_BASE64 void registerFunctionBase64Encode(FunctionFactory &); @@ -70,6 +72,7 @@ void registerFunctionsString(FunctionFactory & factory) registerFunctionNormalizedQueryHash(factory); registerFunctionCountMatches(factory); registerFunctionEncodeXMLComponent(factory); + registerFunctionDecodeXMLComponent(factory); #if USE_BASE64 registerFunctionBase64Encode(factory); registerFunctionBase64Decode(factory); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 7e64deef64d..9ebebab6658 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -220,6 +220,7 @@ SRCS( currentUser.cpp dateDiff.cpp date_trunc.cpp + decodeXMLComponent.cpp decrypt.cpp defaultValueOfArgumentType.cpp defaultValueOfTypeName.cpp diff --git a/tests/queries/0_stateless/01621_decode_XML.reference b/tests/queries/0_stateless/01621_decode_XML.reference new file mode 100644 index 00000000000..dab7a2d14e1 --- /dev/null +++ b/tests/queries/0_stateless/01621_decode_XML.reference @@ -0,0 +1,4 @@ +Hello, "world"! +<123> +&clickhouse +\'foo\' diff --git a/tests/queries/0_stateless/01621_decode_XML.sql b/tests/queries/0_stateless/01621_decode_XML.sql new file mode 100644 index 00000000000..4c9404e6925 --- /dev/null +++ b/tests/queries/0_stateless/01621_decode_XML.sql @@ -0,0 +1,4 @@ +SELECT decodeXMLComponent('Hello, "world"!'); +SELECT decodeXMLComponent('<123>'); +SELECT decodeXMLComponent('&clickhouse'); +SELECT decodeXMLComponent(''foo''); \ No newline at end of file From 9e5864f69441074bb49f6058b3bf48f71b5dc24b Mon Sep 17 00:00:00 2001 From: nauta <870284156@qq.com> Date: Sat, 26 Dec 2020 16:31:05 +0800 Subject: [PATCH 036/264] fix --- contrib/openssl | 1 - contrib/ryu | 1 - 2 files changed, 2 deletions(-) delete mode 160000 contrib/openssl delete mode 160000 contrib/ryu diff --git a/contrib/openssl b/contrib/openssl deleted file mode 160000 index 237260dd6a4..00000000000 --- a/contrib/openssl +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 237260dd6a4bca5cb5a321d366a8a9c807957455 diff --git a/contrib/ryu b/contrib/ryu deleted file mode 160000 index 5b4a853534b..00000000000 --- a/contrib/ryu +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 5b4a853534b47438b4d97935370f6b2397137c2b From 839a4bfbd8572ab3ec14a729628431ab87a805b5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 26 Dec 2020 15:01:14 +0300 Subject: [PATCH 037/264] Update decodeXMLComponent.cpp --- src/Functions/decodeXMLComponent.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index e546a8ba075..8060c112268 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -182,10 +182,10 @@ namespace using FunctionDecodeXMLComponent = FunctionStringToString; -} // namespace +} void registerFunctionDecodeXMLComponent(FunctionFactory & factory) { factory.registerFunction(); } -} // namespace DB +} From 6cdab36a7d2888280c808b01d7486e6a9c5652e6 Mon Sep 17 00:00:00 2001 From: nauta <870284156@qq.com> Date: Sat, 26 Dec 2020 20:18:48 +0800 Subject: [PATCH 038/264] fix bug --- src/Functions/decodeXMLComponent.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index 8060c112268..1cae1e2956c 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -26,7 +26,7 @@ namespace ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { - res_data.resize(data.size() * 6); + res_data.resize(data.size()); size_t size = offsets.size(); res_offsets.resize(size); From 2d8e35b4bd5984ced872a3f945444c4be48690a0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Dec 2020 00:36:31 +0000 Subject: [PATCH 039/264] Add libpq and libpqxx --- .gitmodules | 6 +++ CMakeLists.txt | 1 + cmake/find/libpqxx.cmake | 30 +++++++++++ contrib/CMakeLists.txt | 5 ++ contrib/libpq | 1 + contrib/libpq-cmake/CMakeLists.txt | 77 ++++++++++++++++++++++++++++ contrib/libpqxx | 1 + contrib/libpqxx-cmake/CMakeLists.txt | 33 ++++++++++++ src/CMakeLists.txt | 5 ++ 9 files changed, 159 insertions(+) create mode 100644 cmake/find/libpqxx.cmake create mode 160000 contrib/libpq create mode 100644 contrib/libpq-cmake/CMakeLists.txt create mode 160000 contrib/libpqxx create mode 100644 contrib/libpqxx-cmake/CMakeLists.txt diff --git a/.gitmodules b/.gitmodules index 63173402993..066d4c5e809 100644 --- a/.gitmodules +++ b/.gitmodules @@ -209,6 +209,12 @@ [submodule "contrib/fast_float"] path = contrib/fast_float url = https://github.com/fastfloat/fast_float +[submodule "contrib/libpq"] + path = contrib/libpq + url = https://github.com/kssenii/libpq +[submodule "contrib/libpqxx"] + path = contrib/libpqxx + url = https://github.com/jtv/libpqxx [submodule "contrib/boringssl"] path = contrib/boringssl url = https://github.com/ClickHouse-Extras/boringssl.git diff --git a/CMakeLists.txt b/CMakeLists.txt index 2e42ae3b6bd..ae4f976f336 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -468,6 +468,7 @@ include (cmake/find/rapidjson.cmake) include (cmake/find/fastops.cmake) include (cmake/find/odbc.cmake) include (cmake/find/rocksdb.cmake) +include (cmake/find/libpqxx.cmake) include (cmake/find/nuraft.cmake) diff --git a/cmake/find/libpqxx.cmake b/cmake/find/libpqxx.cmake new file mode 100644 index 00000000000..b635fb80878 --- /dev/null +++ b/cmake/find/libpqxx.cmake @@ -0,0 +1,30 @@ +option(ENABLE_LIBPQXX "Enalbe libpqxx" ${ENABLE_LIBRARIES}) + +if (NOT ENABLE_LIBPQXX) + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpqxx/CMakeLists.txt") + message (WARNING "submodule contrib/libpqxx is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libpqxx library") + set (USE_LIBPQXX 0) + return() +endif() + +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpq") + message (ERROR "submodule contrib/libpq is missing. to fix try run: \n git submodule update --init --recursive") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libpq needed for libpqxx") + set (USE_LIBPQXX 0) + return() +endif() + +set (USE_LIBPQXX 1) +set (LIBPQXX_LIBRARY libpqxx) +set (LIBPQ_LIBRARY libpq) + +set (LIBPQXX_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpqxx/include") +set (LIBPQ_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpq") + +message (STATUS "Using libpqxx=${USE_LIBPQXX}: ${LIBPQXX_INCLUDE_DIR} : ${LIBPQXX_LIBRARY}") +message (STATUS "Using libpq: ${LIBPQ_ROOT_DIR} : ${LIBPQ_INCLUDE_DIR} : ${LIBPQ_LIBRARY}") + diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 3bd08de01b2..8502d9d79ef 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -307,6 +307,11 @@ if (USE_INTERNAL_ROCKSDB_LIBRARY) add_subdirectory(rocksdb-cmake) endif() +if (USE_LIBPQXX) + add_subdirectory (libpq-cmake) + add_subdirectory (libpqxx-cmake) +endif() + if (USE_NURAFT) add_subdirectory(nuraft-cmake) endif() diff --git a/contrib/libpq b/contrib/libpq new file mode 160000 index 00000000000..d94250f9145 --- /dev/null +++ b/contrib/libpq @@ -0,0 +1 @@ +Subproject commit d94250f91457f11de6a37f7a6ba162601ddead27 diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt new file mode 100644 index 00000000000..8678f0141c7 --- /dev/null +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -0,0 +1,77 @@ +set(LIBPQ_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libpq) + +set(SRCS + ${LIBPQ_SOURCE_DIR}/fe-auth.c + ${LIBPQ_SOURCE_DIR}/fe-auth-scram.c + ${LIBPQ_SOURCE_DIR}/fe-connect.c + ${LIBPQ_SOURCE_DIR}/fe-exec.c +# ${LIBPQ_SOURCE_DIR}/fe-gssapi-common.c + ${LIBPQ_SOURCE_DIR}/fe-lobj.c + ${LIBPQ_SOURCE_DIR}/fe-misc.c + ${LIBPQ_SOURCE_DIR}/fe-print.c + ${LIBPQ_SOURCE_DIR}/fe-protocol2.c + ${LIBPQ_SOURCE_DIR}/fe-protocol3.c + ${LIBPQ_SOURCE_DIR}/fe-secure.c + ${LIBPQ_SOURCE_DIR}/fe-secure-common.c +# ${LIBPQ_SOURCE_DIR}/fe-secure-gssapi.c + ${LIBPQ_SOURCE_DIR}/fe-secure-openssl.c + ${LIBPQ_SOURCE_DIR}/legacy-pqsignal.c + ${LIBPQ_SOURCE_DIR}/libpq-events.c + ${LIBPQ_SOURCE_DIR}/pqexpbuffer.c + + ${LIBPQ_SOURCE_DIR}/common/scram-common.c + ${LIBPQ_SOURCE_DIR}/common/sha2_openssl.c + ${LIBPQ_SOURCE_DIR}/common/md5.c + ${LIBPQ_SOURCE_DIR}/common/saslprep.c + ${LIBPQ_SOURCE_DIR}/common/unicode_norm.c + ${LIBPQ_SOURCE_DIR}/common/ip.c + ${LIBPQ_SOURCE_DIR}/common/jsonapi.c + ${LIBPQ_SOURCE_DIR}/common/wchar.c + ${LIBPQ_SOURCE_DIR}/common/base64.c + ${LIBPQ_SOURCE_DIR}/common/link-canary.c + ${LIBPQ_SOURCE_DIR}/common/fe_memutils.c + ${LIBPQ_SOURCE_DIR}/common/sha2.c + ${LIBPQ_SOURCE_DIR}/common/sha2_openssl.c + ${LIBPQ_SOURCE_DIR}/common/string.c + ${LIBPQ_SOURCE_DIR}/common/pg_get_line.c + ${LIBPQ_SOURCE_DIR}/common/stringinfo.c + ${LIBPQ_SOURCE_DIR}/common/psprintf.c + ${LIBPQ_SOURCE_DIR}/common/encnames.c + + ${LIBPQ_SOURCE_DIR}/port/snprintf.c + ${LIBPQ_SOURCE_DIR}/port/strlcpy.c + ${LIBPQ_SOURCE_DIR}/port/strerror.c + ${LIBPQ_SOURCE_DIR}/port/inet_net_ntop.c + ${LIBPQ_SOURCE_DIR}/port/getpeereid.c + + ${LIBPQ_SOURCE_DIR}/port/chklocale.c + ${LIBPQ_SOURCE_DIR}/port/dirmod.c + ${LIBPQ_SOURCE_DIR}/port/dlopen.c + ${LIBPQ_SOURCE_DIR}/port/erand48.c + ${LIBPQ_SOURCE_DIR}/port/explicit_bzero.c + ${LIBPQ_SOURCE_DIR}/port/fls.c + ${LIBPQ_SOURCE_DIR}/port/getopt.c + ${LIBPQ_SOURCE_DIR}/port/kill.c + ${LIBPQ_SOURCE_DIR}/port/link.c + ${LIBPQ_SOURCE_DIR}/port/noblock.c + ${LIBPQ_SOURCE_DIR}/port/open.c + ${LIBPQ_SOURCE_DIR}/port/path.c + ${LIBPQ_SOURCE_DIR}/port/pg_strong_random.c + ${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c + ${LIBPQ_SOURCE_DIR}/port/thread.c + + ${LIBPQ_SOURCE_DIR}/error/elog.c + ${LIBPQ_SOURCE_DIR}/error/errlog.c + ) + +add_library(libpq ${SRCS}) + +target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}/configs) + +target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}) +target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}/include) + +target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/include/common) +target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/include/libpq) + +target_link_libraries (libpq PRIVATE ssl) diff --git a/contrib/libpqxx b/contrib/libpqxx new file mode 160000 index 00000000000..aa8aa6cb48f --- /dev/null +++ b/contrib/libpqxx @@ -0,0 +1 @@ +Subproject commit aa8aa6cb48fcb8ccabc22a0df78f1250279cd233 diff --git a/contrib/libpqxx-cmake/CMakeLists.txt b/contrib/libpqxx-cmake/CMakeLists.txt new file mode 100644 index 00000000000..993927e9d3c --- /dev/null +++ b/contrib/libpqxx-cmake/CMakeLists.txt @@ -0,0 +1,33 @@ +set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libpqxx) + +set (SRCS + ${LIBRARY_DIR}/src/strconv.cxx + ${LIBRARY_DIR}/src/array.cxx + ${LIBRARY_DIR}/src/binarystring.cxx + ${LIBRARY_DIR}/src/connection.cxx + ${LIBRARY_DIR}/src/cursor.cxx + ${LIBRARY_DIR}/src/encodings.cxx + ${LIBRARY_DIR}/src/errorhandler.cxx + ${LIBRARY_DIR}/src/except.cxx + ${LIBRARY_DIR}/src/field.cxx + ${LIBRARY_DIR}/src/largeobject.cxx + ${LIBRARY_DIR}/src/notification.cxx + ${LIBRARY_DIR}/src/pipeline.cxx + ${LIBRARY_DIR}/src/result.cxx + ${LIBRARY_DIR}/src/robusttransaction.cxx + ${LIBRARY_DIR}/src/sql_cursor.cxx + ${LIBRARY_DIR}/src/stream_from.cxx + ${LIBRARY_DIR}/src/stream_to.cxx + ${LIBRARY_DIR}/src/subtransaction.cxx + ${LIBRARY_DIR}/src/transaction.cxx + ${LIBRARY_DIR}/src/transaction_base.cxx + ${LIBRARY_DIR}/src/row.cxx + ${LIBRARY_DIR}/src/util.cxx + ${LIBRARY_DIR}/src/version.cxx + ) + +add_library(libpqxx ${SRCS}) +target_link_libraries(libpqxx PRIVATE ${LIBPQ_LIBRARY}) + +target_include_directories (libpqxx SYSTEM BEFORE PRIVATE ${LIBRARY_DIR}/include) +target_include_directories (libpqxx PRIVATE ${LIBRARY_DIR}/include/pqxx) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4e04f5607df..0c2d0afe5bb 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -439,6 +439,11 @@ if (USE_ROCKSDB) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${ROCKSDB_INCLUDE_DIR}) endif() +if (USE_LIBPQXX) + dbms_target_link_libraries(PUBLIC ${LIBPQXX_LIBRARY}) + dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${LIBPQXX_INCLUDE_DIR}) +endif() + dbms_target_link_libraries(PRIVATE _boost_context) if (ENABLE_TESTS AND USE_GTEST) From 2f6cb7f2f5207d4e8feaf39cd489467857b5b55a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 21 Nov 2020 01:47:04 +0300 Subject: [PATCH 040/264] Add storage PostgreSQL with read support --- src/Access/AccessType.h | 1 + src/Core/ExternalResultDescription.cpp | 3 + src/Core/ExternalResultDescription.h | 3 +- src/Core/config_core.h.in | 2 +- src/Formats/MySQLBlockInputStream.cpp | 3 + src/Formats/PostgreSQLBlockInputStream.cpp | 262 +++++++++++++++++++++ src/Formats/PostgreSQLBlockInputStream.h | 44 ++++ src/Storages/StoragePostgreSQL.cpp | 137 +++++++++++ src/Storages/StoragePostgreSQL.h | 50 ++++ src/Storages/registerStorages.cpp | 7 + 10 files changed, 510 insertions(+), 2 deletions(-) create mode 100644 src/Formats/PostgreSQLBlockInputStream.cpp create mode 100644 src/Formats/PostgreSQLBlockInputStream.h create mode 100644 src/Storages/StoragePostgreSQL.cpp create mode 100644 src/Storages/StoragePostgreSQL.h diff --git a/src/Access/AccessType.h b/src/Access/AccessType.h index 1a070420fd1..5a84aa66739 100644 --- a/src/Access/AccessType.h +++ b/src/Access/AccessType.h @@ -159,6 +159,7 @@ enum class AccessType M(REMOTE, "", GLOBAL, SOURCES) \ M(MONGO, "", GLOBAL, SOURCES) \ M(MYSQL, "", GLOBAL, SOURCES) \ + M(POSTGRES, "", GLOBAL, SOURCES) \ M(ODBC, "", GLOBAL, SOURCES) \ M(JDBC, "", GLOBAL, SOURCES) \ M(HDFS, "", GLOBAL, SOURCES) \ diff --git a/src/Core/ExternalResultDescription.cpp b/src/Core/ExternalResultDescription.cpp index 7165d73b7d0..792e1c30eae 100644 --- a/src/Core/ExternalResultDescription.cpp +++ b/src/Core/ExternalResultDescription.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -76,6 +77,8 @@ void ExternalResultDescription::init(const Block & sample_block_) types.emplace_back(ValueType::vtDecimal128, is_nullable); else if (typeid_cast *>(type)) types.emplace_back(ValueType::vtDecimal256, is_nullable); + else if (typeid_cast(type)) + types.emplace_back(ValueType::vtArray, is_nullable); else throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE}; } diff --git a/src/Core/ExternalResultDescription.h b/src/Core/ExternalResultDescription.h index f8ba2a6bba2..31114746069 100644 --- a/src/Core/ExternalResultDescription.h +++ b/src/Core/ExternalResultDescription.h @@ -30,7 +30,8 @@ struct ExternalResultDescription vtDecimal32, vtDecimal64, vtDecimal128, - vtDecimal256 + vtDecimal256, + vtArray }; Block sample_block; diff --git a/src/Core/config_core.h.in b/src/Core/config_core.h.in index 5dbac2f69bf..6c7a35abd7c 100644 --- a/src/Core/config_core.h.in +++ b/src/Core/config_core.h.in @@ -12,4 +12,4 @@ #cmakedefine01 USE_OPENCL #cmakedefine01 USE_LDAP #cmakedefine01 USE_ROCKSDB - +#cmakedefine01 USE_LIBPQXX diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 2ff8e8e5fb2..f44c44da60d 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -22,6 +22,7 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; + extern const int NOT_IMPLEMENTED; } MySQLBlockInputStream::Connection::Connection( @@ -110,6 +111,8 @@ namespace data_type.deserializeAsWholeText(column, buffer, FormatSettings{}); break; } + default: + throw Exception("Unsupported value type", ErrorCodes::NOT_IMPLEMENTED); } } diff --git a/src/Formats/PostgreSQLBlockInputStream.cpp b/src/Formats/PostgreSQLBlockInputStream.cpp new file mode 100644 index 00000000000..33d9598eefc --- /dev/null +++ b/src/Formats/PostgreSQLBlockInputStream.cpp @@ -0,0 +1,262 @@ +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX +#include + +#include +#include +#include +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include "PostgreSQLBlockInputStream.h" +#include +#include + +namespace DB +{ + +PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( + std::shared_ptr connection_, + const std::string & query_str_, + const Block & sample_block, + const UInt64 max_block_size_) + : query_str(query_str_) + , max_block_size(max_block_size_) + , connection(connection_) + , work(std::make_unique(*connection)) + , stream(std::make_unique(*work, pqxx::from_query, std::string_view(query_str))) +{ + description.init(sample_block); +} + + +Block PostgreSQLBlockInputStream::readImpl() +{ + /// Check if pqxx::stream_from is finished + if (!stream || !(*stream)) + return Block(); + + MutableColumns columns = description.sample_block.cloneEmptyColumns(); + size_t num_rows = 0; + + while (true) + { + const std::vector * row{stream->read_row()}; + + if (!row) + { + stream->complete(); + break; + } + + if (row->empty()) + break; + + std::string value; + for (const auto idx : ext::range(0, row->size())) + { + value = std::string((*row)[idx]); + LOG_DEBUG((&Poco::Logger::get("PostgreSQL")), "GOT {}", value); + const auto & sample = description.sample_block.getByPosition(idx); + + if (value.data()) + { + if (description.types[idx].second) + { + ColumnNullable & column_nullable = assert_cast(*columns[idx]); + const auto & data_type = assert_cast(*sample.type); + insertValue(column_nullable.getNestedColumn(), value, description.types[idx].first, data_type.getNestedType()); + column_nullable.getNullMapData().emplace_back(0); + } + else + { + insertValue(*columns[idx], value, description.types[idx].first, sample.type); + } + } + else + { + insertDefaultValue(*columns[idx], *sample.column); + } + + } + + if (++num_rows == max_block_size) + break; + } + + return description.sample_block.cloneWithColumns(std::move(columns)); +} + + +void PostgreSQLBlockInputStream::insertValue(IColumn & column, const std::string & value, + const ExternalResultDescription::ValueType type, const DataTypePtr data_type) +{ + switch (type) + { + case ValueType::vtUInt8: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtUInt16: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtUInt32: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtUInt64: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtInt8: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtInt16: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtInt32: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtInt64: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtFloat32: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtFloat64: + assert_cast(column).insertValue(pqxx::from_string(value)); + break; + case ValueType::vtString: + assert_cast(column).insertData(value.data(), value.size()); + break; + case ValueType::vtDate: + //assert_cast(column).insertValue(UInt16(value.getDate().getDayNum())); + break; + case ValueType::vtDateTime: + //assert_cast(column).insertValue(UInt32(value.getDateTime())); + break; + case ValueType::vtUUID: + assert_cast(column).insert(parse(value.data(), value.size())); + break; + case ValueType::vtDateTime64:[[fallthrough]]; + case ValueType::vtDecimal32: [[fallthrough]]; + case ValueType::vtDecimal64: [[fallthrough]]; + case ValueType::vtDecimal128:[[fallthrough]]; + case ValueType::vtDecimal256: + { + ReadBuffer buffer(const_cast(value.data()), value.size(), 0); + data_type->deserializeAsWholeText(column, buffer, FormatSettings{}); + break; + } + case ValueType::vtArray: + { + const auto * array_type = typeid_cast(data_type.get()); + auto nested = array_type->getNestedType(); + + size_t expected_dimensions = 1; + while (isArray(nested)) + { + ++expected_dimensions; + nested = typeid_cast(nested.get())->getNestedType(); + } + auto which = WhichDataType(nested); + + auto get_array([&]() -> Field + { + pqxx::array_parser parser{value}; + std::pair parsed = parser.get_next(); + + std::vector> dimensions(expected_dimensions + 1); + size_t dimension = 0, max_dimension = 0; + bool new_row = false, null_value = false; + + while (parsed.first != pqxx::array_parser::juncture::done) + { + while (parsed.first == pqxx::array_parser::juncture::row_start) + { + ++dimension; + if (dimension > expected_dimensions) + throw Exception("Got more dimensions than expected", ErrorCodes::BAD_ARGUMENTS); + + parsed = parser.get_next(); + new_row = true; + } + + /// TODO: dont forget to add test with null type + std::vector current_dimension_row; + while (parsed.first != pqxx::array_parser::juncture::row_end) + { + if (parsed.first == pqxx::array_parser::juncture::null_value) + null_value = true; + + if (which.isUInt8() || which.isUInt16()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : UInt16()); + else if (which.isUInt32()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : UInt32()); + else if (which.isUInt64()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : UInt64()); + else if (which.isInt8() || which.isInt16()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Int16()); + else if (which.isInt32()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Int32()); + else if (which.isInt64()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Int64()); + //else if (which.isDate()) + //else if (which.isDateTime()) + else if (which.isFloat32()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Float32()); + else if (which.isFloat64()) + current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Float64()); + else if (which.isString() || which.isFixedString()) + current_dimension_row.emplace_back(!null_value ? parsed.second : String()); + else throw Exception("Unexpected type " + nested->getName(), ErrorCodes::BAD_ARGUMENTS); + + parsed = parser.get_next(); + null_value = false; + } + + while (parsed.first == pqxx::array_parser::juncture::row_end) + { + --dimension; + if (std::exchange(new_row, false)) + { + if (dimension + 1 > max_dimension) + max_dimension = dimension + 1; + if (dimension) + dimensions[dimension].emplace_back(Array(current_dimension_row.begin(), current_dimension_row.end())); + else + return Array(current_dimension_row.begin(), current_dimension_row.end()); + } + else if (dimension) + { + dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end())); + dimensions[dimension + 1].clear(); + } + parsed = parser.get_next(); + } + } + + if (max_dimension < expected_dimensions) + throw Exception("Got less dimensions than expected", ErrorCodes::BAD_ARGUMENTS); + + return Array(dimensions[1].begin(), dimensions[1].end()); + }); + + assert_cast(column).insert(get_array()); + break; + } + } +} + +} + +#endif diff --git a/src/Formats/PostgreSQLBlockInputStream.h b/src/Formats/PostgreSQLBlockInputStream.h new file mode 100644 index 00000000000..e8a84999d3a --- /dev/null +++ b/src/Formats/PostgreSQLBlockInputStream.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include +#include + +#include + +namespace DB +{ + +class PostgreSQLBlockInputStream : public IBlockInputStream +{ +public: + PostgreSQLBlockInputStream( + std::shared_ptr connection_, + const std::string & query_str, + const Block & sample_block, + const UInt64 max_block_size_); + + String getName() const override { return "PostgreSQL"; } + Block getHeader() const override { return description.sample_block.cloneEmpty(); } + +private: + using ValueType = ExternalResultDescription::ValueType; + + Block readImpl() override; + void insertValue(IColumn & column, const std::string & value, + const ExternalResultDescription::ValueType type, const DataTypePtr data_type); + void insertDefaultValue(IColumn & column, const IColumn & sample_column) + { + column.insertFrom(sample_column, 0); + } + + const String query_str; + const UInt64 max_block_size; + ExternalResultDescription description; + + std::shared_ptr connection; + std::unique_ptr work; + std::unique_ptr stream; +}; + +} diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp new file mode 100644 index 00000000000..4351b4c115b --- /dev/null +++ b/src/Storages/StoragePostgreSQL.cpp @@ -0,0 +1,137 @@ +#include "StoragePostgreSQL.h" + +#if USE_LIBPQXX + +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +StoragePostgreSQL::StoragePostgreSQL( + const StorageID & table_id_, + const String & remote_database_name_, + const String & remote_table_name_, + const String connection_str, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const Context & context_) + : IStorage(table_id_) + , remote_database_name(remote_database_name_) + , remote_table_name(remote_table_name_) + , global_context(context_) + , connection(std::make_shared(connection_str)) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); + setInMemoryMetadata(storage_metadata); +} + + +Pipe StoragePostgreSQL::read( + const Names & column_names_, + const StorageMetadataPtr & metadata_snapshot, + SelectQueryInfo & query_info_, + const Context & context_, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size_, + unsigned) +{ + metadata_snapshot->check(column_names_, getVirtuals(), getStorageID()); + + String query = transformQueryForExternalDatabase( + query_info_, + metadata_snapshot->getColumns().getOrdinary(), + IdentifierQuotingStyle::DoubleQuotes, + remote_database_name, + remote_table_name, + context_); + + Block sample_block; + for (const String & column_name : column_names_) + { + auto column_data = metadata_snapshot->getColumns().getPhysical(column_name); + WhichDataType which(column_data.type); + + if (which.isEnum()) + column_data.type = std::make_shared(); + + sample_block.insert({ column_data.type, column_data.name }); + } + + return Pipe(std::make_shared( + std::make_shared(connection, query, sample_block, max_block_size_))); +} + + +void registerStoragePostgreSQL(StorageFactory & factory) +{ + factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + + if (engine_args.size() < 5) + throw Exception( + "Storage PostgreSQL requires 5-7 parameters: PostgreSQL('host:port', database, table, 'username', 'password'.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context); + + auto parsed_host_port = parseAddress(engine_args[0]->as().value.safeGet(), 5432); + const String & remote_database = engine_args[1]->as().value.safeGet(); + const String & remote_table = engine_args[2]->as().value.safeGet(); + + String connection_str; + if (remote_database.empty()) + { + connection_str = fmt::format("host={} port={} user={} password={}", + parsed_host_port.first, std::to_string(parsed_host_port.second), + engine_args[3]->as().value.safeGet(), + engine_args[4]->as().value.safeGet()); + } + else + { + connection_str = fmt::format("dbname={} host={} port={} user={} password={}", + remote_database, parsed_host_port.first, std::to_string(parsed_host_port.second), + engine_args[3]->as().value.safeGet(), + engine_args[4]->as().value.safeGet()); + } + + return StoragePostgreSQL::create( + args.table_id, remote_database, remote_table, connection_str, args.columns, args.constraints, args.context); + }, + { + .source_access_type = AccessType::POSTGRES, + }); +} + +} + +#endif diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h new file mode 100644 index 00000000000..9108d9d341f --- /dev/null +++ b/src/Storages/StoragePostgreSQL.h @@ -0,0 +1,50 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX + +#include +#include +#include + +#include "pqxx/pqxx" + +namespace DB +{ +class StoragePostgreSQL final : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; +public: + StoragePostgreSQL( + const StorageID & table_id_, + const std::string & remote_database_name_, + const std::string & remote_table_name_, + const String connection_str, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const Context & context_); + + String getName() const override { return "PostgreSQL"; } + + Pipe read( + const Names & column_names, + const StorageMetadataPtr & /*metadata_snapshot*/, + SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +private: + String remote_database_name; + String remote_table_name; + Context global_context; + + std::shared_ptr connection; +}; +} + +#endif diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 33c1b6245ac..0022ee6bd4f 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -58,6 +58,9 @@ void registerStorageRabbitMQ(StorageFactory & factory); void registerStorageEmbeddedRocksDB(StorageFactory & factory); #endif +#if USE_LIBPQXX +void registerStoragePostgreSQL(StorageFactory & factory); +#endif void registerStorages() { @@ -111,6 +114,10 @@ void registerStorages() #if USE_ROCKSDB registerStorageEmbeddedRocksDB(factory); #endif + + #if USE_LIBPQXX + registerStoragePostgreSQL(factory); + #endif } } From ea817862babb31546b8b43f7bc5224a36a5fb256 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 22 Nov 2020 21:05:54 +0300 Subject: [PATCH 041/264] Better --- src/Formats/PostgreSQLBlockInputStream.cpp | 218 +++++++++++---------- src/Formats/PostgreSQLBlockInputStream.h | 13 +- 2 files changed, 123 insertions(+), 108 deletions(-) diff --git a/src/Formats/PostgreSQLBlockInputStream.cpp b/src/Formats/PostgreSQLBlockInputStream.cpp index 33d9598eefc..cf722ea99ec 100644 --- a/src/Formats/PostgreSQLBlockInputStream.cpp +++ b/src/Formats/PostgreSQLBlockInputStream.cpp @@ -17,16 +17,25 @@ #include #include +#include #include #include -#include "PostgreSQLBlockInputStream.h" #include #include +#include "PostgreSQLBlockInputStream.h" + namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TYPE; +} + + PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( std::shared_ptr connection_, const std::string & query_str_, @@ -35,13 +44,18 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( : query_str(query_str_) , max_block_size(max_block_size_) , connection(connection_) - , work(std::make_unique(*connection)) - , stream(std::make_unique(*work, pqxx::from_query, std::string_view(query_str))) { description.init(sample_block); } +void PostgreSQLBlockInputStream::readPrefix() +{ + work = std::make_unique(*connection); + stream = std::make_unique(*work, pqxx::from_query, std::string_view(query_str)); +} + + Block PostgreSQLBlockInputStream::readImpl() { /// Check if pqxx::stream_from is finished @@ -50,6 +64,7 @@ Block PostgreSQLBlockInputStream::readImpl() MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; + std::string value; while (true) { @@ -57,32 +72,32 @@ Block PostgreSQLBlockInputStream::readImpl() if (!row) { + /// row is nullptr if pqxx::stream_from is finished stream->complete(); + work->commit(); break; } - if (row->empty()) - break; - - std::string value; for (const auto idx : ext::range(0, row->size())) { - value = std::string((*row)[idx]); - LOG_DEBUG((&Poco::Logger::get("PostgreSQL")), "GOT {}", value); const auto & sample = description.sample_block.getByPosition(idx); + if (!num_rows && description.types[idx].first == ValueType::vtArray) + getArrayInfo(idx, sample.type); - if (value.data()) + /// if got NULL type, then pqxx::zview will return nullptr in c_str() + if ((*row)[idx].c_str()) { + value = std::string((*row)[idx]); if (description.types[idx].second) { ColumnNullable & column_nullable = assert_cast(*columns[idx]); const auto & data_type = assert_cast(*sample.type); - insertValue(column_nullable.getNestedColumn(), value, description.types[idx].first, data_type.getNestedType()); + insertValue(column_nullable.getNestedColumn(), value, description.types[idx].first, data_type.getNestedType(), idx); column_nullable.getNullMapData().emplace_back(0); } else { - insertValue(*columns[idx], value, description.types[idx].first, sample.type); + insertValue(*columns[idx], value, description.types[idx].first, sample.type, idx); } } else @@ -101,7 +116,7 @@ Block PostgreSQLBlockInputStream::readImpl() void PostgreSQLBlockInputStream::insertValue(IColumn & column, const std::string & value, - const ExternalResultDescription::ValueType type, const DataTypePtr data_type) + const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx) { switch (type) { @@ -138,125 +153,114 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, const std::string case ValueType::vtString: assert_cast(column).insertData(value.data(), value.size()); break; - case ValueType::vtDate: - //assert_cast(column).insertValue(UInt16(value.getDate().getDayNum())); - break; - case ValueType::vtDateTime: - //assert_cast(column).insertValue(UInt32(value.getDateTime())); - break; case ValueType::vtUUID: assert_cast(column).insert(parse(value.data(), value.size())); break; - case ValueType::vtDateTime64:[[fallthrough]]; - case ValueType::vtDecimal32: [[fallthrough]]; - case ValueType::vtDecimal64: [[fallthrough]]; - case ValueType::vtDecimal128:[[fallthrough]]; - case ValueType::vtDecimal256: + case ValueType::vtDate: { - ReadBuffer buffer(const_cast(value.data()), value.size(), 0); - data_type->deserializeAsWholeText(column, buffer, FormatSettings{}); + ReadBufferFromString istr(value); + data_type->deserializeAsWholeText(column, istr, FormatSettings{}); + break; + } + case ValueType::vtDateTime: + { + ReadBufferFromString istr(value); + data_type->deserializeAsWholeText(column, istr, FormatSettings{}); break; } case ValueType::vtArray: { - const auto * array_type = typeid_cast(data_type.get()); - auto nested = array_type->getNestedType(); + pqxx::array_parser parser{value}; + std::pair parsed = parser.get_next(); - size_t expected_dimensions = 1; - while (isArray(nested)) + size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions; + const auto parse_value = array_info[idx].pqxx_parser; + std::vector> dimensions(expected_dimensions + 1); + + while (parsed.first != pqxx::array_parser::juncture::done) { - ++expected_dimensions; - nested = typeid_cast(nested.get())->getNestedType(); - } - auto which = WhichDataType(nested); + if ((parsed.first == pqxx::array_parser::juncture::row_start) && (++dimension > expected_dimensions)) + throw Exception("Got more dimensions than expected", ErrorCodes::BAD_ARGUMENTS); - auto get_array([&]() -> Field - { - pqxx::array_parser parser{value}; - std::pair parsed = parser.get_next(); + else if (parsed.first == pqxx::array_parser::juncture::string_value) + dimensions[dimension].emplace_back(parse_value(parsed.second)); - std::vector> dimensions(expected_dimensions + 1); - size_t dimension = 0, max_dimension = 0; - bool new_row = false, null_value = false; + else if (parsed.first == pqxx::array_parser::juncture::null_value) + dimensions[dimension].emplace_back(array_info[idx].default_value); - while (parsed.first != pqxx::array_parser::juncture::done) + else if (parsed.first == pqxx::array_parser::juncture::row_end) { - while (parsed.first == pqxx::array_parser::juncture::row_start) - { - ++dimension; - if (dimension > expected_dimensions) - throw Exception("Got more dimensions than expected", ErrorCodes::BAD_ARGUMENTS); + max_dimension = std::max(max_dimension, dimension); - parsed = parser.get_next(); - new_row = true; - } + if (--dimension == 0) + break; - /// TODO: dont forget to add test with null type - std::vector current_dimension_row; - while (parsed.first != pqxx::array_parser::juncture::row_end) - { - if (parsed.first == pqxx::array_parser::juncture::null_value) - null_value = true; - - if (which.isUInt8() || which.isUInt16()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : UInt16()); - else if (which.isUInt32()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : UInt32()); - else if (which.isUInt64()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : UInt64()); - else if (which.isInt8() || which.isInt16()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Int16()); - else if (which.isInt32()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Int32()); - else if (which.isInt64()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Int64()); - //else if (which.isDate()) - //else if (which.isDateTime()) - else if (which.isFloat32()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Float32()); - else if (which.isFloat64()) - current_dimension_row.emplace_back(!null_value ? pqxx::from_string(parsed.second) : Float64()); - else if (which.isString() || which.isFixedString()) - current_dimension_row.emplace_back(!null_value ? parsed.second : String()); - else throw Exception("Unexpected type " + nested->getName(), ErrorCodes::BAD_ARGUMENTS); - - parsed = parser.get_next(); - null_value = false; - } - - while (parsed.first == pqxx::array_parser::juncture::row_end) - { - --dimension; - if (std::exchange(new_row, false)) - { - if (dimension + 1 > max_dimension) - max_dimension = dimension + 1; - if (dimension) - dimensions[dimension].emplace_back(Array(current_dimension_row.begin(), current_dimension_row.end())); - else - return Array(current_dimension_row.begin(), current_dimension_row.end()); - } - else if (dimension) - { - dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end())); - dimensions[dimension + 1].clear(); - } - parsed = parser.get_next(); - } + dimensions[dimension].emplace_back(Array(dimensions[dimension + 1].begin(), dimensions[dimension + 1].end())); + dimensions[dimension + 1].clear(); } - if (max_dimension < expected_dimensions) - throw Exception("Got less dimensions than expected", ErrorCodes::BAD_ARGUMENTS); + parsed = parser.get_next(); + } - return Array(dimensions[1].begin(), dimensions[1].end()); - }); + if (max_dimension < expected_dimensions) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Got less dimensions than expected. ({} instead of {})", max_dimension, expected_dimensions); - assert_cast(column).insert(get_array()); + assert_cast(column).insert(Array(dimensions[1].begin(), dimensions[1].end())); break; } + default: + throw Exception("Value of unsupported type:" + column.getName(), ErrorCodes::UNKNOWN_TYPE); } } + +void PostgreSQLBlockInputStream::getArrayInfo(size_t column_idx, const DataTypePtr data_type) +{ + const auto * array_type = typeid_cast(data_type.get()); + auto nested = array_type->getNestedType(); + + size_t count_dimensions = 1; + while (isArray(nested)) + { + ++count_dimensions; + nested = typeid_cast(nested.get())->getNestedType(); + } + + Field default_value = nested->getDefault(); + if (nested->isNullable()) + nested = typeid_cast(nested.get())->getNestedType(); + + WhichDataType which(nested); + std::function parser; + + if (which.isUInt8() || which.isUInt16()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUInt32()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUInt64()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isInt8() || which.isInt16()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isInt32()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isInt64()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isFloat32()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isFloat64()) + parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isString() || which.isFixedString()) + parser = [&](std::string & field) -> Field { return field; }; + else if (which.isDate()) + parser = [&](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; }; + else if (which.isDateTime()) + parser = [&](std::string & field) -> Field { return time_t{LocalDateTime{field}}; }; + else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type {} for pgArray", nested->getName()); + + array_info[column_idx] = {count_dimensions, default_value, parser}; +} + } #endif diff --git a/src/Formats/PostgreSQLBlockInputStream.h b/src/Formats/PostgreSQLBlockInputStream.h index e8a84999d3a..a4b5ace93f9 100644 --- a/src/Formats/PostgreSQLBlockInputStream.h +++ b/src/Formats/PostgreSQLBlockInputStream.h @@ -24,13 +24,16 @@ public: private: using ValueType = ExternalResultDescription::ValueType; + void readPrefix() override; Block readImpl() override; + void insertValue(IColumn & column, const std::string & value, - const ExternalResultDescription::ValueType type, const DataTypePtr data_type); + const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx); void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); } + void getArrayInfo(size_t column_idx, const DataTypePtr data_type); const String query_str; const UInt64 max_block_size; @@ -39,6 +42,14 @@ private: std::shared_ptr connection; std::unique_ptr work; std::unique_ptr stream; + + struct ArrayInfo + { + size_t num_dimensions; + Field default_value; + std::function pqxx_parser; + }; + std::unordered_map array_info; }; } From 42023f4b959b208681c36784c859836714ced566 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 25 Nov 2020 17:52:11 +0300 Subject: [PATCH 042/264] Support insert into table --- .../PostgreSQLBlockInputStream.cpp | 0 .../PostgreSQLBlockInputStream.h | 0 src/Storages/StoragePostgreSQL.cpp | 69 ++++++++++++++++++- src/Storages/StoragePostgreSQL.h | 41 +++++++++++ 4 files changed, 108 insertions(+), 2 deletions(-) rename src/{Formats => DataStreams}/PostgreSQLBlockInputStream.cpp (100%) rename src/{Formats => DataStreams}/PostgreSQLBlockInputStream.h (100%) diff --git a/src/Formats/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp similarity index 100% rename from src/Formats/PostgreSQLBlockInputStream.cpp rename to src/DataStreams/PostgreSQLBlockInputStream.cpp diff --git a/src/Formats/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h similarity index 100% rename from src/Formats/PostgreSQLBlockInputStream.h rename to src/DataStreams/PostgreSQLBlockInputStream.h diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 4351b4c115b..98d36afb4f8 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -9,14 +9,16 @@ #include #include -#include +#include #include #include +#include #include +#include #include -#include +#include #include #include @@ -90,6 +92,69 @@ Pipe StoragePostgreSQL::read( } +BlockOutputStreamPtr StoragePostgreSQL::write( + const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */) +{ + return std::make_shared( + *this, metadata_snapshot, connection, remote_database_name, remote_table_name); +} + + +void PostgreSQLBlockOutputStream::writePrefix() +{ + work = std::make_unique(*connection); + stream_inserter = std::make_unique(*work, remote_table_name); +} + + +void PostgreSQLBlockOutputStream::write(const Block & block) +{ + const auto columns = block.getColumns(); + const size_t num_rows = block.rows(), num_cols = block.columns(); + const auto data_types = block.getDataTypes(); + const auto settings = FormatSettings{}; + + /// std::optional lets libpqxx to know if value is NULL + std::vector> row(num_cols); + + for (const auto i : ext::range(0, num_rows)) + { + for (const auto j : ext::range(0, num_cols)) + { + if (columns[j]->isNullAt(i)) + { + row[j] = std::nullopt; + } + else + { + WriteBufferFromOwnString ostr; + data_types[j]->serializeAsText(*columns[j], i, ostr, settings); + row[j] = std::optional(ostr.str()); + + if (isArray(data_types[j])) + { + char r; + std::replace_if(row[j]->begin(), row[j]->end(), [&](char c) + { + return ((c == '[') && (r = '{')) || ((c == ']') && (r = '}')); + }, r); + } + } + } + + /// pqxx::stream_to uses COPY instead of insert query, so it is faster if inserting large number of rows + stream_inserter->write_values(row); + } +} + + +void PostgreSQLBlockOutputStream::writeSuffix() +{ + stream_inserter->complete(); + work->commit(); +} + + void registerStoragePostgreSQL(StorageFactory & factory) { factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args) diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 9108d9d341f..3b032574bb5 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -10,6 +10,7 @@ #include #include +#include #include "pqxx/pqxx" namespace DB @@ -38,13 +39,53 @@ public: size_t max_block_size, unsigned num_streams) override; + BlockOutputStreamPtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, const Context & context) override; + private: + friend class PostgreSQLBlockOutputStream; + String remote_database_name; String remote_table_name; Context global_context; std::shared_ptr connection; }; + + +class PostgreSQLBlockOutputStream : public IBlockOutputStream +{ +public: + explicit PostgreSQLBlockOutputStream( + const StoragePostgreSQL & storage_, + const StorageMetadataPtr & metadata_snapshot_, + std::shared_ptr connection_, + const std::string & remote_database_name_, + const std::string & remote_table_name_) + : storage(storage_) + , metadata_snapshot(metadata_snapshot_) + , connection(connection_) + , remote_database_name(remote_database_name_) + , remote_table_name(remote_table_name_) + { + } + + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } + + void writePrefix() override; + void write(const Block & block) override; + void writeSuffix() override; + + const StoragePostgreSQL & storage; + +private: + StorageMetadataPtr metadata_snapshot; + std::shared_ptr connection; + std::string remote_database_name; + std::string remote_table_name; + + std::unique_ptr work; + std::unique_ptr stream_inserter; +}; } #endif From 69f6714461d03dcaa27e54e2974246ae4d7ce52d Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 26 Nov 2020 00:06:33 +0300 Subject: [PATCH 043/264] Add table function --- .../PostgreSQLBlockInputStream.cpp | 104 ++++++----- src/DataStreams/PostgreSQLBlockInputStream.h | 20 +- src/Storages/StoragePostgreSQL.cpp | 67 +++---- src/Storages/StoragePostgreSQL.h | 22 +-- .../TableFunctionPostgreSQL.cpp | 173 ++++++++++++++++++ src/TableFunctions/TableFunctionPostgreSQL.h | 38 ++++ src/TableFunctions/registerTableFunctions.cpp | 4 + src/TableFunctions/registerTableFunctions.h | 5 + 8 files changed, 330 insertions(+), 103 deletions(-) create mode 100644 src/TableFunctions/TableFunctionPostgreSQL.cpp create mode 100644 src/TableFunctions/TableFunctionPostgreSQL.h diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index cf722ea99ec..afa6d40f3de 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -1,30 +1,22 @@ -#if !defined(ARCADIA_BUILD) -#include "config_core.h" -#endif +#include "PostgreSQLBlockInputStream.h" #if USE_LIBPQXX -#include - #include #include #include #include #include - #include #include #include - +#include +#include #include #include #include - #include #include #include -#include - -#include "PostgreSQLBlockInputStream.h" namespace DB { @@ -37,7 +29,7 @@ namespace ErrorCodes PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( - std::shared_ptr connection_, + ConnectionPtr connection_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_) @@ -51,8 +43,8 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( void PostgreSQLBlockInputStream::readPrefix() { - work = std::make_unique(*connection); - stream = std::make_unique(*work, pqxx::from_query, std::string_view(query_str)); + tx = std::make_unique(*connection); + stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); } @@ -74,7 +66,7 @@ Block PostgreSQLBlockInputStream::readImpl() { /// row is nullptr if pqxx::stream_from is finished stream->complete(); - work->commit(); + tx->commit(); break; } @@ -82,22 +74,21 @@ Block PostgreSQLBlockInputStream::readImpl() { const auto & sample = description.sample_block.getByPosition(idx); if (!num_rows && description.types[idx].first == ValueType::vtArray) - getArrayInfo(idx, sample.type); + prepareArrayParser(idx, sample.type); /// if got NULL type, then pqxx::zview will return nullptr in c_str() if ((*row)[idx].c_str()) { - value = std::string((*row)[idx]); if (description.types[idx].second) { ColumnNullable & column_nullable = assert_cast(*columns[idx]); const auto & data_type = assert_cast(*sample.type); - insertValue(column_nullable.getNestedColumn(), value, description.types[idx].first, data_type.getNestedType(), idx); + insertValue(column_nullable.getNestedColumn(), (*row)[idx], description.types[idx].first, data_type.getNestedType(), idx); column_nullable.getNullMapData().emplace_back(0); } else { - insertValue(*columns[idx], value, description.types[idx].first, sample.type, idx); + insertValue(*columns[idx], (*row)[idx], description.types[idx].first, sample.type, idx); } } else @@ -115,7 +106,7 @@ Block PostgreSQLBlockInputStream::readImpl() } -void PostgreSQLBlockInputStream::insertValue(IColumn & column, const std::string & value, +void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view value, const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx) { switch (type) @@ -156,18 +147,6 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, const std::string case ValueType::vtUUID: assert_cast(column).insert(parse(value.data(), value.size())); break; - case ValueType::vtDate: - { - ReadBufferFromString istr(value); - data_type->deserializeAsWholeText(column, istr, FormatSettings{}); - break; - } - case ValueType::vtDateTime: - { - ReadBufferFromString istr(value); - data_type->deserializeAsWholeText(column, istr, FormatSettings{}); - break; - } case ValueType::vtArray: { pqxx::array_parser parser{value}; @@ -209,13 +188,24 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, const std::string assert_cast(column).insert(Array(dimensions[1].begin(), dimensions[1].end())); break; } + case ValueType::vtDate: [[fallthrough]]; + case ValueType::vtDateTime: [[fallthrough]]; + case ValueType::vtDateTime64:[[fallthrough]]; + case ValueType::vtDecimal32: [[fallthrough]]; + case ValueType::vtDecimal64: [[fallthrough]]; + case ValueType::vtDecimal128: + { + ReadBufferFromString istr(value); + data_type->deserializeAsWholeText(column, istr, FormatSettings{}); + break; + } default: - throw Exception("Value of unsupported type:" + column.getName(), ErrorCodes::UNKNOWN_TYPE); + throw Exception("Value of unsupported type:" + column.getName(), ErrorCodes::UNKNOWN_TYPE); } } -void PostgreSQLBlockInputStream::getArrayInfo(size_t column_idx, const DataTypePtr data_type) +void PostgreSQLBlockInputStream::prepareArrayParser(size_t column_idx, const DataTypePtr data_type) { const auto * array_type = typeid_cast(data_type.get()); auto nested = array_type->getNestedType(); @@ -235,28 +225,50 @@ void PostgreSQLBlockInputStream::getArrayInfo(size_t column_idx, const DataTypeP std::function parser; if (which.isUInt8() || which.isUInt16()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isUInt32()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isUInt64()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isInt8() || which.isInt16()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isInt32()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isInt64()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isFloat32()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isFloat64()) - parser = [&](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isString() || which.isFixedString()) - parser = [&](std::string & field) -> Field { return field; }; + parser = [](std::string & field) -> Field { return field; }; else if (which.isDate()) - parser = [&](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; }; + parser = [](std::string & field) -> Field { return UInt16{LocalDate{field}.getDayNum()}; }; else if (which.isDateTime()) - parser = [&](std::string & field) -> Field { return time_t{LocalDateTime{field}}; }; - else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type {} for pgArray", nested->getName()); + parser = [](std::string & field) -> Field { return time_t{LocalDateTime{field}}; }; + else if (which.isDecimal32()) + parser = [nested](std::string & field) -> Field + { + auto type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; + else if (which.isDecimal64()) + parser = [nested](std::string & field) -> Field + { + auto type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; + else if (which.isDecimal128()) + parser = [nested](std::string & field) -> Field + { + auto type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName()); array_info[column_idx] = {count_dimensions, default_value, parser}; } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index a4b5ace93f9..2820845e4b5 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -1,19 +1,25 @@ #pragma once +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX #include #include #include - +#include #include namespace DB { +using ConnectionPtr = std::shared_ptr; class PostgreSQLBlockInputStream : public IBlockInputStream { public: PostgreSQLBlockInputStream( - std::shared_ptr connection_, + ConnectionPtr connection_, const std::string & query_str, const Block & sample_block, const UInt64 max_block_size_); @@ -27,20 +33,20 @@ private: void readPrefix() override; Block readImpl() override; - void insertValue(IColumn & column, const std::string & value, + void insertValue(IColumn & column, std::string_view value, const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx); void insertDefaultValue(IColumn & column, const IColumn & sample_column) { column.insertFrom(sample_column, 0); } - void getArrayInfo(size_t column_idx, const DataTypePtr data_type); + void prepareArrayParser(size_t column_idx, const DataTypePtr data_type); const String query_str; const UInt64 max_block_size; ExternalResultDescription description; - std::shared_ptr connection; - std::unique_ptr work; + ConnectionPtr connection; + std::unique_ptr tx; std::unique_ptr stream; struct ArrayInfo @@ -53,3 +59,5 @@ private: }; } + +#endif diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 98d36afb4f8..91a6b542b8b 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -4,27 +4,20 @@ #include #include - #include #include - #include #include - #include #include #include #include #include - #include #include - -#include -#include - #include #include +#include namespace DB { @@ -37,17 +30,17 @@ namespace ErrorCodes StoragePostgreSQL::StoragePostgreSQL( const StorageID & table_id_, - const String & remote_database_name_, const String & remote_table_name_, - const String connection_str, + ConnectionPtr connection_, + const String connection_str_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) : IStorage(table_id_) - , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) , global_context(context_) - , connection(std::make_shared(connection_str)) + , connection(std::move(connection_)) + , connection_str(connection_str_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -68,25 +61,20 @@ Pipe StoragePostgreSQL::read( metadata_snapshot->check(column_names_, getVirtuals(), getStorageID()); String query = transformQueryForExternalDatabase( - query_info_, - metadata_snapshot->getColumns().getOrdinary(), - IdentifierQuotingStyle::DoubleQuotes, - remote_database_name, - remote_table_name, - context_); + query_info_, metadata_snapshot->getColumns().getOrdinary(), + IdentifierQuotingStyle::DoubleQuotes, "", remote_table_name, context_); Block sample_block; for (const String & column_name : column_names_) { auto column_data = metadata_snapshot->getColumns().getPhysical(column_name); WhichDataType which(column_data.type); - if (which.isEnum()) column_data.type = std::make_shared(); - sample_block.insert({ column_data.type, column_data.name }); } + checkConnection(connection); return Pipe(std::make_shared( std::make_shared(connection, query, sample_block, max_block_size_))); } @@ -95,13 +83,13 @@ Pipe StoragePostgreSQL::read( BlockOutputStreamPtr StoragePostgreSQL::write( const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */) { - return std::make_shared( - *this, metadata_snapshot, connection, remote_database_name, remote_table_name); + return std::make_shared(*this, metadata_snapshot, connection, remote_table_name); } void PostgreSQLBlockOutputStream::writePrefix() { + storage.checkConnection(connection); work = std::make_unique(*connection); stream_inserter = std::make_unique(*work, remote_table_name); } @@ -155,6 +143,16 @@ void PostgreSQLBlockOutputStream::writeSuffix() } +void StoragePostgreSQL::checkConnection(ConnectionPtr & pg_connection) const +{ + if (!pg_connection->is_open()) + { + pg_connection->close(); + pg_connection = std::make_shared(connection_str); + } +} + + void registerStoragePostgreSQL(StorageFactory & factory) { factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args) @@ -163,34 +161,25 @@ void registerStoragePostgreSQL(StorageFactory & factory) if (engine_args.size() < 5) throw Exception( - "Storage PostgreSQL requires 5-7 parameters: PostgreSQL('host:port', database, table, 'username', 'password'.", + "Storage PostgreSQL requires 5-7 parameters: PostgreSQL('host:port', 'database', 'table', 'username', 'password'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (auto & engine_arg : engine_args) engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.local_context); auto parsed_host_port = parseAddress(engine_args[0]->as().value.safeGet(), 5432); - const String & remote_database = engine_args[1]->as().value.safeGet(); const String & remote_table = engine_args[2]->as().value.safeGet(); String connection_str; - if (remote_database.empty()) - { - connection_str = fmt::format("host={} port={} user={} password={}", - parsed_host_port.first, std::to_string(parsed_host_port.second), - engine_args[3]->as().value.safeGet(), - engine_args[4]->as().value.safeGet()); - } - else - { - connection_str = fmt::format("dbname={} host={} port={} user={} password={}", - remote_database, parsed_host_port.first, std::to_string(parsed_host_port.second), - engine_args[3]->as().value.safeGet(), - engine_args[4]->as().value.safeGet()); - } + connection_str = fmt::format("dbname={} host={} port={} user={} password={}", + engine_args[1]->as().value.safeGet(), + parsed_host_port.first, std::to_string(parsed_host_port.second), + engine_args[3]->as().value.safeGet(), + engine_args[4]->as().value.safeGet()); + auto connection = std::make_shared(connection_str); return StoragePostgreSQL::create( - args.table_id, remote_database, remote_table, connection_str, args.columns, args.constraints, args.context); + args.table_id, remote_table, connection, connection_str, args.columns, args.constraints, args.context); }, { .source_access_type = AccessType::POSTGRES, diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 3b032574bb5..af061f79545 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -5,24 +5,25 @@ #endif #if USE_LIBPQXX - #include #include #include - #include #include "pqxx/pqxx" namespace DB { + +using ConnectionPtr = std::shared_ptr; + class StoragePostgreSQL final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; public: StoragePostgreSQL( const StorageID & table_id_, - const std::string & remote_database_name_, const std::string & remote_table_name_, + ConnectionPtr connection_, const String connection_str, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, @@ -43,12 +44,13 @@ public: private: friend class PostgreSQLBlockOutputStream; + void checkConnection(ConnectionPtr & connection) const; - String remote_database_name; String remote_table_name; Context global_context; - std::shared_ptr connection; + ConnectionPtr connection; + const String connection_str; }; @@ -58,13 +60,11 @@ public: explicit PostgreSQLBlockOutputStream( const StoragePostgreSQL & storage_, const StorageMetadataPtr & metadata_snapshot_, - std::shared_ptr connection_, - const std::string & remote_database_name_, + ConnectionPtr connection_, const std::string & remote_table_name_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) , connection(connection_) - , remote_database_name(remote_database_name_) , remote_table_name(remote_table_name_) { } @@ -75,12 +75,10 @@ public: void write(const Block & block) override; void writeSuffix() override; - const StoragePostgreSQL & storage; - private: + const StoragePostgreSQL & storage; StorageMetadataPtr metadata_snapshot; - std::shared_ptr connection; - std::string remote_database_name; + ConnectionPtr connection; std::string remote_table_name; std::unique_ptr work; diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp new file mode 100644 index 00000000000..388560c028b --- /dev/null +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -0,0 +1,173 @@ +#include + +#if USE_LIBPQXX +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerTableFunctions.h" +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_TYPE; +} + + +StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, + const Context & context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +{ + auto columns = getActualTableStructure(context); + auto result = std::make_shared( + StorageID(getDatabaseName(), table_name), remote_table_name, + connection, connection_str, columns, ConstraintsDescription{}, context); + + result->startup(); + return result; +} + + +ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(const Context & context) const +{ + const bool use_nulls = context.getSettingsRef().external_table_functions_use_nulls; + auto columns = NamesAndTypesList(); + + std::string query = fmt::format( + "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " + "attnotnull AS not_null, attndims AS dims " + "FROM pg_attribute " + "WHERE attrelid = '{}'::regclass " + "AND NOT attisdropped AND attnum > 0", remote_table_name); + pqxx::read_transaction tx(*connection); + pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); + std::tuple row; + + while (stream >> row) + { + columns.push_back(NameAndTypePair( + std::get<0>(row), + getDataType(std::get<1>(row), use_nulls && (std::get<2>(row) == "f"), std::get<3>(row)))); + } + stream.complete(); + tx.commit(); + + return ColumnsDescription{columns}; +} + + +DataTypePtr TableFunctionPostgreSQL::getDataType(std::string & type, bool is_nullable, uint16_t dimensions) const +{ + DataTypePtr res; + + if (dimensions) + { + /// No matter how many dimensions, in type we will get only one '[]' (i.e. Integer[]) + type.resize(type.size() - 2); + } + + if (type == "smallint") + res = std::make_shared(); + else if (type == "integer") + res = std::make_shared(); + else if (type == "bigint") + res = std::make_shared(); + else if (type == "real") + res = std::make_shared(); + else if (type == "double precision") + res = std::make_shared(); + else if (type == "serial") + res = std::make_shared(); + else if (type == "bigserial") + res = std::make_shared(); + else if (type.starts_with("timestamp")) + res = std::make_shared(); + else if (type == "date") + res = std::make_shared(); + else if (type.starts_with("numeric")) + { + /// Numeric and decimal will both be numeric + /// Will get numeric(precision, scale) string, need to extract precision and scale + std::vector result; + boost::split(result, type, [](char c){ return c == '(' || c == ',' || c == ')'; }); + for (std::string & key : result) + boost::trim(key); + + /// If precision or scale are not specified, postgres creates a column in which numeric values of + /// any precision and scale can be stored, so may be maxPrecision may be used instead of exception + if (result.size() < 3) + throw Exception("Numeric lacks precision and scale in its definition", ErrorCodes::UNKNOWN_TYPE); + + uint32_t precision = std::atoi(result[1].data()); + uint32_t scale = std::atoi(result[2].data()); + + if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + } + + if (!res) + res = std::make_shared(); + if (is_nullable) + res = std::make_shared(res); + while (dimensions--) + res = std::make_shared(res); + + return res; +} + + +void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, const Context & context) +{ + const auto & func_args = ast_function->as(); + + if (!func_args.arguments) + throw Exception("Table function 'PostgreSQL' must have arguments.", ErrorCodes::BAD_ARGUMENTS); + + ASTs & args = func_args.arguments->children; + + if (args.size() < 5) + throw Exception("Table function 'PostgreSQL' requires 5 parameters: " + "PostgreSQL('host:port', 'database', 'table', 'user', 'password').", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + auto parsed_host_port = parseAddress(args[0]->as().value.safeGet(), 5432); + remote_table_name = args[2]->as().value.safeGet(); + + connection_str = fmt::format("dbname={} host={} port={} user={} password={}", + args[1]->as().value.safeGet(), + parsed_host_port.first, std::to_string(parsed_host_port.second), + args[3]->as().value.safeGet(), + args[4]->as().value.safeGet()); + connection = std::make_shared(connection_str); +} + + +void registerTableFunctionPostgreSQL(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + +#endif diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h new file mode 100644 index 00000000000..e1cd3c77195 --- /dev/null +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -0,0 +1,38 @@ +#pragma once +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX +#include +#include "pqxx/pqxx" + +namespace DB +{ + +class TableFunctionPostgreSQL : public ITableFunction +{ +public: + static constexpr auto name = "postgresql"; + std::string getName() const override { return name; } + +private: + StoragePtr executeImpl( + const ASTPtr & ast_function, const Context & context, + const std::string & table_name, ColumnsDescription cached_columns) const override; + + const char * getStorageTypeName() const override { return "PostgreSQL"; } + + ColumnsDescription getActualTableStructure(const Context & context) const override; + void parseArguments(const ASTPtr & ast_function, const Context & context) override; + + DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimensions) const; + + String connection_str; + String remote_table_name; + std::shared_ptr connection; +}; + +} + +#endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 09255c2bd74..a6640bbb0e9 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -36,6 +36,10 @@ void registerTableFunctions() #if USE_MYSQL registerTableFunctionMySQL(factory); #endif + +#if USE_LIBPQXX + registerTableFunctionPostgreSQL(factory); +#endif } } diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index ab05187eeab..0d8438a8ad8 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -37,6 +37,11 @@ void registerTableFunctionView(TableFunctionFactory & factory); void registerTableFunctionMySQL(TableFunctionFactory & factory); #endif +#if USE_LIBPQXX +void registerTableFunctionPostgreSQL(TableFunctionFactory & factory); +#endif + + void registerTableFunctions(); From aa3484515d2b44e5bc9d57ee3ec382883ef2232a Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 9 Dec 2020 00:40:18 +0300 Subject: [PATCH 044/264] Better --- .../PostgreSQLBlockInputStream.cpp | 40 ++++++++++--------- src/DataStreams/PostgreSQLBlockInputStream.h | 2 +- src/Storages/StoragePostgreSQL.cpp | 18 +++++---- .../TableFunctionPostgreSQL.cpp | 8 ++-- src/TableFunctions/TableFunctionPostgreSQL.h | 3 +- 5 files changed, 38 insertions(+), 33 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index afa6d40f3de..bba46983081 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -38,6 +38,9 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( , connection(connection_) { description.init(sample_block); + for (const auto idx : ext::range(0, description.sample_block.columns())) + if (description.types[idx].first == ValueType::vtArray) + prepareArrayInfo(idx, description.sample_block.getByPosition(idx).type); } @@ -56,7 +59,6 @@ Block PostgreSQLBlockInputStream::readImpl() MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; - std::string value; while (true) { @@ -73,8 +75,6 @@ Block PostgreSQLBlockInputStream::readImpl() for (const auto idx : ext::range(0, row->size())) { const auto & sample = description.sample_block.getByPosition(idx); - if (!num_rows && description.types[idx].first == ValueType::vtArray) - prepareArrayParser(idx, sample.type); /// if got NULL type, then pqxx::zview will return nullptr in c_str() if ((*row)[idx].c_str()) @@ -147,6 +147,21 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view case ValueType::vtUUID: assert_cast(column).insert(parse(value.data(), value.size())); break; + case ValueType::vtDate: + assert_cast(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()}); + break; + case ValueType::vtDateTime: + assert_cast(column).insertValue(time_t{LocalDateTime{std::string(value)}}); + break; + case ValueType::vtDateTime64:[[fallthrough]]; + case ValueType::vtDecimal32: [[fallthrough]]; + case ValueType::vtDecimal64: [[fallthrough]]; + case ValueType::vtDecimal128: + { + ReadBufferFromString istr(value); + data_type->deserializeAsWholeText(column, istr, FormatSettings{}); + break; + } case ValueType::vtArray: { pqxx::array_parser parser{value}; @@ -188,24 +203,13 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view assert_cast(column).insert(Array(dimensions[1].begin(), dimensions[1].end())); break; } - case ValueType::vtDate: [[fallthrough]]; - case ValueType::vtDateTime: [[fallthrough]]; - case ValueType::vtDateTime64:[[fallthrough]]; - case ValueType::vtDecimal32: [[fallthrough]]; - case ValueType::vtDecimal64: [[fallthrough]]; - case ValueType::vtDecimal128: - { - ReadBufferFromString istr(value); - data_type->deserializeAsWholeText(column, istr, FormatSettings{}); - break; - } default: throw Exception("Value of unsupported type:" + column.getName(), ErrorCodes::UNKNOWN_TYPE); } } -void PostgreSQLBlockInputStream::prepareArrayParser(size_t column_idx, const DataTypePtr data_type) +void PostgreSQLBlockInputStream::prepareArrayInfo(size_t column_idx, const DataTypePtr data_type) { const auto * array_type = typeid_cast(data_type.get()); auto nested = array_type->getNestedType(); @@ -249,21 +253,21 @@ void PostgreSQLBlockInputStream::prepareArrayParser(size_t column_idx, const Dat else if (which.isDecimal32()) parser = [nested](std::string & field) -> Field { - auto type = typeid_cast *>(nested.get()); + const auto & type = typeid_cast *>(nested.get()); DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); return convertFieldToType(field, res); }; else if (which.isDecimal64()) parser = [nested](std::string & field) -> Field { - auto type = typeid_cast *>(nested.get()); + const auto & type = typeid_cast *>(nested.get()); DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); return convertFieldToType(field, res); }; else if (which.isDecimal128()) parser = [nested](std::string & field) -> Field { - auto type = typeid_cast *>(nested.get()); + const auto & type = typeid_cast *>(nested.get()); DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); return convertFieldToType(field, res); }; diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 2820845e4b5..038b53853f3 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -39,7 +39,7 @@ private: { column.insertFrom(sample_column, 0); } - void prepareArrayParser(size_t column_idx, const DataTypePtr data_type); + void prepareArrayInfo(size_t column_idx, const DataTypePtr data_type); const String query_str; const UInt64 max_block_size; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 91a6b542b8b..ebbe49e78bf 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -91,7 +91,6 @@ void PostgreSQLBlockOutputStream::writePrefix() { storage.checkConnection(connection); work = std::make_unique(*connection); - stream_inserter = std::make_unique(*work, remote_table_name); } @@ -100,7 +99,9 @@ void PostgreSQLBlockOutputStream::write(const Block & block) const auto columns = block.getColumns(); const size_t num_rows = block.rows(), num_cols = block.columns(); const auto data_types = block.getDataTypes(); - const auto settings = FormatSettings{}; + + if (!stream_inserter) + stream_inserter = std::make_unique(*work, remote_table_name, block.getNames()); /// std::optional lets libpqxx to know if value is NULL std::vector> row(num_cols); @@ -116,7 +117,7 @@ void PostgreSQLBlockOutputStream::write(const Block & block) else { WriteBufferFromOwnString ostr; - data_types[j]->serializeAsText(*columns[j], i, ostr, settings); + data_types[j]->serializeAsText(*columns[j], i, ostr, FormatSettings{}); row[j] = std::optional(ostr.str()); if (isArray(data_types[j])) @@ -130,7 +131,7 @@ void PostgreSQLBlockOutputStream::write(const Block & block) } } - /// pqxx::stream_to uses COPY instead of insert query, so it is faster if inserting large number of rows + /// pqxx::stream_to is much faster than simple insert, especially for large number of rows stream_inserter->write_values(row); } } @@ -138,7 +139,8 @@ void PostgreSQLBlockOutputStream::write(const Block & block) void PostgreSQLBlockOutputStream::writeSuffix() { - stream_inserter->complete(); + if (stream_inserter) + stream_inserter->complete(); work->commit(); } @@ -159,9 +161,9 @@ void registerStoragePostgreSQL(StorageFactory & factory) { ASTs & engine_args = args.engine_args; - if (engine_args.size() < 5) - throw Exception( - "Storage PostgreSQL requires 5-7 parameters: PostgreSQL('host:port', 'database', 'table', 'username', 'password'.", + if (engine_args.size() != 5) + throw Exception("Storage PostgreSQL requires 5 parameters: " + "PostgreSQL('host:port', 'database', 'table', 'username', 'password'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (auto & engine_arg : engine_args) diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 388560c028b..7b8e98c50f6 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -74,11 +74,9 @@ DataTypePtr TableFunctionPostgreSQL::getDataType(std::string & type, bool is_nul { DataTypePtr res; + /// Get rid of trailing '[]' for arrays if (dimensions) - { - /// No matter how many dimensions, in type we will get only one '[]' (i.e. Integer[]) type.resize(type.size() - 2); - } if (type == "smallint") res = std::make_shared(); @@ -100,7 +98,7 @@ DataTypePtr TableFunctionPostgreSQL::getDataType(std::string & type, bool is_nul res = std::make_shared(); else if (type.starts_with("numeric")) { - /// Numeric and decimal will both be numeric + /// Numeric and decimal will both end up here as numeric /// Will get numeric(precision, scale) string, need to extract precision and scale std::vector result; boost::split(result, type, [](char c){ return c == '(' || c == ',' || c == ')'; }); @@ -143,7 +141,7 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, const ASTs & args = func_args.arguments->children; - if (args.size() < 5) + if (args.size() != 5) throw Exception("Table function 'PostgreSQL' requires 5 parameters: " "PostgreSQL('host:port', 'database', 'table', 'user', 'password').", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index e1cd3c77195..756d6b2996f 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -9,6 +9,7 @@ namespace DB { +using ConnectionPtr = std::shared_ptr; class TableFunctionPostgreSQL : public ITableFunction { @@ -30,7 +31,7 @@ private: String connection_str; String remote_table_name; - std::shared_ptr connection; + ConnectionPtr connection; }; } From 9cb2c754649696d4f7713b2d1dadac5865500bda Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 11 Dec 2020 17:01:00 +0000 Subject: [PATCH 045/264] Add tests for storage --- .../test_storage_postgresql/__init__.py | 0 .../test_storage_postgresql/test.py | 94 +++++++++++++++++++ 2 files changed, 94 insertions(+) create mode 100644 tests/integration/test_storage_postgresql/__init__.py create mode 100644 tests/integration/test_storage_postgresql/test.py diff --git a/tests/integration/test_storage_postgresql/__init__.py b/tests/integration/test_storage_postgresql/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py new file mode 100644 index 00000000000..38e3cad916c --- /dev/null +++ b/tests/integration/test_storage_postgresql/test.py @@ -0,0 +1,94 @@ +import time + +import pytest +import psycopg2 +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry +from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=[], with_postgres=True) + +def get_postgres_conn(database=False): + if database == True: + conn_string = "host='localhost' dbname='clickhouse' user='postgres' password='mysecretpassword'" + else: + conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn = psycopg2.connect(conn_string) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + +def create_postgres_db(conn, name): + cursor = conn.cursor() + cursor.execute("CREATE DATABASE {}".format(name)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + postgres_conn = get_postgres_conn() + print("postgres connected") + create_postgres_db(postgres_conn, 'clickhouse') + yield cluster + + finally: + cluster.shutdown() + + +def test_postgres_conversions(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + table_name = 'test_types' + cursor.execute( + '''CREATE TABLE IF NOT EXISTS {} ( + a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, + h timestamp, i date, j numeric(5, 5), k decimal(5, 5))'''.format(table_name)) + node1.query(''' + INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword') VALUES + (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.2, 0.2)'''.format(table_name)) + result = node1.query(''' + SELECT * FROM postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name)) + assert(result == '-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.20000\t0.20000\n') + + table_name = 'test_array_dimensions' + cursor.execute( + '''CREATE TABLE IF NOT EXISTS {} (a date[] NOT NULL, b integer[][][], c decimal(5, 2)[][][][][][])'''.format(table_name)) + result = node1.query(''' + DESCRIBE TABLE postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name)) + expected ='a\tArray(Date)\t\t\t\t\t\nb\tArray(Array(Array(Nullable(Int32))))\t\t\t\t\t\nc\tArray(Array(Array(Array(Array(Array(Nullable(Decimal(5, 2))))))))' + assert(result.rstrip() == expected) + + node1.query(''' + INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword') + VALUES (['2000-05-12', '2000-05-12'], [[[1, 1], [NULL, NULL]], [[3, 3], [3, 3]], [[4, 4], [5, 5]]], [[[[[[0.1], [0.2], [0.3]]]]]])'''.format(table_name)) + result = node1.query(''' + SELECT * FROM postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name)) + assert(result == '''['2000-05-12','2000-05-12']\t[[[1,1],[NULL,NULL]],[[3,3],[3,3]],[[4,4],[5,5]]]\t[[[[[[0.10],[0.20],[0.30]]]]]]\n''') + + +def test_postgres_select_insert(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + table_name = 'test_many' + table = '''postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name) + cursor.execute('CREATE TABLE IF NOT EXISTS {} (a integer, b text, c integer)'.format(table_name)) + + result = node1.query(''' + INSERT INTO TABLE FUNCTION {} + SELECT number, concat('name_', toString(number)), 3 from numbers(10000)'''.format(table)) + check1 = "SELECT count() FROM {}".format(table) + check2 = "SELECT Sum(c) FROM {}".format(table) + check3 = "SELECT count(c) FROM {} WHERE a % 2 == 0".format(table) + check4 = "SELECT count() FROM {} WHERE b LIKE concat('name_', toString(1))".format(table) + assert (node1.query(check1)).rstrip() == '10000' + assert (node1.query(check2)).rstrip() == '30000' + assert (node1.query(check3)).rstrip() == '5000' + assert (node1.query(check4)).rstrip() == '1' + + +if __name__ == '__main__': + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() From 375e8e9736d4d430754806ace2636fa8dcbdfd50 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Dec 2020 00:14:16 +0300 Subject: [PATCH 046/264] Add postgres dictionary source --- .../PostgreSQLBlockInputStream.cpp | 3 + src/DataStreams/PostgreSQLBlockInputStream.h | 2 +- .../PostgreSQLDictionarySource.cpp | 177 ++++++++++++++++++ src/Dictionaries/PostgreSQLDictionarySource.h | 71 +++++++ src/Dictionaries/registerDictionaries.cpp | 1 + src/Dictionaries/registerDictionaries.h | 1 + .../test_dictionaries_postgresql/__init__.py | 0 .../configs/config.xml | 4 + .../configs/log_conf.xml | 12 ++ .../configs/postgres_dict.xml | 73 ++++++++ .../test_dictionaries_postgresql/test.py | 123 ++++++++++++ 11 files changed, 466 insertions(+), 1 deletion(-) create mode 100644 src/Dictionaries/PostgreSQLDictionarySource.cpp create mode 100644 src/Dictionaries/PostgreSQLDictionarySource.h create mode 100644 tests/integration/test_dictionaries_postgresql/__init__.py create mode 100644 tests/integration/test_dictionaries_postgresql/configs/config.xml create mode 100644 tests/integration/test_dictionaries_postgresql/configs/log_conf.xml create mode 100644 tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml create mode 100644 tests/integration/test_dictionaries_postgresql/test.py diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index bba46983081..86dcd105c5e 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -41,6 +41,9 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( for (const auto idx : ext::range(0, description.sample_block.columns())) if (description.types[idx].first == ValueType::vtArray) prepareArrayInfo(idx, description.sample_block.getByPosition(idx).type); + /// pqxx::stream_from uses COPY command, but when selecting from dictionary will get ';', it is not needed + if (query_str.ends_with(';')) + query_str.resize(query_str.size() - 1); } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 038b53853f3..3e52303a75d 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -41,7 +41,7 @@ private: } void prepareArrayInfo(size_t column_idx, const DataTypePtr data_type); - const String query_str; + String query_str; const UInt64 max_block_size; ExternalResultDescription description; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp new file mode 100644 index 00000000000..9fcc9f56b53 --- /dev/null +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -0,0 +1,177 @@ +#include "PostgreSQLDictionarySource.h" + +#if USE_LIBPQXX + +# include +# include +# include +# include +# include +#include +#include "readInvalidateQuery.h" +#include "DictionarySourceFactory.h" + +namespace DB +{ +static const UInt64 max_block_size = 8192; + + +PostgreSQLDictionarySource::PostgreSQLDictionarySource( + const DictionaryStructure & dict_struct_, + const Poco::Util::AbstractConfiguration & config_, + const std::string & config_prefix, + const std::string & connection_str, + const Block & sample_block_) + : dict_struct{dict_struct_} + , sample_block(sample_block_) + , connection(std::make_shared(connection_str)) + , log(&Poco::Logger::get("PostgreSQLDictionarySource")) + , db(config_.getString(fmt::format("{}.db", config_prefix), "")) + , table(config_.getString(fmt::format("{}.table", config_prefix), "")) + , where(config_.getString(fmt::format("{}.where", config_prefix), "")) + , query_builder(dict_struct, "", "", table, where, IdentifierQuotingStyle::DoubleQuotes) + , load_all_query(query_builder.composeLoadAllQuery()) + , invalidate_query(config_.getString(fmt::format("{}.invalidate_query", config_prefix), "")) +{ +} + + +/// copy-constructor is provided in order to support cloneability +PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionarySource & other) + : dict_struct(other.dict_struct) + , sample_block(other.sample_block) + , connection(other.connection) + , log(&Poco::Logger::get("PostgreSQLDictionarySource")) + , db(other.db) + , table(other.table) + , where(other.where) + , query_builder(dict_struct, "", "", table, where, IdentifierQuotingStyle::DoubleQuotes) + , load_all_query(query_builder.composeLoadAllQuery()) + , invalidate_query(other.invalidate_query) + , update_time(other.update_time) + , invalidate_query_response(other.invalidate_query_response) +{ +} + + +BlockInputStreamPtr PostgreSQLDictionarySource::loadAll() +{ + LOG_TRACE(log, load_all_query); + return std::make_shared( + connection, load_all_query, sample_block, max_block_size); +} + + +BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() +{ + auto load_update_query = getUpdateFieldAndDate(); + LOG_TRACE(log, load_update_query); + return std::make_shared(connection, load_update_query, sample_block, max_block_size); +} + + +BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) +{ + const auto query = query_builder.composeLoadIdsQuery(ids); + return std::make_shared(connection, query, sample_block, max_block_size); +} + + +BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +{ + const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); + return std::make_shared(connection, query, sample_block, max_block_size); +} + + +bool PostgreSQLDictionarySource::isModified() const +{ + if (!invalidate_query.empty()) + { + auto response = doInvalidateQuery(invalidate_query); + if (response == invalidate_query_response) + return false; + invalidate_query_response = response; + } + return true; +} + + +std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & request) const +{ + Block invalidate_sample_block; + ColumnPtr column(ColumnString::create()); + invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); + PostgreSQLBlockInputStream block_input_stream(connection, request, invalidate_sample_block, 1); + return readInvalidateQuery(block_input_stream); +} + + +bool PostgreSQLDictionarySource::hasUpdateField() const +{ + return !update_field.empty(); +} + + +std::string PostgreSQLDictionarySource::getUpdateFieldAndDate() +{ + if (update_time != std::chrono::system_clock::from_time_t(0)) + { + auto tmp_time = update_time; + update_time = std::chrono::system_clock::now(); + time_t hr_time = std::chrono::system_clock::to_time_t(tmp_time) - 1; + std::string str_time = std::to_string(LocalDateTime(hr_time)); + return query_builder.composeUpdateQuery(update_field, str_time); + } + else + { + update_time = std::chrono::system_clock::now(); + return query_builder.composeLoadAllQuery(); + } +} + + +bool PostgreSQLDictionarySource::supportsSelectiveLoad() const +{ + return true; +} + + +DictionarySourcePtr PostgreSQLDictionarySource::clone() const +{ + return std::make_unique(*this); +} + + +std::string PostgreSQLDictionarySource::toString() const +{ + return "PostgreSQL: " + db + '.' + table + (where.empty() ? "" : ", where: " + where); +} + + +void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) +{ + auto create_table_source = [=](const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & root_config_prefix, + Block & sample_block, + const Context & /* context */, + const std::string & /* default_database */, + bool /* check_config */) -> DictionarySourcePtr + { + const auto config_prefix = root_config_prefix + ".postgresql"; + auto connection_str = fmt::format("dbname={} host={} port={} user={} password={}", + config.getString(fmt::format("{}.db", config_prefix), ""), + config.getString(fmt::format("{}.host", config_prefix), ""), + config.getUInt(fmt::format("{}.port", config_prefix), 0), + config.getString(fmt::format("{}.user", config_prefix), ""), + config.getString(fmt::format("{}.password", config_prefix), "")); + + return std::make_unique( + dict_struct, config, config_prefix, connection_str, sample_block); + }; + factory.registerSource("postgresql", create_table_source); +} +} + +#endif diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h new file mode 100644 index 00000000000..b56135ca0b2 --- /dev/null +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -0,0 +1,71 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX + +# include +# include "DictionaryStructure.h" +# include "ExternalQueryBuilder.h" +# include "IDictionarySource.h" +#include +#include + +#include + +namespace DB +{ +using ConnectionPtr = std::shared_ptr; + +/// Allows loading dictionaries from a PostgreSQL database +class PostgreSQLDictionarySource final : public IDictionarySource +{ +public: + PostgreSQLDictionarySource( + const DictionaryStructure & dict_struct_, + const Poco::Util::AbstractConfiguration & config_, + const std::string & config_prefix, + const std::string & connection_str, + const Block & sample_block_); + + /// copy-constructor is provided in order to support cloneability + PostgreSQLDictionarySource(const PostgreSQLDictionarySource & other); + PostgreSQLDictionarySource & operator=(const PostgreSQLDictionarySource &) = delete; + + BlockInputStreamPtr loadAll() override; + BlockInputStreamPtr loadUpdatedAll() override; + BlockInputStreamPtr loadIds(const std::vector & ids) override; + BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + + bool isModified() const override; + bool supportsSelectiveLoad() const override; + bool hasUpdateField() const override; + + DictionarySourcePtr clone() const override; + std::string toString() const override; + +private: + std::string getUpdateFieldAndDate(); + std::string doInvalidateQuery(const std::string & request) const; + + const DictionaryStructure dict_struct; + Block sample_block; + ConnectionPtr connection; + Poco::Logger * log; + + const std::string db; + const std::string table; + const std::string where; + ExternalQueryBuilder query_builder; + const std::string load_all_query; + std::string invalidate_query; + std::chrono::time_point update_time; + const std::string update_field; + mutable std::string invalidate_query_response; + +}; + +} +#endif diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 7f1eb74a9ad..1de00cdeaab 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -16,6 +16,7 @@ void registerDictionaries() registerDictionarySourceCassandra(source_factory); registerDictionarySourceXDBC(source_factory); registerDictionarySourceJDBC(source_factory); + registerDictionarySourcePostgreSQL(source_factory); registerDictionarySourceExecutable(source_factory); registerDictionarySourceHTTP(source_factory); registerDictionarySourceLibrary(source_factory); diff --git a/src/Dictionaries/registerDictionaries.h b/src/Dictionaries/registerDictionaries.h index 8b38393377f..51fea84e869 100644 --- a/src/Dictionaries/registerDictionaries.h +++ b/src/Dictionaries/registerDictionaries.h @@ -13,6 +13,7 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & source_factory) void registerDictionarySourceRedis(DictionarySourceFactory & source_factory); void registerDictionarySourceXDBC(DictionarySourceFactory & source_factory); void registerDictionarySourceJDBC(DictionarySourceFactory & source_factory); +void registerDictionarySourcePostgreSQL(DictionarySourceFactory & source_factory); void registerDictionarySourceExecutable(DictionarySourceFactory & source_factory); void registerDictionarySourceHTTP(DictionarySourceFactory & source_factory); void registerDictionarySourceLibrary(DictionarySourceFactory & source_factory); diff --git a/tests/integration/test_dictionaries_postgresql/__init__.py b/tests/integration/test_dictionaries_postgresql/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_dictionaries_postgresql/configs/config.xml b/tests/integration/test_dictionaries_postgresql/configs/config.xml new file mode 100644 index 00000000000..77d92134765 --- /dev/null +++ b/tests/integration/test_dictionaries_postgresql/configs/config.xml @@ -0,0 +1,4 @@ + + + /etc/clickhouse-server/config.d/postgres_dict.xml + diff --git a/tests/integration/test_dictionaries_postgresql/configs/log_conf.xml b/tests/integration/test_dictionaries_postgresql/configs/log_conf.xml new file mode 100644 index 00000000000..b52d833cde8 --- /dev/null +++ b/tests/integration/test_dictionaries_postgresql/configs/log_conf.xml @@ -0,0 +1,12 @@ + + + + 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 + + diff --git a/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml b/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml new file mode 100644 index 00000000000..aeb3787a89e --- /dev/null +++ b/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml @@ -0,0 +1,73 @@ + + + + dict0 + + + clickhouse + postgres1 + 5432 + postgres + mysecretpassword + test0
+
+ + + + + + + id + UInt32 + + + id + UInt32 + + + + value + UInt32 + + + + 1 +
+ + dict1 + + + clickhouse + postgres1 + 5432 + postgres + mysecretpassword + test1
+ SELECT value FROM test1 WHERE id = 0 +
+ + + + + + + id + UInt32 + + + id + UInt32 + + + + value + UInt32 + + + + + 1 + 1 + +
+
diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py new file mode 100644 index 00000000000..1c0d94d56a6 --- /dev/null +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -0,0 +1,123 @@ +import pytest +import time +import psycopg2 +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry +from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/config.xml', 'configs/postgres_dict.xml', 'configs/log_conf.xml'], with_postgres=True) + +postgres_dict_table_template = """ + CREATE TABLE IF NOT EXISTS {} ( + id Integer NOT NULL, value Integer NOT NULL, PRIMARY KEY (id)) + """ +click_dict_table_template = """ + CREATE TABLE IF NOT EXISTS `test`.`dict_table_{}` ( + `id` UInt64, `value` UInt32 + ) ENGINE = Dictionary({}) + """ + +def get_postgres_conn(database=False): + if database == True: + conn_string = "host='localhost' dbname='clickhouse' user='postgres' password='mysecretpassword'" + else: + conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn = psycopg2.connect(conn_string) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + +def create_postgres_db(conn, name): + cursor = conn.cursor() + cursor.execute("CREATE DATABASE {}".format(name)) + +def create_postgres_table(conn, table_name): + cursor = conn.cursor() + cursor.execute(postgres_dict_table_template.format(table_name)) + +def create_and_fill_postgres_table(table_name, index=0): + table_name = table_name + str(index) + conn = get_postgres_conn(True) + create_postgres_table(conn, table_name) + # Fill postgres table using clickhouse postgres table function and check + table_func = '''postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name) + node1.query('''INSERT INTO TABLE FUNCTION {} SELECT number, number from numbers(10000) + '''.format(table_func, table_name)) + result = node1.query("SELECT count() FROM {}".format(table_func)) + assert result.rstrip() == '10000' + +def create_dict(table_name, index=0): + node1.query(click_dict_table_template.format(table_name + str(index), 'dict' + str(index))) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + postgres_conn = get_postgres_conn() + node1.query("CREATE DATABASE IF NOT EXISTS test") + print("postgres connected") + create_postgres_db(postgres_conn, 'clickhouse') + yield cluster + + finally: + cluster.shutdown() + + +def test_load_dictionaries(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + table_name = 'test' + create_and_fill_postgres_table(table_name) + create_dict(table_name) + table_name += str(0) + dict_name = 'dict0' + + node1.query("SYSTEM RELOAD DICTIONARIES") + assert node1.query("SELECT count() FROM `test`.`dict_table_{}`".format(table_name)).rstrip() == '10000' + assert node1.query("SELECT dictGetUInt32('{}', 'id', toUInt64(0))".format(dict_name)) == '0\n' + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(9999))".format(dict_name)) == '9999\n' + cursor.execute("DROP TABLE IF EXISTS {}".format(table_name)) + + +def test_invalidate_query(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + table_name = 'test' + create_and_fill_postgres_table(table_name, 0) + create_and_fill_postgres_table(table_name, 1) + + # this dict has no invalidate query + dict_name = 'dict0' + create_dict(table_name) + node1.query("SYSTEM RELOAD DICTIONARIES") + first_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(dict_name)) + time.sleep(4) + second_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(dict_name)) + assert first_update_time != second_update_time + + # this dict has invalidate query: SELECT value FROM test1 WHERE id = 0 + dict_name = 'dict1' + create_dict(table_name, 1) + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(1))".format(dict_name)) == "1\n" + + first_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(dict_name)) + time.sleep(4) + second_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(table_name)) + assert first_update_time != second_update_time + + # no update should be made + cursor.execute("UPDATE {} SET value=value*2 WHERE id > 0".format(table_name+str(1))) + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(1))".format(dict_name)) == '1\n' + + # update should happen + cursor.execute("UPDATE {} SET value=value+1 WHERE id=0".format(table_name+str(1))) + time.sleep(5) + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(1))".format(dict_name)) == '2\n' + + +if __name__ == '__main__': + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() From d95e8e2d74aca2b3fc9ddebf5604519ca29b8b38 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 13 Dec 2020 22:48:40 +0000 Subject: [PATCH 047/264] Better --- .../PostgreSQLDictionarySource.cpp | 29 ++++++++---- src/Dictionaries/PostgreSQLDictionarySource.h | 10 ++-- src/Storages/StoragePostgreSQL.cpp | 1 - .../configs/postgres_dict.xml | 35 +------------- .../test_dictionaries_postgresql/test.py | 46 +++++++++---------- 5 files changed, 45 insertions(+), 76 deletions(-) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 9fcc9f56b53..cb3f9804e8a 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -1,21 +1,20 @@ #include "PostgreSQLDictionarySource.h" -#if USE_LIBPQXX +#include +#include "DictionarySourceFactory.h" +#include "registerDictionaries.h" -# include -# include -# include -# include -# include +#if USE_LIBPQXX +#include +#include #include #include "readInvalidateQuery.h" -#include "DictionarySourceFactory.h" + namespace DB { static const UInt64 max_block_size = 8192; - PostgreSQLDictionarySource::PostgreSQLDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config_, @@ -32,6 +31,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( , query_builder(dict_struct, "", "", table, where, IdentifierQuotingStyle::DoubleQuotes) , load_all_query(query_builder.composeLoadAllQuery()) , invalidate_query(config_.getString(fmt::format("{}.invalidate_query", config_prefix), "")) + , update_field(config_.getString(fmt::format("{}.update_field", config_prefix), "")) { } @@ -49,6 +49,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar , load_all_query(query_builder.composeLoadAllQuery()) , invalidate_query(other.invalidate_query) , update_time(other.update_time) + , update_field(other.update_field) , invalidate_query_response(other.invalidate_query_response) { } @@ -69,7 +70,6 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() return std::make_shared(connection, load_update_query, sample_block, max_block_size); } - BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); @@ -159,6 +159,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) const std::string & /* default_database */, bool /* check_config */) -> DictionarySourcePtr { +#if USE_LIBPQXX const auto config_prefix = root_config_prefix + ".postgresql"; auto connection_str = fmt::format("dbname={} host={} port={} user={} password={}", config.getString(fmt::format("{}.db", config_prefix), ""), @@ -169,9 +170,17 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) return std::make_unique( dict_struct, config, config_prefix, connection_str, sample_block); +#else + (void)dict_struct; + (void)config; + (void)root_config_prefix; + (void)sample_block; + throw Exception{"Dictionary source of type `postgresql` is disabled because ClickHouse was built without postgresql support.", + ErrorCodes::SUPPORT_IS_DISABLED}; +#endif }; factory.registerSource("postgresql", create_table_source); } } - #endif + diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index b56135ca0b2..30e90d7dde5 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -3,16 +3,14 @@ #if !defined(ARCADIA_BUILD) #include "config_core.h" #endif +#include "DictionaryStructure.h" +#include "IDictionarySource.h" #if USE_LIBPQXX - -# include -# include "DictionaryStructure.h" -# include "ExternalQueryBuilder.h" -# include "IDictionarySource.h" +#include "ExternalQueryBuilder.h" #include +#include #include - #include namespace DB diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index ebbe49e78bf..7be396425b0 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -130,7 +130,6 @@ void PostgreSQLBlockOutputStream::write(const Block & block) } } } - /// pqxx::stream_to is much faster than simple insert, especially for large number of rows stream_inserter->write_values(row); } diff --git a/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml b/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml index aeb3787a89e..e3224b39a7d 100644 --- a/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml +++ b/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml @@ -10,40 +10,7 @@ postgres mysecretpassword test0
- - - - - - - - id - UInt32 - - - id - UInt32 - - - - value - UInt32 - - - - 1 - - - dict1 - - - clickhouse - postgres1 - 5432 - postgres - mysecretpassword - test1
- SELECT value FROM test1 WHERE id = 0 + SELECT value FROM test0 WHERE id = 0
diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index 1c0d94d56a6..cd7d575a999 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -36,8 +36,7 @@ def create_postgres_table(conn, table_name): cursor = conn.cursor() cursor.execute(postgres_dict_table_template.format(table_name)) -def create_and_fill_postgres_table(table_name, index=0): - table_name = table_name + str(index) +def create_and_fill_postgres_table(table_name): conn = get_postgres_conn(True) create_postgres_table(conn, table_name) # Fill postgres table using clickhouse postgres table function and check @@ -48,7 +47,7 @@ def create_and_fill_postgres_table(table_name, index=0): assert result.rstrip() == '10000' def create_dict(table_name, index=0): - node1.query(click_dict_table_template.format(table_name + str(index), 'dict' + str(index))) + node1.query(click_dict_table_template.format(table_name, 'dict' + str(index))) @pytest.fixture(scope="module") @@ -68,10 +67,9 @@ def started_cluster(): def test_load_dictionaries(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() - table_name = 'test' + table_name = 'test0' create_and_fill_postgres_table(table_name) create_dict(table_name) - table_name += str(0) dict_name = 'dict0' node1.query("SYSTEM RELOAD DICTIONARIES") @@ -84,36 +82,34 @@ def test_load_dictionaries(started_cluster): def test_invalidate_query(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() - table_name = 'test' - create_and_fill_postgres_table(table_name, 0) - create_and_fill_postgres_table(table_name, 1) + table_name = 'test0' + create_and_fill_postgres_table(table_name) - # this dict has no invalidate query + # invalidate query: SELECT value FROM test0 WHERE id = 0 dict_name = 'dict0' create_dict(table_name) - node1.query("SYSTEM RELOAD DICTIONARIES") - first_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(dict_name)) - time.sleep(4) - second_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(dict_name)) - assert first_update_time != second_update_time - - # this dict has invalidate query: SELECT value FROM test1 WHERE id = 0 - dict_name = 'dict1' - create_dict(table_name, 1) + node1.query("SYSTEM RELOAD DICTIONARY {}".format(dict_name)) + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(0))".format(dict_name)) == "0\n" assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(1))".format(dict_name)) == "1\n" - first_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(dict_name)) - time.sleep(4) - second_update_time = node1.query("SELECT last_successful_update_time FROM system.dictionaries WHERE name = '{}'".format(table_name)) - assert first_update_time != second_update_time + # update should happen + cursor.execute("UPDATE {} SET value=value+1 WHERE id = 0".format(table_name)) + while True: + result = node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(0))".format(dict_name)) + if result != '0\n': + break + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(0))".format(dict_name)) == '1\n' - # no update should be made - cursor.execute("UPDATE {} SET value=value*2 WHERE id > 0".format(table_name+str(1))) + # no update should happen + cursor.execute("UPDATE {} SET value=value*2 WHERE id != 0".format(table_name)) + time.sleep(5) + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(0))".format(dict_name)) == '1\n' assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(1))".format(dict_name)) == '1\n' # update should happen - cursor.execute("UPDATE {} SET value=value+1 WHERE id=0".format(table_name+str(1))) + cursor.execute("UPDATE {} SET value=value+1 WHERE id = 0".format(table_name)) time.sleep(5) + assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(0))".format(dict_name)) == '2\n' assert node1.query("SELECT dictGetUInt32('{}', 'value', toUInt64(1))".format(dict_name)) == '2\n' From 2ab07fbd71186667783d266145e52fffa27e0199 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 14 Dec 2020 21:33:02 +0000 Subject: [PATCH 048/264] Update libraries --- cmake/find/libpqxx.cmake | 2 +- contrib/libpq | 2 +- contrib/libpq-cmake/CMakeLists.txt | 21 +----- contrib/libpqxx | 2 +- contrib/libpqxx-cmake/CMakeLists.txt | 101 +++++++++++++++++++-------- 5 files changed, 77 insertions(+), 51 deletions(-) diff --git a/cmake/find/libpqxx.cmake b/cmake/find/libpqxx.cmake index b635fb80878..8bfb48a0e54 100644 --- a/cmake/find/libpqxx.cmake +++ b/cmake/find/libpqxx.cmake @@ -11,7 +11,7 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpqxx/CMakeLists.txt") return() endif() -if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpq") +if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpq/include") message (ERROR "submodule contrib/libpq is missing. to fix try run: \n git submodule update --init --recursive") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libpq needed for libpqxx") set (USE_LIBPQXX 0) diff --git a/contrib/libpq b/contrib/libpq index d94250f9145..d53cfd7f523 160000 --- a/contrib/libpq +++ b/contrib/libpq @@ -1 +1 @@ -Subproject commit d94250f91457f11de6a37f7a6ba162601ddead27 +Subproject commit d53cfd7f5237e0765ca4c12c6408fdd722b954e1 diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index 8678f0141c7..d58e27371af 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -5,7 +5,6 @@ set(SRCS ${LIBPQ_SOURCE_DIR}/fe-auth-scram.c ${LIBPQ_SOURCE_DIR}/fe-connect.c ${LIBPQ_SOURCE_DIR}/fe-exec.c -# ${LIBPQ_SOURCE_DIR}/fe-gssapi-common.c ${LIBPQ_SOURCE_DIR}/fe-lobj.c ${LIBPQ_SOURCE_DIR}/fe-misc.c ${LIBPQ_SOURCE_DIR}/fe-print.c @@ -13,7 +12,6 @@ set(SRCS ${LIBPQ_SOURCE_DIR}/fe-protocol3.c ${LIBPQ_SOURCE_DIR}/fe-secure.c ${LIBPQ_SOURCE_DIR}/fe-secure-common.c -# ${LIBPQ_SOURCE_DIR}/fe-secure-gssapi.c ${LIBPQ_SOURCE_DIR}/fe-secure-openssl.c ${LIBPQ_SOURCE_DIR}/legacy-pqsignal.c ${LIBPQ_SOURCE_DIR}/libpq-events.c @@ -30,8 +28,6 @@ set(SRCS ${LIBPQ_SOURCE_DIR}/common/base64.c ${LIBPQ_SOURCE_DIR}/common/link-canary.c ${LIBPQ_SOURCE_DIR}/common/fe_memutils.c - ${LIBPQ_SOURCE_DIR}/common/sha2.c - ${LIBPQ_SOURCE_DIR}/common/sha2_openssl.c ${LIBPQ_SOURCE_DIR}/common/string.c ${LIBPQ_SOURCE_DIR}/common/pg_get_line.c ${LIBPQ_SOURCE_DIR}/common/stringinfo.c @@ -43,34 +39,19 @@ set(SRCS ${LIBPQ_SOURCE_DIR}/port/strerror.c ${LIBPQ_SOURCE_DIR}/port/inet_net_ntop.c ${LIBPQ_SOURCE_DIR}/port/getpeereid.c - ${LIBPQ_SOURCE_DIR}/port/chklocale.c - ${LIBPQ_SOURCE_DIR}/port/dirmod.c - ${LIBPQ_SOURCE_DIR}/port/dlopen.c - ${LIBPQ_SOURCE_DIR}/port/erand48.c - ${LIBPQ_SOURCE_DIR}/port/explicit_bzero.c - ${LIBPQ_SOURCE_DIR}/port/fls.c - ${LIBPQ_SOURCE_DIR}/port/getopt.c - ${LIBPQ_SOURCE_DIR}/port/kill.c - ${LIBPQ_SOURCE_DIR}/port/link.c ${LIBPQ_SOURCE_DIR}/port/noblock.c - ${LIBPQ_SOURCE_DIR}/port/open.c - ${LIBPQ_SOURCE_DIR}/port/path.c ${LIBPQ_SOURCE_DIR}/port/pg_strong_random.c ${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c ${LIBPQ_SOURCE_DIR}/port/thread.c - - ${LIBPQ_SOURCE_DIR}/error/elog.c - ${LIBPQ_SOURCE_DIR}/error/errlog.c ) add_library(libpq ${SRCS}) -target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}/configs) - target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}) target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}/include) +target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/configs) target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/include/common) target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/include/libpq) diff --git a/contrib/libpqxx b/contrib/libpqxx index aa8aa6cb48f..58d2a028d16 160000 --- a/contrib/libpqxx +++ b/contrib/libpqxx @@ -1 +1 @@ -Subproject commit aa8aa6cb48fcb8ccabc22a0df78f1250279cd233 +Subproject commit 58d2a028d1600225ac3a478d6b3a06ba2f0c01f6 diff --git a/contrib/libpqxx-cmake/CMakeLists.txt b/contrib/libpqxx-cmake/CMakeLists.txt index 993927e9d3c..ed372951f82 100644 --- a/contrib/libpqxx-cmake/CMakeLists.txt +++ b/contrib/libpqxx-cmake/CMakeLists.txt @@ -1,33 +1,78 @@ set (LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/libpqxx) set (SRCS - ${LIBRARY_DIR}/src/strconv.cxx - ${LIBRARY_DIR}/src/array.cxx - ${LIBRARY_DIR}/src/binarystring.cxx - ${LIBRARY_DIR}/src/connection.cxx - ${LIBRARY_DIR}/src/cursor.cxx - ${LIBRARY_DIR}/src/encodings.cxx - ${LIBRARY_DIR}/src/errorhandler.cxx - ${LIBRARY_DIR}/src/except.cxx - ${LIBRARY_DIR}/src/field.cxx - ${LIBRARY_DIR}/src/largeobject.cxx - ${LIBRARY_DIR}/src/notification.cxx - ${LIBRARY_DIR}/src/pipeline.cxx - ${LIBRARY_DIR}/src/result.cxx - ${LIBRARY_DIR}/src/robusttransaction.cxx - ${LIBRARY_DIR}/src/sql_cursor.cxx - ${LIBRARY_DIR}/src/stream_from.cxx - ${LIBRARY_DIR}/src/stream_to.cxx - ${LIBRARY_DIR}/src/subtransaction.cxx - ${LIBRARY_DIR}/src/transaction.cxx - ${LIBRARY_DIR}/src/transaction_base.cxx - ${LIBRARY_DIR}/src/row.cxx - ${LIBRARY_DIR}/src/util.cxx - ${LIBRARY_DIR}/src/version.cxx - ) + ${LIBRARY_DIR}/src/strconv.cxx + ${LIBRARY_DIR}/src/array.cxx + ${LIBRARY_DIR}/src/binarystring.cxx + ${LIBRARY_DIR}/src/connection.cxx + ${LIBRARY_DIR}/src/cursor.cxx + ${LIBRARY_DIR}/src/encodings.cxx + ${LIBRARY_DIR}/src/errorhandler.cxx + ${LIBRARY_DIR}/src/except.cxx + ${LIBRARY_DIR}/src/field.cxx + ${LIBRARY_DIR}/src/largeobject.cxx + ${LIBRARY_DIR}/src/notification.cxx + ${LIBRARY_DIR}/src/pipeline.cxx + ${LIBRARY_DIR}/src/result.cxx + ${LIBRARY_DIR}/src/robusttransaction.cxx + ${LIBRARY_DIR}/src/sql_cursor.cxx + ${LIBRARY_DIR}/src/stream_from.cxx + ${LIBRARY_DIR}/src/stream_to.cxx + ${LIBRARY_DIR}/src/subtransaction.cxx + ${LIBRARY_DIR}/src/transaction.cxx + ${LIBRARY_DIR}/src/transaction_base.cxx + ${LIBRARY_DIR}/src/row.cxx + ${LIBRARY_DIR}/src/util.cxx + ${LIBRARY_DIR}/src/version.cxx +) -add_library(libpqxx ${SRCS}) -target_link_libraries(libpqxx PRIVATE ${LIBPQ_LIBRARY}) +# Need to explicitly include each header file, because in the directory include/pqxx there are also files +# like just 'array'. So if including the whole directory with `target_include_directories`, it will make +# conflicts with all includes of . +set (HDRS + ${LIBRARY_DIR}/include/pqxx/array.hxx + ${LIBRARY_DIR}/include/pqxx/binarystring.hxx + ${LIBRARY_DIR}/include/pqxx/composite.hxx + ${LIBRARY_DIR}/include/pqxx/connection.hxx + ${LIBRARY_DIR}/include/pqxx/cursor.hxx + ${LIBRARY_DIR}/include/pqxx/dbtransaction.hxx + ${LIBRARY_DIR}/include/pqxx/errorhandler.hxx + ${LIBRARY_DIR}/include/pqxx/except.hxx + ${LIBRARY_DIR}/include/pqxx/field.hxx + ${LIBRARY_DIR}/include/pqxx/isolation.hxx + ${LIBRARY_DIR}/include/pqxx/largeobject.hxx + ${LIBRARY_DIR}/include/pqxx/nontransaction.hxx + ${LIBRARY_DIR}/include/pqxx/notification.hxx + ${LIBRARY_DIR}/include/pqxx/pipeline.hxx + ${LIBRARY_DIR}/include/pqxx/prepared_statement.hxx + ${LIBRARY_DIR}/include/pqxx/result.hxx + ${LIBRARY_DIR}/include/pqxx/robusttransaction.hxx + ${LIBRARY_DIR}/include/pqxx/row.hxx + ${LIBRARY_DIR}/include/pqxx/separated_list.hxx + ${LIBRARY_DIR}/include/pqxx/strconv.hxx + ${LIBRARY_DIR}/include/pqxx/stream_from.hxx + ${LIBRARY_DIR}/include/pqxx/stream_to.hxx + ${LIBRARY_DIR}/include/pqxx/subtransaction.hxx + ${LIBRARY_DIR}/include/pqxx/transaction.hxx + ${LIBRARY_DIR}/include/pqxx/transaction_base.hxx + ${LIBRARY_DIR}/include/pqxx/types.hxx + ${LIBRARY_DIR}/include/pqxx/util.hxx + ${LIBRARY_DIR}/include/pqxx/version.hxx + ${LIBRARY_DIR}/include/pqxx/zview.hxx +) + +add_library(libpqxx ${SRCS} ${HDRS}) + +target_link_libraries(libpqxx PUBLIC ${LIBPQ_LIBRARY}) +target_include_directories (libpqxx PRIVATE ${LIBRARY_DIR}/include) + +# crutch +set(CM_CONFIG_H_IN "${LIBRARY_DIR}/include/pqxx/config.h.in") +set(CM_CONFIG_PUB "${LIBRARY_DIR}/include/pqxx/config-public-compiler.h") +set(CM_CONFIG_INT "${LIBRARY_DIR}/include/pqxx/config-internal-compiler.h") +set(CM_CONFIG_PQ "${LIBRARY_DIR}/include/pqxx/config-internal-libpq.h") + +configure_file("${CM_CONFIG_H_IN}" "${CM_CONFIG_INT}" @ONLY) +configure_file("${CM_CONFIG_H_IN}" "${CM_CONFIG_PUB}" @ONLY) +configure_file("${CM_CONFIG_H_IN}" "${CM_CONFIG_PQ}" @ONLY) -target_include_directories (libpqxx SYSTEM BEFORE PRIVATE ${LIBRARY_DIR}/include) -target_include_directories (libpqxx PRIVATE ${LIBRARY_DIR}/include/pqxx) From 00a37404ca3b8b6a6516f48d374332485f168d80 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 21 Dec 2020 19:20:56 +0000 Subject: [PATCH 049/264] Add postgresql database engine --- src/CMakeLists.txt | 4 + src/Databases/DatabaseFactory.cpp | 42 ++- .../PostgreSQL/DatabasePostgreSQL.cpp | 306 ++++++++++++++++++ src/Databases/PostgreSQL/DatabasePostgreSQL.h | 86 +++++ .../PostgreSQL/FetchFromPostgreSQL.cpp | 120 +++++++ .../PostgreSQL/FetchFromPostgreSQL.h | 18 ++ .../PostgreSQLDictionarySource.cpp | 19 +- src/Dictionaries/PostgreSQLDictionarySource.h | 6 +- src/Storages/StoragePostgreSQL.cpp | 24 +- src/Storages/StoragePostgreSQL.h | 44 ++- .../TableFunctionPostgreSQL.cpp | 98 +----- src/TableFunctions/TableFunctionPostgreSQL.h | 6 +- .../configs/postgres_dict.xml | 5 +- .../test_dictionaries_postgresql/test.py | 2 +- .../__init__.py | 0 .../test_postgresql_database_engine/test.py | 122 +++++++ 16 files changed, 758 insertions(+), 144 deletions(-) create mode 100644 src/Databases/PostgreSQL/DatabasePostgreSQL.cpp create mode 100644 src/Databases/PostgreSQL/DatabasePostgreSQL.h create mode 100644 src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp create mode 100644 src/Databases/PostgreSQL/FetchFromPostgreSQL.h create mode 100644 tests/integration/test_postgresql_database_engine/__init__.py create mode 100644 tests/integration/test_postgresql_database_engine/test.py diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0c2d0afe5bb..364d79d7d8c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -79,6 +79,10 @@ if (USE_AMQPCPP) add_headers_and_sources(dbms Storages/RabbitMQ) endif() +if (USE_LIBPQXX) + add_headers_and_sources(dbms Databases/PostgreSQL) +endif() + if (USE_ROCKSDB) add_headers_and_sources(dbms Storages/RocksDB) endif() diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 0ac80706bd8..703df2ee44b 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -28,6 +28,10 @@ # include #endif +#if USE_LIBPQXX +#include +#endif + namespace DB { @@ -80,7 +84,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const String & engine_name = engine_define->engine->name; const UUID & uuid = create.uuid; - if (engine_name != "MySQL" && engine_name != "MaterializeMySQL" && engine_name != "Lazy" && engine_define->engine->arguments) + if (engine_name != "MySQL" && engine_name != "MaterializeMySQL" && engine_name != "Lazy" && engine_name != "PostgreSQL" && engine_define->engine->arguments) throw Exception("Database engine " + engine_name + " cannot have arguments", ErrorCodes::BAD_ARGUMENTS); if (engine_define->engine->parameters || engine_define->partition_by || engine_define->primary_key || engine_define->order_by || @@ -168,6 +172,42 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String return std::make_shared(database_name, metadata_path, cache_expiration_time_seconds, context); } +#if USE_LIBPQXX + + else if (engine_name == "PostgreSQL") + { + const ASTFunction * engine = engine_define->engine; + + if (!engine->arguments || engine->arguments->children.size() != 4) + throw Exception(fmt::format( + "{} Database require postgres_host_port, postgres_dbname, " + "postgres_username, mysql_password arguments.", engine_name), + ErrorCodes::BAD_ARGUMENTS); + + ASTs & engine_args = engine->arguments->children; + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, context); + + const auto & host_port = safeGetLiteralValue(engine_args[0], engine_name); + const auto & postgres_database_name = safeGetLiteralValue(engine_args[1], engine_name); + const auto & username = safeGetLiteralValue(engine_args[2], engine_name); + const auto & password = safeGetLiteralValue(engine_args[3], engine_name); + + auto parsed_host_port = parseAddress(host_port, 5432); + String connection_str; + connection_str = fmt::format("dbname={} host={} port={} user={} password={}", + postgres_database_name, parsed_host_port.first, std::to_string(parsed_host_port.second), + username, password); + /// no connection is made here + auto connection = std::make_shared(connection_str); + + return std::make_shared( + context, metadata_path, engine_define, database_name, postgres_database_name, connection); + } + +#endif + throw Exception("Unknown database engine: " + engine_name, ErrorCodes::UNKNOWN_DATABASE_ENGINE); } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp new file mode 100644 index 00000000000..cf61cb884e8 --- /dev/null +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -0,0 +1,306 @@ +#include + +#if USE_LIBPQXX +#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 NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int UNKNOWN_TABLE; + extern const int TABLE_IS_DROPPED; + extern const int TABLE_ALREADY_EXISTS; + extern const int UNEXPECTED_AST_STRUCTURE; +} + +DatabasePostgreSQL::DatabasePostgreSQL( + const Context & context, + const String & metadata_path_, + const ASTStorage * database_engine_define_, + const String & dbname_, + const String & postgres_dbname, + PGConnectionPtr connection_) + : IDatabase(dbname_) + , global_context(context.getGlobalContext()) + , metadata_path(metadata_path_) + , database_engine_define(database_engine_define_->clone()) + , dbname(postgres_dbname) + , connection(std::move(connection_)) +{ +} + + +bool DatabasePostgreSQL::empty() const +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "empty"); + std::lock_guard lock(mutex); + + auto tables_list = fetchTablesList(); + + for (const auto & table_name : tables_list) + if (!detached_tables.count(table_name)) + return false; + + return true; +} + + +DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( + const Context & context, const FilterByNameFunction & /* filter_by_table_name */) +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "getTablesIterator"); + std::lock_guard lock(mutex); + + Tables tables; + auto table_names = fetchTablesList(); + + for (auto & table_name : table_names) + if (!detached_tables.count(table_name)) + tables[table_name] = fetchTable(table_name, context); + + return std::make_unique(tables, database_name); +} + + +std::unordered_set DatabasePostgreSQL::fetchTablesList() const +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "fetchTablesList"); + + std::unordered_set tables; + std::string query = "SELECT tablename FROM pg_catalog.pg_tables " + "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'"; + pqxx::read_transaction tx(*connection->conn()); + + for (auto table_name : tx.stream(query)) + tables.insert(std::get<0>(table_name)); + + return tables; +} + + +bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const +{ + pqxx::nontransaction tx(*connection->conn()); + pqxx::result result = tx.exec(fmt::format( + "SELECT attname FROM pg_attribute " + "WHERE attrelid = '{}'::regclass " + "AND NOT attisdropped AND attnum > 0", table_name)); + + if (result.empty()) + return false; + + return true; +} + + +bool DatabasePostgreSQL::isTableExist(const String & table_name, const Context & /* context */) const +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "isTableExists"); + std::lock_guard lock(mutex); + + if (detached_tables.count(table_name)) + return false; + + return checkPostgresTable(table_name); +} + + +StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, const Context & context) const +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "tryGetTable"); + std::lock_guard lock(mutex); + + if (detached_tables.count(table_name)) + return StoragePtr{}; + else + return fetchTable(table_name, context); +} + + +StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Context & context) const +{ + auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; + auto columns = fetchTableStructure(connection->conn(), table_name, use_nulls); + + if (!columns) + return StoragePtr{}; + + return StoragePostgreSQL::create( + StorageID(database_name, table_name), table_name, + connection, ColumnsDescription{*columns}, ConstraintsDescription{}, context); +} + + +void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr & /* storage */, const String &) +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "attachTable"); + std::lock_guard lock{mutex}; + + if (!checkPostgresTable(table_name)) + throw Exception(fmt::format("Cannot attach table {}.{} because it does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); + + if (!detached_tables.count(table_name)) + throw Exception(fmt::format("Cannot attach table {}.{}. It already exists", database_name, table_name), ErrorCodes::TABLE_ALREADY_EXISTS); + + detached_tables.erase(table_name); +} + + +StoragePtr DatabasePostgreSQL::detachTable(const String & table_name) +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "detachTable"); + std::lock_guard lock{mutex}; + + if (!checkPostgresTable(table_name)) + throw Exception(fmt::format("Cannot detach table {}.{} because it does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); + + if (detached_tables.count(table_name)) + throw Exception(fmt::format("Cannot detach table {}.{}. It is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); + + detached_tables.emplace(table_name); + return StoragePtr{}; +} + + +void DatabasePostgreSQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "createTable"); + + const auto & create = create_query->as(); + + if (!create->attach) + throw Exception("PostgreSQL database engine does not support create table", ErrorCodes::NOT_IMPLEMENTED); + + attachTable(table_name, storage, {}); +} + + +void DatabasePostgreSQL::dropTable(const Context &, const String & table_name, bool /*no_delay*/) +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "detachPermanently"); + std::lock_guard lock{mutex}; + + if (!checkPostgresTable(table_name)) + throw Exception(fmt::format("Cannot drop table {}.{} because it does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); + + if (detached_tables.count(table_name)) + throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); + + detached_tables.emplace(table_name); +} + + +void DatabasePostgreSQL::drop(const Context & /*context*/) +{ + Poco::File(getMetadataPath()).remove(true); +} + + +ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "getDatabaseQuery"); + + const auto & create_query = std::make_shared(); + create_query->database = getDatabaseName(); + create_query->set(create_query->storage, database_engine_define); + return create_query; +} + + +ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const +{ + LOG_TRACE(&Poco::Logger::get("kssenii"), "getTableQueryImpl"); + + auto storage = fetchTable(table_name, context); + if (!storage) + { + if (throw_on_error) + throw Exception(fmt::format("PostgreSQL table {}.{} does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); + + return nullptr; + } + + /// Get create table query from storage + + auto create_table_query = std::make_shared(); + auto table_storage_define = database_engine_define->clone(); + create_table_query->set(create_table_query->storage, table_storage_define); + + auto columns_declare_list = std::make_shared(); + auto columns_expression_list = std::make_shared(); + + columns_declare_list->set(columns_declare_list->columns, columns_expression_list); + create_table_query->set(create_table_query->columns_list, columns_declare_list); + + { + /// init create query. + auto table_id = storage->getStorageID(); + create_table_query->table = table_id.table_name; + create_table_query->database = table_id.database_name; + + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) + { + const auto & column_declaration = std::make_shared(); + column_declaration->name = column_type_and_name.name; + + std::function convert_datatype_to_query = [&](const DataTypePtr & data_type) -> ASTPtr + { + WhichDataType which(data_type); + if (!which.isNullable()) + return std::make_shared(data_type->getName()); + return makeASTFunction("Nullable", convert_datatype_to_query(typeid_cast(data_type.get())->getNestedType())); + }; + + column_declaration->type = convert_datatype_to_query(column_type_and_name.type); + columns_expression_list->children.emplace_back(column_declaration); + } + + ASTStorage * ast_storage = table_storage_define->as(); + ASTs storage_children = ast_storage->children; + auto storage_engine_arguments = ast_storage->engine->arguments; + + /// Add table_name to engine arguments + auto mysql_table_name = std::make_shared(table_id.table_name); + storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); + + /// Unset settings + storage_children.erase( + std::remove_if(storage_children.begin(), storage_children.end(), + [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }), + storage_children.end()); + ast_storage->settings = nullptr; + } + + return create_table_query; +} + +} + +#endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h new file mode 100644 index 00000000000..14a48efd578 --- /dev/null +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -0,0 +1,86 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ + +class Context; + +class DatabasePostgreSQL final : public IDatabase +{ + +public: + DatabasePostgreSQL( + const Context & context, + const String & metadata_path_, + const ASTStorage * database_engine_define, + const String & dbname_, + const String & postgres_dbname, + PGConnectionPtr connection_); + + String getEngineName() const override { return "PostgreSQL"; } + String getMetadataPath() const override { return metadata_path; }; + + bool canContainMergeTreeTables() const override { return false; } + bool canContainDistributedTables() const override { return false; } + bool shouldBeEmptyOnDetach() const override { return false; } + + ASTPtr getCreateDatabaseQuery() const override; + + bool empty() const override; + + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; + + bool isTableExist(const String & name, const Context & context) const override; + StoragePtr tryGetTable(const String & name, const Context & context) const override; + + void createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; + void dropTable(const Context &, const String & table_name, bool no_delay) override; + + void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; + StoragePtr detachTable(const String & table_name) override; + + void drop(const Context & /*context*/) override; + void shutdown() override {}; + +protected: + ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override; + +private: + const Context & global_context; + String metadata_path; + ASTPtr database_engine_define; + String dbname; + PGConnectionPtr connection; + + std::unordered_set detached_tables; + + bool checkPostgresTable(const String & table_name) const; + std::unordered_set fetchTablesList() const; + StoragePtr fetchTable(const String & table_name, const Context & context) const; +}; + +} + +#endif diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp new file mode 100644 index 00000000000..afe7f6a9a7d --- /dev/null +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -0,0 +1,120 @@ +#include + +#if USE_LIBPQXX + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_TYPE; +} + +/// These functions are also used for postgresql table function + +std::shared_ptr fetchTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls) +{ + auto columns = NamesAndTypesList(); + + std::string query = fmt::format( + "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " + "attnotnull AS not_null, attndims AS dims " + "FROM pg_attribute " + "WHERE attrelid = '{}'::regclass " + "AND NOT attisdropped AND attnum > 0", postgres_table_name); + pqxx::read_transaction tx(*connection); + pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); + std::tuple row; + + /// No rows to be fetched + if (!stream) + return nullptr; + + while (stream >> row) + { + columns.push_back(NameAndTypePair( + std::get<0>(row), + getDataType(std::get<1>(row), use_nulls && (std::get<2>(row) == "f"), std::get<3>(row)))); + } + stream.complete(); + tx.commit(); + + return std::make_shared(columns); +} + + +DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimensions) +{ + DataTypePtr res; + + /// Get rid of trailing '[]' for arrays + if (dimensions) + type.resize(type.size() - 2); + + if (type == "smallint") + res = std::make_shared(); + else if (type == "integer") + res = std::make_shared(); + else if (type == "bigint") + res = std::make_shared(); + else if (type == "real") + res = std::make_shared(); + else if (type == "double precision") + res = std::make_shared(); + else if (type == "serial") + res = std::make_shared(); + else if (type == "bigserial") + res = std::make_shared(); + else if (type.starts_with("timestamp")) + res = std::make_shared(); + else if (type == "date") + res = std::make_shared(); + else if (type.starts_with("numeric")) + { + /// Numeric and decimal will both end up here as numeric + /// Will get numeric(precision, scale) string, need to extract precision and scale + std::vector result; + boost::split(result, type, [](char c){ return c == '(' || c == ',' || c == ')'; }); + for (std::string & key : result) + boost::trim(key); + + /// If precision or scale are not specified, postgres creates a column in which numeric values of + /// any precision and scale can be stored, so may be maxPrecision may be used instead of exception + if (result.size() < 3) + throw Exception("Numeric lacks precision and scale in its definition", ErrorCodes::UNKNOWN_TYPE); + + uint32_t precision = std::atoi(result[1].data()); + uint32_t scale = std::atoi(result[2].data()); + + if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + } + + if (!res) + res = std::make_shared(); + if (is_nullable) + res = std::make_shared(res); + while (dimensions--) + res = std::make_shared(res); + + return res; +} + +} + +#endif diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h new file mode 100644 index 00000000000..1010fd2404b --- /dev/null +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h @@ -0,0 +1,18 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX +#include + +namespace DB +{ + +std::shared_ptr fetchTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls); +DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimensions); + +} + +#endif diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index cb3f9804e8a..ebe12f897a7 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -19,11 +19,11 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix, - const std::string & connection_str, + PGConnectionPtr connection_, const Block & sample_block_) : dict_struct{dict_struct_} , sample_block(sample_block_) - , connection(std::make_shared(connection_str)) + , connection(std::move(connection_)) , log(&Poco::Logger::get("PostgreSQLDictionarySource")) , db(config_.getString(fmt::format("{}.db", config_prefix), "")) , table(config_.getString(fmt::format("{}.table", config_prefix), "")) @@ -40,7 +40,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionarySource & other) : dict_struct(other.dict_struct) , sample_block(other.sample_block) - , connection(other.connection) + , connection(std::make_shared(other.connection->conn_str())) , log(&Poco::Logger::get("PostgreSQLDictionarySource")) , db(other.db) , table(other.table) @@ -59,7 +59,7 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); return std::make_shared( - connection, load_all_query, sample_block, max_block_size); + connection->conn(), load_all_query, sample_block, max_block_size); } @@ -67,20 +67,20 @@ BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() { auto load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); - return std::make_shared(connection, load_update_query, sample_block, max_block_size); + return std::make_shared(connection->conn(), load_update_query, sample_block, max_block_size); } BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); - return std::make_shared(connection, query, sample_block, max_block_size); + return std::make_shared(connection->conn(), query, sample_block, max_block_size); } BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); - return std::make_shared(connection, query, sample_block, max_block_size); + return std::make_shared(connection->conn(), query, sample_block, max_block_size); } @@ -102,7 +102,7 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - PostgreSQLBlockInputStream block_input_stream(connection, request, invalidate_sample_block, 1); + PostgreSQLBlockInputStream block_input_stream(connection->conn(), request, invalidate_sample_block, 1); return readInvalidateQuery(block_input_stream); } @@ -167,9 +167,10 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) config.getUInt(fmt::format("{}.port", config_prefix), 0), config.getString(fmt::format("{}.user", config_prefix), ""), config.getString(fmt::format("{}.password", config_prefix), "")); + auto connection = std::make_shared(connection_str); return std::make_unique( - dict_struct, config, config_prefix, connection_str, sample_block); + dict_struct, config, config_prefix, connection, sample_block); #else (void)dict_struct; (void)config; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 30e90d7dde5..e59ab81674c 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -11,11 +11,11 @@ #include #include #include +#include #include namespace DB { -using ConnectionPtr = std::shared_ptr; /// Allows loading dictionaries from a PostgreSQL database class PostgreSQLDictionarySource final : public IDictionarySource @@ -25,7 +25,7 @@ public: const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix, - const std::string & connection_str, + PGConnectionPtr connection_, const Block & sample_block_); /// copy-constructor is provided in order to support cloneability @@ -50,7 +50,7 @@ private: const DictionaryStructure dict_struct; Block sample_block; - ConnectionPtr connection; + PGConnectionPtr connection; Poco::Logger * log; const std::string db; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 7be396425b0..7ecb0f5dbc8 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -31,8 +31,7 @@ namespace ErrorCodes StoragePostgreSQL::StoragePostgreSQL( const StorageID & table_id_, const String & remote_table_name_, - ConnectionPtr connection_, - const String connection_str_, + PGConnectionPtr connection_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) @@ -40,7 +39,6 @@ StoragePostgreSQL::StoragePostgreSQL( , remote_table_name(remote_table_name_) , global_context(context_) , connection(std::move(connection_)) - , connection_str(connection_str_) { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); @@ -74,22 +72,20 @@ Pipe StoragePostgreSQL::read( sample_block.insert({ column_data.type, column_data.name }); } - checkConnection(connection); return Pipe(std::make_shared( - std::make_shared(connection, query, sample_block, max_block_size_))); + std::make_shared(connection->conn(), query, sample_block, max_block_size_))); } BlockOutputStreamPtr StoragePostgreSQL::write( const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */) { - return std::make_shared(*this, metadata_snapshot, connection, remote_table_name); + return std::make_shared(metadata_snapshot, connection->conn(), remote_table_name); } void PostgreSQLBlockOutputStream::writePrefix() { - storage.checkConnection(connection); work = std::make_unique(*connection); } @@ -144,16 +140,6 @@ void PostgreSQLBlockOutputStream::writeSuffix() } -void StoragePostgreSQL::checkConnection(ConnectionPtr & pg_connection) const -{ - if (!pg_connection->is_open()) - { - pg_connection->close(); - pg_connection = std::make_shared(connection_str); - } -} - - void registerStoragePostgreSQL(StorageFactory & factory) { factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args) @@ -178,9 +164,9 @@ void registerStoragePostgreSQL(StorageFactory & factory) engine_args[3]->as().value.safeGet(), engine_args[4]->as().value.safeGet()); - auto connection = std::make_shared(connection_str); + auto connection = std::make_shared(connection_str); return StoragePostgreSQL::create( - args.table_id, remote_table, connection, connection_str, args.columns, args.constraints, args.context); + args.table_id, remote_table, connection, args.columns, args.constraints, args.context); }, { .source_access_type = AccessType::POSTGRES, diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index af061f79545..c7f20828115 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -14,6 +14,8 @@ namespace DB { +class PGConnection; +using PGConnectionPtr = std::shared_ptr; using ConnectionPtr = std::shared_ptr; class StoragePostgreSQL final : public ext::shared_ptr_helper, public IStorage @@ -23,8 +25,7 @@ public: StoragePostgreSQL( const StorageID & table_id_, const std::string & remote_table_name_, - ConnectionPtr connection_, - const String connection_str, + PGConnectionPtr connection_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_); @@ -44,13 +45,10 @@ public: private: friend class PostgreSQLBlockOutputStream; - void checkConnection(ConnectionPtr & connection) const; String remote_table_name; Context global_context; - - ConnectionPtr connection; - const String connection_str; + PGConnectionPtr connection; }; @@ -58,12 +56,10 @@ class PostgreSQLBlockOutputStream : public IBlockOutputStream { public: explicit PostgreSQLBlockOutputStream( - const StoragePostgreSQL & storage_, const StorageMetadataPtr & metadata_snapshot_, ConnectionPtr connection_, const std::string & remote_table_name_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) + : metadata_snapshot(metadata_snapshot_) , connection(connection_) , remote_table_name(remote_table_name_) { @@ -76,7 +72,6 @@ public: void writeSuffix() override; private: - const StoragePostgreSQL & storage; StorageMetadataPtr metadata_snapshot; ConnectionPtr connection; std::string remote_table_name; @@ -84,6 +79,35 @@ private: std::unique_ptr work; std::unique_ptr stream_inserter; }; + + +/// Tiny connection class to make it more convenient to use. +class PGConnection +{ +public: + PGConnection(std::string & connection_str_) : connection_str(connection_str_) {} + PGConnection(const PGConnection &) = delete; + PGConnection operator =(const PGConnection &) = delete; + + ConnectionPtr conn() + { + checkUpdateConnection(); + return connection; + } + + std::string & conn_str() { return connection_str; } + +private: + ConnectionPtr connection; + std::string connection_str; + + void checkUpdateConnection() + { + if (!connection || !connection->is_open()) + connection = std::make_unique(connection_str); + } +}; + } #endif diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 7b8e98c50f6..a853a69ad5e 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -1,13 +1,6 @@ #include #if USE_LIBPQXX -#include -#include -#include -#include -#include -#include -#include #include #include #include @@ -16,7 +9,7 @@ #include #include #include "registerTableFunctions.h" -#include +#include namespace DB { @@ -35,7 +28,7 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, auto columns = getActualTableStructure(context); auto result = std::make_shared( StorageID(getDatabaseName(), table_name), remote_table_name, - connection, connection_str, columns, ConstraintsDescription{}, context); + connection, columns, ConstraintsDescription{}, context); result->startup(); return result; @@ -45,90 +38,9 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(const Context & context) const { const bool use_nulls = context.getSettingsRef().external_table_functions_use_nulls; - auto columns = NamesAndTypesList(); + auto columns = fetchTableStructure(connection->conn(), remote_table_name, use_nulls); - std::string query = fmt::format( - "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " - "attnotnull AS not_null, attndims AS dims " - "FROM pg_attribute " - "WHERE attrelid = '{}'::regclass " - "AND NOT attisdropped AND attnum > 0", remote_table_name); - pqxx::read_transaction tx(*connection); - pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); - std::tuple row; - - while (stream >> row) - { - columns.push_back(NameAndTypePair( - std::get<0>(row), - getDataType(std::get<1>(row), use_nulls && (std::get<2>(row) == "f"), std::get<3>(row)))); - } - stream.complete(); - tx.commit(); - - return ColumnsDescription{columns}; -} - - -DataTypePtr TableFunctionPostgreSQL::getDataType(std::string & type, bool is_nullable, uint16_t dimensions) const -{ - DataTypePtr res; - - /// Get rid of trailing '[]' for arrays - if (dimensions) - type.resize(type.size() - 2); - - if (type == "smallint") - res = std::make_shared(); - else if (type == "integer") - res = std::make_shared(); - else if (type == "bigint") - res = std::make_shared(); - else if (type == "real") - res = std::make_shared(); - else if (type == "double precision") - res = std::make_shared(); - else if (type == "serial") - res = std::make_shared(); - else if (type == "bigserial") - res = std::make_shared(); - else if (type.starts_with("timestamp")) - res = std::make_shared(); - else if (type == "date") - res = std::make_shared(); - else if (type.starts_with("numeric")) - { - /// Numeric and decimal will both end up here as numeric - /// Will get numeric(precision, scale) string, need to extract precision and scale - std::vector result; - boost::split(result, type, [](char c){ return c == '(' || c == ',' || c == ')'; }); - for (std::string & key : result) - boost::trim(key); - - /// If precision or scale are not specified, postgres creates a column in which numeric values of - /// any precision and scale can be stored, so may be maxPrecision may be used instead of exception - if (result.size() < 3) - throw Exception("Numeric lacks precision and scale in its definition", ErrorCodes::UNKNOWN_TYPE); - - uint32_t precision = std::atoi(result[1].data()); - uint32_t scale = std::atoi(result[2].data()); - - if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - } - - if (!res) - res = std::make_shared(); - if (is_nullable) - res = std::make_shared(res); - while (dimensions--) - res = std::make_shared(res); - - return res; + return ColumnsDescription{*columns}; } @@ -157,7 +69,7 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, const parsed_host_port.first, std::to_string(parsed_host_port.second), args[3]->as().value.safeGet(), args[4]->as().value.safeGet()); - connection = std::make_shared(connection_str); + connection = std::make_shared(connection_str); } diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index 756d6b2996f..5bd24322d3b 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -5,11 +5,11 @@ #if USE_LIBPQXX #include +#include #include "pqxx/pqxx" namespace DB { -using ConnectionPtr = std::shared_ptr; class TableFunctionPostgreSQL : public ITableFunction { @@ -27,11 +27,9 @@ private: ColumnsDescription getActualTableStructure(const Context & context) const override; void parseArguments(const ASTPtr & ast_function, const Context & context) override; - DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimensions) const; - String connection_str; String remote_table_name; - ConnectionPtr connection; + PGConnectionPtr connection; }; } diff --git a/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml b/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml index e3224b39a7d..2572930a798 100644 --- a/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml +++ b/tests/integration/test_dictionaries_postgresql/configs/postgres_dict.xml @@ -32,9 +32,6 @@ - - 1 - 1 - + 1
diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index cd7d575a999..b83c00409af 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -1,8 +1,8 @@ import pytest import time import psycopg2 + from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT cluster = ClickHouseCluster(__file__) diff --git a/tests/integration/test_postgresql_database_engine/__init__.py b/tests/integration/test_postgresql_database_engine/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py new file mode 100644 index 00000000000..70f6a02f9e6 --- /dev/null +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -0,0 +1,122 @@ +import pytest +import time +import psycopg2 + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry +from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=[], with_postgres=True) + +postgres_table_template = """ + CREATE TABLE IF NOT EXISTS {} ( + id Integer NOT NULL, value Integer, PRIMARY KEY (id)) + """ + +def get_postgres_conn(database=False): + if database == True: + conn_string = "host='localhost' dbname='test_database' user='postgres' password='mysecretpassword'" + else: + conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn = psycopg2.connect(conn_string) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + +def create_postgres_db(cursor, name): + cursor.execute("CREATE DATABASE {}".format(name)) + +def create_postgres_table(cursor, table_name): + # database was specified in connection string + cursor.execute(postgres_table_template.format(table_name)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + conn = get_postgres_conn() + cursor = conn.cursor() + create_postgres_db(cursor, 'test_database') + yield cluster + + finally: + cluster.shutdown() + + +def test_postgres_database_engine_with_postgres_ddl(started_cluster): + # connect to database as well + conn = get_postgres_conn(True) + cursor = conn.cursor() + + node1.query( + "CREATE DATABASE test_database ENGINE = PostgreSQL('postgres1:5432', 'test_database', 'postgres', 'mysecretpassword')") + assert 'test_database' in node1.query('SHOW DATABASES') + + create_postgres_table(cursor, 'test_table') + assert 'test_table' in node1.query('SHOW TABLES FROM test_database') + + cursor.execute('ALTER TABLE test_table ADD COLUMN data Text') + assert 'data' in node1.query("SELECT name FROM system.columns WHERE table = 'test_table' AND database = 'test_database'") + + node1.query("INSERT INTO test_database.test_table SELECT 101, 101, toString(101)") + assert node1.query("SELECT data FROM test_database.test_table WHERE id = 101").rstrip() == '101' + + cursor.execute('ALTER TABLE test_table DROP COLUMN data') + assert 'data' not in node1.query("SELECT name FROM system.columns WHERE table = 'test_table' AND database = 'test_database'") + + cursor.execute('DROP TABLE test_table;') + assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') + + node1.query("DROP DATABASE test_database") + assert 'test_database' not in node1.query('SHOW DATABASES') + + +def test_postgresql_database_engine_with_clickhouse_ddl(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + + node1.query( + "CREATE DATABASE test_database ENGINE = PostgreSQL('postgres1:5432', 'test_database', 'postgres', 'mysecretpassword')") + + create_postgres_table(cursor, 'test_table') + assert 'test_table' in node1.query('SHOW TABLES FROM test_database') + + node1.query("DROP TABLE test_database.test_table") + assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') + + node1.query("ATTACH TABLE test_database.test_table") + assert 'test_table' in node1.query('SHOW TABLES FROM test_database') + + node1.query("DETACH TABLE test_database.test_table") + assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') + + node1.query("ATTACH TABLE test_database.test_table") + assert 'test_table' in node1.query('SHOW TABLES FROM test_database') + + node1.query("DROP DATABASE test_database") + assert 'test_database' not in node1.query('SHOW DATABASES') + + +def test_postgresql_database_engine_queries(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + + node1.query( + "CREATE DATABASE test_database ENGINE = PostgreSQL('postgres1:5432', 'test_database', 'postgres', 'mysecretpassword')") + + create_postgres_table(cursor, 'test_table') + assert node1.query("SELECT count() FROM test_database.test_table").rstrip() == '0' + + node1.query("INSERT INTO test_database.test_table SELECT number, number from numbers(10000)") + assert node1.query("SELECT count() FROM test_database.test_table").rstrip() == '10000' + + node1.query("DROP DATABASE test_database") + assert 'test_database' not in node1.query('SHOW DATABASES') + + +if __name__ == '__main__': + cluster.start() + input("Cluster created, press any key to destroy...") + cluster.shutdown() From 82074bfa653af395118039e79c138fe86fda8612 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 27 Dec 2020 14:17:15 +0000 Subject: [PATCH 050/264] Update libpq --- .gitmodules | 6 +++--- contrib/libpq | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.gitmodules b/.gitmodules index 066d4c5e809..954cea58a15 100644 --- a/.gitmodules +++ b/.gitmodules @@ -209,12 +209,12 @@ [submodule "contrib/fast_float"] path = contrib/fast_float url = https://github.com/fastfloat/fast_float -[submodule "contrib/libpq"] - path = contrib/libpq - url = https://github.com/kssenii/libpq [submodule "contrib/libpqxx"] path = contrib/libpqxx url = https://github.com/jtv/libpqxx +[submodule "contrib/libpq"] + path = contrib/libpq + url = https://github.com/kssenii/libpq [submodule "contrib/boringssl"] path = contrib/boringssl url = https://github.com/ClickHouse-Extras/boringssl.git diff --git a/contrib/libpq b/contrib/libpq index d53cfd7f523..e2154c47525 160000 --- a/contrib/libpq +++ b/contrib/libpq @@ -1 +1 @@ -Subproject commit d53cfd7f5237e0765ca4c12c6408fdd722b954e1 +Subproject commit e2154c4752515623e52f437e2c95138b2c54b5d1 From 8f8920a7ee3997201d49fe1ee44eafaed570514d Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 26 Dec 2020 19:38:10 +0000 Subject: [PATCH 051/264] Add table cache, better drop table --- src/Core/ExternalResultDescription.cpp | 2 +- .../PostgreSQLBlockInputStream.cpp | 2 +- src/Databases/DatabaseFactory.cpp | 12 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 272 ++++++++++++------ src/Databases/PostgreSQL/DatabasePostgreSQL.h | 27 +- .../PostgreSQL/FetchFromPostgreSQL.cpp | 8 +- .../PostgreSQLDictionarySource.cpp | 6 + src/Storages/StoragePostgreSQL.cpp | 1 - src/Storages/StoragePostgreSQL.h | 1 + .../TableFunctionPostgreSQL.cpp | 1 - src/TableFunctions/registerTableFunctions.h | 2 - .../test_postgresql_database_engine/test.py | 46 ++- 12 files changed, 249 insertions(+), 131 deletions(-) diff --git a/src/Core/ExternalResultDescription.cpp b/src/Core/ExternalResultDescription.cpp index 792e1c30eae..c891988ec11 100644 --- a/src/Core/ExternalResultDescription.cpp +++ b/src/Core/ExternalResultDescription.cpp @@ -77,7 +77,7 @@ void ExternalResultDescription::init(const Block & sample_block_) types.emplace_back(ValueType::vtDecimal128, is_nullable); else if (typeid_cast *>(type)) types.emplace_back(ValueType::vtDecimal256, is_nullable); - else if (typeid_cast(type)) + else if (typeid_cast(type)) types.emplace_back(ValueType::vtArray, is_nullable); else throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE}; diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 86dcd105c5e..bffab4d4841 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -41,7 +41,7 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( for (const auto idx : ext::range(0, description.sample_block.columns())) if (description.types[idx].first == ValueType::vtArray) prepareArrayInfo(idx, description.sample_block.getByPosition(idx).type); - /// pqxx::stream_from uses COPY command, but when selecting from dictionary will get ';', it is not needed + /// pqxx::stream_from uses COPY command, will get error if ';' is present if (query_str.ends_with(';')) query_str.resize(query_str.size() - 1); } diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 703df2ee44b..457b99a88d3 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -178,10 +178,10 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String { const ASTFunction * engine = engine_define->engine; - if (!engine->arguments || engine->arguments->children.size() != 4) + if (!engine->arguments || engine->arguments->children.size() < 4 || engine->arguments->children.size() > 5) throw Exception(fmt::format( - "{} Database require postgres_host_port, postgres_dbname, " - "postgres_username, mysql_password arguments.", engine_name), + "{} Database require host:port, database_name, username, password arguments " + "[, use_table_cache = 0].", engine_name), ErrorCodes::BAD_ARGUMENTS); ASTs & engine_args = engine->arguments->children; @@ -194,6 +194,10 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String const auto & username = safeGetLiteralValue(engine_args[2], engine_name); const auto & password = safeGetLiteralValue(engine_args[3], engine_name); + auto use_table_cache = 0; + if (engine->arguments->children.size() == 5) + use_table_cache = safeGetLiteralValue(engine_args[4], engine_name); + auto parsed_host_port = parseAddress(host_port, 5432); String connection_str; connection_str = fmt::format("dbname={} host={} port={} user={} password={}", @@ -203,7 +207,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String auto connection = std::make_shared(connection_str); return std::make_shared( - context, metadata_path, engine_define, database_name, postgres_database_name, connection); + context, metadata_path, engine_define, database_name, postgres_database_name, connection, use_table_cache); } #endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index cf61cb884e8..49723dc2793 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -1,12 +1,8 @@ #include #if USE_LIBPQXX -#include -#include + #include -#include -#include -#include #include #include #include @@ -14,15 +10,8 @@ #include #include #include -#include -#include #include #include - -#include - -#include -#include #include @@ -32,39 +21,43 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; extern const int UNKNOWN_TABLE; extern const int TABLE_IS_DROPPED; extern const int TABLE_ALREADY_EXISTS; - extern const int UNEXPECTED_AST_STRUCTURE; } +static const auto suffix = ".removed"; +static const auto cleaner_reschedule_ms = 60000; + DatabasePostgreSQL::DatabasePostgreSQL( const Context & context, const String & metadata_path_, const ASTStorage * database_engine_define_, const String & dbname_, const String & postgres_dbname, - PGConnectionPtr connection_) + PGConnectionPtr connection_, + const bool cache_tables_) : IDatabase(dbname_) , global_context(context.getGlobalContext()) , metadata_path(metadata_path_) , database_engine_define(database_engine_define_->clone()) , dbname(postgres_dbname) , connection(std::move(connection_)) + , cache_tables(cache_tables_) { + cleaner_task = context.getSchedulePool().createTask("PostgreSQLCleanerTask", [this]{ removeOutdatedTables(); }); + cleaner_task->deactivate(); } bool DatabasePostgreSQL::empty() const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "empty"); std::lock_guard lock(mutex); auto tables_list = fetchTablesList(); for (const auto & table_name : tables_list) - if (!detached_tables.count(table_name)) + if (!detached_or_dropped.count(table_name)) return false; return true; @@ -74,15 +67,14 @@ bool DatabasePostgreSQL::empty() const DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( const Context & context, const FilterByNameFunction & /* filter_by_table_name */) { - LOG_TRACE(&Poco::Logger::get("kssenii"), "getTablesIterator"); std::lock_guard lock(mutex); Tables tables; auto table_names = fetchTablesList(); for (auto & table_name : table_names) - if (!detached_tables.count(table_name)) - tables[table_name] = fetchTable(table_name, context); + if (!detached_or_dropped.count(table_name)) + tables[table_name] = fetchTable(table_name, context, true); return std::make_unique(tables, database_name); } @@ -90,11 +82,10 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( std::unordered_set DatabasePostgreSQL::fetchTablesList() const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "fetchTablesList"); - std::unordered_set tables; std::string query = "SELECT tablename FROM pg_catalog.pg_tables " "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'"; + /// Already connected to the needed database, search will be done there pqxx::read_transaction tx(*connection->conn()); for (auto table_name : tx.stream(query)) @@ -108,9 +99,8 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const { pqxx::nontransaction tx(*connection->conn()); pqxx::result result = tx.exec(fmt::format( - "SELECT attname FROM pg_attribute " - "WHERE attrelid = '{}'::regclass " - "AND NOT attisdropped AND attnum > 0", table_name)); + "SELECT tablename FROM pg_catalog.pg_tables " + "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema' AND tablename = '{}'", table_name)); if (result.empty()) return false; @@ -121,10 +111,9 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const bool DatabasePostgreSQL::isTableExist(const String & table_name, const Context & /* context */) const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "isTableExists"); std::lock_guard lock(mutex); - if (detached_tables.count(table_name)) + if (detached_or_dropped.count(table_name)) return false; return checkPostgresTable(table_name); @@ -133,65 +122,94 @@ bool DatabasePostgreSQL::isTableExist(const String & table_name, const Context & StoragePtr DatabasePostgreSQL::tryGetTable(const String & table_name, const Context & context) const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "tryGetTable"); std::lock_guard lock(mutex); - if (detached_tables.count(table_name)) - return StoragePtr{}; - else - return fetchTable(table_name, context); + if (!detached_or_dropped.count(table_name)) + return fetchTable(table_name, context, false); + + return StoragePtr{}; } -StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Context & context) const +StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Context & context, const bool table_checked) const { - auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; - auto columns = fetchTableStructure(connection->conn(), table_name, use_nulls); + if (!cache_tables || !cached_tables.count(table_name)) + { + if (!table_checked && !checkPostgresTable(table_name)) + return StoragePtr{}; - if (!columns) - return StoragePtr{}; + auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; + auto columns = fetchTableStructure(connection->conn(), table_name, use_nulls); - return StoragePostgreSQL::create( - StorageID(database_name, table_name), table_name, - connection, ColumnsDescription{*columns}, ConstraintsDescription{}, context); + if (!columns) + return StoragePtr{}; + + auto storage = StoragePostgreSQL::create( + StorageID(database_name, table_name), table_name, std::make_shared(connection->conn_str()), + ColumnsDescription{*columns}, ConstraintsDescription{}, context); + + /// There is no easy (embedded) way in postgres to check table modification time, so if `cache_tables` == 1 (default: 0) + /// table structure is cached and not checked for being modififed, but it will be updated during detach->attach. + if (cache_tables) + cached_tables[table_name] = storage; + + return storage; + } + + if (table_checked || checkPostgresTable(table_name)) + { + return cached_tables[table_name]; + } + + /// Table does not exist anymore + cached_tables.erase(table_name); + return StoragePtr{}; } -void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr & /* storage */, const String &) +void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr & storage, const String &) { - LOG_TRACE(&Poco::Logger::get("kssenii"), "attachTable"); std::lock_guard lock{mutex}; if (!checkPostgresTable(table_name)) throw Exception(fmt::format("Cannot attach table {}.{} because it does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); - if (!detached_tables.count(table_name)) + if (!detached_or_dropped.count(table_name)) throw Exception(fmt::format("Cannot attach table {}.{}. It already exists", database_name, table_name), ErrorCodes::TABLE_ALREADY_EXISTS); - detached_tables.erase(table_name); + if (cache_tables) + cached_tables[table_name] = storage; + + detached_or_dropped.erase(table_name); + + Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + if (table_marked_as_removed.exists()) + table_marked_as_removed.remove(); } StoragePtr DatabasePostgreSQL::detachTable(const String & table_name) { - LOG_TRACE(&Poco::Logger::get("kssenii"), "detachTable"); std::lock_guard lock{mutex}; + if (detached_or_dropped.count(table_name)) + throw Exception(fmt::format("Cannot detach table {}.{}. It is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); + if (!checkPostgresTable(table_name)) throw Exception(fmt::format("Cannot detach table {}.{} because it does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); - if (detached_tables.count(table_name)) - throw Exception(fmt::format("Cannot detach table {}.{}. It is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); + if (cache_tables) + cached_tables.erase(table_name); - detached_tables.emplace(table_name); + detached_or_dropped.emplace(table_name); + + /// not used anywhere (for postgres database) return StoragePtr{}; } void DatabasePostgreSQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) { - LOG_TRACE(&Poco::Logger::get("kssenii"), "createTable"); - const auto & create = create_query->as(); if (!create->attach) @@ -201,18 +219,31 @@ void DatabasePostgreSQL::createTable(const Context &, const String & table_name, } -void DatabasePostgreSQL::dropTable(const Context &, const String & table_name, bool /*no_delay*/) +void DatabasePostgreSQL::dropTable(const Context &, const String & table_name, bool /* no_delay */) { - LOG_TRACE(&Poco::Logger::get("kssenii"), "detachPermanently"); std::lock_guard lock{mutex}; if (!checkPostgresTable(table_name)) throw Exception(fmt::format("Cannot drop table {}.{} because it does not exist", database_name, table_name), ErrorCodes::UNKNOWN_TABLE); - if (detached_tables.count(table_name)) + if (detached_or_dropped.count(table_name)) throw Exception(fmt::format("Table {}.{} is already dropped/detached", database_name, table_name), ErrorCodes::TABLE_IS_DROPPED); - detached_tables.emplace(table_name); + Poco::File mark_table_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + + try + { + mark_table_removed.createFile(); + } + catch (...) + { + throw; + } + + if (cache_tables) + cached_tables.erase(table_name); + + detached_or_dropped.emplace(table_name); } @@ -222,10 +253,71 @@ void DatabasePostgreSQL::drop(const Context & /*context*/) } +void DatabasePostgreSQL::loadStoredObjects(Context & /* context */, bool, bool /*force_attach*/) +{ + { + std::lock_guard lock{mutex}; + Poco::DirectoryIterator iterator(getMetadataPath()); + + /// Check for previously droppped tables + for (Poco::DirectoryIterator end; iterator != end; ++iterator) + { + if (iterator->isFile() && endsWith(iterator.name(), suffix)) + { + const auto & file_name = iterator.name(); + const auto & table_name = unescapeForFileName(file_name.substr(0, file_name.size() - strlen(suffix))); + detached_or_dropped.emplace(table_name); + } + } + } + + cleaner_task->activateAndSchedule(); +} + + +void DatabasePostgreSQL::removeOutdatedTables() +{ + std::lock_guard lock{mutex}; + auto actual_tables = fetchTablesList(); + + if (cache_tables) + { + /// (Tables are cached only after being accessed at least once) + for (auto iter = cached_tables.begin(); iter != cached_tables.end();) + { + if (!actual_tables.count(iter->first)) + iter = cached_tables.erase(iter); + else + ++iter; + } + } + + for (auto iter = detached_or_dropped.begin(); iter != detached_or_dropped.end();) + { + if (!actual_tables.count(*iter)) + { + auto table_name = *iter; + iter = detached_or_dropped.erase(iter); + Poco::File table_marked_as_removed(getMetadataPath() + '/' + escapeForFileName(table_name) + suffix); + if (table_marked_as_removed.exists()) + table_marked_as_removed.remove(); + } + else + ++iter; + } + + cleaner_task->scheduleAfter(cleaner_reschedule_ms); +} + + +void DatabasePostgreSQL::shutdown() +{ + cleaner_task->deactivate(); +} + + ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "getDatabaseQuery"); - const auto & create_query = std::make_shared(); create_query->database = getDatabaseName(); create_query->set(create_query->storage, database_engine_define); @@ -235,9 +327,7 @@ ASTPtr DatabasePostgreSQL::getCreateDatabaseQuery() const ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "getTableQueryImpl"); - - auto storage = fetchTable(table_name, context); + auto storage = fetchTable(table_name, context, false); if (!storage) { if (throw_on_error) @@ -246,8 +336,6 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, co return nullptr; } - /// Get create table query from storage - auto create_table_query = std::make_shared(); auto table_storage_define = database_engine_define->clone(); create_table_query->set(create_table_query->storage, table_storage_define); @@ -258,46 +346,40 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, co columns_declare_list->set(columns_declare_list->columns, columns_expression_list); create_table_query->set(create_table_query->columns_list, columns_declare_list); + /// init create query. + auto table_id = storage->getStorageID(); + create_table_query->table = table_id.table_name; + create_table_query->database = table_id.database_name; + + auto metadata_snapshot = storage->getInMemoryMetadataPtr(); + for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) { - /// init create query. - auto table_id = storage->getStorageID(); - create_table_query->table = table_id.table_name; - create_table_query->database = table_id.database_name; + const auto & column_declaration = std::make_shared(); + column_declaration->name = column_type_and_name.name; - auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) + std::function convert_datatype_to_query = [&](const DataTypePtr & data_type) -> ASTPtr { - const auto & column_declaration = std::make_shared(); - column_declaration->name = column_type_and_name.name; + WhichDataType which(data_type); + if (!which.isNullable()) + return std::make_shared(data_type->getName()); + return makeASTFunction("Nullable", convert_datatype_to_query(typeid_cast(data_type.get())->getNestedType())); + }; - std::function convert_datatype_to_query = [&](const DataTypePtr & data_type) -> ASTPtr - { - WhichDataType which(data_type); - if (!which.isNullable()) - return std::make_shared(data_type->getName()); - return makeASTFunction("Nullable", convert_datatype_to_query(typeid_cast(data_type.get())->getNestedType())); - }; - - column_declaration->type = convert_datatype_to_query(column_type_and_name.type); - columns_expression_list->children.emplace_back(column_declaration); - } - - ASTStorage * ast_storage = table_storage_define->as(); - ASTs storage_children = ast_storage->children; - auto storage_engine_arguments = ast_storage->engine->arguments; - - /// Add table_name to engine arguments - auto mysql_table_name = std::make_shared(table_id.table_name); - storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); - - /// Unset settings - storage_children.erase( - std::remove_if(storage_children.begin(), storage_children.end(), - [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }), - storage_children.end()); - ast_storage->settings = nullptr; + column_declaration->type = convert_datatype_to_query(column_type_and_name.type); + columns_expression_list->children.emplace_back(column_declaration); } + ASTStorage * ast_storage = table_storage_define->as(); + ASTs storage_children = ast_storage->children; + auto storage_engine_arguments = ast_storage->engine->arguments; + + /// Remove extra engine argument (`use_table_cache`) + if (storage_engine_arguments->children.size() > 4) + storage_engine_arguments->children.resize(storage_engine_arguments->children.size() - 1); + + /// Add table_name to engine arguments + storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, std::make_shared(table_id.table_name)); + return create_table_query; } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 14a48efd578..7a10b1cb319 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -6,19 +6,9 @@ #if USE_LIBPQXX -#include -#include #include +#include #include - -#include -#include -#include -#include -#include -#include -#include - #include #include @@ -37,7 +27,8 @@ public: const ASTStorage * database_engine_define, const String & dbname_, const String & postgres_dbname, - PGConnectionPtr connection_); + PGConnectionPtr connection_, + const bool cache_tables_); String getEngineName() const override { return "PostgreSQL"; } String getMetadataPath() const override { return metadata_path; }; @@ -50,6 +41,8 @@ public: bool empty() const override; + void loadStoredObjects(Context &, bool, bool force_attach) override; + DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override; bool isTableExist(const String & name, const Context & context) const override; @@ -62,7 +55,7 @@ public: StoragePtr detachTable(const String & table_name) override; void drop(const Context & /*context*/) override; - void shutdown() override {}; + void shutdown() override; protected: ASTPtr getCreateTableQueryImpl(const String & table_name, const Context & context, bool throw_on_error) const override; @@ -73,12 +66,16 @@ private: ASTPtr database_engine_define; String dbname; PGConnectionPtr connection; + const bool cache_tables; - std::unordered_set detached_tables; + mutable Tables cached_tables; + std::unordered_set detached_or_dropped; + BackgroundSchedulePool::TaskHolder cleaner_task; bool checkPostgresTable(const String & table_name) const; std::unordered_set fetchTablesList() const; - StoragePtr fetchTable(const String & table_name, const Context & context) const; + StoragePtr fetchTable(const String & table_name, const Context & context, const bool table_checked) const; + void removeOutdatedTables(); }; } diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index afe7f6a9a7d..1d52e2502f0 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -21,7 +21,6 @@ namespace ErrorCodes extern const int UNKNOWN_TYPE; } -/// These functions are also used for postgresql table function std::shared_ptr fetchTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls) { @@ -37,10 +36,6 @@ std::shared_ptr fetchTableStructure(ConnectionPtr connection, pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); std::tuple row; - /// No rows to be fetched - if (!stream) - return nullptr; - while (stream >> row) { columns.push_back(NameAndTypePair( @@ -50,6 +45,9 @@ std::shared_ptr fetchTableStructure(ConnectionPtr connection, stream.complete(); tx.commit(); + if (columns.empty()) + return nullptr; + return std::make_shared(columns); } diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index ebe12f897a7..7b9a69b1b79 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -13,6 +13,12 @@ namespace DB { + +namespace ErrorCodes +{ + extern const int SUPPORT_IS_DISABLED; +} + static const UInt64 max_block_size = 8192; PostgreSQLDictionarySource::PostgreSQLDictionarySource( diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 7ecb0f5dbc8..4acc1dce286 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -25,7 +25,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; } StoragePostgreSQL::StoragePostgreSQL( diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index c7f20828115..4a25e3459e5 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -82,6 +82,7 @@ private: /// Tiny connection class to make it more convenient to use. +/// Connection is not made untill actually used. class PGConnection { public: diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index a853a69ad5e..21bf957860d 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -18,7 +18,6 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_TYPE; } diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 0d8438a8ad8..7e9a8ab5b61 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -41,8 +41,6 @@ void registerTableFunctionMySQL(TableFunctionFactory & factory); void registerTableFunctionPostgreSQL(TableFunctionFactory & factory); #endif - - void registerTableFunctions(); } diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index 70f6a02f9e6..c76a3bdaa41 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -60,15 +60,9 @@ def test_postgres_database_engine_with_postgres_ddl(started_cluster): cursor.execute('ALTER TABLE test_table ADD COLUMN data Text') assert 'data' in node1.query("SELECT name FROM system.columns WHERE table = 'test_table' AND database = 'test_database'") - node1.query("INSERT INTO test_database.test_table SELECT 101, 101, toString(101)") - assert node1.query("SELECT data FROM test_database.test_table WHERE id = 101").rstrip() == '101' - cursor.execute('ALTER TABLE test_table DROP COLUMN data') assert 'data' not in node1.query("SELECT name FROM system.columns WHERE table = 'test_table' AND database = 'test_database'") - cursor.execute('DROP TABLE test_table;') - assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') - node1.query("DROP DATABASE test_database") assert 'test_database' not in node1.query('SHOW DATABASES') @@ -112,6 +106,46 @@ def test_postgresql_database_engine_queries(started_cluster): node1.query("INSERT INTO test_database.test_table SELECT number, number from numbers(10000)") assert node1.query("SELECT count() FROM test_database.test_table").rstrip() == '10000' + cursor.execute('DROP TABLE test_table;') + assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') + + node1.query("DROP DATABASE test_database") + assert 'test_database' not in node1.query('SHOW DATABASES') + + +def test_postgresql_database_engine_table_cache(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + + node1.query( + "CREATE DATABASE test_database ENGINE = PostgreSQL('postgres1:5432', 'test_database', 'postgres', 'mysecretpassword', 1)") + + create_postgres_table(cursor, 'test_table') + assert node1.query('DESCRIBE TABLE test_database.test_table').rstrip() == 'id\tInt32\t\t\t\t\t\nvalue\tNullable(Int32)' + + cursor.execute('ALTER TABLE test_table ADD COLUMN data Text') + assert node1.query('DESCRIBE TABLE test_database.test_table').rstrip() == 'id\tInt32\t\t\t\t\t\nvalue\tNullable(Int32)' + + node1.query("DETACH TABLE test_database.test_table") + assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') + + node1.query("ATTACH TABLE test_database.test_table") + assert 'test_table' in node1.query('SHOW TABLES FROM test_database') + + assert node1.query('DESCRIBE TABLE test_database.test_table').rstrip() == 'id\tInt32\t\t\t\t\t\nvalue\tNullable(Int32)\t\t\t\t\t\ndata\tNullable(String)' + + node1.query("DROP TABLE test_database.test_table") + assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') + + node1.query("ATTACH TABLE test_database.test_table") + assert 'test_table' in node1.query('SHOW TABLES FROM test_database') + + node1.query("INSERT INTO test_database.test_table SELECT number, number, toString(number) from numbers(10000)") + assert node1.query("SELECT count() FROM test_database.test_table").rstrip() == '10000' + + cursor.execute('DROP TABLE test_table;') + assert 'test_table' not in node1.query('SHOW TABLES FROM test_database') + node1.query("DROP DATABASE test_database") assert 'test_database' not in node1.query('SHOW DATABASES') From 9b25890674e0551cfe40abd8bd288b5b943778cd Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 28 Dec 2020 12:41:31 +0000 Subject: [PATCH 052/264] Generate ya.make, fix fast test --- src/DataStreams/ya.make | 1 + src/Databases/ya.make | 2 ++ src/Dictionaries/PostgreSQLDictionarySource.cpp | 8 +++++--- src/Dictionaries/ya.make | 1 + src/Storages/ya.make | 1 + src/TableFunctions/ya.make | 1 + 6 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index 858bf7081e7..83d88f31316 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -36,6 +36,7 @@ SRCS( NativeBlockInputStream.cpp NativeBlockOutputStream.cpp ParallelParsingBlockInputStream.cpp + PostgreSQLBlockInputStream.cpp PushingToViewsBlockOutputStream.cpp RemoteBlockInputStream.cpp RemoteBlockOutputStream.cpp diff --git a/src/Databases/ya.make b/src/Databases/ya.make index 0dc44386088..5ae5b2260e8 100644 --- a/src/Databases/ya.make +++ b/src/Databases/ya.make @@ -25,6 +25,8 @@ SRCS( MySQL/MaterializeMetadata.cpp MySQL/MaterializeMySQLSettings.cpp MySQL/MaterializeMySQLSyncThread.cpp + PostgreSQL/DatabasePostgreSQL.cpp + PostgreSQL/FetchFromPostgreSQL.cpp ) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 7b9a69b1b79..ecebd60775d 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -9,7 +9,7 @@ #include #include #include "readInvalidateQuery.h" - +#endif namespace DB { @@ -19,6 +19,8 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; } +#if USE_LIBPQXX + static const UInt64 max_block_size = 8192; PostgreSQLDictionarySource::PostgreSQLDictionarySource( @@ -154,6 +156,7 @@ std::string PostgreSQLDictionarySource::toString() const return "PostgreSQL: " + db + '.' + table + (where.empty() ? "" : ", where: " + where); } +#endif void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) { @@ -188,6 +191,5 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) }; factory.registerSource("postgresql", create_table_source); } -} -#endif +} diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 19a0f5008b8..159d9b07ac0 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -61,6 +61,7 @@ SRCS( PolygonDictionary.cpp PolygonDictionaryImplementations.cpp PolygonDictionaryUtils.cpp + PostgreSQLDictionarySource.cpp RangeHashedDictionary.cpp RedisBlockInputStream.cpp RedisDictionarySource.cpp diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 27aa9e3ac3f..56b72e711e4 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -130,6 +130,7 @@ SRCS( StorageMongoDB.cpp StorageMySQL.cpp StorageNull.cpp + StoragePostgreSQL.cpp StorageReplicatedMergeTree.cpp StorageSet.cpp StorageStripeLog.cpp diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index 7bcf5fc53b3..036e8ebee19 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -20,6 +20,7 @@ SRCS( TableFunctionMySQL.cpp TableFunctionNull.cpp TableFunctionNumbers.cpp + TableFunctionPostgreSQL.cpp TableFunctionRemote.cpp TableFunctionURL.cpp TableFunctionValues.cpp From c5cf556aaf9cdf9ebcc562d808b50a61d67840d9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 28 Dec 2020 13:56:46 +0000 Subject: [PATCH 053/264] Fix fast test --- tests/queries/0_stateless/01271_show_privileges.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index 5f5a39b906b..791f83cca05 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -108,6 +108,7 @@ URL [] GLOBAL SOURCES REMOTE [] GLOBAL SOURCES MONGO [] GLOBAL SOURCES MYSQL [] GLOBAL SOURCES +POSTGRES [] GLOBAL SOURCES ODBC [] GLOBAL SOURCES JDBC [] GLOBAL SOURCES HDFS [] GLOBAL SOURCES From fa3297bf725e3977718e77e727742ff0859552f7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 28 Dec 2020 19:16:10 +0000 Subject: [PATCH 054/264] Fix build, update libpq --- contrib/libpq | 2 +- src/Databases/DatabaseFactory.cpp | 7 +++++-- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 7 ++----- src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/contrib/libpq b/contrib/libpq index e2154c47525..e8728ef5146 160000 --- a/contrib/libpq +++ b/contrib/libpq @@ -1 +1 @@ -Subproject commit e2154c4752515623e52f437e2c95138b2c54b5d1 +Subproject commit e8728ef514677e1a974b63950d789c3ed30c4fe5 diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 457b99a88d3..4360420668d 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -23,11 +23,14 @@ # include # include # include -# include -# include # include #endif +#if USE_MYSQL || USE_LIBPQXX +#include +#include +#endif + #if USE_LIBPQXX #include #endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 49723dc2793..4d171f88ab9 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -72,7 +72,7 @@ DatabaseTablesIteratorPtr DatabasePostgreSQL::getTablesIterator( Tables tables; auto table_names = fetchTablesList(); - for (auto & table_name : table_names) + for (const auto & table_name : table_names) if (!detached_or_dropped.count(table_name)) tables[table_name] = fetchTable(table_name, context, true); @@ -102,10 +102,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const "SELECT tablename FROM pg_catalog.pg_tables " "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema' AND tablename = '{}'", table_name)); - if (result.empty()) - return false; - - return true; + return !result.empty(); } diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index 1d52e2502f0..bc0abce30df 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -92,8 +92,8 @@ DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimension if (result.size() < 3) throw Exception("Numeric lacks precision and scale in its definition", ErrorCodes::UNKNOWN_TYPE); - uint32_t precision = std::atoi(result[1].data()); - uint32_t scale = std::atoi(result[2].data()); + uint32_t precision = pqxx::from_string(result[1]); + uint32_t scale = pqxx::from_string(result[2]); if (precision <= DecimalUtils::maxPrecision()) res = std::make_shared>(precision, scale); From ff6f76e2d1d67ff52ead1d1c0c7acddafa47f2a5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Dec 2020 06:41:55 +0000 Subject: [PATCH 055/264] Fix build, update libpq --- cmake/find/libpqxx.cmake | 6 ++++++ contrib/libpq | 2 +- contrib/libpq-cmake/CMakeLists.txt | 1 + 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/cmake/find/libpqxx.cmake b/cmake/find/libpqxx.cmake index 8bfb48a0e54..d55420eaa7b 100644 --- a/cmake/find/libpqxx.cmake +++ b/cmake/find/libpqxx.cmake @@ -4,6 +4,12 @@ if (NOT ENABLE_LIBPQXX) return() endif() +if (OS_FREEBSD OR OS_DARWIN) + message (${RECONFIGURE_MESSAGE_LEVEL} "libpq library is disabled on FreeBSD and Darwin") + set (USE_LIBPQXX 0) + return() +endif() + if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpqxx/CMakeLists.txt") message (WARNING "submodule contrib/libpqxx is missing. to fix try run: \n git submodule update --init --recursive") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libpqxx library") diff --git a/contrib/libpq b/contrib/libpq index e8728ef5146..44e08e3436b 160000 --- a/contrib/libpq +++ b/contrib/libpq @@ -1 +1 @@ -Subproject commit e8728ef514677e1a974b63950d789c3ed30c4fe5 +Subproject commit 44e08e3436ba898e2813a2581aca22f301ab0f85 diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index d58e27371af..11a3a3aa4ea 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -33,6 +33,7 @@ set(SRCS ${LIBPQ_SOURCE_DIR}/common/stringinfo.c ${LIBPQ_SOURCE_DIR}/common/psprintf.c ${LIBPQ_SOURCE_DIR}/common/encnames.c + ${LIBPQ_SOURCE_DIR}/common/logging.c ${LIBPQ_SOURCE_DIR}/port/snprintf.c ${LIBPQ_SOURCE_DIR}/port/strlcpy.c From f58f54610a0c46ae99753273899c70e3d9f084df Mon Sep 17 00:00:00 2001 From: George Date: Tue, 29 Dec 2020 13:54:17 +0300 Subject: [PATCH 056/264] Edited English description --- .../functions/string-search-functions.md | 146 +++++++++++++++++- 1 file changed, 139 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 3cde7dd71d4..d430c3346e0 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -536,11 +536,11 @@ For case-insensitive search or/and in UTF-8 format use functions `ngramSearchCas !!! note "Note" For UTF-8 case we use 3-gram distance. All these are not perfectly fair n-gram distances. We use 2-byte hashes to hash n-grams and then calculate the (non-)symmetric difference between these hash tables – collisions may occur. With UTF-8 case-insensitive format we do not use fair `tolower` function – we zero the 5-th bit (starting from zero) of each codepoint byte and first bit of zeroth byte if bytes more than one – this works for Latin and mostly for all Cyrillic letters. -## countSubstrings(haystack, needle) {#countSubstrings} +## countSubstrings {#countSubstrings} -Count the number of substring occurrences +Counts the number of substring occurrences. -For a case-insensitive search, use the function `countSubstringsCaseInsensitive` (or `countSubstringsCaseInsensitiveUTF8`). +For a case-insensitive search, use [countSubstringsCaseInsensitive](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitive) or [countSubstringsCaseInsensitiveUTF8](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitiveUTF8) functions. **Syntax** @@ -558,14 +558,14 @@ countSubstrings(haystack, needle[, start_pos]) - Number of occurrences. -Type: `Integer`. +Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** Query: ``` sql -SELECT countSubstrings('foobar.com', '.') +SELECT countSubstrings('foobar.com', '.'); ``` Result: @@ -579,7 +579,7 @@ Result: Query: ``` sql -SELECT countSubstrings('aaaa', 'aa') +SELECT countSubstrings('aaaa', 'aa'); ``` Result: @@ -590,8 +590,140 @@ Result: └───────────────────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/query_language/functions/string_search_functions/) +Query: + +```sql +SELECT countSubstrings('abc___abc', 'abc', 4); +``` + +Result: + +``` text +┌─countSubstrings('abc___abc', 'abc', 4)─┐ +│ 1 │ +└────────────────────────────────────────┘ +``` + +## countSubstringsCaseInsensitive {#countSubstringsCaseInsensitive} + +Counts the number of substring occurrences case-insensitive. + +**Syntax** + +``` sql +countSubstringsCaseInsensitive(haystack, needle[, start_pos]) +``` + +**Parameters** + +- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md) + +**Returned values** + +- Number of occurrences. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Examples** + +Query: + +``` sql +select countSubstringsCaseInsensitive('aba', 'B'); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitive('aba', 'B')─┐ +│ 1 │ +└────────────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT countSubstringsCaseInsensitive('foobar.com', 'CoM'); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitive('foobar.com', 'CoM')─┐ +│ 1 │ +└─────────────────────────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT countSubstringsCaseInsensitive('abC___abC', 'aBc', 2); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitive('abC___abC', 'aBc', 2)─┐ +│ 1 │ +└───────────────────────────────────────────────────────┘ +``` + +## countSubstringsCaseInsensitiveUTF8 {#countSubstringsCaseInsensitiveUTF8} + +Counts the number of substring occurrences in `UTF8` case-insensitive. + +**Syntax** + +``` sql +SELECT countSubstringsCaseInsensitiveUTF8(haystack, needle[, start_pos]) +``` + +**Parameters** + +- `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md) + +**Returned values** + +- Number of occurrences. + +Type: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Examples** + +Query: + +``` sql +SELECT countSubstringsCaseInsensitiveUTF8('абв', 'A'); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitiveUTF8('абв', 'A')─┐ +│ 1 │ +└────────────────────────────────────────────────┘ +``` + +Query: + +```sql +SELECT countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв'); +``` + +Result: + +``` text +┌─countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв')─┐ +│ 3 │ +└────────────────────────────────────────────────────────────┘ +``` ## countMatches(haystack, pattern) {#countmatcheshaystack-pattern} Returns the number of regular expression matches for a `pattern` in a `haystack`. + +[Original article](https://clickhouse.tech/docs/en/query_language/functions/string_search_functions/) From 56a757d8182285b1190a431f4db4bdbe6f295e9d Mon Sep 17 00:00:00 2001 From: George Date: Tue, 29 Dec 2020 14:07:30 +0300 Subject: [PATCH 057/264] Added translation and updated the description --- .../functions/string-search-functions.md | 8 +- .../functions/string-search-functions.md | 186 ++++++++++++++++++ 2 files changed, 190 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index d430c3346e0..3e1a6586e03 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -552,7 +552,7 @@ countSubstrings(haystack, needle[, start_pos]) - `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). - `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md) +- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). **Returned values** @@ -618,7 +618,7 @@ countSubstringsCaseInsensitive(haystack, needle[, start_pos]) - `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). - `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md) +- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). **Returned values** @@ -672,7 +672,7 @@ Result: ## countSubstringsCaseInsensitiveUTF8 {#countSubstringsCaseInsensitiveUTF8} -Counts the number of substring occurrences in `UTF8` case-insensitive. +Counts the number of substring occurrences in `UTF-8` case-insensitive. **Syntax** @@ -684,7 +684,7 @@ SELECT countSubstringsCaseInsensitiveUTF8(haystack, needle[, start_pos]) - `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). - `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md) +- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). **Returned values** diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index d2f1119783b..fd2fee9d462 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -522,4 +522,190 @@ SELECT * FROM Months WHERE ilike(name, '%j%') !!! note "Примечание" Для случая UTF-8 мы используем триграммное расстояние. Вычисление n-граммного расстояния не совсем честное. Мы используем 2-х байтные хэши для хэширования n-грамм, а затем вычисляем (не)симметрическую разность между хэш таблицами – могут возникнуть коллизии. В формате UTF-8 без учета регистра мы не используем честную функцию `tolower` – мы обнуляем 5-й бит (нумерация с нуля) каждого байта кодовой точки, а также первый бит нулевого байта, если байтов больше 1 – это работает для латиницы и почти для всех кириллических букв. +## countSubstrings {#countSubstrings} + +Вычисляет количество вхождений подстроки. + +Для поиска без учета регистра, используйте функции [countSubstringsCaseInsensitive](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitive) или [countSubstringsCaseInsensitiveUTF8](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitiveUTF8) + +**Синтаксис** + +``` sql +countSubstrings(haystack, needle[, start_pos]) +``` + +**Параметры** + +- `haystack` — строка, в которой ведется поиск. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — искомая подстрока. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – позиция первого символа в строке, с которого начнется поиск. Необязательный параметр. [UInt](../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- Число вхождений. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Примеры** + +Запрос: + +``` sql +SELECT countSubstrings('foobar.com', '.'); +``` + +Результат: + +``` text +┌─countSubstrings('foobar.com', '.')─┐ +│ 1 │ +└────────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT countSubstrings('aaaa', 'aa'); +``` + +Результат: + +``` text +┌─countSubstrings('aaaa', 'aa')─┐ +│ 2 │ +└───────────────────────────────┘ +``` + +Запрос: + +```sql +SELECT countSubstrings('abc___abc', 'abc', 4); +``` + +Результат: + +``` text +┌─countSubstrings('abc___abc', 'abc', 4)─┐ +│ 1 │ +└────────────────────────────────────────┘ +``` + +## countSubstringsCaseInsensitive {#countSubstringsCaseInsensitive} + +Вычисляет количество вхождений подстроки без учета регистра. + +**Синтаксис** + +``` sql +countSubstringsCaseInsensitive(haystack, needle[, start_pos]) +``` + +**Параметры** + +- `haystack` — строка, в которой ведется поиск. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — искомая подстрока. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – позиция первого символа в строке, с которого начнется поиск. Необязательный параметр. [UInt](../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- Число вхождений. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Примеры** + +Запрос: + +``` sql +select countSubstringsCaseInsensitive('aba', 'B'); +``` + +Результат: + +``` text +┌─countSubstringsCaseInsensitive('aba', 'B')─┐ +│ 1 │ +└────────────────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT countSubstringsCaseInsensitive('foobar.com', 'CoM'); +``` + +Результат: + +``` text +┌─countSubstringsCaseInsensitive('foobar.com', 'CoM')─┐ +│ 1 │ +└─────────────────────────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT countSubstringsCaseInsensitive('abC___abC', 'aBc', 2); +``` + +Результат: + +``` text +┌─countSubstringsCaseInsensitive('abC___abC', 'aBc', 2)─┐ +│ 1 │ +└───────────────────────────────────────────────────────┘ +``` + +## countSubstringsCaseInsensitiveUTF8 {#countSubstringsCaseInsensitiveUTF8} + +Вычисляет количество вхождений подстроки в `UTF-8` без учета регистра. + +**Синтаксис** + +``` sql +SELECT countSubstringsCaseInsensitiveUTF8(haystack, needle[, start_pos]) +``` + +**Параметры** + +- `haystack` — строка, в которой ведется поиск. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `needle` — искомая подстрока. [String](../../sql-reference/syntax.md#syntax-string-literal). +- `start_pos` – позиция первого символа в строке, с которого начнется поиск. Необязательный параметр. [UInt](../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения** + +- Число вхождений. + +Тип: [UInt64](../../sql-reference/data-types/int-uint.md). + +**Примеры** + +Запрос: + +``` sql +SELECT countSubstringsCaseInsensitiveUTF8('абв', 'A'); +``` + +Результат: + +``` text +┌─countSubstringsCaseInsensitiveUTF8('абв', 'A')─┐ +│ 1 │ +└────────────────────────────────────────────────┘ +``` + +Запрос: + +```sql +SELECT countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв'); +``` + +Результат: + +``` text +┌─countSubstringsCaseInsensitiveUTF8('аБв__АбВ__абв', 'Абв')─┐ +│ 3 │ +└────────────────────────────────────────────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/string_search_functions/) From 13627226c39d03ae0fca5f9b18d6a98e241641f4 Mon Sep 17 00:00:00 2001 From: George Date: Tue, 29 Dec 2020 14:11:48 +0300 Subject: [PATCH 058/264] fixed typos --- docs/en/sql-reference/functions/string-search-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 3e1a6586e03..7617f0d7b68 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -552,7 +552,7 @@ countSubstrings(haystack, needle[, start_pos]) - `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). - `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). +- `start_pos` – Position of the first character in the string to start search. Optional. [UInt](../../sql-reference/data-types/int-uint.md). **Returned values** @@ -618,7 +618,7 @@ countSubstringsCaseInsensitive(haystack, needle[, start_pos]) - `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). - `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). +- `start_pos` – Position of the first character in the string to start search. Optional. [UInt](../../sql-reference/data-types/int-uint.md). **Returned values** @@ -684,7 +684,7 @@ SELECT countSubstringsCaseInsensitiveUTF8(haystack, needle[, start_pos]) - `haystack` — The string to search in. [String](../../sql-reference/syntax.md#syntax-string-literal). - `needle` — The substring to search for. [String](../../sql-reference/syntax.md#syntax-string-literal). -- `start_pos` – Optional parameter, position of the first character in the string to start search. [UInt](../../sql-reference/data-types/int-uint.md). +- `start_pos` – Position of the first character in the string to start search. Optional. [UInt](../../sql-reference/data-types/int-uint.md). **Returned values** From e654564aa0ca93d673c6add8c74ea5597b0e2161 Mon Sep 17 00:00:00 2001 From: George Date: Tue, 29 Dec 2020 14:30:47 +0300 Subject: [PATCH 059/264] Minor improvements --- docs/en/sql-reference/functions/string-search-functions.md | 6 +++--- docs/ru/sql-reference/functions/string-search-functions.md | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 7617f0d7b68..82ef5610868 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -538,7 +538,7 @@ For case-insensitive search or/and in UTF-8 format use functions `ngramSearchCas ## countSubstrings {#countSubstrings} -Counts the number of substring occurrences. +Returns the number of substring occurrences. For a case-insensitive search, use [countSubstringsCaseInsensitive](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitive) or [countSubstringsCaseInsensitiveUTF8](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitiveUTF8) functions. @@ -606,7 +606,7 @@ Result: ## countSubstringsCaseInsensitive {#countSubstringsCaseInsensitive} -Counts the number of substring occurrences case-insensitive. +Returns the number of substring occurrences case-insensitive. **Syntax** @@ -672,7 +672,7 @@ Result: ## countSubstringsCaseInsensitiveUTF8 {#countSubstringsCaseInsensitiveUTF8} -Counts the number of substring occurrences in `UTF-8` case-insensitive. +Returns the number of substring occurrences in `UTF-8` case-insensitive. **Syntax** diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index fd2fee9d462..e4d9316cbf3 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -524,7 +524,7 @@ SELECT * FROM Months WHERE ilike(name, '%j%') ## countSubstrings {#countSubstrings} -Вычисляет количество вхождений подстроки. +Возвращает количество вхождений подстроки. Для поиска без учета регистра, используйте функции [countSubstringsCaseInsensitive](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitive) или [countSubstringsCaseInsensitiveUTF8](../../sql-reference/functions/string-search-functions.md#countSubstringsCaseInsensitiveUTF8) @@ -592,7 +592,7 @@ SELECT countSubstrings('abc___abc', 'abc', 4); ## countSubstringsCaseInsensitive {#countSubstringsCaseInsensitive} -Вычисляет количество вхождений подстроки без учета регистра. +Возвращает количество вхождений подстроки без учета регистра. **Синтаксис** @@ -658,7 +658,7 @@ SELECT countSubstringsCaseInsensitive('abC___abC', 'aBc', 2); ## countSubstringsCaseInsensitiveUTF8 {#countSubstringsCaseInsensitiveUTF8} -Вычисляет количество вхождений подстроки в `UTF-8` без учета регистра. +Возвращает количество вхождений подстроки в `UTF-8` без учета регистра. **Синтаксис** From 8efd85bef2984523a96e2464e447d17623d62ba0 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Dec 2020 16:56:50 +0000 Subject: [PATCH 060/264] Fix build --- contrib/libpq | 2 +- contrib/libpq-cmake/CMakeLists.txt | 3 --- src/DataStreams/PostgreSQLBlockInputStream.cpp | 1 + 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/contrib/libpq b/contrib/libpq index 44e08e3436b..31e2ff4e8fc 160000 --- a/contrib/libpq +++ b/contrib/libpq @@ -1 +1 @@ -Subproject commit 44e08e3436ba898e2813a2581aca22f301ab0f85 +Subproject commit 31e2ff4e8fc4b50208ee86cb33a77c14531f95a0 diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index 11a3a3aa4ea..32da3ef5a4e 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -51,9 +51,6 @@ add_library(libpq ${SRCS}) target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}) target_include_directories (libpq PUBLIC ${LIBPQ_SOURCE_DIR}/include) - target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/configs) -target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/include/common) -target_include_directories (libpq PRIVATE ${LIBPQ_SOURCE_DIR}/include/libpq) target_link_libraries (libpq PRIVATE ssl) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index bffab4d4841..df60f88fecd 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -144,6 +144,7 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view case ValueType::vtFloat64: assert_cast(column).insertValue(pqxx::from_string(value)); break; + case ValueType::vtFixedString:[[fallthrough]]; case ValueType::vtString: assert_cast(column).insertData(value.data(), value.size()); break; From d146974058f940aae69065d9dbc4ebd3941fc5c7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 29 Dec 2020 20:05:42 +0000 Subject: [PATCH 061/264] Update libpq --- cmake/find/libpqxx.cmake | 6 ------ contrib/libpq | 2 +- contrib/libpq-cmake/CMakeLists.txt | 1 + 3 files changed, 2 insertions(+), 7 deletions(-) diff --git a/cmake/find/libpqxx.cmake b/cmake/find/libpqxx.cmake index d55420eaa7b..8bfb48a0e54 100644 --- a/cmake/find/libpqxx.cmake +++ b/cmake/find/libpqxx.cmake @@ -4,12 +4,6 @@ if (NOT ENABLE_LIBPQXX) return() endif() -if (OS_FREEBSD OR OS_DARWIN) - message (${RECONFIGURE_MESSAGE_LEVEL} "libpq library is disabled on FreeBSD and Darwin") - set (USE_LIBPQXX 0) - return() -endif() - if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpqxx/CMakeLists.txt") message (WARNING "submodule contrib/libpqxx is missing. to fix try run: \n git submodule update --init --recursive") message (${RECONFIGURE_MESSAGE_LEVEL} "Can't find internal libpqxx library") diff --git a/contrib/libpq b/contrib/libpq index 31e2ff4e8fc..e2d1a14f079 160000 --- a/contrib/libpq +++ b/contrib/libpq @@ -1 +1 @@ -Subproject commit 31e2ff4e8fc4b50208ee86cb33a77c14531f95a0 +Subproject commit e2d1a14f079bda7b4bebc203f41355bb433fc145 diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index 32da3ef5a4e..e538e4f4d93 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -45,6 +45,7 @@ set(SRCS ${LIBPQ_SOURCE_DIR}/port/pg_strong_random.c ${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c ${LIBPQ_SOURCE_DIR}/port/thread.c + ${LIBPQ_SOURCE_DIR}/port/path.c ) add_library(libpq ${SRCS}) From 235ea220ec80b88182dde9206ded1240901d982c Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 18 Dec 2020 23:09:39 +0300 Subject: [PATCH 062/264] SELECT count() FROM table now requires at least one column to be accessible in the table. --- src/Interpreters/InterpreterSelectQuery.cpp | 34 ++++++++++++++++++- src/Interpreters/TreeRewriter.cpp | 6 ++-- src/Interpreters/TreeRewriter.h | 7 ++++ .../test_select_access_rights/test.py | 22 ++++++++++++ 4 files changed, 66 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index dd9df3f36a0..8cc4b08c61b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -13,6 +13,7 @@ #include #include +#include #include @@ -99,6 +100,7 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; + extern const int ACCESS_DENIED; } /// Assumes `storage` is set and the table filter (row-level security) is not empty. @@ -211,6 +213,36 @@ static void rewriteMultipleJoins(ASTPtr & query, const TablesWithColumns & table JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query); } +/// Checks that the current user has the SELECT privilege. +static void checkAccessRightsForSelect( + const Context & context, + const StorageID & table_id, + const StorageMetadataPtr & table_metadata, + const Strings & required_columns, + const TreeRewriterResult & syntax_analyzer_result) +{ + if (!syntax_analyzer_result.has_explicit_columns && table_metadata && !table_metadata->getColumns().empty()) + { + /// For a trivial query like "SELECT count() FROM table" access is granted if at least + /// one column is accessible. + /// In this case just checking access for `required_columns` doesn't work correctly + /// because `required_columns` will contain the name of a column of minimum size (see TreeRewriterResult::collectUsedColumns()) + /// which is probably not the same column as the column the current user has access to. + auto access = context.getAccess(); + for (const auto & column : table_metadata->getColumns()) + { + if (access->isGranted(AccessType::SELECT, table_id.database_name, table_id.table_name, column.name)) + return; + } + throw Exception(context.getUserName() + " : Not enough privileges. " + "To execute this query it's necessary to have grant SELECT for at least one column on " + table_id.getFullTableName(), + ErrorCodes::ACCESS_DENIED); + } + + /// General check. + context.checkAccess(AccessType::SELECT, table_id, required_columns); +} + /// Returns true if we should ignore quotas and limits for a specified table in the system database. static bool shouldIgnoreQuotaAndLimits(const StorageID & table_id) { @@ -466,7 +498,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( { /// The current user should have the SELECT privilege. /// If this table_id is for a table function we don't check access rights here because in this case they have been already checked in ITableFunction::execute(). - context->checkAccess(AccessType::SELECT, table_id, required_columns); + checkAccessRightsForSelect(*context, table_id, metadata_snapshot, required_columns, *syntax_analyzer_result); /// Remove limits for some tables in the `system` database. if (shouldIgnoreQuotaAndLimits(table_id) && (joined_tables.tablesCount() <= 1)) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 2b801500958..29a6b102fc9 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -588,11 +588,13 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select required.insert(column_name_type.name); } - /// You need to read at least one column to find the number of rows. - if (is_select && required.empty()) + /// Figure out if we're able to use the trivial count optimization. + has_explicit_columns = !required.empty(); + if (is_select && !has_explicit_columns) { optimize_trivial_count = true; + /// You need to read at least one column to find the number of rows. /// We will find a column with minimum . /// Because it is the column that is cheapest to read. struct ColumnSizeTuple diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index d9f98ee40bd..35f82d2505f 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -53,6 +53,13 @@ struct TreeRewriterResult /// Predicate optimizer overrides the sub queries bool rewrite_subqueries = false; + /// Whether the query contains explicit columns like "SELECT column1 + column2 FROM table1". + /// Queries like "SELECT count() FROM table1", "SELECT 1" don't contain explicit columns. + bool has_explicit_columns = false; + + /// Whether it's possible to use the trivial count optimization, + /// i.e. use a fast call of IStorage::totalRows() (or IStorage::totalRowsByPartitionPredicate()) + /// instead of actual retrieving columns and counting rows. bool optimize_trivial_count = false; /// Cache isRemote() call for storage, because it may be too heavy. diff --git a/tests/integration/test_select_access_rights/test.py b/tests/integration/test_select_access_rights/test.py index ccea77d6fb7..14eae1700d0 100644 --- a/tests/integration/test_select_access_rights/test.py +++ b/tests/integration/test_select_access_rights/test.py @@ -155,3 +155,25 @@ def test_select_union(): instance.query("REVOKE SELECT ON default.table1 FROM A") assert "it's necessary to have grant SELECT(a, b) ON default.table1" in instance.query_and_get_error(select_query, user = 'A') + + +def test_select_count(): + instance.query("CREATE TABLE table1(x String, y UInt8) ENGINE = MergeTree ORDER BY tuple()") + + select_query = "SELECT count() FROM table1" + assert "it's necessary to have grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user = 'A') + + instance.query("GRANT SELECT(x) ON default.table1 TO A") + assert instance.query(select_query, user = 'A') == "0\n" + + instance.query("REVOKE SELECT(x) ON default.table1 FROM A") + assert "it's necessary to have grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user = 'A') + + instance.query("GRANT SELECT(y) ON default.table1 TO A") + assert instance.query(select_query, user = 'A') == "0\n" + + instance.query("REVOKE SELECT(y) ON default.table1 FROM A") + assert "it's necessary to have grant SELECT for at least one column on default.table1" in instance.query_and_get_error(select_query, user = 'A') + + instance.query("GRANT SELECT ON default.table1 TO A") + assert instance.query(select_query, user = 'A') == "0\n" From 9bc571eacc45bac84de4db2aaf14f28ff25714fc Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 30 Dec 2020 17:58:43 +0300 Subject: [PATCH 063/264] Update KeyCondition.cpp --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 735d764eb8e..2f172675e99 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -883,7 +883,7 @@ public: RIGHT_CONST, }; - FunctionWithOptionalConstArg(const FunctionBasePtr & func_) : func(func_) { } + explicit FunctionWithOptionalConstArg(const FunctionBasePtr & func_) : func(func_) {} FunctionWithOptionalConstArg(const FunctionBasePtr & func_, const ColumnWithTypeAndName & const_arg_, Kind kind_) : func(func_), const_arg(const_arg_), kind(kind_) { From 62b06de97eaca9e88f6843e96ce58b902e04b04b Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 30 Dec 2020 16:00:05 +0000 Subject: [PATCH 064/264] Try fix build --- cmake/find/libpqxx.cmake | 8 +++++--- contrib/libpq | 2 +- contrib/libpq-cmake/CMakeLists.txt | 1 + src/Databases/PostgreSQL/DatabasePostgreSQL.h | 2 +- 4 files changed, 8 insertions(+), 5 deletions(-) diff --git a/cmake/find/libpqxx.cmake b/cmake/find/libpqxx.cmake index 8bfb48a0e54..b641f114d05 100644 --- a/cmake/find/libpqxx.cmake +++ b/cmake/find/libpqxx.cmake @@ -18,13 +18,15 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libpq/include") return() endif() +if (NOT USE_INTERNAL_SSL_LIBRARY) + set (USE_LIBPQXX 0) + message (STATUS "Using libpqxx=${USE_LIBPQXX}. (Internal ssl library is missing)") +else () set (USE_LIBPQXX 1) set (LIBPQXX_LIBRARY libpqxx) set (LIBPQ_LIBRARY libpq) - set (LIBPQXX_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpqxx/include") set (LIBPQ_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/libpq") - message (STATUS "Using libpqxx=${USE_LIBPQXX}: ${LIBPQXX_INCLUDE_DIR} : ${LIBPQXX_LIBRARY}") message (STATUS "Using libpq: ${LIBPQ_ROOT_DIR} : ${LIBPQ_INCLUDE_DIR} : ${LIBPQ_LIBRARY}") - +endif() diff --git a/contrib/libpq b/contrib/libpq index e2d1a14f079..8e7e9058547 160000 --- a/contrib/libpq +++ b/contrib/libpq @@ -1 +1 @@ -Subproject commit e2d1a14f079bda7b4bebc203f41355bb433fc145 +Subproject commit 8e7e905854714a7fbb49c124dbc45c7bd4b98e07 diff --git a/contrib/libpq-cmake/CMakeLists.txt b/contrib/libpq-cmake/CMakeLists.txt index e538e4f4d93..34c57799a8a 100644 --- a/contrib/libpq-cmake/CMakeLists.txt +++ b/contrib/libpq-cmake/CMakeLists.txt @@ -46,6 +46,7 @@ set(SRCS ${LIBPQ_SOURCE_DIR}/port/pgstrcasecmp.c ${LIBPQ_SOURCE_DIR}/port/thread.c ${LIBPQ_SOURCE_DIR}/port/path.c + ${LIBPQ_SOURCE_DIR}/port/explicit_bzero.c ) add_library(libpq ${SRCS}) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 7a10b1cb319..a80b90086c2 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -31,7 +31,7 @@ public: const bool cache_tables_); String getEngineName() const override { return "PostgreSQL"; } - String getMetadataPath() const override { return metadata_path; }; + String getMetadataPath() const override { return metadata_path; } bool canContainMergeTreeTables() const override { return false; } bool canContainDistributedTables() const override { return false; } From f93e30bed6003b01abab792b67f44940faa875af Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 31 Dec 2020 11:06:15 +0800 Subject: [PATCH 065/264] Fix warning --- src/Storages/MergeTree/KeyCondition.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 2f172675e99..60f896844ac 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -898,8 +898,7 @@ public: ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName & arguments) const override { return func->prepare(arguments); } ColumnPtr - execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run = false) - const override + execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const override { if (kind == Kind::LEFT_CONST) { From e100aec086ad89e8ff843d5d0d4be2268ff62a08 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 31 Dec 2020 11:15:05 +0000 Subject: [PATCH 066/264] Fix typos check --- cmake/find/libpqxx.cmake | 1 - src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Storages/StoragePostgreSQL.h | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/cmake/find/libpqxx.cmake b/cmake/find/libpqxx.cmake index b641f114d05..f981df19aaa 100644 --- a/cmake/find/libpqxx.cmake +++ b/cmake/find/libpqxx.cmake @@ -20,7 +20,6 @@ endif() if (NOT USE_INTERNAL_SSL_LIBRARY) set (USE_LIBPQXX 0) - message (STATUS "Using libpqxx=${USE_LIBPQXX}. (Internal ssl library is missing)") else () set (USE_LIBPQXX 1) set (LIBPQXX_LIBRARY libpqxx) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 4d171f88ab9..44793856f82 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -256,7 +256,7 @@ void DatabasePostgreSQL::loadStoredObjects(Context & /* context */, bool, bool / std::lock_guard lock{mutex}; Poco::DirectoryIterator iterator(getMetadataPath()); - /// Check for previously droppped tables + /// Check for previously dropped tables for (Poco::DirectoryIterator end; iterator != end; ++iterator) { if (iterator->isFile() && endsWith(iterator.name(), suffix)) diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 4a25e3459e5..426908775be 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -82,7 +82,7 @@ private: /// Tiny connection class to make it more convenient to use. -/// Connection is not made untill actually used. +/// Connection is not made until actually used. class PGConnection { public: From 0cb82c26a36714da03c0c604cb6d120db0c36d65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 31 Dec 2020 17:36:21 +0300 Subject: [PATCH 067/264] Skip Arcadia --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index c1f7c14f58a..335b39438f4 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -179,3 +179,4 @@ 01558_ttest_scipy 01561_mann_whitney_scipy 01601_custom_tld +01602_insert_into_table_function_cluster From 76288a88b6caac1d122d4b80e92995ab84eb0f28 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 2 Jan 2021 00:16:13 +0300 Subject: [PATCH 068/264] Update IDataType.cpp --- src/DataTypes/IDataType.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 26fccb24fe4..5582a8698e0 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -11,7 +11,7 @@ #include #include #include -#include + namespace DB { From ceab9180b4bf853d325931e2825ce37b891f2326 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 2 Jan 2021 00:16:47 +0300 Subject: [PATCH 069/264] Update DataTypeArray.cpp --- src/DataTypes/DataTypeArray.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index e68d0d56c11..79e0196c9f1 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -16,10 +16,10 @@ #include #include -#include #include + namespace DB { From 89595e184f89fda449d392966af57333275b3361 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 2 Jan 2021 00:26:41 +0300 Subject: [PATCH 070/264] Consistency --- src/Columns/ColumnMap.h | 4 ++-- src/DataTypes/DataTypeMap.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index d92522fc28f..abab522e68f 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -85,8 +85,8 @@ public: const ColumnArray & getNestedColumn() const { return assert_cast(*nested); } ColumnArray & getNestedColumn() { return assert_cast(*nested); } - const ColumnPtr & getNestedPtr() const { return nested; } - ColumnPtr & getNestedPtr() { return nested; } + const ColumnPtr & getNestedColumnPtr() const { return nested; } + ColumnPtr & getNestedColumnPtr() { return nested; } const ColumnTuple & getNestedData() const { return assert_cast(getNestedColumn().getData()); } ColumnTuple & getNestedData() { return assert_cast(getNestedColumn().getData()); } diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 84152de65f6..3f59e1d3654 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -323,7 +323,7 @@ void DataTypeMap::deserializeBinaryBulkWithMultipleStreamsImpl( SubstreamsCache * cache) const { auto & column_map = assert_cast(column); - nested->deserializeBinaryBulkWithMultipleStreams(column_map.getNestedPtr(), limit, settings, state, cache); + nested->deserializeBinaryBulkWithMultipleStreams(column_map.getNestedColumnPtr(), limit, settings, state, cache); } void DataTypeMap::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const From 5953c2fcbe583ff66278d7abcb3e9d822fadf83b Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Jan 2021 03:21:53 +0000 Subject: [PATCH 071/264] fix --- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../01602_insert_into_table_function_cluster.sql | 15 ++++++++------- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 2a193dc7249..cd2f4a5eb0f 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -55,7 +55,7 @@ namespace ErrorCodes InterpreterInsertQuery::InterpreterInsertQuery( const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_, bool no_squash_, bool no_destination_) - : query_ptr(query_ptr_) + : query_ptr(query_ptr_->clone()) , context(context_) , allow_materialized(allow_materialized_) , no_squash(no_squash_) diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql index 53a33676ef8..0cb8949410e 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql @@ -1,22 +1,23 @@ DROP TABLE IF EXISTS default.x; +DROP TABLE IF EXISTS default.y; CREATE TABLE default.x AS system.numbers ENGINE = MergeTree ORDER BY number; CREATE TABLE default.y AS system.numbers ENGINE = MergeTree ORDER BY number; -- Just one shard, sharding key isn't necessary -INSERT INTO FUNCTION cluster('test_shard_localhost', default, x) SELECT * FROM numbers(10); -INSERT INTO FUNCTION cluster('test_shard_localhost', default, x, rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION cluster('test_shard_localhost', currentDatabase(), x) SELECT * FROM numbers(10); +INSERT INTO FUNCTION cluster('test_shard_localhost', currentDatabase(), x, rand()) SELECT * FROM numbers(10); -- More than one shard, sharding key is necessary -INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', default, x) SELECT * FROM numbers(10); --{ serverError 55 } -INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', default, x, rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', currentDatabase(), x) SELECT * FROM numbers(10); --{ serverError 55 } +INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', currentDatabase(), x, rand()) SELECT * FROM numbers(10); -INSERT INTO FUNCTION remote('127.0.0.{1,2}', default, y, 'default') SELECT * FROM numbers(10); -- { serverError 55 } -INSERT INTO FUNCTION remote('127.0.0.{1,2}', default, y, 'default', rand()) SELECT * FROM numbers(10); +INSERT INTO FUNCTION remote('127.0.0.{1,2}', currentDatabase(), y, 'default') SELECT * FROM numbers(10); -- { serverError 55 } +INSERT INTO FUNCTION remote('127.0.0.{1,2}', currentDatabase(), y, 'default', rand()) SELECT * FROM numbers(10); SELECT * FROM default.x ORDER BY number; -SELECT * FROM remote('127.0.0.{1,2}', default, y) ORDER BY number; +SELECT * FROM remote('127.0.0.{1,2}', currentDatabase(), y) ORDER BY number; DROP TABLE default.x; DROP TABLE default.y; From fddb71df8fd19636feebfef3dd7eee6bb312f0d4 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 3 Jan 2021 08:47:30 +0000 Subject: [PATCH 072/264] fix test --- .../01602_insert_into_table_function_cluster.sql | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql index 0cb8949410e..006cef24080 100644 --- a/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql +++ b/tests/queries/0_stateless/01602_insert_into_table_function_cluster.sql @@ -1,8 +1,8 @@ -DROP TABLE IF EXISTS default.x; -DROP TABLE IF EXISTS default.y; +DROP TABLE IF EXISTS x; +DROP TABLE IF EXISTS y; -CREATE TABLE default.x AS system.numbers ENGINE = MergeTree ORDER BY number; -CREATE TABLE default.y AS system.numbers ENGINE = MergeTree ORDER BY number; +CREATE TABLE x AS system.numbers ENGINE = MergeTree ORDER BY number; +CREATE TABLE y AS system.numbers ENGINE = MergeTree ORDER BY number; -- Just one shard, sharding key isn't necessary INSERT INTO FUNCTION cluster('test_shard_localhost', currentDatabase(), x) SELECT * FROM numbers(10); @@ -15,9 +15,9 @@ INSERT INTO FUNCTION cluster('test_cluster_two_shards_localhost', currentDatabas INSERT INTO FUNCTION remote('127.0.0.{1,2}', currentDatabase(), y, 'default') SELECT * FROM numbers(10); -- { serverError 55 } INSERT INTO FUNCTION remote('127.0.0.{1,2}', currentDatabase(), y, 'default', rand()) SELECT * FROM numbers(10); -SELECT * FROM default.x ORDER BY number; +SELECT * FROM x ORDER BY number; SELECT * FROM remote('127.0.0.{1,2}', currentDatabase(), y) ORDER BY number; -DROP TABLE default.x; -DROP TABLE default.y; +DROP TABLE x; +DROP TABLE y; From 5cbc25c647ff4d6564d20ffa8830a1a503da23c6 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 3 Dec 2020 07:18:12 +0300 Subject: [PATCH 073/264] Added global setting `s3_max_connections`. --- src/Core/Settings.h | 1 + src/Disks/S3/registerDiskS3.cpp | 1 + src/IO/HTTPCommon.cpp | 107 ++++++++++++++++++++----- src/IO/HTTPCommon.h | 5 +- src/IO/S3/PocoHTTPClient.cpp | 30 ++++--- src/IO/S3/PocoHTTPClient.h | 1 + src/IO/S3/PocoHTTPResponseStream.cpp | 19 ----- src/IO/S3/PocoHTTPResponseStream.h | 21 ----- src/IO/S3/SessionAwareAwsStream.h | 27 +++++++ src/IO/S3Common.cpp | 9 +-- src/IO/S3Common.h | 4 +- src/Storages/StorageS3.cpp | 9 ++- src/Storages/StorageS3.h | 1 + src/TableFunctions/TableFunctionS3.cpp | 2 + 14 files changed, 152 insertions(+), 85 deletions(-) delete mode 100644 src/IO/S3/PocoHTTPResponseStream.cpp delete mode 100644 src/IO/S3/PocoHTTPResponseStream.h create mode 100644 src/IO/S3/SessionAwareAwsStream.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 00d4682332d..0b20f5fb033 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -67,6 +67,7 @@ class IColumn; M(UInt64, s3_min_upload_part_size, 512*1024*1024, "The minimum size of part to upload during multipart upload to S3.", 0) \ M(UInt64, s3_max_single_part_upload_size, 64*1024*1024, "The maximum size of object to upload using singlepart upload to S3.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ + M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ M(Bool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index fd658d95327..dc7020c9617 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -120,6 +120,7 @@ void registerDiskS3(DiskFactory & factory) cfg.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", 10000); cfg.httpRequestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", 5000); + cfg.maxConnections = config.getUInt(config_prefix + ".max_connections", 100); cfg.endpointOverride = uri.endpoint; auto proxy_config = getProxyConfiguration(config_prefix, config); diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 91136d1fded..39951477779 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -106,23 +106,68 @@ namespace const std::string host; const UInt16 port; bool https; + const String proxy_host; + const UInt16 proxy_port; + bool proxy_https; + bool resolve_host; using Base = PoolBase; ObjectPtr allocObject() override { - return makeHTTPSessionImpl(host, port, https, true); + auto session = makeHTTPSessionImpl(host, port, https, true, resolve_host); + if (!proxy_host.empty()) + { + const String proxy_scheme = proxy_https ? "https" : "http"; + session->setProxyHost(proxy_host); + session->setProxyPort(proxy_port); + +#if !defined(ARCADIA_BUILD) + session->setProxyProtocol(proxy_scheme); + + /// Turn on tunnel mode if proxy scheme is HTTP while endpoint scheme is HTTPS. + session->setProxyTunnel(!proxy_https && https); +#endif + } + return session; } public: - SingleEndpointHTTPSessionPool(const std::string & host_, UInt16 port_, bool https_, size_t max_pool_size_) - : Base(max_pool_size_, &Poco::Logger::get("HTTPSessionPool")), host(host_), port(port_), https(https_) + SingleEndpointHTTPSessionPool( + const std::string & host_, + UInt16 port_, + bool https_, + const std::string & proxy_host_, + UInt16 proxy_port_, + bool proxy_https_, + size_t max_pool_size_, + bool resolve_host_ = true) + : Base(max_pool_size_, &Poco::Logger::get("HTTPSessionPool")) + , host(host_) + , port(port_) + , https(https_) + , proxy_host(proxy_host_) + , proxy_port(proxy_port_) + , proxy_https(proxy_https_) + , resolve_host(resolve_host_) { } }; class HTTPSessionPool : private boost::noncopyable { + public: + struct Key + { + String target_host; + UInt16 target_port; + bool is_target_https; + String proxy_host; + UInt16 proxy_port; + bool is_proxy_https; + + bool operator ==(const Key &) const = default; + }; + private: - using Key = std::tuple; using PoolPtr = std::shared_ptr; using Entry = SingleEndpointHTTPSessionPool::Entry; @@ -131,9 +176,12 @@ namespace size_t operator()(const Key & k) const { SipHash s; - s.update(std::get<0>(k)); - s.update(std::get<1>(k)); - s.update(std::get<2>(k)); + s.update(k.target_host); + s.update(k.target_port); + s.update(k.is_target_https); + s.update(k.proxy_host); + s.update(k.proxy_port); + s.update(k.is_proxy_https); return s.get64(); } }; @@ -153,18 +201,32 @@ namespace Entry getSession( const Poco::URI & uri, + const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, - size_t max_connections_per_endpoint) + size_t max_connections_per_endpoint, + bool resolve_host = true) { std::unique_lock lock(mutex); const std::string & host = uri.getHost(); UInt16 port = uri.getPort(); bool https = isHTTPS(uri); - auto key = std::make_tuple(host, port, https); + + + String proxy_host; + UInt16 proxy_port = 0; + bool proxy_https = false; + if (!proxy_uri.empty()) + { + proxy_host = proxy_uri.getHost(); + proxy_port = proxy_uri.getPort(); + proxy_https = isHTTPS(proxy_uri); + } + + HTTPSessionPool::Key key{host, port, https, proxy_host, proxy_port, proxy_https}; auto pool_ptr = endpoints_pool.find(key); if (pool_ptr == endpoints_pool.end()) std::tie(pool_ptr, std::ignore) = endpoints_pool.emplace( - key, std::make_shared(host, port, https, max_connections_per_endpoint)); + key, std::make_shared(host, port, https, proxy_host, proxy_port, proxy_https, max_connections_per_endpoint, resolve_host)); auto retry_timeout = timeouts.connection_timeout.totalMicroseconds(); auto session = pool_ptr->second->get(retry_timeout); @@ -178,13 +240,17 @@ namespace if (!msg.empty()) { LOG_TRACE((&Poco::Logger::get("HTTPCommon")), "Failed communicating with {} with error '{}' will try to reconnect session", host, msg); - /// Host can change IP - const auto ip = DNSResolver::instance().resolveHost(host).toString(); - if (ip != session->getHost()) + + if (resolve_host) { - session->reset(); - session->setHost(ip); - session->attachSessionData({}); + /// Host can change IP + const auto ip = DNSResolver::instance().resolveHost(host).toString(); + if (ip != session->getHost()) + { + session->reset(); + session->setHost(ip); + session->attachSessionData({}); + } } } } @@ -218,9 +284,14 @@ HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & } -PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size) +PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, bool resolve_host) { - return HTTPSessionPool::instance().getSession(uri, timeouts, per_endpoint_pool_size); + return makePooledHTTPSession(uri, {}, timeouts, per_endpoint_pool_size, resolve_host); +} + +PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, bool resolve_host) +{ + return HTTPSessionPool::instance().getSession(uri, proxy_uri, timeouts, per_endpoint_pool_size, resolve_host); } bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status) { return status == Poco::Net::HTTPResponse::HTTP_MOVED_PERMANENTLY || status == Poco::Net::HTTPResponse::HTTP_FOUND || status == Poco::Net::HTTPResponse::HTTP_SEE_OTHER || status == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT; } diff --git a/src/IO/HTTPCommon.h b/src/IO/HTTPCommon.h index db0abe8fc6e..4a81d23a8a3 100644 --- a/src/IO/HTTPCommon.h +++ b/src/IO/HTTPCommon.h @@ -50,8 +50,9 @@ void setResponseDefaultHeaders(Poco::Net::HTTPServerResponse & response, unsigne /// Create session object to perform requests and set required parameters. HTTPSessionPtr makeHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, bool resolve_host = true); -/// As previous method creates session, but tooks it from pool -PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size); +/// As previous method creates session, but tooks it from pool, without and with proxy uri. +PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, bool resolve_host = true); +PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const Poco::URI & proxy_uri, const ConnectionTimeouts & timeouts, size_t per_endpoint_pool_size, bool resolve_host = true); bool isRedirect(const Poco::Net::HTTPResponse::HTTPStatus status); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 57916251d9b..2389f9a2192 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -6,7 +6,7 @@ #include #include -#include +#include #include #include #include @@ -86,6 +86,7 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & clientConfigu )) , remote_host_filter(clientConfiguration.remote_host_filter) , s3_max_redirects(clientConfiguration.s3_max_redirects) + , max_connections(clientConfiguration.maxConnections) { } @@ -164,28 +165,24 @@ void PocoHTTPClient::makeRequestInternal( { for (unsigned int attempt = 0; attempt <= s3_max_redirects; ++attempt) { - Poco::URI poco_uri(uri); - - /// Reverse proxy can replace host header with resolved ip address instead of host name. - /// This can lead to request signature difference on S3 side. - auto session = makeHTTPSession(poco_uri, timeouts, false); + Poco::URI target_uri(uri); + Poco::URI proxy_uri; auto request_configuration = per_request_configuration(request); if (!request_configuration.proxyHost.empty()) { - /// Turn on tunnel mode if proxy scheme is HTTP while endpoint scheme is HTTPS. - bool use_tunnel = request_configuration.proxyScheme == Aws::Http::Scheme::HTTP && poco_uri.getScheme() == "https"; - session->setProxy( - request_configuration.proxyHost, - request_configuration.proxyPort, - Aws::Http::SchemeMapper::ToString(request_configuration.proxyScheme), - use_tunnel - ); + proxy_uri.setScheme(Aws::Http::SchemeMapper::ToString(request_configuration.proxyScheme)); + proxy_uri.setHost(request_configuration.proxyHost); + proxy_uri.setPort(request_configuration.proxyPort); } + /// Reverse proxy can replace host header with resolved ip address instead of host name. + /// This can lead to request signature difference on S3 side. + auto session = makePooledHTTPSession(target_uri, proxy_uri, timeouts, max_connections, false); + Poco::Net::HTTPRequest poco_request(Poco::Net::HTTPRequest::HTTP_1_1); - poco_request.setURI(poco_uri.getPathAndQuery()); + poco_request.setURI(target_uri.getPathAndQuery()); switch (request.GetMethod()) { @@ -281,7 +278,7 @@ void PocoHTTPClient::makeRequestInternal( } } else - response->GetResponseStream().SetUnderlyingStream(std::make_shared(session, response_body_stream)); + response->GetResponseStream().SetUnderlyingStream(std::make_shared>(session, response_body_stream)); return; } @@ -297,6 +294,7 @@ void PocoHTTPClient::makeRequestInternal( ProfileEvents::increment(select_metric(S3MetricType::Errors)); } } + } #endif diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 560f0a455f0..e4fc453f388 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -56,6 +56,7 @@ private: ConnectionTimeouts timeouts; const RemoteHostFilter & remote_host_filter; unsigned int s3_max_redirects; + unsigned int max_connections; }; } diff --git a/src/IO/S3/PocoHTTPResponseStream.cpp b/src/IO/S3/PocoHTTPResponseStream.cpp deleted file mode 100644 index 93f85d65e30..00000000000 --- a/src/IO/S3/PocoHTTPResponseStream.cpp +++ /dev/null @@ -1,19 +0,0 @@ -#include - -#if USE_AWS_S3 - - -#include "PocoHTTPResponseStream.h" - -#include - -namespace DB::S3 -{ -PocoHTTPResponseStream::PocoHTTPResponseStream(std::shared_ptr session_, std::istream & response_stream_) - : Aws::IOStream(response_stream_.rdbuf()), session(std::move(session_)) -{ -} - -} - -#endif diff --git a/src/IO/S3/PocoHTTPResponseStream.h b/src/IO/S3/PocoHTTPResponseStream.h deleted file mode 100644 index fe3df6e52a7..00000000000 --- a/src/IO/S3/PocoHTTPResponseStream.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include -#include - -namespace DB::S3 -{ -/** - * Wrapper of IStream to store response stream and corresponding HTTP session. - */ -class PocoHTTPResponseStream : public Aws::IOStream -{ -public: - PocoHTTPResponseStream(std::shared_ptr session_, std::istream & response_stream_); - -private: - /// Poco HTTP session is holder of response stream. - std::shared_ptr session; -}; - -} diff --git a/src/IO/S3/SessionAwareAwsStream.h b/src/IO/S3/SessionAwareAwsStream.h new file mode 100644 index 00000000000..70ddafba067 --- /dev/null +++ b/src/IO/S3/SessionAwareAwsStream.h @@ -0,0 +1,27 @@ +#pragma once + +#include + +#include + + +namespace DB::S3 +{ +/** + * Wrapper of IOStream to store response stream and corresponding HTTP session. + */ +template +class SessionAwareAwsStream : public Aws::IOStream +{ +public: + SessionAwareAwsStream(Session session_, std::iostream & response_stream_) + : Aws::IStream(response_stream_.rdbuf()), session(std::move(session_)) + { + } + +private: + /// Poco HTTP session is holder of response stream. + Session session; +}; + +} diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 06c51e058a0..d4c4ba9bb02 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -280,7 +280,7 @@ namespace S3 } std::shared_ptr ClientFactory::create( // NOLINT - Aws::Client::ClientConfiguration & cfg, + const Aws::Client::ClientConfiguration & cfg, bool is_virtual_hosted_style, const String & access_key_id, const String & secret_access_key, @@ -306,7 +306,7 @@ namespace S3 } std::shared_ptr ClientFactory::create( // NOLINT - const String & endpoint, + const Aws::Client::ClientConfiguration & cfg, bool is_virtual_hosted_style, const String & access_key_id, const String & secret_access_key, @@ -315,10 +315,7 @@ namespace S3 const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects) { - PocoHTTPClientConfiguration client_configuration({}, remote_host_filter, s3_max_redirects); - - if (!endpoint.empty()) - client_configuration.endpointOverride = endpoint; + PocoHTTPClientConfiguration client_configuration(cfg, remote_host_filter, s3_max_redirects); client_configuration.updateSchemeAndRegion(); diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 664c07d5bf4..e2ec0785811 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -41,7 +41,7 @@ public: unsigned int s3_max_redirects); std::shared_ptr create( - Aws::Client::ClientConfiguration & cfg, + const Aws::Client::ClientConfiguration & cfg, bool is_virtual_hosted_style, const String & access_key_id, const String & secret_access_key, @@ -50,7 +50,7 @@ public: unsigned int s3_max_redirects); std::shared_ptr create( - const String & endpoint, + const Aws::Client::ClientConfiguration & cfg, bool is_virtual_hosted_style, const String & access_key_id, const String & secret_access_key, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 5d7fc0cdaa9..1920a84e447 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -196,6 +196,7 @@ StorageS3::StorageS3( const String & format_name_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, + UInt64 max_connections_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, @@ -220,8 +221,12 @@ StorageS3::StorageS3( if (access_key_id_.empty()) credentials = Aws::Auth::AWSCredentials(std::move(settings.access_key_id), std::move(settings.secret_access_key)); + Aws::Client::ClientConfiguration client_configuration; + client_configuration.endpointOverride = uri_.endpoint; + client_configuration.maxConnections = max_connections_; + client = S3::ClientFactory::instance().create( - uri_.endpoint, + client_configuration, uri_.is_virtual_hosted_style, credentials.GetAWSAccessKeyId(), credentials.GetAWSSecretKey(), @@ -374,6 +379,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) UInt64 min_upload_part_size = args.local_context.getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = args.local_context.getSettingsRef().s3_max_single_part_upload_size; + UInt64 max_connections = args.local_context.getSettingsRef().s3_max_connections; String compression_method; String format_name; @@ -396,6 +402,7 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory) format_name, min_upload_part_size, max_single_part_upload_size, + max_connections, args.columns, args.constraints, args.context, diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index f436fb85c90..f006de39c99 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -32,6 +32,7 @@ public: const String & format_name_, UInt64 min_upload_part_size_, UInt64 max_single_part_upload_size_, + UInt64 max_connections_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_, diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index cc7877b204e..6dc9230ca46 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -68,6 +68,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const C S3::URI s3_uri (uri); UInt64 min_upload_part_size = context.getSettingsRef().s3_min_upload_part_size; UInt64 max_single_part_upload_size = context.getSettingsRef().s3_max_single_part_upload_size; + UInt64 max_connections = context.getSettingsRef().s3_max_connections; StoragePtr storage = StorageS3::create( s3_uri, @@ -77,6 +78,7 @@ StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, const C format, min_upload_part_size, max_single_part_upload_size, + max_connections, getActualTableStructure(context), ConstraintsDescription{}, const_cast(context), From 59fb76e8bc04a76b1a791b684075a11a7a4e17a4 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 14 Dec 2020 10:08:29 +0300 Subject: [PATCH 074/264] Fix. --- src/IO/S3/SessionAwareAwsStream.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/SessionAwareAwsStream.h b/src/IO/S3/SessionAwareAwsStream.h index 70ddafba067..f64be5dac16 100644 --- a/src/IO/S3/SessionAwareAwsStream.h +++ b/src/IO/S3/SessionAwareAwsStream.h @@ -11,10 +11,10 @@ namespace DB::S3 * Wrapper of IOStream to store response stream and corresponding HTTP session. */ template -class SessionAwareAwsStream : public Aws::IOStream +class SessionAwareAwsStream : public Aws::IStream { public: - SessionAwareAwsStream(Session session_, std::iostream & response_stream_) + SessionAwareAwsStream(Session session_, std::istream & response_stream_) : Aws::IStream(response_stream_.rdbuf()), session(std::move(session_)) { } From ad6fe2a8b138ac118bb5767e6a4a8ce4daae1c93 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 21 Dec 2020 10:48:26 +0300 Subject: [PATCH 075/264] Build fix. --- src/IO/HTTPCommon.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index 39951477779..6a3ebb97734 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -164,7 +164,11 @@ namespace UInt16 proxy_port; bool is_proxy_https; - bool operator ==(const Key &) const = default; + bool operator ==(const Key & rhs) const + { + return std::tie(target_host, target_port, is_target_https, proxy_host, proxy_port, is_proxy_https) + == std::tie(rhs.target_host, rhs.target_port, rhs.is_target_https, rhs.proxy_host, rhs.proxy_port, rhs.is_proxy_https); + } }; private: From ec8e3f2409c2979c9e687c775a7dcb91496bc5b7 Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Tue, 5 Jan 2021 21:08:36 +0800 Subject: [PATCH 076/264] add decode numeric entities --- src/Functions/decodeXMLComponent.cpp | 70 +++++++++++++++++-- .../0_stateless/01621_decode_XML.reference | 20 ++++++ .../queries/0_stateless/01621_decode_XML.sql | 24 ++++++- 3 files changed, 107 insertions(+), 7 deletions(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index 1cae1e2956c..b2f42eabd02 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -3,6 +3,7 @@ #include #include +#include namespace DB { namespace ErrorCodes @@ -53,6 +54,8 @@ namespace } private: + static const int min_XML_number = 32; + static const int max_XML_number = 126; static size_t execute(const char * src, size_t src_size, char * dst) { const char * src_prev_pos = src; @@ -72,7 +75,7 @@ namespace else if (*src_curr_pos == '&') { src_next_pos = find_first_symbols<';'>(src_curr_pos, src_end); - if (src_next_pos == src_end || src_next_pos - src_curr_pos < 3) + if (src_next_pos == src_end) { src_curr_pos = src_end; break; @@ -99,11 +102,12 @@ namespace } else { - src_curr_pos = src_next_pos + 1; + ++src_curr_pos; size_t bytes_to_copy = src_curr_pos - src_prev_pos; memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); dst_pos += bytes_to_copy; src_prev_pos = src_curr_pos; + continue; } src_curr_pos += 4; } @@ -118,13 +122,34 @@ namespace ++dst_pos; src_prev_pos = src_curr_pos + 5; } + else if (*(src_curr_pos + 1) == '#' && isdigit(*(src_curr_pos + 2)) && isdigit(*(src_curr_pos + 3))) + { + char numeric_character = decodeNumberPart(src_curr_pos + 2); + if (numeric_character == '\0') + { + size_t bytes_to_copy = src_next_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_prev_pos = src_curr_pos + 5; + } + else + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + *dst_pos = '\0' + numeric_character; + ++dst_pos; + src_prev_pos = src_curr_pos + 5; + } + } else { - src_curr_pos = src_next_pos + 1; + ++src_curr_pos; size_t bytes_to_copy = src_curr_pos - src_prev_pos; memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); dst_pos += bytes_to_copy; src_prev_pos = src_curr_pos; + continue; } src_curr_pos += 5; } @@ -148,19 +173,42 @@ namespace ++dst_pos; src_prev_pos = src_curr_pos + 6; } + else if ( + *(src_curr_pos + 1) == '#' && isdigit(*(src_curr_pos + 2)) && isdigit(*(src_curr_pos + 3)) + && isdigit(*(src_curr_pos + 4))) + { + char numeric_character = decodeNumberPart(src_curr_pos + 2); + if (numeric_character == '\0') + { + size_t bytes_to_copy = src_next_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_prev_pos = src_curr_pos + 6; + } + else + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + *dst_pos = '\0' + numeric_character; + ++dst_pos; + src_prev_pos = src_curr_pos + 6; + } + } else { - src_curr_pos = src_next_pos + 1; + ++src_curr_pos; size_t bytes_to_copy = src_curr_pos - src_prev_pos; memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); dst_pos += bytes_to_copy; src_prev_pos = src_curr_pos; + continue; } src_curr_pos += 6; } else { - src_curr_pos = src_next_pos + 1; + ++src_curr_pos; size_t bytes_to_copy = src_curr_pos - src_prev_pos; memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); dst_pos += bytes_to_copy; @@ -178,6 +226,16 @@ namespace return dst_pos - dst; } + + static inline char decodeNumberPart(const char * src) + { + auto ans = strtol(src, nullptr, 10); + if (ans >= min_XML_number && ans <= max_XML_number) + { + return '\0' + ans; + } + return '\0'; + } }; using FunctionDecodeXMLComponent = FunctionStringToString; @@ -188,4 +246,4 @@ void registerFunctionDecodeXMLComponent(FunctionFactory & factory) { factory.registerFunction(); } -} +} \ No newline at end of file diff --git a/tests/queries/0_stateless/01621_decode_XML.reference b/tests/queries/0_stateless/01621_decode_XML.reference index dab7a2d14e1..854f453ca10 100644 --- a/tests/queries/0_stateless/01621_decode_XML.reference +++ b/tests/queries/0_stateless/01621_decode_XML.reference @@ -2,3 +2,23 @@ Hello, "world"! <123> &clickhouse \'foo\' +Hello, && world +Hello, &;& world +Hello, &a;& world +Hello, <t;& world +Hello, <t& world +Hello, &t;& world +� +  + !"#$%&\'()*+,-./012 +)*+,-./0123456789:;< +=>?@ABCDEFGHIJKLMNOP +QRSTUVWXYZ[\\]^_`abcd +efghijklmnopqrstuvwx +yz{|}~€‚ƒ„…†‡ˆ‰Š‹Œ +Ž‘’“”•–—˜™š›œžŸ  +¡¢£¤¥¦§¨©ª«¬­®¯°±²³´ +µ¶·¸¹º»¼½¾¿ÀÁÂÃÄÅÆÇÈ +ÉÊËÌÍÎÏÐÑÒÓÔÕÖ×ØÙÚÛÜ +ÝÞßàáâãäåæçèéêëìíîïð +ñòóôõö÷øùúûüýþÿĀ diff --git a/tests/queries/0_stateless/01621_decode_XML.sql b/tests/queries/0_stateless/01621_decode_XML.sql index 4c9404e6925..04319ad1759 100644 --- a/tests/queries/0_stateless/01621_decode_XML.sql +++ b/tests/queries/0_stateless/01621_decode_XML.sql @@ -1,4 +1,26 @@ SELECT decodeXMLComponent('Hello, "world"!'); SELECT decodeXMLComponent('<123>'); SELECT decodeXMLComponent('&clickhouse'); -SELECT decodeXMLComponent(''foo''); \ No newline at end of file +SELECT decodeXMLComponent(''foo''); +SELECT decodeXMLComponent('Hello, && world'); +SELECT decodeXMLComponent('Hello, &;& world'); +SELECT decodeXMLComponent('Hello, &a;& world'); +SELECT decodeXMLComponent('Hello, <t;& world'); +SELECT decodeXMLComponent('Hello, <t& world'); +SELECT decodeXMLComponent('Hello, &t;& world'); + +--decode numeric entities +SELECT decodeXMLComponent('� '); +SELECT decodeXMLComponent(' '); +SELECT decodeXMLComponent(' !"#$%&'()*+,-./012'); +SELECT decodeXMLComponent(')*+,-./0123456789:;<'); +SELECT decodeXMLComponent('=>?@ABCDEFGHIJKLMNOP'); +SELECT decodeXMLComponent('QRSTUVWXYZ[\]^_`abcd'); +SELECT decodeXMLComponent('efghijklmnopqrstuvwx'); +SELECT decodeXMLComponent('yz{|}~€‚ƒ„…†‡ˆ‰Š‹Œ'); +SELECT decodeXMLComponent('Ž‘’“”•–—˜™š›œžŸ '); +SELECT decodeXMLComponent('¡¢£¤¥¦§¨©ª«¬­®¯°±²³´'); +SELECT decodeXMLComponent('µ¶·¸¹º»¼½¾¿ÀÁÂÃÄÅÆÇÈ'); +SELECT decodeXMLComponent('ÉÊËÌÍÎÏÐÑÒÓÔÕÖ×ØÙÚÛÜ'); +SELECT decodeXMLComponent('ÝÞßàáâãäåæçèéêëìíîïð'); +SELECT decodeXMLComponent('ñòóôõö÷øùúûüýþÿĀ'); \ No newline at end of file From 48ac1ce81ac4454e4affe49211ab6b30098b624b Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Tue, 5 Jan 2021 21:41:18 +0800 Subject: [PATCH 077/264] fix --- src/Functions/decodeXMLComponent.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index b2f42eabd02..2fdefa8a6c7 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -246,4 +246,4 @@ void registerFunctionDecodeXMLComponent(FunctionFactory & factory) { factory.registerFunction(); } -} \ No newline at end of file +} From 709556832fac124ff712944ae28061317f0a5f3f Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Tue, 5 Jan 2021 22:28:09 +0800 Subject: [PATCH 078/264] fix typos --- src/Functions/decodeXMLComponent.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index 2fdefa8a6c7..7aef3445614 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -229,10 +229,10 @@ namespace static inline char decodeNumberPart(const char * src) { - auto ans = strtol(src, nullptr, 10); - if (ans >= min_XML_number && ans <= max_XML_number) + auto numberic_ans = strtol(src, nullptr, 10); + if (numberic_ans >= min_XML_number && ans <= max_XML_number) { - return '\0' + ans; + return '\0' + numberic_ans; } return '\0'; } From 6ecf505f48d53ce0814eb1d5d4482e903a104f3d Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Tue, 5 Jan 2021 23:08:36 +0800 Subject: [PATCH 079/264] fix typos --- src/Functions/decodeXMLComponent.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index 7aef3445614..a4cad7834b2 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -230,7 +230,7 @@ namespace static inline char decodeNumberPart(const char * src) { auto numberic_ans = strtol(src, nullptr, 10); - if (numberic_ans >= min_XML_number && ans <= max_XML_number) + if (numberic_ans >= min_XML_number && numberic_ans <= max_XML_number) { return '\0' + numberic_ans; } From 60d4db421caf94f0c1828c95b673924d10fb4658 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 06:45:12 +0300 Subject: [PATCH 080/264] Fix Arcadia --- src/DataStreams/ya.make.in | 2 +- src/Databases/DatabaseFactory.cpp | 2 +- src/Databases/ya.make.in | 2 +- src/Dictionaries/ya.make.in | 2 +- src/TableFunctions/ya.make.in | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/DataStreams/ya.make.in b/src/DataStreams/ya.make.in index 1624ddb799f..eeef4b9b95d 100644 --- a/src/DataStreams/ya.make.in +++ b/src/DataStreams/ya.make.in @@ -12,7 +12,7 @@ NO_COMPILER_WARNINGS() SRCS( - + ) END() diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 4360420668d..e3456c2e53b 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -32,7 +32,7 @@ #endif #if USE_LIBPQXX -#include +#include // Y_IGNORE #endif namespace DB diff --git a/src/Databases/ya.make.in b/src/Databases/ya.make.in index 0c5692a9bfa..b0e8faf223c 100644 --- a/src/Databases/ya.make.in +++ b/src/Databases/ya.make.in @@ -8,7 +8,7 @@ PEERDIR( SRCS( - + ) END() diff --git a/src/Dictionaries/ya.make.in b/src/Dictionaries/ya.make.in index d11ab4b0840..5df5803e7f4 100644 --- a/src/Dictionaries/ya.make.in +++ b/src/Dictionaries/ya.make.in @@ -15,7 +15,7 @@ NO_COMPILER_WARNINGS() SRCS( - + ) END() diff --git a/src/TableFunctions/ya.make.in b/src/TableFunctions/ya.make.in index 6465d033067..0d5aa172cb4 100644 --- a/src/TableFunctions/ya.make.in +++ b/src/TableFunctions/ya.make.in @@ -8,7 +8,7 @@ PEERDIR( SRCS( - + ) END() From 0b6c0d474f9a98b332ffdfbb00696465d6b60ea6 Mon Sep 17 00:00:00 2001 From: templarzq Date: Thu, 7 Jan 2021 15:33:42 +0800 Subject: [PATCH 081/264] Update ColumnAggregateFunction.cpp --- src/Columns/ColumnAggregateFunction.cpp | 31 +++++++++++++++++++++++++ 1 file changed, 31 insertions(+) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 2f3a766b8f5..44928fefe56 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -608,5 +608,36 @@ ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & func(src_.func), src(src_.getPtr()), data(src_.data.begin(), src_.data.end()) { } + +//override method cloneResized +MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { + //create a new col to return + auto cloneCol = cloneEmpty(); + auto res = typeid_cast(cloneCol.get()); + if (size == 0) + return cloneCol; + + size_t from_size = data.size(); + auto & res_data = res->data; + //copy data to cloned column + if (size <= from_size) + { + res_data.resize(size); + res->insertRangeFrom(*this, 0, size); + } + else + { + res_data.resize(from_size); + if (from_size > 0) + { + res->insertRangeFrom(*this, 0, from_size); + } + res->ensureOwnership(); + for(int i=0;iinsertDefault(); + } + } + return cloneCol; +} } From 1769a51f70d4204ecced22d679471f8681b84a17 Mon Sep 17 00:00:00 2001 From: templarzq Date: Thu, 7 Jan 2021 15:34:32 +0800 Subject: [PATCH 082/264] Update ColumnAggregateFunction.h --- src/Columns/ColumnAggregateFunction.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index f257351a4d0..0ba8e561ab6 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -208,6 +208,8 @@ public: void getExtremes(Field & min, Field & max) const override; bool structureEquals(const IColumn &) const override; + + MutableColumnPtr cloneResized(size_t size) const override; }; From 8af19c325148792554ce5ab0a5cbde66d372c19a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 15:29:02 +0300 Subject: [PATCH 083/264] Fix Arcadia --- src/DataStreams/ya.make | 1 - src/Databases/ya.make | 2 -- src/Dictionaries/ya.make | 1 - src/TableFunctions/ya.make | 1 - 4 files changed, 5 deletions(-) diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index c82e0a25354..8648f233f26 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -35,7 +35,6 @@ SRCS( MongoDBBlockInputStream.cpp NativeBlockInputStream.cpp NativeBlockOutputStream.cpp - PostgreSQLBlockInputStream.cpp PushingToViewsBlockOutputStream.cpp RemoteBlockInputStream.cpp RemoteBlockOutputStream.cpp diff --git a/src/Databases/ya.make b/src/Databases/ya.make index 5ae5b2260e8..0dc44386088 100644 --- a/src/Databases/ya.make +++ b/src/Databases/ya.make @@ -25,8 +25,6 @@ SRCS( MySQL/MaterializeMetadata.cpp MySQL/MaterializeMySQLSettings.cpp MySQL/MaterializeMySQLSyncThread.cpp - PostgreSQL/DatabasePostgreSQL.cpp - PostgreSQL/FetchFromPostgreSQL.cpp ) diff --git a/src/Dictionaries/ya.make b/src/Dictionaries/ya.make index 159d9b07ac0..19a0f5008b8 100644 --- a/src/Dictionaries/ya.make +++ b/src/Dictionaries/ya.make @@ -61,7 +61,6 @@ SRCS( PolygonDictionary.cpp PolygonDictionaryImplementations.cpp PolygonDictionaryUtils.cpp - PostgreSQLDictionarySource.cpp RangeHashedDictionary.cpp RedisBlockInputStream.cpp RedisDictionarySource.cpp diff --git a/src/TableFunctions/ya.make b/src/TableFunctions/ya.make index 036e8ebee19..7bcf5fc53b3 100644 --- a/src/TableFunctions/ya.make +++ b/src/TableFunctions/ya.make @@ -20,7 +20,6 @@ SRCS( TableFunctionMySQL.cpp TableFunctionNull.cpp TableFunctionNumbers.cpp - TableFunctionPostgreSQL.cpp TableFunctionRemote.cpp TableFunctionURL.cpp TableFunctionValues.cpp From e7a7f557d1d110e4472b6c5597333242fed93fa7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Jan 2021 18:40:54 +0300 Subject: [PATCH 084/264] Fix Arcadia --- src/Dictionaries/registerDictionaries.cpp | 36 +++++++++++++++++++++++ src/Dictionaries/registerDictionaries.h | 32 -------------------- src/Storages/ya.make | 1 - src/Storages/ya.make.in | 2 +- 4 files changed, 37 insertions(+), 34 deletions(-) diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 1de00cdeaab..abcc0ce06ad 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -4,6 +4,40 @@ namespace DB { +class DictionarySourceFactory; + +void registerDictionarySourceFile(DictionarySourceFactory & source_factory); +void registerDictionarySourceMysql(DictionarySourceFactory & source_factory); +void registerDictionarySourceClickHouse(DictionarySourceFactory & source_factory); +void registerDictionarySourceMongoDB(DictionarySourceFactory & source_factory); +void registerDictionarySourceCassandra(DictionarySourceFactory & source_factory); +void registerDictionarySourceRedis(DictionarySourceFactory & source_factory); +void registerDictionarySourceXDBC(DictionarySourceFactory & source_factory); +void registerDictionarySourceJDBC(DictionarySourceFactory & source_factory); +#if !defined(ARCADIA_BUILD) +void registerDictionarySourcePostgreSQL(DictionarySourceFactory & source_factory); +#endif +void registerDictionarySourceExecutable(DictionarySourceFactory & source_factory); +void registerDictionarySourceHTTP(DictionarySourceFactory & source_factory); +void registerDictionarySourceLibrary(DictionarySourceFactory & source_factory); + +class DictionaryFactory; +void registerDictionaryRangeHashed(DictionaryFactory & factory); +void registerDictionaryComplexKeyHashed(DictionaryFactory & factory); +void registerDictionaryComplexKeyCache(DictionaryFactory & factory); +void registerDictionaryComplexKeyDirect(DictionaryFactory & factory); +void registerDictionaryTrie(DictionaryFactory & factory); +void registerDictionaryFlat(DictionaryFactory & factory); +void registerDictionaryHashed(DictionaryFactory & factory); +void registerDictionaryCache(DictionaryFactory & factory); +#if defined(__linux__) || defined(__FreeBSD__) +void registerDictionarySSDCache(DictionaryFactory & factory); +void registerDictionarySSDComplexKeyCache(DictionaryFactory & factory); +#endif +void registerDictionaryPolygon(DictionaryFactory & factory); +void registerDictionaryDirect(DictionaryFactory & factory); + + void registerDictionaries() { { @@ -16,7 +50,9 @@ void registerDictionaries() registerDictionarySourceCassandra(source_factory); registerDictionarySourceXDBC(source_factory); registerDictionarySourceJDBC(source_factory); +#if !defined(ARCADIA_BUILD) registerDictionarySourcePostgreSQL(source_factory); +#endif registerDictionarySourceExecutable(source_factory); registerDictionarySourceHTTP(source_factory); registerDictionarySourceLibrary(source_factory); diff --git a/src/Dictionaries/registerDictionaries.h b/src/Dictionaries/registerDictionaries.h index 51fea84e869..e8480277c2c 100644 --- a/src/Dictionaries/registerDictionaries.h +++ b/src/Dictionaries/registerDictionaries.h @@ -2,37 +2,5 @@ namespace DB { - -class DictionarySourceFactory; - -void registerDictionarySourceFile(DictionarySourceFactory & source_factory); -void registerDictionarySourceMysql(DictionarySourceFactory & source_factory); -void registerDictionarySourceClickHouse(DictionarySourceFactory & source_factory); -void registerDictionarySourceMongoDB(DictionarySourceFactory & source_factory); -void registerDictionarySourceCassandra(DictionarySourceFactory & source_factory); -void registerDictionarySourceRedis(DictionarySourceFactory & source_factory); -void registerDictionarySourceXDBC(DictionarySourceFactory & source_factory); -void registerDictionarySourceJDBC(DictionarySourceFactory & source_factory); -void registerDictionarySourcePostgreSQL(DictionarySourceFactory & source_factory); -void registerDictionarySourceExecutable(DictionarySourceFactory & source_factory); -void registerDictionarySourceHTTP(DictionarySourceFactory & source_factory); -void registerDictionarySourceLibrary(DictionarySourceFactory & source_factory); - -class DictionaryFactory; -void registerDictionaryRangeHashed(DictionaryFactory & factory); -void registerDictionaryComplexKeyHashed(DictionaryFactory & factory); -void registerDictionaryComplexKeyCache(DictionaryFactory & factory); -void registerDictionaryComplexKeyDirect(DictionaryFactory & factory); -void registerDictionaryTrie(DictionaryFactory & factory); -void registerDictionaryFlat(DictionaryFactory & factory); -void registerDictionaryHashed(DictionaryFactory & factory); -void registerDictionaryCache(DictionaryFactory & factory); -#if defined(__linux__) || defined(__FreeBSD__) -void registerDictionarySSDCache(DictionaryFactory & factory); -void registerDictionarySSDComplexKeyCache(DictionaryFactory & factory); -#endif -void registerDictionaryPolygon(DictionaryFactory & factory); -void registerDictionaryDirect(DictionaryFactory & factory); - void registerDictionaries(); } diff --git a/src/Storages/ya.make b/src/Storages/ya.make index 56b72e711e4..27aa9e3ac3f 100644 --- a/src/Storages/ya.make +++ b/src/Storages/ya.make @@ -130,7 +130,6 @@ SRCS( StorageMongoDB.cpp StorageMySQL.cpp StorageNull.cpp - StoragePostgreSQL.cpp StorageReplicatedMergeTree.cpp StorageSet.cpp StorageStripeLog.cpp diff --git a/src/Storages/ya.make.in b/src/Storages/ya.make.in index 4311ff5e16b..2e8727b53fd 100644 --- a/src/Storages/ya.make.in +++ b/src/Storages/ya.make.in @@ -10,7 +10,7 @@ PEERDIR( SRCS( - + ) END() From ab9a3eaecba2dcc8e85a3d9057e7f0afce89964c Mon Sep 17 00:00:00 2001 From: templarzq Date: Fri, 8 Jan 2021 14:21:44 +0800 Subject: [PATCH 085/264] Update ColumnAggregateFunction.cpp --- src/Columns/ColumnAggregateFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 44928fefe56..5feb4d3bf6a 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -633,7 +633,7 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { res->insertRangeFrom(*this, 0, from_size); } res->ensureOwnership(); - for(int i=0;iinsertDefault(); } } From c7a3ae305ac4c58243fe11f8776ac28fac25c786 Mon Sep 17 00:00:00 2001 From: templarzq Date: Fri, 8 Jan 2021 15:14:42 +0800 Subject: [PATCH 086/264] Update ColumnAggregateFunction.cpp fix style check --- src/Columns/ColumnAggregateFunction.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 5feb4d3bf6a..ca291e14ed4 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -610,7 +610,8 @@ ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & } //override method cloneResized -MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { +MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const +{ //create a new col to return auto cloneCol = cloneEmpty(); auto res = typeid_cast(cloneCol.get()); @@ -633,7 +634,8 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { res->insertRangeFrom(*this, 0, from_size); } res->ensureOwnership(); - for(size_t i=0;iinsertDefault(); } } From ee8e3f94e7fb1c8ae751cee0a22ce4c7c0df95c5 Mon Sep 17 00:00:00 2001 From: templarzq Date: Fri, 8 Jan 2021 16:08:12 +0800 Subject: [PATCH 087/264] Update ColumnAggregateFunction.cpp convert eol to LF --- src/Columns/ColumnAggregateFunction.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index ca291e14ed4..d160f75f609 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -608,7 +608,6 @@ ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & func(src_.func), src(src_.getPtr()), data(src_.data.begin(), src_.data.end()) { } - //override method cloneResized MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { @@ -641,5 +640,4 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const } return cloneCol; } - } From f32f6a086398642f6fd1b493f3811f8d042db01d Mon Sep 17 00:00:00 2001 From: templarzq Date: Fri, 8 Jan 2021 16:10:36 +0800 Subject: [PATCH 088/264] Update ColumnAggregateFunction.h --- src/Columns/ColumnAggregateFunction.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 0ba8e561ab6..45b26f852eb 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -208,8 +208,8 @@ public: void getExtremes(Field & min, Field & max) const override; bool structureEquals(const IColumn &) const override; - - MutableColumnPtr cloneResized(size_t size) const override; + + MutableColumnPtr cloneResized(size_t size) const; }; From b57c3532892c49f3b001765d295c77d3d293f928 Mon Sep 17 00:00:00 2001 From: templarzq Date: Fri, 8 Jan 2021 16:23:33 +0800 Subject: [PATCH 089/264] Update ColumnAggregateFunction.h --- src/Columns/ColumnAggregateFunction.h | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 45b26f852eb..3cce2da3410 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -209,8 +209,6 @@ public: bool structureEquals(const IColumn &) const override; - MutableColumnPtr cloneResized(size_t size) const; + MutableColumnPtr cloneResized(size_t size) const override; }; - - } From 8359289283f98ba32d88ce87fba2e38156a0af80 Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Sat, 9 Jan 2021 19:08:19 +0800 Subject: [PATCH 090/264] add unicode decode --- src/Functions/decodeXMLComponent.cpp | 189 +++++++++++++----- .../0_stateless/01621_decode_XML.reference | 17 +- .../queries/0_stateless/01621_decode_XML.sql | 18 +- 3 files changed, 150 insertions(+), 74 deletions(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index a4cad7834b2..8c46976a718 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -1,9 +1,10 @@ #include #include #include +#include #include -#include +#include namespace DB { namespace ErrorCodes @@ -54,8 +55,8 @@ namespace } private: - static const int min_XML_number = 32; - static const int max_XML_number = 126; + static const int max_legal_unicode_value = 0x10FFFF; + static const int max_legal_unicode_bits = 7; static size_t execute(const char * src, size_t src_size, char * dst) { const char * src_prev_pos = src; @@ -80,6 +81,32 @@ namespace src_curr_pos = src_end; break; } + else if (isValidNumeric(src_curr_pos, src_next_pos)) + { + std::vector decodeNumericChars; + decodeNumericPart(src_curr_pos + 2, src_next_pos, decodeNumericChars); + if (decodeNumericChars.empty()) + { + ++src_curr_pos; + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_prev_pos = src_curr_pos; + } + else + { + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + for (size_t i = 0; i < decodeNumericChars.size(); i++) + { + *dst_pos = decodeNumericChars[i]; + ++dst_pos; + } + src_prev_pos = src_next_pos + 1; + } + src_curr_pos = src_next_pos + 1; + } else if (src_next_pos - src_curr_pos == 3) { if (strncmp(src_curr_pos, "<", 3) == 0) @@ -122,26 +149,6 @@ namespace ++dst_pos; src_prev_pos = src_curr_pos + 5; } - else if (*(src_curr_pos + 1) == '#' && isdigit(*(src_curr_pos + 2)) && isdigit(*(src_curr_pos + 3))) - { - char numeric_character = decodeNumberPart(src_curr_pos + 2); - if (numeric_character == '\0') - { - size_t bytes_to_copy = src_next_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - src_prev_pos = src_curr_pos + 5; - } - else - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - *dst_pos = '\0' + numeric_character; - ++dst_pos; - src_prev_pos = src_curr_pos + 5; - } - } else { ++src_curr_pos; @@ -173,28 +180,6 @@ namespace ++dst_pos; src_prev_pos = src_curr_pos + 6; } - else if ( - *(src_curr_pos + 1) == '#' && isdigit(*(src_curr_pos + 2)) && isdigit(*(src_curr_pos + 3)) - && isdigit(*(src_curr_pos + 4))) - { - char numeric_character = decodeNumberPart(src_curr_pos + 2); - if (numeric_character == '\0') - { - size_t bytes_to_copy = src_next_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - src_prev_pos = src_curr_pos + 6; - } - else - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - *dst_pos = '\0' + numeric_character; - ++dst_pos; - src_prev_pos = src_curr_pos + 6; - } - } else { ++src_curr_pos; @@ -227,14 +212,118 @@ namespace return dst_pos - dst; } - static inline char decodeNumberPart(const char * src) + static void decodeNumericPart(const char * src, const char * end, std::vector & decodeNumericChars) { - auto numberic_ans = strtol(src, nullptr, 10); - if (numberic_ans >= min_XML_number && numberic_ans <= max_XML_number) + int numeric_ans; + if (*src == 'x' || *src == 'X') { - return '\0' + numberic_ans; + numeric_ans = hexOrDecStrToInt(src + 1, end, 16); + } + else + { + numeric_ans = hexOrDecStrToInt(src, end, 10); + } + const auto num_bits = numBitsCount(numeric_ans); + if (num_bits <= 7) + { + decodeNumericChars.push_back('\0' + (numeric_ans & 0x7F)); + } + else if (num_bits <= 11) + { + decodeNumericChars.push_back('\0' + ((numeric_ans >> 6) & 0x1F) + 0xC0); + decodeNumericChars.push_back('\0' + (numeric_ans & 0x3F) + 0x80); + } + else if (num_bits <= 16) + { + decodeNumericChars.push_back('\0' + ((numeric_ans >> 12) & 0x0F) + 0xE0); + decodeNumericChars.push_back('\0' + ((numeric_ans >> 6) & 0x3F) + 0x80); + decodeNumericChars.push_back('\0' + (numeric_ans & 0x3F) + 0x80); + } + else if ((num_bits <= 21) && (numeric_ans <= max_legal_unicode_value)) + { + decodeNumericChars.push_back('\0' + ((numeric_ans >> 18) & 0x07) + 0xF0); + decodeNumericChars.push_back('\0' + ((numeric_ans >> 12) & 0x3F) + 0x80); + decodeNumericChars.push_back('\0' + ((numeric_ans >> 6) & 0x3F) + 0x80); + decodeNumericChars.push_back('\0' + (numeric_ans & 0x3F) + 0x80); + } + } + + static int hexOrDecStrToInt(const char * src, const char * end, int base) + { + int numeric_ans = 0; + int pos = 0; + if (base == 16) + { + while (src + pos != end) + { + if (isNumericASCII(*(src + pos))) + { + numeric_ans = numeric_ans * base + (*(src + pos) - '0'); + } + else if (*(src + pos) >= 'a' && *(src + pos) <= 'f') + { + numeric_ans = numeric_ans * base + (*(src + pos) - 'a' + 10); + } + else if (*(src + pos) >= 'A' && *(src + pos) <= 'F') + { + numeric_ans = numeric_ans * base + (*(src + pos) - 'A' + 10); + } + ++pos; + } + } + else + { + while (src + pos != end) + { + numeric_ans = numeric_ans * base + (*(src + pos) - '0'); + ++pos; + } + } + return numeric_ans; + } + static int numBitsCount(int integer) + { + size_t num_bits = 0; + while (integer > 0) + { + ++num_bits; + integer >>= 1; + } + return num_bits; + } + static bool isValidNumeric(const char * src, const char * end) + { + int pos; + if (*src != '&' || *(src + 1) != '#' || (end - (src + 2) > max_legal_unicode_bits)) + { + return false; + } + if (*(src + 2) == 'x' || *(src + 2) == 'X') + { + pos = 3; + while (src + pos != end) + { + if (!isHexDigit(*(src + pos))) + { + return false; + } + ++pos; + } + return true; + } + else + { + pos = 2; + while (src + pos != end) + { + if (!isNumericASCII(*(src + pos))) + { + return false; + } + ++pos; + } + return true; } - return '\0'; } }; diff --git a/tests/queries/0_stateless/01621_decode_XML.reference b/tests/queries/0_stateless/01621_decode_XML.reference index 854f453ca10..3463fa0788c 100644 --- a/tests/queries/0_stateless/01621_decode_XML.reference +++ b/tests/queries/0_stateless/01621_decode_XML.reference @@ -8,17 +8,10 @@ Hello, &a;& world Hello, <t;& world Hello, <t& world Hello, &t;& world -� -  - !"#$%&\'()*+,-./012 + !"#$%&\'()*+,-./012 )*+,-./0123456789:;< =>?@ABCDEFGHIJKLMNOP -QRSTUVWXYZ[\\]^_`abcd -efghijklmnopqrstuvwx -yz{|}~€‚ƒ„…†‡ˆ‰Š‹Œ -Ž‘’“”•–—˜™š›œžŸ  -¡¢£¤¥¦§¨©ª«¬­®¯°±²³´ -µ¶·¸¹º»¼½¾¿ÀÁÂÃÄÅÆÇÈ -ÉÊËÌÍÎÏÐÑÒÓÔÕÖ×ØÙÚÛÜ -ÝÞßàáâãäåæçèéêëìíîïð -ñòóôõö÷øùúûüýþÿĀ +为什么 +为什么 +�\'123 +ЦЦЮЮЫㄱ diff --git a/tests/queries/0_stateless/01621_decode_XML.sql b/tests/queries/0_stateless/01621_decode_XML.sql index 04319ad1759..b111520db4c 100644 --- a/tests/queries/0_stateless/01621_decode_XML.sql +++ b/tests/queries/0_stateless/01621_decode_XML.sql @@ -10,17 +10,11 @@ SELECT decodeXMLComponent('Hello, <t& world'); SELECT decodeXMLComponent('Hello, &t;& world'); --decode numeric entities -SELECT decodeXMLComponent('� '); -SELECT decodeXMLComponent(' '); -SELECT decodeXMLComponent(' !"#$%&'()*+,-./012'); + +SELECT decodeXMLComponent(' !"#$%&'()*+,-./012'); SELECT decodeXMLComponent(')*+,-./0123456789:;<'); SELECT decodeXMLComponent('=>?@ABCDEFGHIJKLMNOP'); -SELECT decodeXMLComponent('QRSTUVWXYZ[\]^_`abcd'); -SELECT decodeXMLComponent('efghijklmnopqrstuvwx'); -SELECT decodeXMLComponent('yz{|}~€‚ƒ„…†‡ˆ‰Š‹Œ'); -SELECT decodeXMLComponent('Ž‘’“”•–—˜™š›œžŸ '); -SELECT decodeXMLComponent('¡¢£¤¥¦§¨©ª«¬­®¯°±²³´'); -SELECT decodeXMLComponent('µ¶·¸¹º»¼½¾¿ÀÁÂÃÄÅÆÇÈ'); -SELECT decodeXMLComponent('ÉÊËÌÍÎÏÐÑÒÓÔÕÖ×ØÙÚÛÜ'); -SELECT decodeXMLComponent('ÝÞßàáâãäåæçèéêëìíîïð'); -SELECT decodeXMLComponent('ñòóôõö÷øùúûüýþÿĀ'); \ No newline at end of file +SELECT decodeXMLComponent('为'); +SELECT decodeXMLComponent('为'); +SELECT decodeXMLComponent('�'123'); +SELECT decodeXMLComponent('ЦЦЮЮЫㄱ'); \ No newline at end of file From 33996d41f5d19cfc1cc66b1bf46c6b20843802ba Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Sat, 9 Jan 2021 19:15:09 +0800 Subject: [PATCH 091/264] add test cases reference --- tests/queries/0_stateless/01621_decode_XML.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01621_decode_XML.reference b/tests/queries/0_stateless/01621_decode_XML.reference index 3463fa0788c..d4fa75bbf94 100644 --- a/tests/queries/0_stateless/01621_decode_XML.reference +++ b/tests/queries/0_stateless/01621_decode_XML.reference @@ -11,7 +11,7 @@ Hello, &t;& world !"#$%&\'()*+,-./012 )*+,-./0123456789:;< =>?@ABCDEFGHIJKLMNOP -为什么 -为什么 +为 +为 �\'123 ЦЦЮЮЫㄱ From 445bc1c65cb859e661065e174bfc3fbb8166a85e Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Sat, 9 Jan 2021 22:39:25 +0800 Subject: [PATCH 092/264] fix bug --- src/Functions/decodeXMLComponent.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index 8c46976a718..ff25db76290 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -83,9 +83,9 @@ namespace } else if (isValidNumeric(src_curr_pos, src_next_pos)) { - std::vector decodeNumericChars; - decodeNumericPart(src_curr_pos + 2, src_next_pos, decodeNumericChars); - if (decodeNumericChars.empty()) + std::vector decode_numeric_chars; + decodeNumericPart(src_curr_pos + 2, src_next_pos, decode_numeric_chars); + if (decode_numeric_chars.empty()) { ++src_curr_pos; size_t bytes_to_copy = src_curr_pos - src_prev_pos; @@ -98,9 +98,9 @@ namespace size_t bytes_to_copy = src_curr_pos - src_prev_pos; memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); dst_pos += bytes_to_copy; - for (size_t i = 0; i < decodeNumericChars.size(); i++) + for (auto cur_char : decode_numeric_chars) { - *dst_pos = decodeNumericChars[i]; + *dst_pos = cur_char; ++dst_pos; } src_prev_pos = src_next_pos + 1; From 0088d66d5941d7bf0fc1b84e2a2ecae8dce68bca Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 9 Jan 2021 22:42:42 +0000 Subject: [PATCH 093/264] Hopefully better array conversions for insert query --- src/Databases/DatabaseFactory.cpp | 2 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 4 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 4 +- .../PostgreSQLDictionarySource.cpp | 6 +- src/Dictionaries/PostgreSQLDictionarySource.h | 4 +- src/Storages/StoragePostgreSQL.cpp | 246 +++++++++++++----- src/Storages/StoragePostgreSQL.h | 45 +--- .../TableFunctionPostgreSQL.cpp | 2 +- src/TableFunctions/TableFunctionPostgreSQL.h | 2 +- .../test_storage_postgresql/test.py | 106 +++++--- 10 files changed, 283 insertions(+), 138 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index e3456c2e53b..7bfb977e7cb 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -207,7 +207,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String postgres_database_name, parsed_host_port.first, std::to_string(parsed_host_port.second), username, password); /// no connection is made here - auto connection = std::make_shared(connection_str); + auto connection = std::make_shared(connection_str); return std::make_shared( context, metadata_path, engine_define, database_name, postgres_database_name, connection, use_table_cache); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 44793856f82..1e3214ea0b6 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -35,7 +35,7 @@ DatabasePostgreSQL::DatabasePostgreSQL( const ASTStorage * database_engine_define_, const String & dbname_, const String & postgres_dbname, - PGConnectionPtr connection_, + PostgreSQLConnectionPtr connection_, const bool cache_tables_) : IDatabase(dbname_) , global_context(context.getGlobalContext()) @@ -142,7 +142,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte return StoragePtr{}; auto storage = StoragePostgreSQL::create( - StorageID(database_name, table_name), table_name, std::make_shared(connection->conn_str()), + StorageID(database_name, table_name), table_name, std::make_shared(connection->conn_str()), ColumnsDescription{*columns}, ConstraintsDescription{}, context); /// There is no easy (embedded) way in postgres to check table modification time, so if `cache_tables` == 1 (default: 0) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index a80b90086c2..78d25c6c000 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -27,7 +27,7 @@ public: const ASTStorage * database_engine_define, const String & dbname_, const String & postgres_dbname, - PGConnectionPtr connection_, + PostgreSQLConnectionPtr connection_, const bool cache_tables_); String getEngineName() const override { return "PostgreSQL"; } @@ -65,7 +65,7 @@ private: String metadata_path; ASTPtr database_engine_define; String dbname; - PGConnectionPtr connection; + PostgreSQLConnectionPtr connection; const bool cache_tables; mutable Tables cached_tables; diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index ecebd60775d..417ef9b318d 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -27,7 +27,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix, - PGConnectionPtr connection_, + PostgreSQLConnectionPtr connection_, const Block & sample_block_) : dict_struct{dict_struct_} , sample_block(sample_block_) @@ -48,7 +48,7 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource( PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionarySource & other) : dict_struct(other.dict_struct) , sample_block(other.sample_block) - , connection(std::make_shared(other.connection->conn_str())) + , connection(std::make_shared(other.connection->conn_str())) , log(&Poco::Logger::get("PostgreSQLDictionarySource")) , db(other.db) , table(other.table) @@ -176,7 +176,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) config.getUInt(fmt::format("{}.port", config_prefix), 0), config.getString(fmt::format("{}.user", config_prefix), ""), config.getString(fmt::format("{}.password", config_prefix), "")); - auto connection = std::make_shared(connection_str); + auto connection = std::make_shared(connection_str); return std::make_unique( dict_struct, config, config_prefix, connection, sample_block); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index e59ab81674c..eaec84bfee1 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -25,7 +25,7 @@ public: const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config_, const std::string & config_prefix, - PGConnectionPtr connection_, + PostgreSQLConnectionPtr connection_, const Block & sample_block_); /// copy-constructor is provided in order to support cloneability @@ -50,7 +50,7 @@ private: const DictionaryStructure dict_struct; Block sample_block; - PGConnectionPtr connection; + PostgreSQLConnectionPtr connection; Poco::Logger * log; const std::string db; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 4acc1dce286..af0731ffd12 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -7,6 +7,15 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -19,18 +28,20 @@ #include #include + namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int NOT_IMPLEMENTED; } StoragePostgreSQL::StoragePostgreSQL( const StorageID & table_id_, const String & remote_table_name_, - PGConnectionPtr connection_, + PostgreSQLConnectionPtr connection_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_) @@ -76,6 +87,181 @@ Pipe StoragePostgreSQL::read( } +class PostgreSQLBlockOutputStream : public IBlockOutputStream +{ +public: + explicit PostgreSQLBlockOutputStream( + const StorageMetadataPtr & metadata_snapshot_, + ConnectionPtr connection_, + const std::string & remote_table_name_) + : metadata_snapshot(metadata_snapshot_) + , connection(connection_) + , remote_table_name(remote_table_name_) + { + } + + Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } + + void writePrefix() override + { + work = std::make_unique(*connection); + } + + void write(const Block & block) override + { + const auto columns = block.getColumns(); + const size_t num_rows = block.rows(), num_cols = block.columns(); + const auto data_types = block.getDataTypes(); + + if (!stream_inserter) + stream_inserter = std::make_unique(*work, remote_table_name, block.getNames()); + + /// std::optional lets libpqxx to know if value is NULL + std::vector> row(num_cols); + + for (const auto i : ext::range(0, num_rows)) + { + for (const auto j : ext::range(0, num_cols)) + { + if (columns[j]->isNullAt(i)) + { + row[j] = std::nullopt; + } + else + { + WriteBufferFromOwnString ostr; + + if (isArray(data_types[j])) + { + parseArray((*columns[j])[i], data_types[j], ostr); + } + else + { + data_types[j]->serializeAsText(*columns[j], i, ostr, FormatSettings{}); + } + + row[j] = ostr.str(); + } + } + /// pqxx::stream_to is much faster than simple insert, especially for large number of rows + stream_inserter->write_values(row); + } + } + + void writeSuffix() override + { + if (stream_inserter) + stream_inserter->complete(); + work->commit(); + } + + /// Cannot just use serializeAsText for array data type even though it converts perfectly + /// any dimension number array into text format, because it incloses in '[]' and for postgres it must be '{}'. + void parseArray(Field array_field, DataTypePtr data_type, WriteBuffer & ostr) + { + const auto * array_type = typeid_cast(data_type.get()); + const auto nested = array_type->getNestedType(); + const auto & array = array_field.get(); + + if (!isArray(nested)) + { + writeText(clickhouseToPostgresArray(array, data_type), ostr); + return; + } + + writeChar('{', ostr); + + const auto * nested_array_type = typeid_cast(nested.get()); + for (auto iter = array.begin(); iter != array.end(); ++iter) + { + if (iter != array.begin()) + writeText(", ", ostr); + + if (!isArray(nested_array_type->getNestedType())) + { + writeText(clickhouseToPostgresArray(iter->get(), nested), ostr); + } + else + { + parseArray(*iter, nested, ostr); + } + } + + writeChar('}', ostr); + } + + /// Conversion is done via column casting because with writeText(Array..) got incorrect conversion + /// of Date and DateTime data types and it added extra quotes for values inside array. + std::string clickhouseToPostgresArray(const Array & array_field, DataTypePtr data_type) + { + auto nested = typeid_cast(data_type.get())->getNestedType(); + ColumnPtr nested_column(createNested(nested)); + const auto array_column{ColumnArray::create(nested_column)}; + const_cast(array_column.get())->insert(array_field); + WriteBufferFromOwnString ostr; + data_type->serializeAsText(*array_column, 0, ostr, FormatSettings{}); + + /// ostr is guaranteed to be at least '[]', i.e. size is at least 2 and 2 only if ostr.str() == '[]' + return '{' + std::string(ostr.str().begin() + 1, ostr.str().end() - 1) + '}'; + } + + ColumnPtr createNested(DataTypePtr nested) + { + bool is_nullable = false; + if (nested->isNullable()) + { + is_nullable = true; + nested = typeid_cast(nested.get())->getNestedType(); + } + + WhichDataType which(nested); + ColumnPtr nested_column; + if (which.isString() || which.isFixedString()) nested_column = ColumnString::create(); + else if (which.isInt8() || which.isInt16()) nested_column = ColumnInt16::create(); + else if (which.isUInt8() || which.isUInt16()) nested_column = ColumnUInt16::create(); + else if (which.isInt32()) nested_column = ColumnInt32::create(); + else if (which.isUInt32()) nested_column = ColumnUInt32::create(); + else if (which.isInt64()) nested_column = ColumnInt64::create(); + else if (which.isUInt64()) nested_column = ColumnUInt64::create(); + else if (which.isFloat32()) nested_column = ColumnFloat32::create(); + else if (which.isFloat64()) nested_column = ColumnFloat64::create(); + else if (which.isDate()) nested_column = ColumnUInt16::create(); + else if (which.isDateTime()) nested_column = ColumnUInt32::create(); + else if (which.isDecimal32()) + { + const auto & type = typeid_cast *>(nested.get()); + nested_column = ColumnDecimal::create(0, type->getScale()); + } + else if (which.isDecimal64()) + { + const auto & type = typeid_cast *>(nested.get()); + nested_column = ColumnDecimal::create(0, type->getScale()); + } + else if (which.isDecimal128()) + { + const auto & type = typeid_cast *>(nested.get()); + nested_column = ColumnDecimal::create(0, type->getScale()); + } + else + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversion not supported"); + + if (is_nullable) + return ColumnNullable::create(nested_column, ColumnUInt8::create(nested_column->size(), 0)); + + return nested_column; + } + + +private: + StorageMetadataPtr metadata_snapshot; + ConnectionPtr connection; + std::string remote_table_name; + + std::unique_ptr work; + std::unique_ptr stream_inserter; +}; + + BlockOutputStreamPtr StoragePostgreSQL::write( const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /* context */) { @@ -83,62 +269,6 @@ BlockOutputStreamPtr StoragePostgreSQL::write( } -void PostgreSQLBlockOutputStream::writePrefix() -{ - work = std::make_unique(*connection); -} - - -void PostgreSQLBlockOutputStream::write(const Block & block) -{ - const auto columns = block.getColumns(); - const size_t num_rows = block.rows(), num_cols = block.columns(); - const auto data_types = block.getDataTypes(); - - if (!stream_inserter) - stream_inserter = std::make_unique(*work, remote_table_name, block.getNames()); - - /// std::optional lets libpqxx to know if value is NULL - std::vector> row(num_cols); - - for (const auto i : ext::range(0, num_rows)) - { - for (const auto j : ext::range(0, num_cols)) - { - if (columns[j]->isNullAt(i)) - { - row[j] = std::nullopt; - } - else - { - WriteBufferFromOwnString ostr; - data_types[j]->serializeAsText(*columns[j], i, ostr, FormatSettings{}); - row[j] = std::optional(ostr.str()); - - if (isArray(data_types[j])) - { - char r; - std::replace_if(row[j]->begin(), row[j]->end(), [&](char c) - { - return ((c == '[') && (r = '{')) || ((c == ']') && (r = '}')); - }, r); - } - } - } - /// pqxx::stream_to is much faster than simple insert, especially for large number of rows - stream_inserter->write_values(row); - } -} - - -void PostgreSQLBlockOutputStream::writeSuffix() -{ - if (stream_inserter) - stream_inserter->complete(); - work->commit(); -} - - void registerStoragePostgreSQL(StorageFactory & factory) { factory.registerStorage("PostgreSQL", [](const StorageFactory::Arguments & args) @@ -163,7 +293,7 @@ void registerStoragePostgreSQL(StorageFactory & factory) engine_args[3]->as().value.safeGet(), engine_args[4]->as().value.safeGet()); - auto connection = std::make_shared(connection_str); + auto connection = std::make_shared(connection_str); return StoragePostgreSQL::create( args.table_id, remote_table, connection, args.columns, args.constraints, args.context); }, diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 426908775be..6d1764b2080 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -14,8 +14,8 @@ namespace DB { -class PGConnection; -using PGConnectionPtr = std::shared_ptr; +class PostgreSQLConnection; +using PostgreSQLConnectionPtr = std::shared_ptr; using ConnectionPtr = std::shared_ptr; class StoragePostgreSQL final : public ext::shared_ptr_helper, public IStorage @@ -25,7 +25,7 @@ public: StoragePostgreSQL( const StorageID & table_id_, const std::string & remote_table_name_, - PGConnectionPtr connection_, + PostgreSQLConnectionPtr connection_, const ColumnsDescription & columns_, const ConstraintsDescription & constraints_, const Context & context_); @@ -48,47 +48,18 @@ private: String remote_table_name; Context global_context; - PGConnectionPtr connection; -}; - - -class PostgreSQLBlockOutputStream : public IBlockOutputStream -{ -public: - explicit PostgreSQLBlockOutputStream( - const StorageMetadataPtr & metadata_snapshot_, - ConnectionPtr connection_, - const std::string & remote_table_name_) - : metadata_snapshot(metadata_snapshot_) - , connection(connection_) - , remote_table_name(remote_table_name_) - { - } - - Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } - - void writePrefix() override; - void write(const Block & block) override; - void writeSuffix() override; - -private: - StorageMetadataPtr metadata_snapshot; - ConnectionPtr connection; - std::string remote_table_name; - - std::unique_ptr work; - std::unique_ptr stream_inserter; + PostgreSQLConnectionPtr connection; }; /// Tiny connection class to make it more convenient to use. /// Connection is not made until actually used. -class PGConnection +class PostgreSQLConnection { public: - PGConnection(std::string & connection_str_) : connection_str(connection_str_) {} - PGConnection(const PGConnection &) = delete; - PGConnection operator =(const PGConnection &) = delete; + PostgreSQLConnection(const std::string & connection_str_) : connection_str(connection_str_) {} + PostgreSQLConnection(const PostgreSQLConnection &) = delete; + PostgreSQLConnection operator =(const PostgreSQLConnection &) = delete; ConnectionPtr conn() { diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 21bf957860d..92a965cb277 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -68,7 +68,7 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, const parsed_host_port.first, std::to_string(parsed_host_port.second), args[3]->as().value.safeGet(), args[4]->as().value.safeGet()); - connection = std::make_shared(connection_str); + connection = std::make_shared(connection_str); } diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index 5bd24322d3b..9bdb4e2e32f 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -29,7 +29,7 @@ private: String connection_str; String remote_table_name; - PGConnectionPtr connection; + PostgreSQLConnectionPtr connection; }; } diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 38e3cad916c..4f567c19f2b 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -37,37 +37,6 @@ def started_cluster(): cluster.shutdown() -def test_postgres_conversions(started_cluster): - conn = get_postgres_conn(True) - cursor = conn.cursor() - table_name = 'test_types' - cursor.execute( - '''CREATE TABLE IF NOT EXISTS {} ( - a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, - h timestamp, i date, j numeric(5, 5), k decimal(5, 5))'''.format(table_name)) - node1.query(''' - INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword') VALUES - (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.2, 0.2)'''.format(table_name)) - result = node1.query(''' - SELECT * FROM postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name)) - assert(result == '-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.20000\t0.20000\n') - - table_name = 'test_array_dimensions' - cursor.execute( - '''CREATE TABLE IF NOT EXISTS {} (a date[] NOT NULL, b integer[][][], c decimal(5, 2)[][][][][][])'''.format(table_name)) - result = node1.query(''' - DESCRIBE TABLE postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name)) - expected ='a\tArray(Date)\t\t\t\t\t\nb\tArray(Array(Array(Nullable(Int32))))\t\t\t\t\t\nc\tArray(Array(Array(Array(Array(Array(Nullable(Decimal(5, 2))))))))' - assert(result.rstrip() == expected) - - node1.query(''' - INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword') - VALUES (['2000-05-12', '2000-05-12'], [[[1, 1], [NULL, NULL]], [[3, 3], [3, 3]], [[4, 4], [5, 5]]], [[[[[[0.1], [0.2], [0.3]]]]]])'''.format(table_name)) - result = node1.query(''' - SELECT * FROM postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name)) - assert(result == '''['2000-05-12','2000-05-12']\t[[[1,1],[NULL,NULL]],[[3,3],[3,3]],[[4,4],[5,5]]]\t[[[[[[0.10],[0.20],[0.30]]]]]]\n''') - - def test_postgres_select_insert(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() @@ -88,6 +57,81 @@ def test_postgres_select_insert(started_cluster): assert (node1.query(check4)).rstrip() == '1' +def test_postgres_conversions(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + cursor.execute( + '''CREATE TABLE IF NOT EXISTS test_types ( + a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, + h timestamp, i date, j numeric(5, 5), k decimal(5, 5))''') + node1.query(''' + INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword') VALUES + (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.2, 0.2)''') + result = node1.query(''' + SELECT * FROM postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword')''') + assert(result == '-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.20000\t0.20000\n') + + cursor.execute( + '''CREATE TABLE IF NOT EXISTS test_array_dimensions + ( + a Date[] NOT NULL, -- Date + b Timestamp[] NOT NULL, -- DateTime + c real[][] NOT NULL, -- Float32 + d double precision[][] NOT NULL, -- Float64 + e decimal(5, 5)[][][] NOT NULL, -- Decimal32 + f integer[][][] NOT NULL, -- Int32 + g Text[][][][][] NOT NULL, -- String + h Integer[][][], -- Nullable(Int32) + i Char(2)[][][][], -- Nullable(String) + k Char(2)[] -- Nullable(String) + )''') + + result = node1.query(''' + DESCRIBE TABLE postgresql('postgres1:5432', 'clickhouse', 'test_array_dimensions', 'postgres', 'mysecretpassword')''') + expected = ('a\tArray(Date)\t\t\t\t\t\n' + + 'b\tArray(DateTime)\t\t\t\t\t\n' + + 'c\tArray(Array(Float32))\t\t\t\t\t\n' + + 'd\tArray(Array(Float64))\t\t\t\t\t\n' + + 'e\tArray(Array(Array(Decimal(5, 5))))\t\t\t\t\t\n' + + 'f\tArray(Array(Array(Int32)))\t\t\t\t\t\n' + + 'g\tArray(Array(Array(Array(Array(String)))))\t\t\t\t\t\n' + + 'h\tArray(Array(Array(Nullable(Int32))))\t\t\t\t\t\n' + + 'i\tArray(Array(Array(Array(Nullable(String)))))\t\t\t\t\t\n' + + 'k\tArray(Nullable(String))' + ) + assert(result.rstrip() == expected) + + node1.query("INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', 'test_array_dimensions', 'postgres', 'mysecretpassword') " + "VALUES (" + "['2000-05-12', '2000-05-12'], " + "['2000-05-12 12:12:12', '2000-05-12 12:12:12'], " + "[[1.12345], [1.12345], [1.12345]], " + "[[1.1234567891], [1.1234567891], [1.1234567891]], " + "[[[0.11111, 0.11111]], [[0.22222, 0.22222]], [[0.33333, 0.33333]]], " + "[[[1, 1], [1, 1]], [[3, 3], [3, 3]], [[4, 4], [5, 5]]], " + "[[[[['winx', 'winx', 'winx']]]]], " + "[[[1, NULL], [NULL, 1]], [[NULL, NULL], [NULL, NULL]], [[4, 4], [5, 5]]], " + "[[[[NULL]]]], " + "[]" + ")") + + result = node1.query(''' + SELECT * FROM postgresql('postgres1:5432', 'clickhouse', 'test_array_dimensions', 'postgres', 'mysecretpassword')''') + expected = ( + "['2000-05-12','2000-05-12']\t" + + "['2000-05-12 12:12:12','2000-05-12 12:12:12']\t" + + "[[1.12345],[1.12345],[1.12345]]\t" + + "[[1.1234567891],[1.1234567891],[1.1234567891]]\t" + + "[[[0.11111,0.11111]],[[0.22222,0.22222]],[[0.33333,0.33333]]]\t" + "[[[1,1],[1,1]],[[3,3],[3,3]],[[4,4],[5,5]]]\t" + "[[[[['winx','winx','winx']]]]]\t" + "[[[1,NULL],[NULL,1]],[[NULL,NULL],[NULL,NULL]],[[4,4],[5,5]]]\t" + "[[[[NULL]]]]\t" + "[]\n" + ) + assert(result == expected) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From d7456f8ddd8b92cfe3f1a28bc49450c5df9b2caa Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Sun, 10 Jan 2021 16:18:26 +0800 Subject: [PATCH 094/264] update decodeXMLComponent.cpp --- src/Functions/decodeXMLComponent.cpp | 88 +++++++++++----------------- 1 file changed, 35 insertions(+), 53 deletions(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index ff25db76290..603aed08f53 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -2,9 +2,9 @@ #include #include #include +#include #include -#include namespace DB { namespace ErrorCodes @@ -83,28 +83,29 @@ namespace } else if (isValidNumeric(src_curr_pos, src_next_pos)) { - std::vector decode_numeric_chars; - decodeNumericPart(src_curr_pos + 2, src_next_pos, decode_numeric_chars); - if (decode_numeric_chars.empty()) + int numeric_entity; + size_t bytes_to_copy = src_curr_pos - src_prev_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + if (*(src_curr_pos + 2) == 'x' || *(src_curr_pos + 2) == 'X') { - ++src_curr_pos; - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - src_prev_pos = src_curr_pos; + numeric_entity = hexOrDecStrToInt(src_curr_pos + 3, src_next_pos, 0x10); } else { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - for (auto cur_char : decode_numeric_chars) - { - *dst_pos = cur_char; - ++dst_pos; - } - src_prev_pos = src_next_pos + 1; + numeric_entity = hexOrDecStrToInt(src_curr_pos + 2, src_next_pos, 10); } + if (numeric_entity > max_legal_unicode_value) + { + bytes_to_copy = src_next_pos - src_curr_pos + 1; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_curr_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + } + else + { + decodeNumericPart(numeric_entity, dst_pos); + } + src_prev_pos = src_next_pos + 1; src_curr_pos = src_next_pos + 1; } else if (src_next_pos - src_curr_pos == 3) @@ -212,39 +213,31 @@ namespace return dst_pos - dst; } - static void decodeNumericPart(const char * src, const char * end, std::vector & decodeNumericChars) + static void decodeNumericPart(int numeric_entity, char *& dst_pos) { - int numeric_ans; - if (*src == 'x' || *src == 'X') - { - numeric_ans = hexOrDecStrToInt(src + 1, end, 16); - } - else - { - numeric_ans = hexOrDecStrToInt(src, end, 10); - } - const auto num_bits = numBitsCount(numeric_ans); + const auto num_bits = numBitsCount(numeric_entity); if (num_bits <= 7) { - decodeNumericChars.push_back('\0' + (numeric_ans & 0x7F)); + *(dst_pos++) = '\0' + (numeric_entity & 0x7F); } else if (num_bits <= 11) { - decodeNumericChars.push_back('\0' + ((numeric_ans >> 6) & 0x1F) + 0xC0); - decodeNumericChars.push_back('\0' + (numeric_ans & 0x3F) + 0x80); + *(dst_pos++) = '\0' + ((numeric_entity >> 6) & 0x1F) + 0xC0; + + *(dst_pos++) = '\0' + (numeric_entity & 0x3F) + 0x80; } else if (num_bits <= 16) { - decodeNumericChars.push_back('\0' + ((numeric_ans >> 12) & 0x0F) + 0xE0); - decodeNumericChars.push_back('\0' + ((numeric_ans >> 6) & 0x3F) + 0x80); - decodeNumericChars.push_back('\0' + (numeric_ans & 0x3F) + 0x80); + *(dst_pos++) = '\0' + ((numeric_entity >> 12) & 0x0F) + 0xE0; + *(dst_pos++) = '\0' + ((numeric_entity >> 6) & 0x3F) + 0x80; + *(dst_pos++) = '\0' + (numeric_entity & 0x3F) + 0x80; } - else if ((num_bits <= 21) && (numeric_ans <= max_legal_unicode_value)) + else { - decodeNumericChars.push_back('\0' + ((numeric_ans >> 18) & 0x07) + 0xF0); - decodeNumericChars.push_back('\0' + ((numeric_ans >> 12) & 0x3F) + 0x80); - decodeNumericChars.push_back('\0' + ((numeric_ans >> 6) & 0x3F) + 0x80); - decodeNumericChars.push_back('\0' + (numeric_ans & 0x3F) + 0x80); + *(dst_pos++) = '\0' + ((numeric_entity >> 18) & 0x07) + 0xF0; + *(dst_pos++) = '\0' + ((numeric_entity >> 12) & 0x3F) + 0x80; + *(dst_pos++) = '\0' + ((numeric_entity >> 6) & 0x3F) + 0x80; + *(dst_pos++) = '\0' + (numeric_entity & 0x3F) + 0x80; } } @@ -252,22 +245,11 @@ namespace { int numeric_ans = 0; int pos = 0; - if (base == 16) + if (base == 0x10) { while (src + pos != end) { - if (isNumericASCII(*(src + pos))) - { - numeric_ans = numeric_ans * base + (*(src + pos) - '0'); - } - else if (*(src + pos) >= 'a' && *(src + pos) <= 'f') - { - numeric_ans = numeric_ans * base + (*(src + pos) - 'a' + 10); - } - else if (*(src + pos) >= 'A' && *(src + pos) <= 'F') - { - numeric_ans = numeric_ans * base + (*(src + pos) - 'A' + 10); - } + numeric_ans = numeric_ans * 0x10 + static_cast(unhex(*(src + pos))); ++pos; } } From a0982bfd554bad1dda0d082f676f37074bf09f65 Mon Sep 17 00:00:00 2001 From: nautaa <870284156@qq.com> Date: Sun, 10 Jan 2021 16:21:21 +0800 Subject: [PATCH 095/264] update decodeXMLComponent.cpp --- src/Functions/decodeXMLComponent.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index 603aed08f53..ffc10d3a7bb 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -223,7 +223,6 @@ namespace else if (num_bits <= 11) { *(dst_pos++) = '\0' + ((numeric_entity >> 6) & 0x1F) + 0xC0; - *(dst_pos++) = '\0' + (numeric_entity & 0x3F) + 0x80; } else if (num_bits <= 16) From fc9de76f7d8e1b93a2deff1dd8c76e11260eb39e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Jan 2021 09:00:35 +0000 Subject: [PATCH 096/264] Fix incorrect getCreateTableQueryImpl with multidim arrays --- .../PostgreSQL/DatabasePostgreSQL.cpp | 26 +++++++++------ src/Databases/PostgreSQL/DatabasePostgreSQL.h | 1 + .../test_postgresql_database_engine/test.py | 33 +++++++++++++++++++ 3 files changed, 50 insertions(+), 10 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 1e3214ea0b6..a972bb9c7a8 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -3,6 +3,7 @@ #if USE_LIBPQXX #include +#include #include #include #include @@ -353,16 +354,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, co { const auto & column_declaration = std::make_shared(); column_declaration->name = column_type_and_name.name; - - std::function convert_datatype_to_query = [&](const DataTypePtr & data_type) -> ASTPtr - { - WhichDataType which(data_type); - if (!which.isNullable()) - return std::make_shared(data_type->getName()); - return makeASTFunction("Nullable", convert_datatype_to_query(typeid_cast(data_type.get())->getNestedType())); - }; - - column_declaration->type = convert_datatype_to_query(column_type_and_name.type); + column_declaration->type = getColumnDeclaration(column_type_and_name.type); columns_expression_list->children.emplace_back(column_declaration); } @@ -380,6 +372,20 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, co return create_table_query; } + +ASTPtr DatabasePostgreSQL::getColumnDeclaration(const DataTypePtr & data_type) const +{ + WhichDataType which(data_type); + + if (which.isNullable()) + return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + + if (which.isArray()) + return makeASTFunction("Array", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); + + return std::make_shared(data_type->getName()); +} + } #endif diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 78d25c6c000..6b55a802b41 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -76,6 +76,7 @@ private: std::unordered_set fetchTablesList() const; StoragePtr fetchTable(const String & table_name, const Context & context, const bool table_checked) const; void removeOutdatedTables(); + ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; }; } diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index c76a3bdaa41..b3f7c0fa9af 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -113,6 +113,39 @@ def test_postgresql_database_engine_queries(started_cluster): assert 'test_database' not in node1.query('SHOW DATABASES') +def test_get_create_table_query_with_multidim_arrays(started_cluster): + conn = get_postgres_conn(True) + cursor = conn.cursor() + + node1.query( + "CREATE DATABASE test_database ENGINE = PostgreSQL('postgres1:5432', 'test_database', 'postgres', 'mysecretpassword')") + + cursor.execute(""" + CREATE TABLE IF NOT EXISTS array_columns ( + b Integer[][][] NOT NULL, + c Integer[][][] + )""") + + node1.query("DETACH TABLE test_database.array_columns") + node1.query("ATTACH TABLE test_database.array_columns") + + node1.query("INSERT INTO test_database.array_columns " + "VALUES (" + "[[[1, 1], [1, 1]], [[3, 3], [3, 3]], [[4, 4], [5, 5]]], " + "[[[1, NULL], [NULL, 1]], [[NULL, NULL], [NULL, NULL]], [[4, 4], [5, 5]]] " + ")") + result = node1.query(''' + SELECT * FROM test_database.array_columns''') + expected = ( + "[[[1,1],[1,1]],[[3,3],[3,3]],[[4,4],[5,5]]]\t" + "[[[1,NULL],[NULL,1]],[[NULL,NULL],[NULL,NULL]],[[4,4],[5,5]]]\n" + ) + assert(result == expected) + + node1.query("DROP DATABASE test_database") + assert 'test_database' not in node1.query('SHOW DATABASES') + + def test_postgresql_database_engine_table_cache(started_cluster): conn = get_postgres_conn(True) cursor = conn.cursor() From d952b0897ea4fd361ca2ff62342213ae8de82013 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Jan 2021 10:03:01 +0000 Subject: [PATCH 097/264] Minor adjustments --- .../PostgreSQLBlockInputStream.cpp | 22 ++++--- src/DataStreams/PostgreSQLBlockInputStream.h | 1 + .../PostgreSQL/DatabasePostgreSQL.cpp | 24 +++++--- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 6 +- .../PostgreSQL/FetchFromPostgreSQL.cpp | 60 ++++++++++--------- .../PostgreSQL/FetchFromPostgreSQL.h | 4 +- src/Storages/StoragePostgreSQL.cpp | 27 ++++++--- .../TableFunctionPostgreSQL.cpp | 2 +- 8 files changed, 91 insertions(+), 55 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index df60f88fecd..774e3afcdb3 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -24,7 +24,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_TYPE; } @@ -67,13 +66,9 @@ Block PostgreSQLBlockInputStream::readImpl() { const std::vector * row{stream->read_row()}; + /// row is nullptr if pqxx::stream_from is finished if (!row) - { - /// row is nullptr if pqxx::stream_from is finished - stream->complete(); - tx->commit(); break; - } for (const auto idx : ext::range(0, row->size())) { @@ -109,6 +104,16 @@ Block PostgreSQLBlockInputStream::readImpl() } +void PostgreSQLBlockInputStream::readSuffix() +{ + if (stream) + { + stream->complete(); + tx->commit(); + } +} + + void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view value, const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx) { @@ -160,7 +165,8 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view case ValueType::vtDateTime64:[[fallthrough]]; case ValueType::vtDecimal32: [[fallthrough]]; case ValueType::vtDecimal64: [[fallthrough]]; - case ValueType::vtDecimal128: + case ValueType::vtDecimal128: [[fallthrough]]; + case ValueType::vtDecimal256: { ReadBufferFromString istr(value); data_type->deserializeAsWholeText(column, istr, FormatSettings{}); @@ -207,8 +213,6 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view assert_cast(column).insert(Array(dimensions[1].begin(), dimensions[1].end())); break; } - default: - throw Exception("Value of unsupported type:" + column.getName(), ErrorCodes::UNKNOWN_TYPE); } } diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 3e52303a75d..431af203caf 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -32,6 +32,7 @@ private: void readPrefix() override; Block readImpl() override; + void readSuffix() override; void insertValue(IColumn & column, std::string_view value, const ExternalResultDescription::ValueType type, const DataTypePtr data_type, size_t idx); diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index a972bb9c7a8..52c46ed0bce 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -86,7 +86,6 @@ std::unordered_set DatabasePostgreSQL::fetchTablesList() const std::unordered_set tables; std::string query = "SELECT tablename FROM pg_catalog.pg_tables " "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema'"; - /// Already connected to the needed database, search will be done there pqxx::read_transaction tx(*connection->conn()); for (auto table_name : tx.stream(query)) @@ -99,11 +98,22 @@ std::unordered_set DatabasePostgreSQL::fetchTablesList() const bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const { pqxx::nontransaction tx(*connection->conn()); - pqxx::result result = tx.exec(fmt::format( - "SELECT tablename FROM pg_catalog.pg_tables " - "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema' AND tablename = '{}'", table_name)); - return !result.empty(); + try + { + pqxx::result result = tx.exec(fmt::format("select '{}'::regclass", table_name)); + } + catch (pqxx::undefined_table const &) + { + return false; + } + catch (Exception & e) + { + e.addMessage("while checking postgresql table existance"); + throw; + } + + return true; } @@ -137,7 +147,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte return StoragePtr{}; auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; - auto columns = fetchTableStructure(connection->conn(), table_name, use_nulls); + auto columns = fetchPostgreSQLTableStructure(connection->conn(), table_name, use_nulls); if (!columns) return StoragePtr{}; @@ -146,8 +156,6 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, const Conte StorageID(database_name, table_name), table_name, std::make_shared(connection->conn_str()), ColumnsDescription{*columns}, ConstraintsDescription{}, context); - /// There is no easy (embedded) way in postgres to check table modification time, so if `cache_tables` == 1 (default: 0) - /// table structure is cached and not checked for being modififed, but it will be updated during detach->attach. if (cache_tables) cached_tables[table_name] = storage; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 6b55a802b41..68c78b74ba4 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -10,13 +10,17 @@ #include #include #include -#include namespace DB { class Context; +/** Real-time access to table list and table structure from remote PostgreSQL. + * All tables are created after pull-out structure from remote PostgreSQL. + * If `cache_tables` == 1 (default: 0) table structure is cached and not checked for being modififed, + * but it will be updated during detach->attach. + */ class DatabasePostgreSQL final : public IDatabase { diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index bc0abce30df..d74b9baf97b 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -2,6 +2,7 @@ #if USE_LIBPQXX +#include #include #include #include @@ -19,10 +20,11 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_TYPE; + extern const int UNKNOWN_TABLE; } -std::shared_ptr fetchTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls) +std::shared_ptr fetchPostgreSQLTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls) { auto columns = NamesAndTypesList(); @@ -32,18 +34,30 @@ std::shared_ptr fetchTableStructure(ConnectionPtr connection, "FROM pg_attribute " "WHERE attrelid = '{}'::regclass " "AND NOT attisdropped AND attnum > 0", postgres_table_name); - pqxx::read_transaction tx(*connection); - pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); - std::tuple row; - - while (stream >> row) + try { - columns.push_back(NameAndTypePair( - std::get<0>(row), - getDataType(std::get<1>(row), use_nulls && (std::get<2>(row) == "f"), std::get<3>(row)))); + pqxx::read_transaction tx(*connection); + pqxx::stream_from stream(tx, pqxx::from_query, std::string_view(query)); + + std::tuple row; + while (stream >> row) + { + columns.push_back(NameAndTypePair( + std::get<0>(row), + convertPostgreSQLDataType( + std::get<1>(row), + use_nulls && (std::get<2>(row) == "f"), /// 'f' means that postgres `not_null` is false, i.e. value is nullable + std::get<3>(row)))); + } + stream.complete(); + tx.commit(); + } + catch (pqxx::undefined_table const &) + { + throw Exception(fmt::format( + "PostgreSQL table {}.{} does not exist", + connection->dbname(), postgres_table_name), ErrorCodes::UNKNOWN_TABLE); } - stream.complete(); - tx.commit(); if (columns.empty()) return nullptr; @@ -52,12 +66,12 @@ std::shared_ptr fetchTableStructure(ConnectionPtr connection, } -DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimensions) +DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions) { DataTypePtr res; /// Get rid of trailing '[]' for arrays - if (dimensions) + if (dimensions && type.ends_with("[]")) type.resize(type.size() - 2); if (type == "smallint") @@ -80,20 +94,10 @@ DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimension res = std::make_shared(); else if (type.starts_with("numeric")) { - /// Numeric and decimal will both end up here as numeric - /// Will get numeric(precision, scale) string, need to extract precision and scale - std::vector result; - boost::split(result, type, [](char c){ return c == '(' || c == ',' || c == ')'; }); - for (std::string & key : result) - boost::trim(key); - - /// If precision or scale are not specified, postgres creates a column in which numeric values of - /// any precision and scale can be stored, so may be maxPrecision may be used instead of exception - if (result.size() < 3) - throw Exception("Numeric lacks precision and scale in its definition", ErrorCodes::UNKNOWN_TYPE); - - uint32_t precision = pqxx::from_string(result[1]); - uint32_t scale = pqxx::from_string(result[2]); + /// Numeric and decimal will both end up here as numeric. + res = DataTypeFactory::instance().get(type); + uint32_t precision = getDecimalPrecision(*res); + uint32_t scale = getDecimalScale(*res); if (precision <= DecimalUtils::maxPrecision()) res = std::make_shared>(precision, scale); @@ -101,6 +105,8 @@ DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimension res = std::make_shared>(precision, scale); else if (precision <= DecimalUtils::maxPrecision()) res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); } if (!res) diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h index 1010fd2404b..bb6b8d46c84 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h @@ -10,8 +10,8 @@ namespace DB { -std::shared_ptr fetchTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls); -DataTypePtr getDataType(std::string & type, bool is_nullable, uint16_t dimensions); +std::shared_ptr fetchPostgreSQLTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls); +DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions); } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index af0731ffd12..52ded0bafec 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -102,11 +102,13 @@ public: Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } + void writePrefix() override { work = std::make_unique(*connection); } + void write(const Block & block) override { const auto columns = block.getColumns(); @@ -143,24 +145,28 @@ public: row[j] = ostr.str(); } } - /// pqxx::stream_to is much faster than simple insert, especially for large number of rows + stream_inserter->write_values(row); } } + void writeSuffix() override { if (stream_inserter) + { stream_inserter->complete(); - work->commit(); + work->commit(); + } } + /// Cannot just use serializeAsText for array data type even though it converts perfectly /// any dimension number array into text format, because it incloses in '[]' and for postgres it must be '{}'. - void parseArray(Field array_field, DataTypePtr data_type, WriteBuffer & ostr) + void parseArray(const Field & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) { const auto * array_type = typeid_cast(data_type.get()); - const auto nested = array_type->getNestedType(); + const auto & nested = array_type->getNestedType(); const auto & array = array_field.get(); if (!isArray(nested)) @@ -190,13 +196,14 @@ public: writeChar('}', ostr); } + /// Conversion is done via column casting because with writeText(Array..) got incorrect conversion /// of Date and DateTime data types and it added extra quotes for values inside array. - std::string clickhouseToPostgresArray(const Array & array_field, DataTypePtr data_type) + std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type) { auto nested = typeid_cast(data_type.get())->getNestedType(); ColumnPtr nested_column(createNested(nested)); - const auto array_column{ColumnArray::create(nested_column)}; + auto array_column{ColumnArray::create(nested_column)}; const_cast(array_column.get())->insert(array_field); WriteBufferFromOwnString ostr; data_type->serializeAsText(*array_column, 0, ostr, FormatSettings{}); @@ -205,7 +212,8 @@ public: return '{' + std::string(ostr.str().begin() + 1, ostr.str().end() - 1) + '}'; } - ColumnPtr createNested(DataTypePtr nested) + + static ColumnPtr createNested(DataTypePtr nested) { bool is_nullable = false; if (nested->isNullable()) @@ -242,6 +250,11 @@ public: const auto & type = typeid_cast *>(nested.get()); nested_column = ColumnDecimal::create(0, type->getScale()); } + else if (which.isDecimal256()) + { + const auto & type = typeid_cast *>(nested.get()); + nested_column = ColumnDecimal::create(0, type->getScale()); + } else throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversion not supported"); diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 92a965cb277..4f8ffecc89a 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -37,7 +37,7 @@ StoragePtr TableFunctionPostgreSQL::executeImpl(const ASTPtr & /*ast_function*/, ColumnsDescription TableFunctionPostgreSQL::getActualTableStructure(const Context & context) const { const bool use_nulls = context.getSettingsRef().external_table_functions_use_nulls; - auto columns = fetchTableStructure(connection->conn(), remote_table_name, use_nulls); + auto columns = fetchPostgreSQLTableStructure(connection->conn(), remote_table_name, use_nulls); return ColumnsDescription{*columns}; } From c6c6b2d23c0113c2e121f544452b099c5db78e8e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Jan 2021 13:01:15 +0000 Subject: [PATCH 098/264] Fix style and types check --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 2 +- src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp | 1 - src/Storages/StoragePostgreSQL.cpp | 6 ++++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 52c46ed0bce..1ce82ec36b2 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -109,7 +109,7 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const } catch (Exception & e) { - e.addMessage("while checking postgresql table existance"); + e.addMessage("while checking postgresql table existence"); throw; } diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index d74b9baf97b..c3a8c6ba2ef 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -19,7 +19,6 @@ namespace DB namespace ErrorCodes { - extern const int UNKNOWN_TYPE; extern const int UNKNOWN_TABLE; } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 52ded0bafec..c2aec72c66e 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -111,6 +111,9 @@ public: void write(const Block & block) override { + if (!work) + return; + const auto columns = block.getColumns(); const size_t num_rows = block.rows(), num_cols = block.columns(); const auto data_types = block.getDataTypes(); @@ -154,10 +157,9 @@ public: void writeSuffix() override { if (stream_inserter) - { stream_inserter->complete(); + if (work) work->commit(); - } } From f09a53103fd433dac915690275023fedafe04fa2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 00:23:05 +0300 Subject: [PATCH 099/264] Add a test for corruption during distributed send --- .../__init__.py | 0 .../configs/remote_servers.xml | 18 ++ .../configs/users.d/batch.xml | 7 + .../configs/users.d/no_batch.xml | 7 + .../test.py | 165 ++++++++++++++++++ 5 files changed, 197 insertions(+) create mode 100644 tests/integration/test_insert_distributed_async_send/__init__.py create mode 100644 tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml create mode 100644 tests/integration/test_insert_distributed_async_send/configs/users.d/batch.xml create mode 100644 tests/integration/test_insert_distributed_async_send/configs/users.d/no_batch.xml create mode 100644 tests/integration/test_insert_distributed_async_send/test.py diff --git a/tests/integration/test_insert_distributed_async_send/__init__.py b/tests/integration/test_insert_distributed_async_send/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml b/tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml new file mode 100644 index 00000000000..2211d599b25 --- /dev/null +++ b/tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml @@ -0,0 +1,18 @@ + + + + + false + + n1 + 9000 + + + n2 + 9000 + + + + + + diff --git a/tests/integration/test_insert_distributed_async_send/configs/users.d/batch.xml b/tests/integration/test_insert_distributed_async_send/configs/users.d/batch.xml new file mode 100644 index 00000000000..5058ffac147 --- /dev/null +++ b/tests/integration/test_insert_distributed_async_send/configs/users.d/batch.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_insert_distributed_async_send/configs/users.d/no_batch.xml b/tests/integration/test_insert_distributed_async_send/configs/users.d/no_batch.xml new file mode 100644 index 00000000000..806df109670 --- /dev/null +++ b/tests/integration/test_insert_distributed_async_send/configs/users.d/no_batch.xml @@ -0,0 +1,7 @@ + + + + 0 + + + diff --git a/tests/integration/test_insert_distributed_async_send/test.py b/tests/integration/test_insert_distributed_async_send/test.py new file mode 100644 index 00000000000..55a9c90fefd --- /dev/null +++ b/tests/integration/test_insert_distributed_async_send/test.py @@ -0,0 +1,165 @@ +# pylint: disable=unused-argument +# pylint: disable=redefined-outer-name +# pylint: disable=line-too-long + +# NOTES: +# - timeout should not be reduced due to bit flip of the corrupted buffer + +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) + +# n1 -- distributed_directory_monitor_batch_inserts=1 +n1 = cluster.add_instance('n1', main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.d/batch.xml']) +# n2 -- distributed_directory_monitor_batch_inserts=0 +n2 = cluster.add_instance('n2', main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.d/no_batch.xml']) + +batch_params = pytest.mark.parametrize('batch', [ + (1), + (0), +]) + +@pytest.fixture(scope='module', autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def create_tables(): + for _, instance in list(cluster.instances.items()): + instance.query('CREATE TABLE data (key Int, value String) Engine=Memory()') + instance.query(""" + CREATE TABLE dist AS data + Engine=Distributed( + insert_distributed_async_send_cluster, + currentDatabase(), + data, + rand() + ) + """) + # only via SYSTEM FLUSH DISTRIBUTED + instance.query('SYSTEM STOP DISTRIBUTED SENDS dist') + +def drop_tables(): + for _, instance in list(cluster.instances.items()): + instance.query('DROP TABLE IF EXISTS data') + instance.query('DROP TABLE IF EXISTS dist') + +# return amount of bytes of the 2.bin for n2 shard +def insert_data(node): + node.query('INSERT INTO dist SELECT number, randomPrintableASCII(100) FROM numbers(10000)', settings={ + # do not do direct INSERT, always via SYSTEM FLUSH DISTRIBUTED + 'prefer_localhost_replica': 0, + }) + path = get_path_to_dist_batch() + size = int(node.exec_in_container(['bash', '-c', f'wc -c < {path}'])) + assert size > 1<<16 + return size + +def get_node(batch): + if batch: + return n1 + return n2 + +def bootstrap(batch): + drop_tables() + create_tables() + return insert_data(get_node(batch)) + +def get_path_to_dist_batch(file='2.bin'): + # There are: + # - /var/lib/clickhouse/data/default/dist/shard1_replica1/1.bin + # - /var/lib/clickhouse/data/default/dist/shard1_replica2/2.bin + # + # @return the file for the n2 shard + return f'/var/lib/clickhouse/data/default/dist/shard1_replica2/{file}' + +def check_dist_after_corruption(truncate, batch): + node = get_node(batch) + + if batch: + # In batch mode errors are ignored + node.query('SYSTEM FLUSH DISTRIBUTED dist') + else: + if truncate: + with pytest.raises(QueryRuntimeException, match="Cannot read all data. Bytes read:"): + node.query('SYSTEM FLUSH DISTRIBUTED dist') + else: + with pytest.raises(QueryRuntimeException, match="Checksum doesn't match: corrupted data. Reference:"): + node.query('SYSTEM FLUSH DISTRIBUTED dist') + + # send pending files + # (since we have two nodes and corrupt file for only one of them) + node.query('SYSTEM FLUSH DISTRIBUTED dist') + + # but there is broken file + broken = get_path_to_dist_batch('broken') + node.exec_in_container(['bash', '-c', f'ls {broken}/2.bin']) + + assert int(n1.query('SELECT count() FROM data')) == 10000 + assert int(n2.query('SELECT count() FROM data')) == 0 + + +@batch_params +def test_insert_distributed_async_send_success(batch): + bootstrap(batch) + node = get_node(batch) + node.query('SYSTEM FLUSH DISTRIBUTED dist') + assert int(n1.query('SELECT count() FROM data')) == 10000 + assert int(n2.query('SELECT count() FROM data')) == 10000 + +@batch_params +def test_insert_distributed_async_send_truncated_1(batch): + size = bootstrap(batch) + path = get_path_to_dist_batch() + node = get_node(batch) + + new_size = size - 10 + # we cannot use truncate, due to hardlinks + node.exec_in_container(['bash', '-c', f'mv {path} /tmp/bin && head -c {new_size} /tmp/bin > {path}']) + + check_dist_after_corruption(True, batch) + +@batch_params +def test_insert_distributed_async_send_truncated_2(batch): + bootstrap(batch) + path = get_path_to_dist_batch() + node = get_node(batch) + + # we cannot use truncate, due to hardlinks + node.exec_in_container(['bash', '-c', f'mv {path} /tmp/bin && head -c 10000 /tmp/bin > {path}']) + + check_dist_after_corruption(True, batch) + +# The difference from the test_insert_distributed_async_send_corrupted_small +# is that small corruption will be seen only on local node +@batch_params +def test_insert_distributed_async_send_corrupted_big(batch): + size = bootstrap(batch) + path = get_path_to_dist_batch() + + node = get_node(batch) + + from_original_size = size - 8192 + zeros_size = 8192 + node.exec_in_container(['bash', '-c', f'mv {path} /tmp/bin && head -c {from_original_size} /tmp/bin > {path} && head -c {zeros_size} /dev/zero >> {path}']) + + check_dist_after_corruption(False, batch) + +@batch_params +def test_insert_distributed_async_send_corrupted_small(batch): + size = bootstrap(batch) + path = get_path_to_dist_batch() + node = get_node(batch) + + from_original_size = size - 60 + zeros_size = 60 + node.exec_in_container(['bash', '-c', f'mv {path} /tmp/bin && head -c {from_original_size} /tmp/bin > {path} && head -c {zeros_size} /dev/zero >> {path}']) + + check_dist_after_corruption(False, batch) From 676bc83c6de86c45ac8979f98ffc0255e6281c8f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 8 Jan 2021 03:23:41 +0300 Subject: [PATCH 100/264] Check per-block checksum of the distributed batch on the sender before sending This is already done for distributed_directory_monitor_batch_inserts=1, so let's do the same for the non batched mode, since otherwise in case the file will be truncated the receiver will just stuck (since it will wait for the block, but the sender will not send it). --- src/Storages/Distributed/DirectoryMonitor.cpp | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index ade75506a38..903052b9d45 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -325,6 +325,35 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa LOG_TRACE(log, "Started processing `{}`", file_path); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context.getSettingsRef()); + /// Check that the file is valid + /// FIXME: suboptimal + try + { + Block sample_block; + Settings insert_settings; + String insert_query; + ClientInfo client_info; + + /// Determine metadata of the current file and check if it is not broken. + ReadBufferFromFile in{file_path}; + readHeader(in, insert_settings, insert_query, client_info, log); + + CompressedReadBuffer decompressing_in(in); + NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); + block_in.readPrefix(); + + while (Block block = block_in.read()) + { + /// Just do the per-block checksum checks in the CompressedReadBuffer. + } + block_in.readSuffix(); + } + catch (const Exception & e) + { + maybeMarkAsBroken(file_path, e); + throw; + } + try { CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; From fce8b6b5ef80499b4bc8448855a204d7047beb5e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Jan 2021 14:39:46 +0300 Subject: [PATCH 101/264] Refactoring distributed header parsing --- src/Storages/Distributed/DirectoryMonitor.cpp | 154 +++++++++--------- src/Storages/Distributed/DirectoryMonitor.h | 3 - 2 files changed, 76 insertions(+), 81 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 903052b9d45..f94e6700a74 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -78,6 +78,70 @@ namespace } } + struct DistributedHeader + { + Settings insert_settings; + std::string insert_query; + ClientInfo client_info; + }; + + static DistributedHeader readDistributedHeader(ReadBuffer & in, Poco::Logger * log) + { + DistributedHeader header; + + UInt64 query_size; + readVarUInt(query_size, in); + + if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER) + { + /// Read the header as a string. + String header_data; + readStringBinary(header_data, in); + + /// Check the checksum of the header. + CityHash_v1_0_2::uint128 checksum; + readPODBinary(checksum, in); + assertChecksum(checksum, CityHash_v1_0_2::CityHash128(header_data.data(), header_data.size())); + + /// Read the parts of the header. + ReadBufferFromString header_buf(header_data); + + UInt64 initiator_revision; + readVarUInt(initiator_revision, header_buf); + if (DBMS_TCP_PROTOCOL_VERSION < initiator_revision) + { + LOG_WARNING(log, "ClickHouse shard version is older than ClickHouse initiator version. It may lack support for new features."); + } + + readStringBinary(header.insert_query, header_buf); + header.insert_settings.read(header_buf); + + if (header_buf.hasPendingData()) + header.client_info.read(header_buf, initiator_revision); + + /// Add handling new data here, for example: + /// + /// if (header_buf.hasPendingData()) + /// readVarUInt(my_new_data, header_buf); + /// + /// And note that it is safe, because we have checksum and size for header. + + return header; + } + + if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT) + { + header.insert_settings.read(in, SettingsWriteFormat::BINARY); + readStringBinary(header.insert_query, in); + return header; + } + + header.insert_query.resize(query_size); + in.readStrict(header.insert_query.data(), query_size); + + return header; + } + } @@ -330,13 +394,10 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa try { Block sample_block; - Settings insert_settings; - String insert_query; - ClientInfo client_info; /// Determine metadata of the current file and check if it is not broken. ReadBufferFromFile in{file_path}; - readHeader(in, insert_settings, insert_query, client_info, log); + readDistributedHeader(in, log); CompressedReadBuffer decompressing_in(in); NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); @@ -359,16 +420,11 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; ReadBufferFromFile in{file_path}; + const auto & header = readDistributedHeader(in, log); - Settings insert_settings; - std::string insert_query; - ClientInfo client_info; - - readHeader(in, insert_settings, insert_query, client_info, log); - - auto connection = pool->get(timeouts, &insert_settings); - - RemoteBlockOutputStream remote{*connection, timeouts, insert_query, insert_settings, client_info}; + auto connection = pool->get(timeouts, &header.insert_settings); + RemoteBlockOutputStream remote{*connection, timeouts, + header.insert_query, header.insert_settings, header.client_info}; remote.writePrefix(); remote.writePrepared(in); @@ -390,57 +446,6 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa LOG_TRACE(log, "Finished processing `{}`", file_path); } -void StorageDistributedDirectoryMonitor::readHeader( - ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Poco::Logger * log) -{ - UInt64 query_size; - readVarUInt(query_size, in); - - if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER) - { - /// Read the header as a string. - String header; - readStringBinary(header, in); - - /// Check the checksum of the header. - CityHash_v1_0_2::uint128 checksum; - readPODBinary(checksum, in); - assertChecksum(checksum, CityHash_v1_0_2::CityHash128(header.data(), header.size())); - - /// Read the parts of the header. - ReadBufferFromString header_buf(header); - - UInt64 initiator_revision; - readVarUInt(initiator_revision, header_buf); - if (DBMS_TCP_PROTOCOL_VERSION < initiator_revision) - { - LOG_WARNING(log, "ClickHouse shard version is older than ClickHouse initiator version. It may lack support for new features."); - } - - readStringBinary(insert_query, header_buf); - insert_settings.read(header_buf); - - if (header_buf.hasPendingData()) - client_info.read(header_buf, initiator_revision); - - /// Add handling new data here, for example: - /// if (header_buf.hasPendingData()) - /// readVarUInt(my_new_data, header_buf); - - return; - } - - if (query_size == DBMS_DISTRIBUTED_SIGNATURE_HEADER_OLD_FORMAT) - { - insert_settings.read(in, SettingsWriteFormat::BINARY); - readStringBinary(insert_query, in); - return; - } - - insert_query.resize(query_size); - in.readStrict(insert_query.data(), query_size); -} - struct StorageDistributedDirectoryMonitor::BatchHeader { Settings settings; @@ -551,9 +556,6 @@ struct StorageDistributedDirectoryMonitor::Batch bool batch_broken = false; try { - Settings insert_settings; - String insert_query; - ClientInfo client_info; std::unique_ptr remote; bool first = true; @@ -568,12 +570,13 @@ struct StorageDistributedDirectoryMonitor::Batch } ReadBufferFromFile in(file_path->second); - parent.readHeader(in, insert_settings, insert_query, client_info, parent.log); + const auto & header = readDistributedHeader(in, parent.log); if (first) { first = false; - remote = std::make_unique(*connection, timeouts, insert_query, insert_settings, client_info); + remote = std::make_unique(*connection, timeouts, + header.insert_query, header.insert_settings, header.client_info); remote->writePrefix(); } @@ -652,10 +655,7 @@ public: , block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION) , log{&Poco::Logger::get("DirectoryMonitorBlockInputStream")} { - Settings insert_settings; - String insert_query; - ClientInfo client_info; - StorageDistributedDirectoryMonitor::readHeader(in, insert_settings, insert_query, client_info, log); + readDistributedHeader(in, log); block_in.readPrefix(); first_block = block_in.read(); @@ -744,14 +744,12 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map size_t total_rows = 0; size_t total_bytes = 0; Block sample_block; - Settings insert_settings; - String insert_query; - ClientInfo client_info; + DistributedHeader header; try { /// Determine metadata of the current file and check if it is not broken. ReadBufferFromFile in{file_path}; - readHeader(in, insert_settings, insert_query, client_info, log); + header = readDistributedHeader(in, log); CompressedReadBuffer decompressing_in(in); NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); @@ -778,7 +776,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map throw; } - BatchHeader batch_header(std::move(insert_settings), std::move(insert_query), std::move(client_info), std::move(sample_block)); + BatchHeader batch_header(std::move(header.insert_settings), std::move(header.insert_query), std::move(header.client_info), std::move(sample_block)); Batch & batch = header_to_batch.try_emplace(batch_header, *this, files).first->second; batch.file_indices.push_back(file_idx); diff --git a/src/Storages/Distributed/DirectoryMonitor.h b/src/Storages/Distributed/DirectoryMonitor.h index a6175b44d7b..bc897136786 100644 --- a/src/Storages/Distributed/DirectoryMonitor.h +++ b/src/Storages/Distributed/DirectoryMonitor.h @@ -111,9 +111,6 @@ private: CurrentMetrics::Increment metric_pending_files; - /// Read insert query and insert settings for backward compatible. - static void readHeader(ReadBuffer & in, Settings & insert_settings, std::string & insert_query, ClientInfo & client_info, Poco::Logger * log); - friend class DirectoryMonitorBlockInputStream; }; From 819b9d7d56cdf0495ab54b68197bc597631dfe2e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Jan 2021 15:03:22 +0300 Subject: [PATCH 102/264] Add more metadata into distributed .bin files to avoid doing the same on sending Before this patch StorageDistributedDirectoryMonitor reading .bin files in batch mode, just to calculate number of bytes/rows, this is very ineffective, let's just store them in the header (rows/bytes). --- src/Storages/Distributed/DirectoryMonitor.cpp | 75 +++++++++++++------ .../DistributedBlockOutputStream.cpp | 3 + 2 files changed, 57 insertions(+), 21 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index f94e6700a74..92b08527f6a 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -83,6 +83,10 @@ namespace Settings insert_settings; std::string insert_query; ClientInfo client_info; + + /// .bin file cannot have zero rows/bytes. + size_t rows = 0; + size_t bytes = 0; }; static DistributedHeader readDistributedHeader(ReadBuffer & in, Poco::Logger * log) @@ -119,6 +123,12 @@ namespace if (header_buf.hasPendingData()) header.client_info.read(header_buf, initiator_revision); + if (header_buf.hasPendingData()) + { + readVarUInt(header.rows, header_buf); + readVarUInt(header.bytes, header_buf); + } + /// Add handling new data here, for example: /// /// if (header_buf.hasPendingData()) @@ -142,6 +152,19 @@ namespace return header; } + /// FIXME: suboptimal + void verifyDistributedChecksum(ReadBufferFromFile & in) + { + CompressedReadBuffer decompressing_in(in); + NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); + block_in.readPrefix(); + + while (Block block = block_in.read()) + { + /// Just do the per-block checksum checks in the CompressedReadBuffer. + } + block_in.readSuffix(); + } } @@ -390,24 +413,13 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context.getSettingsRef()); /// Check that the file is valid - /// FIXME: suboptimal try { - Block sample_block; - /// Determine metadata of the current file and check if it is not broken. ReadBufferFromFile in{file_path}; readDistributedHeader(in, log); - CompressedReadBuffer decompressing_in(in); - NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); - block_in.readPrefix(); - - while (Block block = block_in.read()) - { - /// Just do the per-block checksum checks in the CompressedReadBuffer. - } - block_in.readSuffix(); + verifyDistributedChecksum(in); } catch (const Exception & e) { @@ -751,19 +763,40 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map ReadBufferFromFile in{file_path}; header = readDistributedHeader(in, log); - CompressedReadBuffer decompressing_in(in); - NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); - block_in.readPrefix(); - - while (Block block = block_in.read()) + if (header.rows) { - total_rows += block.rows(); - total_bytes += block.bytes(); + total_rows += header.rows; + total_bytes += header.bytes; - if (!sample_block) + CompressedReadBuffer decompressing_in(in); + NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); + block_in.readPrefix(); + + /// We still need to read one block for the header. + while (Block block = block_in.read()) + { sample_block = block.cloneEmpty(); + break; + } + + verifyDistributedChecksum(in); + } + else + { + CompressedReadBuffer decompressing_in(in); + NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); + block_in.readPrefix(); + + while (Block block = block_in.read()) + { + total_rows += block.rows(); + total_bytes += block.bytes(); + + if (!sample_block) + sample_block = block.cloneEmpty(); + } + block_in.readSuffix(); } - block_in.readSuffix(); } catch (const Exception & e) { diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 7299953c88e..977d8d3e81c 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -653,9 +653,12 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: writeStringBinary(query_string, header_buf); context.getSettingsRef().write(header_buf); context.getClientInfo().write(header_buf, DBMS_TCP_PROTOCOL_VERSION); + writeVarUInt(block.rows(), header_buf); + writeVarUInt(block.bytes(), header_buf); /// Add new fields here, for example: /// writeVarUInt(my_new_data, header_buf); + /// And note that it is safe, because we have checksum and size for header. /// Write the header. const StringRef header = header_buf.stringRef(); From 6ec59f1304955c959e5ce3ab989cac1d48a60f02 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 10 Jan 2021 15:33:35 +0000 Subject: [PATCH 103/264] Update libpq, tiny fix --- .gitmodules | 2 +- .../PostgreSQLBlockInputStream.cpp | 19 +++++++++++++------ src/Storages/StoragePostgreSQL.cpp | 7 +++---- 3 files changed, 17 insertions(+), 11 deletions(-) diff --git a/.gitmodules b/.gitmodules index 954cea58a15..c7ff7cea7db 100644 --- a/.gitmodules +++ b/.gitmodules @@ -214,7 +214,7 @@ url = https://github.com/jtv/libpqxx [submodule "contrib/libpq"] path = contrib/libpq - url = https://github.com/kssenii/libpq + url = https://github.com/ClickHouse-Extras/libpq [submodule "contrib/boringssl"] path = contrib/boringssl url = https://github.com/ClickHouse-Extras/boringssl.git diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 774e3afcdb3..4d9a142da3d 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -231,23 +231,23 @@ void PostgreSQLBlockInputStream::prepareArrayInfo(size_t column_idx, const DataT Field default_value = nested->getDefault(); if (nested->isNullable()) - nested = typeid_cast(nested.get())->getNestedType(); + nested = static_cast(nested.get())->getNestedType(); WhichDataType which(nested); std::function parser; if (which.isUInt8() || which.isUInt16()) parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isUInt32()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; - else if (which.isUInt64()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isInt8() || which.isInt16()) parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUInt32()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isInt32()) parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + else if (which.isUInt64()) + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isInt64()) - parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; + parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isFloat32()) parser = [](std::string & field) -> Field { return pqxx::from_string(field); }; else if (which.isFloat64()) @@ -279,6 +279,13 @@ void PostgreSQLBlockInputStream::prepareArrayInfo(size_t column_idx, const DataT DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); return convertFieldToType(field, res); }; + else if (which.isDecimal256()) + parser = [nested](std::string & field) -> Field + { + const auto & type = typeid_cast *>(nested.get()); + DataTypeDecimal res(getDecimalPrecision(*type), getDecimalScale(*type)); + return convertFieldToType(field, res); + }; else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Type conversion to {} is not supported", nested->getName()); diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index c2aec72c66e..ea1279bc319 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -158,8 +158,7 @@ public: { if (stream_inserter) stream_inserter->complete(); - if (work) - work->commit(); + work->commit(); } @@ -201,7 +200,7 @@ public: /// Conversion is done via column casting because with writeText(Array..) got incorrect conversion /// of Date and DateTime data types and it added extra quotes for values inside array. - std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type) + static std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type) { auto nested = typeid_cast(data_type.get())->getNestedType(); ColumnPtr nested_column(createNested(nested)); @@ -221,7 +220,7 @@ public: if (nested->isNullable()) { is_nullable = true; - nested = typeid_cast(nested.get())->getNestedType(); + nested = static_cast(nested.get())->getNestedType(); } WhichDataType which(nested); From 946a126799fb880dfea322d4d6a7e6b947d2398f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Jan 2021 18:09:05 +0300 Subject: [PATCH 104/264] Add CheckingCompressedReadBuffer Buffer for reading from a compressed file with just checking checksums of the compressed blocks, without any decompression, so result can be proxied. --- .../CachedCompressedReadBuffer.cpp | 2 +- .../CheckingCompressedReadBuffer.cpp | 24 +++++++++++++++ .../CheckingCompressedReadBuffer.h | 27 +++++++++++++++++ src/Compression/CompressedReadBuffer.cpp | 4 +-- src/Compression/CompressedReadBufferBase.cpp | 29 ++++++++++--------- src/Compression/CompressedReadBufferBase.h | 6 +++- .../CompressedReadBufferFromFile.cpp | 4 +-- src/Compression/ya.make | 1 + 8 files changed, 78 insertions(+), 19 deletions(-) create mode 100644 src/Compression/CheckingCompressedReadBuffer.cpp create mode 100644 src/Compression/CheckingCompressedReadBuffer.h diff --git a/src/Compression/CachedCompressedReadBuffer.cpp b/src/Compression/CachedCompressedReadBuffer.cpp index 3fb45ab0948..4b4d33954a9 100644 --- a/src/Compression/CachedCompressedReadBuffer.cpp +++ b/src/Compression/CachedCompressedReadBuffer.cpp @@ -45,7 +45,7 @@ bool CachedCompressedReadBuffer::nextImpl() size_t size_decompressed; size_t size_compressed_without_checksum; - owned_cell->compressed_size = readCompressedData(size_decompressed, size_compressed_without_checksum); + owned_cell->compressed_size = readCompressedData(size_decompressed, size_compressed_without_checksum, false); if (owned_cell->compressed_size) { diff --git a/src/Compression/CheckingCompressedReadBuffer.cpp b/src/Compression/CheckingCompressedReadBuffer.cpp new file mode 100644 index 00000000000..b27b899c73f --- /dev/null +++ b/src/Compression/CheckingCompressedReadBuffer.cpp @@ -0,0 +1,24 @@ +#include + +namespace DB +{ + +bool CheckingCompressedReadBuffer::nextImpl() +{ + size_t size_decompressed; + size_t size_compressed_without_checksum; + size_t size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, true); + + if (!size_compressed) + return false; + + /// own_compressed_buffer also includes getAdditionalSizeAtTheEndOfBuffer() + /// which should not be accounted here, so size_compressed is used. + /// + /// And BufferBase is used over ReadBuffer, since former reset the working_buffer. + BufferBase::set(own_compressed_buffer.data(), size_compressed, 0); + + return true; +} + +} diff --git a/src/Compression/CheckingCompressedReadBuffer.h b/src/Compression/CheckingCompressedReadBuffer.h new file mode 100644 index 00000000000..7a1d1f7a600 --- /dev/null +++ b/src/Compression/CheckingCompressedReadBuffer.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/** A buffer for reading from a compressed file with just checking checksums of + * the compressed blocks, without any decompression. + */ +class CheckingCompressedReadBuffer : public CompressedReadBufferBase, public ReadBuffer +{ +protected: + bool nextImpl() override; + +public: + CheckingCompressedReadBuffer(ReadBuffer & in_, bool allow_different_codecs_ = false) + : CompressedReadBufferBase(&in_, allow_different_codecs_) + , ReadBuffer(nullptr, 0) + { + } +}; + +} diff --git a/src/Compression/CompressedReadBuffer.cpp b/src/Compression/CompressedReadBuffer.cpp index 043fbdcdc1b..6a082164231 100644 --- a/src/Compression/CompressedReadBuffer.cpp +++ b/src/Compression/CompressedReadBuffer.cpp @@ -9,7 +9,7 @@ bool CompressedReadBuffer::nextImpl() { size_t size_decompressed; size_t size_compressed_without_checksum; - size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); + size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); if (!size_compressed) return false; @@ -40,7 +40,7 @@ size_t CompressedReadBuffer::readBig(char * to, size_t n) size_t size_decompressed; size_t size_compressed_without_checksum; - if (!readCompressedData(size_decompressed, size_compressed_without_checksum)) + if (!readCompressedData(size_decompressed, size_compressed_without_checksum, false)) return bytes_read; auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer(); diff --git a/src/Compression/CompressedReadBufferBase.cpp b/src/Compression/CompressedReadBufferBase.cpp index 5be31913140..8f5b779e4bc 100644 --- a/src/Compression/CompressedReadBufferBase.cpp +++ b/src/Compression/CompressedReadBufferBase.cpp @@ -105,19 +105,18 @@ static void validateChecksum(char * data, size_t size, const Checksum expected_c /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. /// Returns number of compressed bytes read. -size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum) +size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy) { if (compressed_in->eof()) return 0; - Checksum checksum; - compressed_in->readStrict(reinterpret_cast(&checksum), sizeof(Checksum)); - UInt8 header_size = ICompressionCodec::getHeaderSize(); - own_compressed_buffer.resize(header_size); - compressed_in->readStrict(own_compressed_buffer.data(), header_size); + own_compressed_buffer.resize(header_size + sizeof(Checksum)); - uint8_t method = ICompressionCodec::readMethod(own_compressed_buffer.data()); + compressed_in->readStrict(own_compressed_buffer.data(), sizeof(Checksum) + header_size); + char * compressed_header = own_compressed_buffer.data() + sizeof(Checksum); + + uint8_t method = ICompressionCodec::readMethod(compressed_header); if (!codec) { @@ -139,8 +138,8 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, } } - size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(own_compressed_buffer.data()); - size_decompressed = ICompressionCodec::readDecompressedBlockSize(own_compressed_buffer.data()); + size_compressed_without_checksum = ICompressionCodec::readCompressedBlockSize(compressed_header); + size_decompressed = ICompressionCodec::readDecompressedBlockSize(compressed_header); /// This is for clang static analyzer. assert(size_decompressed > 0); @@ -160,8 +159,9 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, auto additional_size_at_the_end_of_buffer = codec->getAdditionalSizeAtTheEndOfBuffer(); /// Is whole compressed block located in 'compressed_in->' buffer? - if (compressed_in->offset() >= header_size && - compressed_in->position() + size_compressed_without_checksum + additional_size_at_the_end_of_buffer - header_size <= compressed_in->buffer().end()) + if (!always_copy && + compressed_in->offset() >= header_size + sizeof(Checksum) && + compressed_in->available() >= (size_compressed_without_checksum - header_size) + additional_size_at_the_end_of_buffer + sizeof(Checksum)) { compressed_in->position() -= header_size; compressed_buffer = compressed_in->position(); @@ -169,13 +169,16 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_decompressed, } else { - own_compressed_buffer.resize(size_compressed_without_checksum + additional_size_at_the_end_of_buffer); - compressed_buffer = own_compressed_buffer.data(); + own_compressed_buffer.resize(sizeof(Checksum) + size_compressed_without_checksum + additional_size_at_the_end_of_buffer); + compressed_buffer = own_compressed_buffer.data() + sizeof(Checksum); compressed_in->readStrict(compressed_buffer + header_size, size_compressed_without_checksum - header_size); } if (!disable_checksum) + { + Checksum & checksum = *reinterpret_cast(own_compressed_buffer.data()); validateChecksum(compressed_buffer, size_compressed_without_checksum, checksum); + } return size_compressed_without_checksum + sizeof(Checksum); } diff --git a/src/Compression/CompressedReadBufferBase.h b/src/Compression/CompressedReadBufferBase.h index 71dc5274d5b..60b8847f639 100644 --- a/src/Compression/CompressedReadBufferBase.h +++ b/src/Compression/CompressedReadBufferBase.h @@ -30,8 +30,12 @@ protected: bool allow_different_codecs; /// Read compressed data into compressed_buffer. Get size of decompressed data from block header. Checksum if need. + /// + /// If always_copy is true then even if the compressed block is already stored in compressed_in.buffer() it will be copied into own_compressed_buffer. + /// This is required for CheckingCompressedReadBuffer, since this is just a proxy. + /// /// Returns number of compressed bytes read. - size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum); + size_t readCompressedData(size_t & size_decompressed, size_t & size_compressed_without_checksum, bool always_copy); void decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum); diff --git a/src/Compression/CompressedReadBufferFromFile.cpp b/src/Compression/CompressedReadBufferFromFile.cpp index f3fa2d6bc10..54f360f417b 100644 --- a/src/Compression/CompressedReadBufferFromFile.cpp +++ b/src/Compression/CompressedReadBufferFromFile.cpp @@ -19,7 +19,7 @@ bool CompressedReadBufferFromFile::nextImpl() { size_t size_decompressed = 0; size_t size_compressed_without_checksum; - size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); + size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); if (!size_compressed) return false; @@ -98,7 +98,7 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n) size_t size_decompressed = 0; size_t size_compressed_without_checksum = 0; - size_t new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum); + size_t new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum, false); size_compressed = 0; /// file_in no longer points to the end of the block in working_buffer. if (!new_size_compressed) return bytes_read; diff --git a/src/Compression/ya.make b/src/Compression/ya.make index d14f26379c5..89b1c73a81f 100644 --- a/src/Compression/ya.make +++ b/src/Compression/ya.make @@ -17,6 +17,7 @@ PEERDIR( SRCS( CachedCompressedReadBuffer.cpp + CheckingCompressedReadBuffer.cpp CompressedReadBuffer.cpp CompressedReadBufferBase.cpp CompressedReadBufferFromFile.cpp From 2565d2ac443401d94cd2642c089c0fd7290b7da4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Jan 2021 15:37:54 +0300 Subject: [PATCH 105/264] Verify compressed headers while sending distributed batches Before this patch the DirectoryMonitor was checking the compressed file by reading it one more time (since w/o this receiver may stuck on truncated file), while this is ineffective and we can just check the checksums before sending. But note that this may decrease batch size that is used for sending over network. --- src/Storages/Distributed/DirectoryMonitor.cpp | 41 ++++--------------- 1 file changed, 7 insertions(+), 34 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 92b08527f6a..8e7a2c5cc39 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -151,20 +152,6 @@ namespace return header; } - - /// FIXME: suboptimal - void verifyDistributedChecksum(ReadBufferFromFile & in) - { - CompressedReadBuffer decompressing_in(in); - NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); - block_in.readPrefix(); - - while (Block block = block_in.read()) - { - /// Just do the per-block checksum checks in the CompressedReadBuffer. - } - block_in.readSuffix(); - } } @@ -412,34 +399,21 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa LOG_TRACE(log, "Started processing `{}`", file_path); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(storage.global_context.getSettingsRef()); - /// Check that the file is valid - try - { - /// Determine metadata of the current file and check if it is not broken. - ReadBufferFromFile in{file_path}; - readDistributedHeader(in, log); - - verifyDistributedChecksum(in); - } - catch (const Exception & e) - { - maybeMarkAsBroken(file_path, e); - throw; - } - try { CurrentMetrics::Increment metric_increment{CurrentMetrics::DistributedSend}; - ReadBufferFromFile in{file_path}; + ReadBufferFromFile in(file_path); const auto & header = readDistributedHeader(in, log); auto connection = pool->get(timeouts, &header.insert_settings); RemoteBlockOutputStream remote{*connection, timeouts, header.insert_query, header.insert_settings, header.client_info}; + CheckingCompressedReadBuffer checking_in(in); + remote.writePrefix(); - remote.writePrepared(in); + remote.writePrepared(checking_in); remote.writeSuffix(); } catch (const Exception & e) @@ -592,7 +566,8 @@ struct StorageDistributedDirectoryMonitor::Batch remote->writePrefix(); } - remote->writePrepared(in); + CheckingCompressedReadBuffer checking_in(in); + remote->writePrepared(checking_in); } if (remote) @@ -778,8 +753,6 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map sample_block = block.cloneEmpty(); break; } - - verifyDistributedChecksum(in); } else { From 56475774d32b45cb49b509c92d9fb47fe2e96385 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Jan 2021 23:57:40 +0300 Subject: [PATCH 106/264] Fix readability-static-definition-in-anonymous-namespace in DirectoryMonitor --- src/Storages/Distributed/DirectoryMonitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 8e7a2c5cc39..62fa2b3f23a 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -90,7 +90,7 @@ namespace size_t bytes = 0; }; - static DistributedHeader readDistributedHeader(ReadBuffer & in, Poco::Logger * log) + DistributedHeader readDistributedHeader(ReadBuffer & in, Poco::Logger * log) { DistributedHeader header; From bf90fcc98e2fc3dc1a29761e3490dd0e6ddabb9c Mon Sep 17 00:00:00 2001 From: templarzq Date: Mon, 11 Jan 2021 10:40:03 +0800 Subject: [PATCH 107/264] Update ColumnAggregateFunction.cpp fix warning --- src/Columns/ColumnAggregateFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index d160f75f609..1808ad1bc15 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -613,7 +613,7 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { //create a new col to return auto cloneCol = cloneEmpty(); - auto res = typeid_cast(cloneCol.get()); + auto* res = typeid_cast(cloneCol.get()); if (size == 0) return cloneCol; From 7d1842cb19593c38ee38b21b142f3b3c84437e3f Mon Sep 17 00:00:00 2001 From: templarzq Date: Mon, 11 Jan 2021 10:49:18 +0800 Subject: [PATCH 108/264] Update ColumnAggregateFunction.cpp --- src/Columns/ColumnAggregateFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 1808ad1bc15..fda59df0a46 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -612,7 +612,7 @@ ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { //create a new col to return - auto cloneCol = cloneEmpty(); + MutableColumnPtr cloneCol = cloneEmpty(); auto* res = typeid_cast(cloneCol.get()); if (size == 0) return cloneCol; From 22c07e7342dca927ffeb3f2640f07ec5a85183ad Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 11 Jan 2021 07:39:48 +0300 Subject: [PATCH 109/264] Update test_check_table expectations Since now CompressedReadBufferBase read checksum and header at once it will expect 25 (16+9) bytes not 16 (only checksum) --- tests/integration/test_check_table/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index 30751b8cbba..916b2ead7f7 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -75,10 +75,10 @@ def test_check_normal_table_corruption(started_cluster): corrupt_data_part_on_disk(node1, "non_replicated_mt", "201902_1_1_0") assert node1.query("CHECK TABLE non_replicated_mt", settings={ - "check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16." + "check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 25." assert node1.query("CHECK TABLE non_replicated_mt", settings={ - "check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 16." + "check_query_single_value_result": 0}).strip() == "201902_1_1_0\t0\tCannot read all data. Bytes read: 2. Bytes expected: 25." node1.query("INSERT INTO non_replicated_mt VALUES (toDate('2019-01-01'), 1, 10), (toDate('2019-01-01'), 2, 12)") @@ -90,7 +90,7 @@ def test_check_normal_table_corruption(started_cluster): remove_checksums_on_disk(node1, "non_replicated_mt", "201901_2_2_0") assert node1.query("CHECK TABLE non_replicated_mt PARTITION 201901", settings={ - "check_query_single_value_result": 0}) == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 16.\\'\n" + "check_query_single_value_result": 0}) == "201901_2_2_0\t0\tCheck of part finished with error: \\'Cannot read all data. Bytes read: 2. Bytes expected: 25.\\'\n" def test_check_replicated_table_simple(started_cluster): From 87041363006adbf64e960d6807e9ae09ee0ae37c Mon Sep 17 00:00:00 2001 From: templarzq Date: Mon, 11 Jan 2021 14:38:21 +0800 Subject: [PATCH 110/264] Update ColumnAggregateFunction.cpp --- src/Columns/ColumnAggregateFunction.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index fda59df0a46..f417e227f97 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -613,7 +613,7 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { //create a new col to return MutableColumnPtr cloneCol = cloneEmpty(); - auto* res = typeid_cast(cloneCol.get()); + auto * res = typeid_cast(cloneCol.get()); if (size == 0) return cloneCol; From 38a9cba8504124401506cf6d3bfd0d57207e4bee Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 11 Jan 2021 10:23:44 +0000 Subject: [PATCH 111/264] Fix --- src/DataStreams/PostgreSQLBlockInputStream.cpp | 2 +- src/DataStreams/PostgreSQLBlockInputStream.h | 1 + src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 7 ++++++- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 1 + src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp | 6 ++++++ src/Databases/PostgreSQL/FetchFromPostgreSQL.h | 1 + src/Dictionaries/PostgreSQLDictionarySource.cpp | 1 + src/Dictionaries/PostgreSQLDictionarySource.h | 1 + src/Storages/StoragePostgreSQL.cpp | 4 +++- src/Storages/StoragePostgreSQL.h | 1 + src/TableFunctions/TableFunctionPostgreSQL.cpp | 1 + src/TableFunctions/TableFunctionPostgreSQL.h | 1 + 12 files changed, 24 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 4d9a142da3d..da6a83fb930 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -18,6 +18,7 @@ #include #include + namespace DB { @@ -26,7 +27,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } - PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( ConnectionPtr connection_, const std::string & query_str_, diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 431af203caf..b88c81cca0a 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -11,6 +11,7 @@ #include #include + namespace DB { using ConnectionPtr = std::shared_ptr; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 1ce82ec36b2..d3ee726b84e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -101,7 +101,12 @@ bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const try { - pqxx::result result = tx.exec(fmt::format("select '{}'::regclass", table_name)); + /// Casting table_name::regclass throws pqxx::indefined_table exception if table_name is incorrect. + pqxx::result result = tx.exec(fmt::format( + "SELECT '{}'::regclass, tablename " + "FROM pg_catalog.pg_tables " + "WHERE schemaname != 'pg_catalog' AND schemaname != 'information_schema' " + "AND tablename = '{}'", table_name, table_name)); } catch (pqxx::undefined_table const &) { diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 68c78b74ba4..950481cb292 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -11,6 +11,7 @@ #include #include + namespace DB { diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index c3a8c6ba2ef..b07c71d33a7 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -14,6 +14,7 @@ #include #include + namespace DB { @@ -57,6 +58,11 @@ std::shared_ptr fetchPostgreSQLTableStructure(ConnectionPtr c "PostgreSQL table {}.{} does not exist", connection->dbname(), postgres_table_name), ErrorCodes::UNKNOWN_TABLE); } + catch (Exception & e) + { + e.addMessage("while fetching postgresql table structure"); + throw; + } if (columns.empty()) return nullptr; diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h index bb6b8d46c84..b0b103251a0 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h @@ -7,6 +7,7 @@ #if USE_LIBPQXX #include + namespace DB { diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 417ef9b318d..67e959c34b0 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -11,6 +11,7 @@ #include "readInvalidateQuery.h" #endif + namespace DB { diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index eaec84bfee1..a826ff15f4f 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -14,6 +14,7 @@ #include #include + namespace DB { diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index ea1279bc319..c45336d1515 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -157,8 +157,10 @@ public: void writeSuffix() override { if (stream_inserter) + { stream_inserter->complete(); - work->commit(); + work->commit(); + } } diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 6d1764b2080..684c9cf2b17 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -11,6 +11,7 @@ #include #include "pqxx/pqxx" + namespace DB { diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index 4f8ffecc89a..bbbc9742015 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -11,6 +11,7 @@ #include "registerTableFunctions.h" #include + namespace DB { diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index 9bdb4e2e32f..7af01ecb053 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -8,6 +8,7 @@ #include #include "pqxx/pqxx" + namespace DB { From 91dc347ff355b3ebda0fa0ae7e3ae75d10705b2d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 11 Jan 2021 14:36:31 +0300 Subject: [PATCH 112/264] add some comments --- src/DataTypes/DataTypeCustom.h | 5 +++++ src/DataTypes/DataTypeOneElementTuple.cpp | 9 ++++----- src/DataTypes/DataTypeOneElementTuple.h | 1 - src/DataTypes/IDataType.h | 5 ++++- src/Storages/StorageTinyLog.cpp | 1 - .../0_stateless/01475_read_subcolumns_storages.sh | 1 + 6 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/DataTypes/DataTypeCustom.h b/src/DataTypes/DataTypeCustom.h index ec6093cf1e4..0fa2e365990 100644 --- a/src/DataTypes/DataTypeCustom.h +++ b/src/DataTypes/DataTypeCustom.h @@ -63,6 +63,11 @@ public: virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const = 0; }; +/** Allows to customize an existing data type by representation with custom substreams. + * Customized data type will be serialized/deserialized to files with different names than base type, + * but binary and text representation will be unchanged. + * E.g it can be used for reading single subcolumns of complex types. + */ class IDataTypeCustomStreams { public: diff --git a/src/DataTypes/DataTypeOneElementTuple.cpp b/src/DataTypes/DataTypeOneElementTuple.cpp index bb3cf37569b..a4169220362 100644 --- a/src/DataTypes/DataTypeOneElementTuple.cpp +++ b/src/DataTypes/DataTypeOneElementTuple.cpp @@ -14,6 +14,10 @@ namespace DB namespace { +/** Custom substreams representation for single subcolumn. + * It serializes/deserializes column as a nested type, but in that way + * if it was a named tuple with one element and a given name. + */ class DataTypeOneElementTupleStreams : public IDataTypeCustomStreams { private: @@ -105,9 +109,4 @@ DataTypePtr createOneElementTuple(const DataTypePtr & type, const String & name, return DataTypeFactory::instance().getCustom(std::move(custom_desc)); } -bool isOneElementTuple(const DataTypePtr & type) -{ - return typeid_cast(type->getCustomStreams()) != nullptr; -} - } diff --git a/src/DataTypes/DataTypeOneElementTuple.h b/src/DataTypes/DataTypeOneElementTuple.h index a452a547364..03b0511ef4a 100644 --- a/src/DataTypes/DataTypeOneElementTuple.h +++ b/src/DataTypes/DataTypeOneElementTuple.h @@ -6,6 +6,5 @@ namespace DB { DataTypePtr createOneElementTuple(const DataTypePtr & type, const String & name, bool escape_delimiter = true); -bool isOneElementTuple(const DataTypePtr & type); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 218316a440a..b67c5ee1846 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -100,9 +100,10 @@ public: }; Type type; - /// Index of tuple element, starting at 1. + /// Index of tuple element, starting at 1 or name. String tuple_element_name; + /// Do we need to escape a dot in filenames for tuple elements. bool escape_tuple_delimiter = true; Substream(Type type_) : type(type_) {} @@ -115,6 +116,8 @@ public: String toString() const; }; + /// Cache for common substreams of one type, but possible different its subcolumns. + /// E.g. sizes of arrays of Nested data type. using SubstreamsCache = std::unordered_map; using StreamCallback = std::function; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index e7d477b77c2..76621e5e9d0 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -176,7 +176,6 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, return nullptr; String stream_name = IDataType::getFileNameForStream(name_and_type, path); - auto & stream = streams[stream_name]; if (!stream) { diff --git a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh index 33f757dfe01..54cd0c073b2 100755 --- a/tests/queries/0_stateless/01475_read_subcolumns_storages.sh +++ b/tests/queries/0_stateless/01475_read_subcolumns_storages.sh @@ -1,6 +1,7 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh set -e From 8dde70b9370959ef638184a1f5a397e0dd2bd499 Mon Sep 17 00:00:00 2001 From: fastio Date: Mon, 11 Jan 2021 21:26:43 +0800 Subject: [PATCH 113/264] Bugfix: attach partition should reset the mutation --- src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 54248e6032c..7046a510f75 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -254,6 +254,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart( part->info.min_block = block_number; part->info.max_block = block_number; part->info.level = 0; + part->info.mutation = 0; part->name = part->getNewName(part->info); From b43bd4d900a0738aa2198afc83c8ace950615034 Mon Sep 17 00:00:00 2001 From: fastio Date: Tue, 12 Jan 2021 12:54:59 +0800 Subject: [PATCH 114/264] test --- .../__init__.py | 0 .../configs/zookeeper_config.xml | 8 +++ .../test.py | 60 +++++++++++++++++++ 3 files changed, 68 insertions(+) create mode 100644 tests/integration/test_fetch_partition_should_reset_mutation/__init__.py create mode 100644 tests/integration/test_fetch_partition_should_reset_mutation/configs/zookeeper_config.xml create mode 100644 tests/integration/test_fetch_partition_should_reset_mutation/test.py diff --git a/tests/integration/test_fetch_partition_should_reset_mutation/__init__.py b/tests/integration/test_fetch_partition_should_reset_mutation/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_fetch_partition_should_reset_mutation/configs/zookeeper_config.xml b/tests/integration/test_fetch_partition_should_reset_mutation/configs/zookeeper_config.xml new file mode 100644 index 00000000000..17cbd681c41 --- /dev/null +++ b/tests/integration/test_fetch_partition_should_reset_mutation/configs/zookeeper_config.xml @@ -0,0 +1,8 @@ + + + + zoo1 + 2181 + + + diff --git a/tests/integration/test_fetch_partition_should_reset_mutation/test.py b/tests/integration/test_fetch_partition_should_reset_mutation/test.py new file mode 100644 index 00000000000..6bd2f30a3cd --- /dev/null +++ b/tests/integration/test_fetch_partition_should_reset_mutation/test.py @@ -0,0 +1,60 @@ +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance("node", main_configs=["configs/zookeeper_config.xml"], with_zookeeper=True) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + + +def test_part_should_reset_mutation(start_cluster): + node.query( + "CREATE TABLE test (i Int64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test', 'node') ORDER BY i;" + ) + node.query("INSERT INTO test SELECT 1, 'a'") + node.query("optimize table test final") + node.query("optimize table test final") + + + expected = TSV('''all_0_0_2\t1\ta''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") + + expected = TSV('''all_0_0_2_4\t1\txxx''') + assert TSV(node.query('SELECT _part, * FROM test')) == expected + + node.query( + "CREATE TABLE restore (i Int64, s String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/restore', 'node') ORDER BY i;" + ) + node.query("ALTER TABLE restore FETCH PARTITION tuple() FROM '/clickhouse/tables/test/'") + node.query("ALTER TABLE restore ATTACH PART 'all_0_0_2_4'") + node.query("INSERT INTO restore select 2, 'a'") + + print(TSV(node.query('SELECT _part, * FROM restore'))) + expected = TSV('''all_0_0_0\t1\txxx\nall_1_1_0\t2\ta''') + assert TSV(node.query('SELECT _part, * FROM restore ORDER BY i')) == expected + + node.query("ALTER TABLE restore UPDATE s='yyy' WHERE 1") + + + expected = TSV('''all_0_0_0_2\t1\tyyy\nall_1_1_0_2\t2\tyyy''') + assert TSV(node.query('SELECT _part, * FROM restore ORDER BY i')) == expected + + node.query("ALTER TABLE restore DELETE WHERE 1") + + + assert node.query("SELECT count() FROM restore").strip() == "0" From 36381ce35ee4d1c1cdd8f62e44520195dc37232a Mon Sep 17 00:00:00 2001 From: templarzq Date: Tue, 12 Jan 2021 14:36:26 +0800 Subject: [PATCH 115/264] Update ColumnAggregateFunction.cpp --- src/Columns/ColumnAggregateFunction.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index f417e227f97..5604bb6d75b 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -612,10 +612,10 @@ ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { //create a new col to return - MutableColumnPtr cloneCol = cloneEmpty(); - auto * res = typeid_cast(cloneCol.get()); + MutableColumnPtr cloned_col = cloneEmpty(); + auto * res = typeid_cast(cloned_col.get()); if (size == 0) - return cloneCol; + return cloned_col; size_t from_size = data.size(); auto & res_data = res->data; @@ -638,6 +638,6 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const res->insertDefault(); } } - return cloneCol; + return cloned_col; } } From 21d3bd525a78701c08f48639c34c7d2ead86cc9e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 12 Jan 2021 17:11:57 +0300 Subject: [PATCH 116/264] Add config for integration tests msan --- tests/ci/ci_config.json | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 1efcf39601e..b2e93d39848 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -477,6 +477,18 @@ "with_coverage": false } }, + "Integration tests (memory)": { + "required_build_properties": { + "compiler": "clang-11", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, "Integration tests flaky check (asan)": { "required_build_properties": { "compiler": "clang-11", From 3cc7e1407d122bf49b8b1e710aed24909ef2584f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 12 Jan 2021 21:45:44 +0300 Subject: [PATCH 117/264] revert previous fix --- programs/local/LocalServer.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5a8d35e204d..22e72be27f2 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -273,10 +273,9 @@ try global_context->setCurrentDatabase(default_database); applyCmdOptions(*global_context); - if (config().has("path")) + String path = global_context->getPath(); + if (!path.empty()) { - String path = global_context->getPath(); - /// Lock path directory before read status.emplace(path + "status", StatusFile::write_full_info); From 5db57001195d06891fc423641bae15692b3be97b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 12 Jan 2021 21:47:01 +0300 Subject: [PATCH 118/264] fix rare deadlock on shutdown of BackgroundSchedulePool --- src/Core/BackgroundSchedulePool.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Core/BackgroundSchedulePool.cpp b/src/Core/BackgroundSchedulePool.cpp index b3aedd6c875..5f2a115562d 100644 --- a/src/Core/BackgroundSchedulePool.cpp +++ b/src/Core/BackgroundSchedulePool.cpp @@ -250,7 +250,16 @@ void BackgroundSchedulePool::threadFunction() while (!shutdown) { - if (Poco::AutoPtr notification = queue.waitDequeueNotification()) + /// We have to wait with timeout to prevent very rare deadlock, caused by the following race condition: + /// 1. Background thread N: threadFunction(): checks for shutdown (it's false) + /// 2. Main thread: ~BackgroundSchedulePool(): sets shutdown to true, calls queue.wakeUpAll(), it triggers + /// all existing Poco::Events inside Poco::NotificationQueue which background threads are waiting on. + /// 3. Background thread N: threadFunction(): calls queue.waitDequeueNotification(), it creates + /// new Poco::Event inside Poco::NotificationQueue and starts to wait on it + /// Background thread N will never be woken up. + /// TODO Do we really need Poco::NotificationQueue? Why not to use std::queue + mutex + condvar or maybe even DB::ThreadPool? + constexpr size_t wait_timeout_ms = 500; + if (Poco::AutoPtr notification = queue.waitDequeueNotification(wait_timeout_ms)) { TaskNotification & task_notification = static_cast(*notification); task_notification.execute(); From 111f4d49e3f4da74da42a32695ac37a4af87eea8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 12 Jan 2021 21:55:12 +0300 Subject: [PATCH 119/264] remove unread data limit from PeekableReadBuffer --- src/IO/PeekableReadBuffer.cpp | 12 ++---------- src/IO/PeekableReadBuffer.h | 4 +--- 2 files changed, 3 insertions(+), 13 deletions(-) diff --git a/src/IO/PeekableReadBuffer.cpp b/src/IO/PeekableReadBuffer.cpp index 8ad0e7b572e..62366a24f95 100644 --- a/src/IO/PeekableReadBuffer.cpp +++ b/src/IO/PeekableReadBuffer.cpp @@ -8,9 +8,8 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/, - size_t unread_limit_ /* = default_limit*/) - : BufferWithOwnMemory(start_size_), sub_buf(sub_buf_), unread_limit(unread_limit_) +PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/) + : BufferWithOwnMemory(start_size_), sub_buf(sub_buf_) { padded &= sub_buf.isPadded(); /// Read from sub-buffer @@ -191,8 +190,6 @@ void PeekableReadBuffer::checkStateCorrect() const } if (currentlyReadFromOwnMemory() && !peeked_size) throw DB::Exception("Pos in empty own buffer", ErrorCodes::LOGICAL_ERROR); - if (unread_limit < memory.size()) - throw DB::Exception("Size limit exceed", ErrorCodes::LOGICAL_ERROR); } void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append) @@ -222,16 +219,11 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append) } else { - if (unread_limit < new_size) - throw DB::Exception("PeekableReadBuffer: Memory limit exceed", ErrorCodes::MEMORY_LIMIT_EXCEEDED); - size_t pos_offset = pos - memory.data(); size_t new_size_amortized = memory.size() * 2; if (new_size_amortized < new_size) new_size_amortized = new_size; - else if (unread_limit < new_size_amortized) - new_size_amortized = unread_limit; memory.resize(new_size_amortized); if (need_update_checkpoint) diff --git a/src/IO/PeekableReadBuffer.h b/src/IO/PeekableReadBuffer.h index 62b6f08f621..3088e59d94c 100644 --- a/src/IO/PeekableReadBuffer.h +++ b/src/IO/PeekableReadBuffer.h @@ -20,8 +20,7 @@ class PeekableReadBuffer : public BufferWithOwnMemory { friend class PeekableReadBufferCheckpoint; public: - explicit PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE, - size_t unread_limit_ = 16 * DBMS_DEFAULT_BUFFER_SIZE); + explicit PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE); ~PeekableReadBuffer() override; @@ -95,7 +94,6 @@ private: ReadBuffer & sub_buf; - const size_t unread_limit; size_t peeked_size = 0; Position checkpoint = nullptr; bool checkpoint_in_own_memory = false; From d3b7fa1a77ff58a4e503d9591e9d84d450e042de Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Wed, 13 Jan 2021 02:05:09 +0300 Subject: [PATCH 120/264] Docs in En --- docs/en/operations/settings/index.md | 1 + docs/en/sql-reference/statements/select/index.md | 11 +++++++++++ 2 files changed, 12 insertions(+) diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index cd483694521..87c8a9bf1b9 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -25,6 +25,7 @@ Ways to configure settings, in order of priority: - When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`. - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). + - Make settings right in the query, in the `SETTINGS` clause. The setting value is applied only to that query and is reset to default or previous value after the query is executed. Settings that can only be made in the server config file are not covered in this section. diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index ed69198ed4d..73ae1392350 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -25,6 +25,7 @@ SELECT [DISTINCT] expr_list [ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] +[SETTINGS ...] [UNION ...] [INTO OUTFILE filename] [FORMAT format] @@ -265,5 +266,15 @@ SELECT * REPLACE(i + 1 AS i) EXCEPT (j) APPLY(sum) from columns_transformers; └─────────────────┴────────┘ ``` +## SETTINGS in SELECT Query {#settings-in-select} + +There are multiple ways to make settings, see [here](../../../operations/settings/index.md). One of them is to specify necessary settings right in the SELECT query. The setting value is applied only to this query and is reset to default or previous value after the query is executed. + +**Example** + +``` sql +SELECT * FROM some_table SETTINGS optimize_read_in_order=1 SETTINGS cast_keep_nullable=1; +``` + [Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/) From d7200ee2ede7b739a682c8f5c06e47c7e624a121 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 13 Jan 2021 02:20:32 +0300 Subject: [PATCH 121/264] minor changes --- src/DataTypes/DataTypeLowCardinality.cpp | 3 +-- src/Storages/ColumnsDescription.cpp | 14 +++++++------- src/Storages/ColumnsDescription.h | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 5 ----- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 - src/Storages/MergeTree/IMergeTreeReader.cpp | 2 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartInMemory.h | 2 +- src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 14 +++----------- src/Storages/MergeTree/MergeTreeReaderCompact.h | 2 +- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 2 +- src/Storages/StorageTinyLog.cpp | 6 +----- 12 files changed, 19 insertions(+), 38 deletions(-) diff --git a/src/DataTypes/DataTypeLowCardinality.cpp b/src/DataTypes/DataTypeLowCardinality.cpp index 0bb4f7d2bee..a433d39c561 100644 --- a/src/DataTypes/DataTypeLowCardinality.cpp +++ b/src/DataTypes/DataTypeLowCardinality.cpp @@ -584,9 +584,8 @@ void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreamsImpl( size_t limit, DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, - SubstreamsCache * cache) const + SubstreamsCache * /* cache */) const { - UNUSED(cache); ColumnLowCardinality & low_cardinality_column = typeid_cast(column); settings.path.push_back(Substream::DictionaryKeys); diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 9443042d13f..b4ddde7c0f3 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -541,12 +541,12 @@ ColumnsDescription ColumnsDescription::parse(const String & str) return result; } -void ColumnsDescription::addSubcolumns(const String & storage_name, const DataTypePtr & storage_type) +void ColumnsDescription::addSubcolumns(const String & name_in_storage, const DataTypePtr & type_in_storage) { - for (const auto & subcolumn_name : storage_type->getSubcolumnNames()) + for (const auto & subcolumn_name : type_in_storage->getSubcolumnNames()) { - auto subcolumn = NameAndTypePair(storage_name, subcolumn_name, - storage_type, storage_type->getSubcolumnType(subcolumn_name)); + auto subcolumn = NameAndTypePair(name_in_storage, subcolumn_name, + type_in_storage, type_in_storage->getSubcolumnType(subcolumn_name)); if (has(subcolumn.name)) throw Exception(ErrorCodes::ILLEGAL_COLUMN, @@ -556,10 +556,10 @@ void ColumnsDescription::addSubcolumns(const String & storage_name, const DataTy } } -void ColumnsDescription::removeSubcolumns(const String & storage_name, const DataTypePtr & storage_type) +void ColumnsDescription::removeSubcolumns(const String & name_in_storage, const DataTypePtr & type_in_storage) { - for (const auto & subcolumn_name : storage_type->getSubcolumnNames()) - subcolumns.erase(storage_name + "." + subcolumn_name); + for (const auto & subcolumn_name : type_in_storage->getSubcolumnNames()) + subcolumns.erase(name_in_storage + "." + subcolumn_name); } Block validateColumnsDefaultsAndGetSampleBlock(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, const Context & context) diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 40ebdb7ec5b..1990c565b65 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -149,8 +149,8 @@ private: SubcolumnsContainer subcolumns; void modifyColumnOrder(const String & column_name, const String & after_column, bool first); - void addSubcolumns(const String & storage_name, const DataTypePtr & storage_type); - void removeSubcolumns(const String & storage_name, const DataTypePtr & storage_type); + void addSubcolumns(const String & name_in_storage, const DataTypePtr & type_in_storage); + void removeSubcolumns(const String & name_in_storage, const DataTypePtr & type_in_storage); }; /// Validate default expressions and corresponding types compatibility, i.e. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 03c582d38a5..593501f94e6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -193,11 +193,6 @@ std::optional IMergeTreeDataPart::getColumnPosition(const String & colum return it->second; } -std::optional IMergeTreeDataPart::getColumnPosition(const NameAndTypePair & column) const -{ - return getColumnPosition(column.name); -} - DayNum IMergeTreeDataPart::getMinDate() const { if (storage.minmax_idx_date_column_pos != -1 && minmax_idx.initialized) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 70caa687362..7e8ebca53e5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -142,7 +142,6 @@ public: /// take place, you must take original name of column for this part from /// storage and pass it to this method. std::optional getColumnPosition(const String & column_name) const; - std::optional getColumnPosition(const NameAndTypePair & column) const; /// Returns the name of a column with minimum compressed size (as returned by getColumnSize()). /// If no checksums are present returns the name of the first physically existing column. diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 356bef68530..0140b32e12c 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -285,7 +285,7 @@ IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const St { if (typeid_cast(part_column.type.get())) { - auto position = data_part->getColumnPosition(part_column); + auto position = data_part->getColumnPosition(part_column.name); if (position && Nested::extractTableName(part_column.name) == table_name) return position; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 234f904ac80..32f54e3b782 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -124,7 +124,7 @@ void MergeTreeDataPartCompact::loadIndexGranularity() bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const { - if (!getColumnPosition(column)) + if (!getColumnPosition(column.name)) return false; auto bin_checksum = checksums.files.find(DATA_FILE_NAME_WITH_EXTENSION); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index ce0bff9e2ba..397d3d2036c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -42,7 +42,7 @@ public: const MergeTreeIndexGranularity & computed_index_granularity) const override; bool isStoredOnDisk() const override { return false; } - bool hasColumnFiles(const NameAndTypePair & column) const override { return !!getColumnPosition(column); } + bool hasColumnFiles(const NameAndTypePair & column) 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 StorageMetadataPtr & metadata_snapshot) const override; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index d6bfac4a5b6..635c59cf19a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -55,7 +55,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { auto column_from_part = getColumnFromPart(*name_and_type); - auto position = data_part->getColumnPosition(column_from_part); + auto position = data_part->getColumnPosition(column_from_part.name); if (!position && typeid_cast(column_from_part.type.get())) { /// If array of Nested column is missing in part, @@ -140,7 +140,6 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, while (read_rows < max_rows_to_read) { size_t rows_to_read = data_part->index_granularity.getMarkRows(from_mark); - std::unordered_map caches; auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) @@ -154,8 +153,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, auto & column = res_columns[pos]; size_t column_size_before_reading = column->size(); - readData(column_from_part, column, from_mark, *column_positions[pos], - rows_to_read, read_only_offsets[pos], caches[column_from_part.getNameInStorage()]); + readData(column_from_part, column, from_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column < rows_to_read) @@ -189,8 +187,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, void MergeTreeReaderCompact::readData( const NameAndTypePair & name_and_type, ColumnPtr & column, - size_t from_mark, size_t column_position, size_t rows_to_read, - bool only_offsets, IDataType::SubstreamsCache & cache) + size_t from_mark, size_t column_position, size_t rows_to_read, bool only_offsets) { const auto & [name, type] = name_and_type; @@ -199,9 +196,6 @@ void MergeTreeReaderCompact::readData( auto buffer_getter = [&](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { - // if (cache.count(IDataType::getSubcolumnNameForStream(substream_path))) - // return nullptr; - if (only_offsets && (substream_path.size() != 1 || substream_path[0].type != IDataType::Substream::ArraySizes)) return nullptr; @@ -228,8 +222,6 @@ void MergeTreeReaderCompact::readData( type->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state); } - UNUSED(cache); - /// The buffer is left in inconsistent state after reading single offsets if (only_offsets) last_read_granule.reset(); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.h b/src/Storages/MergeTree/MergeTreeReaderCompact.h index a37ac304fe9..dbfaa7868fa 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.h +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.h @@ -57,7 +57,7 @@ private: void seekToMark(size_t row_index, size_t column_index); void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark, - size_t column_position, size_t rows_to_read, bool only_offsets, IDataType::SubstreamsCache & cache); + size_t column_position, size_t rows_to_read, bool only_offsets); /// Returns maximal value of granule size in compressed file from @mark_ranges. /// This value is used as size of read buffer. diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 42798ee4dd0..30db54fc8e0 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -174,7 +174,7 @@ void MergeTreeReaderWide::readData( { return [&, stream_for_prefix](const IDataType::SubstreamPath & substream_path) -> ReadBuffer * { - /// If offsets for arrays have already been read. TODO + /// If substream have already been read. if (cache.count(IDataType::getSubcolumnNameForStream(substream_path))) return nullptr; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 76621e5e9d0..a51546bb8ff 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -121,7 +121,7 @@ Chunk TinyLogSource::generate() { Block res; - if (is_finished || (!streams.empty() && streams.begin()->second->compressed.eof())) + if (is_finished || file_sizes.empty() || (!streams.empty() && streams.begin()->second->compressed.eof())) { /** Close the files (before destroying the object). * When many sources are created, but simultaneously reading only a few of them, @@ -132,10 +132,6 @@ Chunk TinyLogSource::generate() return {}; } - /// if there are no files in the folder, it means that the table is empty - if (storage.disk->isDirectoryEmpty(storage.table_path)) - return {}; - std::unordered_map caches; for (const auto & name_type : columns) { From c310f69dc3baf6ce0d8807bab9bfc0b7eceedbb3 Mon Sep 17 00:00:00 2001 From: templarzq Date: Wed, 13 Jan 2021 10:35:33 +0800 Subject: [PATCH 122/264] Update ColumnAggregateFunction.cpp --- src/Columns/ColumnAggregateFunction.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 5604bb6d75b..a142ec196d4 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -619,6 +619,7 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const size_t from_size = data.size(); auto & res_data = res->data; + //copy data to cloned column if (size <= from_size) { From 8b98b467b697417cc2f9dca2c83b2b98124f2218 Mon Sep 17 00:00:00 2001 From: templarzq Date: Wed, 13 Jan 2021 13:57:42 +0800 Subject: [PATCH 123/264] Update ColumnAggregateFunction.cpp --- src/Columns/ColumnAggregateFunction.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index a142ec196d4..5604bb6d75b 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -619,7 +619,6 @@ MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const size_t from_size = data.size(); auto & res_data = res->data; - //copy data to cloned column if (size <= from_size) { From 7650968b1ec7bfa46bd278e1f76ee14054b39492 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 13 Jan 2021 10:19:43 +0300 Subject: [PATCH 124/264] Fix flaky test --- .../test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_fetch_partition_should_reset_mutation/test.py b/tests/integration/test_fetch_partition_should_reset_mutation/test.py index 6bd2f30a3cd..14a91a42031 100644 --- a/tests/integration/test_fetch_partition_should_reset_mutation/test.py +++ b/tests/integration/test_fetch_partition_should_reset_mutation/test.py @@ -29,10 +29,10 @@ def test_part_should_reset_mutation(start_cluster): expected = TSV('''all_0_0_2\t1\ta''') assert TSV(node.query('SELECT _part, * FROM test')) == expected - node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") - node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") - node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") - node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1") + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1", settings={"mutations_sync": "2"}) + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1", settings={"mutations_sync": "2"}) + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1", settings={"mutations_sync": "2"}) + node.query("ALTER TABLE test UPDATE s='xxx' WHERE 1", settings={"mutations_sync": "2"}) expected = TSV('''all_0_0_2_4\t1\txxx''') assert TSV(node.query('SELECT _part, * FROM test')) == expected @@ -48,13 +48,13 @@ def test_part_should_reset_mutation(start_cluster): expected = TSV('''all_0_0_0\t1\txxx\nall_1_1_0\t2\ta''') assert TSV(node.query('SELECT _part, * FROM restore ORDER BY i')) == expected - node.query("ALTER TABLE restore UPDATE s='yyy' WHERE 1") + node.query("ALTER TABLE restore UPDATE s='yyy' WHERE 1", settings={"mutations_sync": "2"}) expected = TSV('''all_0_0_0_2\t1\tyyy\nall_1_1_0_2\t2\tyyy''') assert TSV(node.query('SELECT _part, * FROM restore ORDER BY i')) == expected - node.query("ALTER TABLE restore DELETE WHERE 1") + node.query("ALTER TABLE restore DELETE WHERE 1", settings={"mutations_sync": "2"}) assert node.query("SELECT count() FROM restore").strip() == "0" From e2a2ab6f67b6935ea5bb7de6ac57fe32a0598992 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jan 2021 13:34:58 +0300 Subject: [PATCH 125/264] Add tests from #15889 --- .../01651_bugs_from_15889.reference | 2 + .../0_stateless/01651_bugs_from_15889.sql | 81 +++++++++++++++++++ 2 files changed, 83 insertions(+) create mode 100644 tests/queries/0_stateless/01651_bugs_from_15889.reference create mode 100644 tests/queries/0_stateless/01651_bugs_from_15889.sql diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.reference b/tests/queries/0_stateless/01651_bugs_from_15889.reference new file mode 100644 index 00000000000..77ac542d4fb --- /dev/null +++ b/tests/queries/0_stateless/01651_bugs_from_15889.reference @@ -0,0 +1,2 @@ +0 + diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql new file mode 100644 index 00000000000..fa4b7ca2268 --- /dev/null +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -0,0 +1,81 @@ +DROP TABLE IF EXISTS xp; +DROP TABLE IF EXISTS xp_d; + +CREATE TABLE xp (`A` Date, `B` Int64, `S` String) ENGINE = MergeTree PARTITION BY toYYYYMM(A) ORDER BY B; +INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); + +CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp); + +SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 20 } + +SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } + +DROP TABLE IF EXISTS xp; +DROP TABLE IF EXISTS xp_d; + +set allow_introspection_functions = 1; + +WITH concat(addressToLine(arrayJoin(trace) AS addr), '#') AS symbol +SELECT count() > 7 +FROM system.trace_log AS t +WHERE (query_id = +( + SELECT + [NULL, NULL, NULL, NULL, 0.00009999999747378752, NULL, NULL, NULL, NULL, NULL], + query_id + FROM system.query_log + WHERE (query LIKE '%test cpu time query profiler%') AND (query NOT LIKE '%system%') + ORDER BY event_time DESC + LIMIT 1 +)) AND (symbol LIKE '%Source%'); + + +WITH addressToSymbol(arrayJoin(trace)) AS symbol +SELECT count() > 0 +FROM system.trace_log AS t +WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterday()) AND (trace_type = 'Memory') AND (query_id = +( + SELECT + ignore(ignore(ignore(ignore(65536)), ignore(65537), ignore(2)), ''), + query_id + FROM system.query_log + WHERE (event_date >= yesterday()) AND (query LIKE '%test memory profiler%') + ORDER BY event_time DESC + LIMIT 1 +)); -- { serverError 42 } + +WITH ( + ( + SELECT query_start_time_microseconds + FROM system.query_log + ORDER BY query_start_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT + inf, + query_start_time + FROM system.query_log + ORDER BY query_start_time DESC + LIMIT 1 + ) AS t) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = -9223372036854775808, 'ok', ''); -- { serverError 43 } + +WITH ( + ( + SELECT query_start_time_microseconds + FROM system.query_log + ORDER BY query_start_time DESC + LIMIT 1 + ) AS time_with_microseconds, + ( + SELECT query_start_time + FROM system.query_log + ORDER BY query_start_time DESC + LIMIT 1 + ) AS t) +SELECT if(dateDiff('second', toDateTime(time_with_microseconds), toDateTime(t)) = -9223372036854775808, 'ok', ''); + + + + From 6c4be506fdc1486021b82f403830f8073caefb2b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 13 Jan 2021 16:43:42 +0300 Subject: [PATCH 126/264] add test --- src/IO/PeekableReadBuffer.cpp | 1 - src/IO/tests/gtest_peekable_read_buffer.cpp | 2 +- ...01184_insert_values_huge_strings.reference | 3 +++ .../01184_insert_values_huge_strings.sh | 20 +++++++++++++++++++ 4 files changed, 24 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01184_insert_values_huge_strings.reference create mode 100755 tests/queries/0_stateless/01184_insert_values_huge_strings.sh diff --git a/src/IO/PeekableReadBuffer.cpp b/src/IO/PeekableReadBuffer.cpp index 62366a24f95..11ca9cd6a0b 100644 --- a/src/IO/PeekableReadBuffer.cpp +++ b/src/IO/PeekableReadBuffer.cpp @@ -4,7 +4,6 @@ namespace DB { namespace ErrorCodes { - extern const int MEMORY_LIMIT_EXCEEDED; extern const int LOGICAL_ERROR; } diff --git a/src/IO/tests/gtest_peekable_read_buffer.cpp b/src/IO/tests/gtest_peekable_read_buffer.cpp index fb4b0b799b4..f28f5588006 100644 --- a/src/IO/tests/gtest_peekable_read_buffer.cpp +++ b/src/IO/tests/gtest_peekable_read_buffer.cpp @@ -40,7 +40,7 @@ try DB::ReadBufferFromString b4(s4); DB::ConcatReadBuffer concat({&b1, &b2, &b3, &b4}); - DB::PeekableReadBuffer peekable(concat, 0, 16); + DB::PeekableReadBuffer peekable(concat, 0); ASSERT_TRUE(!peekable.eof()); assertAvailable(peekable, "0123456789"); diff --git a/tests/queries/0_stateless/01184_insert_values_huge_strings.reference b/tests/queries/0_stateless/01184_insert_values_huge_strings.reference new file mode 100644 index 00000000000..1c42cb6d5ed --- /dev/null +++ b/tests/queries/0_stateless/01184_insert_values_huge_strings.reference @@ -0,0 +1,3 @@ +1000100 +1000100 +1000100 diff --git a/tests/queries/0_stateless/01184_insert_values_huge_strings.sh b/tests/queries/0_stateless/01184_insert_values_huge_strings.sh new file mode 100755 index 00000000000..63287582cbd --- /dev/null +++ b/tests/queries/0_stateless/01184_insert_values_huge_strings.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists default.huge_strings" +$CLICKHOUSE_CLIENT -q "create table default.huge_strings (n UInt64, l UInt64, s String, h UInt64) engine=MergeTree order by n" + +for _ in {1..10}; do + $CLICKHOUSE_CLIENT -q "select number, (rand() % 100*1000*1000) as l, repeat(randomString(l/1000/1000), 1000*1000) as s, cityHash64(s) from numbers(10) format Values" | $CLICKHOUSE_CLIENT -q "insert into default.huge_strings values" & + $CLICKHOUSE_CLIENT -q "select number % 10, (rand() % 100) as l, randomString(l) as s, cityHash64(s) from numbers(100000)" | $CLICKHOUSE_CLIENT -q "insert into default.huge_strings format TSV" & +done; +wait + +$CLICKHOUSE_CLIENT -q "select count() from huge_strings" +$CLICKHOUSE_CLIENT -q "select sum(l = length(s)) from huge_strings" +$CLICKHOUSE_CLIENT -q "select sum(h = cityHash64(s)) from huge_strings" + +$CLICKHOUSE_CLIENT -q "drop table huge_strings" From 43f350c0d1376d99ee4c6c11d25424a146016194 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Wed, 13 Jan 2021 15:55:48 +0100 Subject: [PATCH 127/264] Enabling RBAC tests to look for fails. Adding new tests --- tests/testflows/rbac/regression.py | 20 +- .../rbac/requirements/requirements.md | 1475 +- .../rbac/requirements/requirements.py | 18990 +++++++--------- .../rbac/tests/privileges/admin_option.py | 119 + .../tests/privileges/alter/alter_column.py | 5 +- .../privileges/alter/alter_constraint.py | 5 +- .../tests/privileges/alter/alter_delete.py | 3 +- .../tests/privileges/alter/alter_fetch.py | 9 +- .../tests/privileges/alter/alter_freeze.py | 8 +- .../tests/privileges/attach/attach_table.py | 11 +- .../tests/privileges/create/create_table.py | 4 +- .../tests/privileges/detach/detach_view.py | 12 +- .../rbac/tests/privileges/dictGet.py | 646 + .../tests/privileges/distributed_table.py | 12 +- .../rbac/tests/privileges/feature.py | 20 +- .../testflows/rbac/tests/privileges/insert.py | 14 +- .../rbac/tests/privileges/introspection.py | 285 + .../rbac/tests/privileges/public_tables.py | 55 +- .../rbac/tests/privileges/role_admin.py | 4 +- .../testflows/rbac/tests/privileges/select.py | 12 +- .../tests/privileges/show/show_columns.py | 6 +- .../tests/privileges/show/show_databases.py | 8 +- .../privileges/show/show_dictionaries.py | 8 +- .../rbac/tests/privileges/show/show_quotas.py | 6 +- .../rbac/tests/privileges/show/show_roles.py | 6 +- .../privileges/show/show_row_policies.py | 6 +- .../privileges/show/show_settings_profiles.py | 6 +- .../rbac/tests/privileges/show/show_tables.py | 8 +- .../rbac/tests/privileges/show/show_users.py | 6 +- .../rbac/tests/privileges/sources.py | 713 + .../rbac/tests/privileges/system/__init__.py | 0 .../tests/privileges/system/drop_cache.py | 269 + .../rbac/tests/privileges/system/fetches.py | 150 + .../rbac/tests/privileges/system/flush.py | 194 + .../rbac/tests/privileges/system/merges.py | 150 + .../rbac/tests/privileges/system/moves.py | 150 + .../rbac/tests/privileges/system/reload.py | 379 + .../privileges/system/replication_queues.py | 150 + .../privileges/system/restart_replica.py | 99 + .../rbac/tests/privileges/system/sends.py | 314 + .../rbac/tests/privileges/system/shutdown.py | 200 + .../tests/privileges/system/sync_replica.py | 99 + .../tests/privileges/system/ttl_merges.py | 150 + tests/testflows/regression.py | 5 +- 44 files changed, 13707 insertions(+), 11084 deletions(-) create mode 100644 tests/testflows/rbac/tests/privileges/admin_option.py create mode 100644 tests/testflows/rbac/tests/privileges/dictGet.py create mode 100644 tests/testflows/rbac/tests/privileges/introspection.py create mode 100644 tests/testflows/rbac/tests/privileges/sources.py create mode 100644 tests/testflows/rbac/tests/privileges/system/__init__.py create mode 100644 tests/testflows/rbac/tests/privileges/system/drop_cache.py create mode 100644 tests/testflows/rbac/tests/privileges/system/fetches.py create mode 100644 tests/testflows/rbac/tests/privileges/system/flush.py create mode 100644 tests/testflows/rbac/tests/privileges/system/merges.py create mode 100644 tests/testflows/rbac/tests/privileges/system/moves.py create mode 100644 tests/testflows/rbac/tests/privileges/system/reload.py create mode 100644 tests/testflows/rbac/tests/privileges/system/replication_queues.py create mode 100644 tests/testflows/rbac/tests/privileges/system/restart_replica.py create mode 100644 tests/testflows/rbac/tests/privileges/system/sends.py create mode 100644 tests/testflows/rbac/tests/privileges/system/shutdown.py create mode 100644 tests/testflows/rbac/tests/privileges/system/sync_replica.py create mode 100644 tests/testflows/rbac/tests/privileges/system/ttl_merges.py diff --git a/tests/testflows/rbac/regression.py b/tests/testflows/rbac/regression.py index 24878332c8a..e89be3bab20 100755 --- a/tests/testflows/rbac/regression.py +++ b/tests/testflows/rbac/regression.py @@ -25,6 +25,8 @@ issue_17147 = "https://github.com/ClickHouse/ClickHouse/issues/17147" issue_17653 = "https://github.com/ClickHouse/ClickHouse/issues/17653" issue_17655 = "https://github.com/ClickHouse/ClickHouse/issues/17655" issue_17766 = "https://github.com/ClickHouse/ClickHouse/issues/17766" +issue_18110 = "https://github.com/ClickHouse/ClickHouse/issues/18110" +issue_18206 = "https://github.com/ClickHouse/ClickHouse/issues/18206" xfails = { "syntax/show create quota/I show create quota current": @@ -113,8 +115,22 @@ xfails = { [(Fail, issue_17147)], "privileges/show dictionaries/:/check privilege/:/exists/EXISTS with privilege": [(Fail, issue_17655)], - "privileges/public tables/query log": - [(Fail, issue_17766)] + "privileges/public tables/sensitive tables": + [(Fail, issue_18110)], + "privileges/system merges/:/:/:/:/SYSTEM:": + [(Fail, issue_18206)], + "privileges/system ttl merges/:/:/:/:/SYSTEM:": + [(Fail, issue_18206)], + "privileges/system moves/:/:/:/:/SYSTEM:": + [(Fail, issue_18206)], + "privileges/system sends/:/:/:/:/SYSTEM:": + [(Fail, issue_18206)], + "privileges/system fetches/:/:/:/:/SYSTEM:": + [(Fail, issue_18206)], + "privileges/system restart replica/:/:/:/:/SYSTEM:": + [(Fail, issue_18206)], + "privileges/system replication queues/:/:/:/:/SYSTEM:": + [(Fail, issue_18206)], } xflags = { diff --git a/tests/testflows/rbac/requirements/requirements.md b/tests/testflows/rbac/requirements/requirements.md index aea25b04d57..ae6a038c15e 100644 --- a/tests/testflows/rbac/requirements/requirements.md +++ b/tests/testflows/rbac/requirements/requirements.md @@ -153,388 +153,412 @@ * 5.2.8.95 [RQ.SRS-006.RBAC.Grant.Privilege.Insert](#rqsrs-006rbacgrantprivilegeinsert) * 5.2.8.96 [RQ.SRS-006.RBAC.Grant.Privilege.Alter](#rqsrs-006rbacgrantprivilegealter) * 5.2.8.97 [RQ.SRS-006.RBAC.Grant.Privilege.Create](#rqsrs-006rbacgrantprivilegecreate) - * 5.2.8.98 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Database](#rqsrs-006rbacgrantprivilegecreatedatabase) - * 5.2.8.99 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Dictionary](#rqsrs-006rbacgrantprivilegecreatedictionary) - * 5.2.8.100 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Table](#rqsrs-006rbacgrantprivilegecreatetable) - * 5.2.8.101 [RQ.SRS-006.RBAC.Grant.Privilege.Create.TemporaryTable](#rqsrs-006rbacgrantprivilegecreatetemporarytable) - * 5.2.8.102 [RQ.SRS-006.RBAC.Grant.Privilege.Drop](#rqsrs-006rbacgrantprivilegedrop) - * 5.2.8.103 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Database](#rqsrs-006rbacgrantprivilegedropdatabase) - * 5.2.8.104 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Dictionary](#rqsrs-006rbacgrantprivilegedropdictionary) - * 5.2.8.105 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Table](#rqsrs-006rbacgrantprivilegedroptable) - * 5.2.8.106 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.TemporaryTable](#rqsrs-006rbacgrantprivilegedroptemporarytable) - * 5.2.8.107 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate](#rqsrs-006rbacgrantprivilegetruncate) - * 5.2.8.108 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize](#rqsrs-006rbacgrantprivilegeoptimize) - * 5.2.8.109 [RQ.SRS-006.RBAC.Grant.Privilege.Show](#rqsrs-006rbacgrantprivilegeshow) - * 5.2.8.110 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery](#rqsrs-006rbacgrantprivilegekillquery) - * 5.2.8.111 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement](#rqsrs-006rbacgrantprivilegeaccessmanagement) - * 5.2.8.112 [RQ.SRS-006.RBAC.Grant.Privilege.System](#rqsrs-006rbacgrantprivilegesystem) - * 5.2.8.113 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection](#rqsrs-006rbacgrantprivilegeintrospection) - * 5.2.8.114 [RQ.SRS-006.RBAC.Grant.Privilege.Sources](#rqsrs-006rbacgrantprivilegesources) - * 5.2.8.115 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet](#rqsrs-006rbacgrantprivilegedictget) - * 5.2.8.116 [RQ.SRS-006.RBAC.Grant.Privilege.None](#rqsrs-006rbacgrantprivilegenone) - * 5.2.8.117 [RQ.SRS-006.RBAC.Grant.Privilege.All](#rqsrs-006rbacgrantprivilegeall) - * 5.2.8.118 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption](#rqsrs-006rbacgrantprivilegegrantoption) - * 5.2.8.119 [RQ.SRS-006.RBAC.Grant.Privilege.On](#rqsrs-006rbacgrantprivilegeon) - * 5.2.8.120 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns](#rqsrs-006rbacgrantprivilegeprivilegecolumns) - * 5.2.8.121 [RQ.SRS-006.RBAC.Grant.Privilege.OnCluster](#rqsrs-006rbacgrantprivilegeoncluster) - * 5.2.8.122 [RQ.SRS-006.RBAC.Grant.Privilege.Syntax](#rqsrs-006rbacgrantprivilegesyntax) - * 5.2.8.123 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster](#rqsrs-006rbacrevokeprivilegecluster) - * 5.2.8.124 [RQ.SRS-006.RBAC.Revoke.Privilege.Any](#rqsrs-006rbacrevokeprivilegeany) - * 5.2.8.125 [RQ.SRS-006.RBAC.Revoke.Privilege.Select](#rqsrs-006rbacrevokeprivilegeselect) - * 5.2.8.126 [RQ.SRS-006.RBAC.Revoke.Privilege.Insert](#rqsrs-006rbacrevokeprivilegeinsert) - * 5.2.8.127 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter](#rqsrs-006rbacrevokeprivilegealter) - * 5.2.8.128 [RQ.SRS-006.RBAC.Revoke.Privilege.Create](#rqsrs-006rbacrevokeprivilegecreate) - * 5.2.8.129 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Database](#rqsrs-006rbacrevokeprivilegecreatedatabase) - * 5.2.8.130 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Dictionary](#rqsrs-006rbacrevokeprivilegecreatedictionary) - * 5.2.8.131 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Table](#rqsrs-006rbacrevokeprivilegecreatetable) - * 5.2.8.132 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.TemporaryTable](#rqsrs-006rbacrevokeprivilegecreatetemporarytable) - * 5.2.8.133 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop](#rqsrs-006rbacrevokeprivilegedrop) - * 5.2.8.134 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Database](#rqsrs-006rbacrevokeprivilegedropdatabase) - * 5.2.8.135 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Dictionary](#rqsrs-006rbacrevokeprivilegedropdictionary) - * 5.2.8.136 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Table](#rqsrs-006rbacrevokeprivilegedroptable) - * 5.2.8.137 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.TemporaryTable](#rqsrs-006rbacrevokeprivilegedroptemporarytable) - * 5.2.8.138 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate](#rqsrs-006rbacrevokeprivilegetruncate) - * 5.2.8.139 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize](#rqsrs-006rbacrevokeprivilegeoptimize) - * 5.2.8.140 [RQ.SRS-006.RBAC.Revoke.Privilege.Show](#rqsrs-006rbacrevokeprivilegeshow) - * 5.2.8.141 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery](#rqsrs-006rbacrevokeprivilegekillquery) - * 5.2.8.142 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement](#rqsrs-006rbacrevokeprivilegeaccessmanagement) - * 5.2.8.143 [RQ.SRS-006.RBAC.Revoke.Privilege.System](#rqsrs-006rbacrevokeprivilegesystem) - * 5.2.8.144 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection](#rqsrs-006rbacrevokeprivilegeintrospection) - * 5.2.8.145 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources](#rqsrs-006rbacrevokeprivilegesources) - * 5.2.8.146 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet](#rqsrs-006rbacrevokeprivilegedictget) - * 5.2.8.147 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns](#rqsrs-006rbacrevokeprivilegeprivelegecolumns) - * 5.2.8.148 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple](#rqsrs-006rbacrevokeprivilegemultiple) - * 5.2.8.149 [RQ.SRS-006.RBAC.Revoke.Privilege.All](#rqsrs-006rbacrevokeprivilegeall) - * 5.2.8.150 [RQ.SRS-006.RBAC.Revoke.Privilege.None](#rqsrs-006rbacrevokeprivilegenone) - * 5.2.8.151 [RQ.SRS-006.RBAC.Revoke.Privilege.On](#rqsrs-006rbacrevokeprivilegeon) - * 5.2.8.152 [RQ.SRS-006.RBAC.Revoke.Privilege.From](#rqsrs-006rbacrevokeprivilegefrom) - * 5.2.8.153 [RQ.SRS-006.RBAC.Revoke.Privilege.Syntax](#rqsrs-006rbacrevokeprivilegesyntax) - * 5.2.8.154 [RQ.SRS-006.RBAC.PartialRevoke.Syntax](#rqsrs-006rbacpartialrevokesyntax) - * 5.2.8.155 [RQ.SRS-006.RBAC.Grant.Role](#rqsrs-006rbacgrantrole) - * 5.2.8.156 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser](#rqsrs-006rbacgrantrolecurrentuser) - * 5.2.8.157 [RQ.SRS-006.RBAC.Grant.Role.AdminOption](#rqsrs-006rbacgrantroleadminoption) - * 5.2.8.158 [RQ.SRS-006.RBAC.Grant.Role.OnCluster](#rqsrs-006rbacgrantroleoncluster) - * 5.2.8.159 [RQ.SRS-006.RBAC.Grant.Role.Syntax](#rqsrs-006rbacgrantrolesyntax) - * 5.2.8.160 [RQ.SRS-006.RBAC.Revoke.Role](#rqsrs-006rbacrevokerole) - * 5.2.8.161 [RQ.SRS-006.RBAC.Revoke.Role.Keywords](#rqsrs-006rbacrevokerolekeywords) - * 5.2.8.162 [RQ.SRS-006.RBAC.Revoke.Role.Cluster](#rqsrs-006rbacrevokerolecluster) - * 5.2.8.163 [RQ.SRS-006.RBAC.Revoke.AdminOption](#rqsrs-006rbacrevokeadminoption) - * 5.2.8.164 [RQ.SRS-006.RBAC.Revoke.Role.Syntax](#rqsrs-006rbacrevokerolesyntax) - * 5.2.8.165 [RQ.SRS-006.RBAC.Show.Grants](#rqsrs-006rbacshowgrants) - * 5.2.8.166 [RQ.SRS-006.RBAC.Show.Grants.For](#rqsrs-006rbacshowgrantsfor) - * 5.2.8.167 [RQ.SRS-006.RBAC.Show.Grants.Syntax](#rqsrs-006rbacshowgrantssyntax) - * 5.2.8.168 [RQ.SRS-006.RBAC.SettingsProfile.Create](#rqsrs-006rbacsettingsprofilecreate) - * 5.2.8.169 [RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists](#rqsrs-006rbacsettingsprofilecreateifnotexists) - * 5.2.8.170 [RQ.SRS-006.RBAC.SettingsProfile.Create.Replace](#rqsrs-006rbacsettingsprofilecreatereplace) - * 5.2.8.171 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables](#rqsrs-006rbacsettingsprofilecreatevariables) - * 5.2.8.172 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value](#rqsrs-006rbacsettingsprofilecreatevariablesvalue) - * 5.2.8.173 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints](#rqsrs-006rbacsettingsprofilecreatevariablesconstraints) - * 5.2.8.174 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment](#rqsrs-006rbacsettingsprofilecreateassignment) - * 5.2.8.175 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None](#rqsrs-006rbacsettingsprofilecreateassignmentnone) - * 5.2.8.176 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All](#rqsrs-006rbacsettingsprofilecreateassignmentall) - * 5.2.8.177 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilecreateassignmentallexcept) - * 5.2.8.178 [RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit](#rqsrs-006rbacsettingsprofilecreateinherit) - * 5.2.8.179 [RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster](#rqsrs-006rbacsettingsprofilecreateoncluster) - * 5.2.8.180 [RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax](#rqsrs-006rbacsettingsprofilecreatesyntax) - * 5.2.8.181 [RQ.SRS-006.RBAC.SettingsProfile.Alter](#rqsrs-006rbacsettingsprofilealter) - * 5.2.8.182 [RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists](#rqsrs-006rbacsettingsprofilealterifexists) - * 5.2.8.183 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename](#rqsrs-006rbacsettingsprofilealterrename) - * 5.2.8.184 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables](#rqsrs-006rbacsettingsprofilealtervariables) - * 5.2.8.185 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value](#rqsrs-006rbacsettingsprofilealtervariablesvalue) - * 5.2.8.186 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints](#rqsrs-006rbacsettingsprofilealtervariablesconstraints) - * 5.2.8.187 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment](#rqsrs-006rbacsettingsprofilealterassignment) - * 5.2.8.188 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None](#rqsrs-006rbacsettingsprofilealterassignmentnone) - * 5.2.8.189 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All](#rqsrs-006rbacsettingsprofilealterassignmentall) - * 5.2.8.190 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilealterassignmentallexcept) - * 5.2.8.191 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit](#rqsrs-006rbacsettingsprofilealterassignmentinherit) - * 5.2.8.192 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster](#rqsrs-006rbacsettingsprofilealterassignmentoncluster) - * 5.2.8.193 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax](#rqsrs-006rbacsettingsprofilealtersyntax) - * 5.2.8.194 [RQ.SRS-006.RBAC.SettingsProfile.Drop](#rqsrs-006rbacsettingsprofiledrop) - * 5.2.8.195 [RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists](#rqsrs-006rbacsettingsprofiledropifexists) - * 5.2.8.196 [RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster](#rqsrs-006rbacsettingsprofiledroponcluster) - * 5.2.8.197 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax](#rqsrs-006rbacsettingsprofiledropsyntax) - * 5.2.8.198 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile](#rqsrs-006rbacsettingsprofileshowcreatesettingsprofile) - * 5.2.8.199 [RQ.SRS-006.RBAC.Quota.Create](#rqsrs-006rbacquotacreate) - * 5.2.8.200 [RQ.SRS-006.RBAC.Quota.Create.IfNotExists](#rqsrs-006rbacquotacreateifnotexists) - * 5.2.8.201 [RQ.SRS-006.RBAC.Quota.Create.Replace](#rqsrs-006rbacquotacreatereplace) - * 5.2.8.202 [RQ.SRS-006.RBAC.Quota.Create.Cluster](#rqsrs-006rbacquotacreatecluster) - * 5.2.8.203 [RQ.SRS-006.RBAC.Quota.Create.Interval](#rqsrs-006rbacquotacreateinterval) - * 5.2.8.204 [RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized](#rqsrs-006rbacquotacreateintervalrandomized) - * 5.2.8.205 [RQ.SRS-006.RBAC.Quota.Create.Queries](#rqsrs-006rbacquotacreatequeries) - * 5.2.8.206 [RQ.SRS-006.RBAC.Quota.Create.Errors](#rqsrs-006rbacquotacreateerrors) - * 5.2.8.207 [RQ.SRS-006.RBAC.Quota.Create.ResultRows](#rqsrs-006rbacquotacreateresultrows) - * 5.2.8.208 [RQ.SRS-006.RBAC.Quota.Create.ReadRows](#rqsrs-006rbacquotacreatereadrows) - * 5.2.8.209 [RQ.SRS-006.RBAC.Quota.Create.ResultBytes](#rqsrs-006rbacquotacreateresultbytes) - * 5.2.8.210 [RQ.SRS-006.RBAC.Quota.Create.ReadBytes](#rqsrs-006rbacquotacreatereadbytes) - * 5.2.8.211 [RQ.SRS-006.RBAC.Quota.Create.ExecutionTime](#rqsrs-006rbacquotacreateexecutiontime) - * 5.2.8.212 [RQ.SRS-006.RBAC.Quota.Create.NoLimits](#rqsrs-006rbacquotacreatenolimits) - * 5.2.8.213 [RQ.SRS-006.RBAC.Quota.Create.TrackingOnly](#rqsrs-006rbacquotacreatetrackingonly) - * 5.2.8.214 [RQ.SRS-006.RBAC.Quota.Create.KeyedBy](#rqsrs-006rbacquotacreatekeyedby) - * 5.2.8.215 [RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions](#rqsrs-006rbacquotacreatekeyedbyoptions) - * 5.2.8.216 [RQ.SRS-006.RBAC.Quota.Create.Assignment](#rqsrs-006rbacquotacreateassignment) - * 5.2.8.217 [RQ.SRS-006.RBAC.Quota.Create.Assignment.None](#rqsrs-006rbacquotacreateassignmentnone) - * 5.2.8.218 [RQ.SRS-006.RBAC.Quota.Create.Assignment.All](#rqsrs-006rbacquotacreateassignmentall) - * 5.2.8.219 [RQ.SRS-006.RBAC.Quota.Create.Assignment.Except](#rqsrs-006rbacquotacreateassignmentexcept) - * 5.2.8.220 [RQ.SRS-006.RBAC.Quota.Create.Syntax](#rqsrs-006rbacquotacreatesyntax) - * 5.2.8.221 [RQ.SRS-006.RBAC.Quota.Alter](#rqsrs-006rbacquotaalter) - * 5.2.8.222 [RQ.SRS-006.RBAC.Quota.Alter.IfExists](#rqsrs-006rbacquotaalterifexists) - * 5.2.8.223 [RQ.SRS-006.RBAC.Quota.Alter.Rename](#rqsrs-006rbacquotaalterrename) - * 5.2.8.224 [RQ.SRS-006.RBAC.Quota.Alter.Cluster](#rqsrs-006rbacquotaaltercluster) - * 5.2.8.225 [RQ.SRS-006.RBAC.Quota.Alter.Interval](#rqsrs-006rbacquotaalterinterval) - * 5.2.8.226 [RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized](#rqsrs-006rbacquotaalterintervalrandomized) - * 5.2.8.227 [RQ.SRS-006.RBAC.Quota.Alter.Queries](#rqsrs-006rbacquotaalterqueries) - * 5.2.8.228 [RQ.SRS-006.RBAC.Quota.Alter.Errors](#rqsrs-006rbacquotaaltererrors) - * 5.2.8.229 [RQ.SRS-006.RBAC.Quota.Alter.ResultRows](#rqsrs-006rbacquotaalterresultrows) - * 5.2.8.230 [RQ.SRS-006.RBAC.Quota.Alter.ReadRows](#rqsrs-006rbacquotaalterreadrows) - * 5.2.8.231 [RQ.SRS-006.RBAC.Quota.ALter.ResultBytes](#rqsrs-006rbacquotaalterresultbytes) - * 5.2.8.232 [RQ.SRS-006.RBAC.Quota.Alter.ReadBytes](#rqsrs-006rbacquotaalterreadbytes) - * 5.2.8.233 [RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime](#rqsrs-006rbacquotaalterexecutiontime) - * 5.2.8.234 [RQ.SRS-006.RBAC.Quota.Alter.NoLimits](#rqsrs-006rbacquotaalternolimits) - * 5.2.8.235 [RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly](#rqsrs-006rbacquotaaltertrackingonly) - * 5.2.8.236 [RQ.SRS-006.RBAC.Quota.Alter.KeyedBy](#rqsrs-006rbacquotaalterkeyedby) - * 5.2.8.237 [RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions](#rqsrs-006rbacquotaalterkeyedbyoptions) - * 5.2.8.238 [RQ.SRS-006.RBAC.Quota.Alter.Assignment](#rqsrs-006rbacquotaalterassignment) - * 5.2.8.239 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.None](#rqsrs-006rbacquotaalterassignmentnone) - * 5.2.8.240 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.All](#rqsrs-006rbacquotaalterassignmentall) - * 5.2.8.241 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except](#rqsrs-006rbacquotaalterassignmentexcept) - * 5.2.8.242 [RQ.SRS-006.RBAC.Quota.Alter.Syntax](#rqsrs-006rbacquotaaltersyntax) - * 5.2.8.243 [RQ.SRS-006.RBAC.Quota.Drop](#rqsrs-006rbacquotadrop) - * 5.2.8.244 [RQ.SRS-006.RBAC.Quota.Drop.IfExists](#rqsrs-006rbacquotadropifexists) - * 5.2.8.245 [RQ.SRS-006.RBAC.Quota.Drop.Cluster](#rqsrs-006rbacquotadropcluster) - * 5.2.8.246 [RQ.SRS-006.RBAC.Quota.Drop.Syntax](#rqsrs-006rbacquotadropsyntax) - * 5.2.8.247 [RQ.SRS-006.RBAC.Quota.ShowQuotas](#rqsrs-006rbacquotashowquotas) - * 5.2.8.248 [RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile](#rqsrs-006rbacquotashowquotasintooutfile) - * 5.2.8.249 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Format](#rqsrs-006rbacquotashowquotasformat) - * 5.2.8.250 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings](#rqsrs-006rbacquotashowquotassettings) - * 5.2.8.251 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax](#rqsrs-006rbacquotashowquotassyntax) - * 5.2.8.252 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name](#rqsrs-006rbacquotashowcreatequotaname) - * 5.2.8.253 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current](#rqsrs-006rbacquotashowcreatequotacurrent) - * 5.2.8.254 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax](#rqsrs-006rbacquotashowcreatequotasyntax) - * 5.2.8.255 [RQ.SRS-006.RBAC.RowPolicy.Create](#rqsrs-006rbacrowpolicycreate) - * 5.2.8.256 [RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists](#rqsrs-006rbacrowpolicycreateifnotexists) - * 5.2.8.257 [RQ.SRS-006.RBAC.RowPolicy.Create.Replace](#rqsrs-006rbacrowpolicycreatereplace) - * 5.2.8.258 [RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster](#rqsrs-006rbacrowpolicycreateoncluster) - * 5.2.8.259 [RQ.SRS-006.RBAC.RowPolicy.Create.On](#rqsrs-006rbacrowpolicycreateon) - * 5.2.8.260 [RQ.SRS-006.RBAC.RowPolicy.Create.Access](#rqsrs-006rbacrowpolicycreateaccess) - * 5.2.8.261 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive](#rqsrs-006rbacrowpolicycreateaccesspermissive) - * 5.2.8.262 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive](#rqsrs-006rbacrowpolicycreateaccessrestrictive) - * 5.2.8.263 [RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect](#rqsrs-006rbacrowpolicycreateforselect) - * 5.2.8.264 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition](#rqsrs-006rbacrowpolicycreatecondition) - * 5.2.8.265 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment](#rqsrs-006rbacrowpolicycreateassignment) - * 5.2.8.266 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None](#rqsrs-006rbacrowpolicycreateassignmentnone) - * 5.2.8.267 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All](#rqsrs-006rbacrowpolicycreateassignmentall) - * 5.2.8.268 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept](#rqsrs-006rbacrowpolicycreateassignmentallexcept) - * 5.2.8.269 [RQ.SRS-006.RBAC.RowPolicy.Create.Syntax](#rqsrs-006rbacrowpolicycreatesyntax) - * 5.2.8.270 [RQ.SRS-006.RBAC.RowPolicy.Alter](#rqsrs-006rbacrowpolicyalter) - * 5.2.8.271 [RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists](#rqsrs-006rbacrowpolicyalterifexists) - * 5.2.8.272 [RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect](#rqsrs-006rbacrowpolicyalterforselect) - * 5.2.8.273 [RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster](#rqsrs-006rbacrowpolicyalteroncluster) - * 5.2.8.274 [RQ.SRS-006.RBAC.RowPolicy.Alter.On](#rqsrs-006rbacrowpolicyalteron) - * 5.2.8.275 [RQ.SRS-006.RBAC.RowPolicy.Alter.Rename](#rqsrs-006rbacrowpolicyalterrename) - * 5.2.8.276 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access](#rqsrs-006rbacrowpolicyalteraccess) - * 5.2.8.277 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive](#rqsrs-006rbacrowpolicyalteraccesspermissive) - * 5.2.8.278 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive](#rqsrs-006rbacrowpolicyalteraccessrestrictive) - * 5.2.8.279 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition](#rqsrs-006rbacrowpolicyaltercondition) - * 5.2.8.280 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None](#rqsrs-006rbacrowpolicyalterconditionnone) - * 5.2.8.281 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment](#rqsrs-006rbacrowpolicyalterassignment) - * 5.2.8.282 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None](#rqsrs-006rbacrowpolicyalterassignmentnone) - * 5.2.8.283 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All](#rqsrs-006rbacrowpolicyalterassignmentall) - * 5.2.8.284 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept](#rqsrs-006rbacrowpolicyalterassignmentallexcept) - * 5.2.8.285 [RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax](#rqsrs-006rbacrowpolicyaltersyntax) - * 5.2.8.286 [RQ.SRS-006.RBAC.RowPolicy.Drop](#rqsrs-006rbacrowpolicydrop) - * 5.2.8.287 [RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists](#rqsrs-006rbacrowpolicydropifexists) - * 5.2.8.288 [RQ.SRS-006.RBAC.RowPolicy.Drop.On](#rqsrs-006rbacrowpolicydropon) - * 5.2.8.289 [RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster](#rqsrs-006rbacrowpolicydroponcluster) - * 5.2.8.290 [RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax](#rqsrs-006rbacrowpolicydropsyntax) - * 5.2.8.291 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy](#rqsrs-006rbacrowpolicyshowcreaterowpolicy) - * 5.2.8.292 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On](#rqsrs-006rbacrowpolicyshowcreaterowpolicyon) - * 5.2.8.293 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax](#rqsrs-006rbacrowpolicyshowcreaterowpolicysyntax) - * 5.2.8.294 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies](#rqsrs-006rbacrowpolicyshowrowpolicies) - * 5.2.8.295 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On](#rqsrs-006rbacrowpolicyshowrowpolicieson) - * 5.2.8.296 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) + * 5.2.8.98 [RQ.SRS-006.RBAC.Grant.Privilege.Drop](#rqsrs-006rbacgrantprivilegedrop) + * 5.2.8.99 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate](#rqsrs-006rbacgrantprivilegetruncate) + * 5.2.8.100 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize](#rqsrs-006rbacgrantprivilegeoptimize) + * 5.2.8.101 [RQ.SRS-006.RBAC.Grant.Privilege.Show](#rqsrs-006rbacgrantprivilegeshow) + * 5.2.8.102 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery](#rqsrs-006rbacgrantprivilegekillquery) + * 5.2.8.103 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement](#rqsrs-006rbacgrantprivilegeaccessmanagement) + * 5.2.8.104 [RQ.SRS-006.RBAC.Grant.Privilege.System](#rqsrs-006rbacgrantprivilegesystem) + * 5.2.8.105 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection](#rqsrs-006rbacgrantprivilegeintrospection) + * 5.2.8.106 [RQ.SRS-006.RBAC.Grant.Privilege.Sources](#rqsrs-006rbacgrantprivilegesources) + * 5.2.8.107 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet](#rqsrs-006rbacgrantprivilegedictget) + * 5.2.8.108 [RQ.SRS-006.RBAC.Grant.Privilege.None](#rqsrs-006rbacgrantprivilegenone) + * 5.2.8.109 [RQ.SRS-006.RBAC.Grant.Privilege.All](#rqsrs-006rbacgrantprivilegeall) + * 5.2.8.110 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption](#rqsrs-006rbacgrantprivilegegrantoption) + * 5.2.8.111 [RQ.SRS-006.RBAC.Grant.Privilege.On](#rqsrs-006rbacgrantprivilegeon) + * 5.2.8.112 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns](#rqsrs-006rbacgrantprivilegeprivilegecolumns) + * 5.2.8.113 [RQ.SRS-006.RBAC.Grant.Privilege.OnCluster](#rqsrs-006rbacgrantprivilegeoncluster) + * 5.2.8.114 [RQ.SRS-006.RBAC.Grant.Privilege.Syntax](#rqsrs-006rbacgrantprivilegesyntax) + * 5.2.8.115 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster](#rqsrs-006rbacrevokeprivilegecluster) + * 5.2.8.116 [RQ.SRS-006.RBAC.Revoke.Privilege.Any](#rqsrs-006rbacrevokeprivilegeany) + * 5.2.8.117 [RQ.SRS-006.RBAC.Revoke.Privilege.Select](#rqsrs-006rbacrevokeprivilegeselect) + * 5.2.8.118 [RQ.SRS-006.RBAC.Revoke.Privilege.Insert](#rqsrs-006rbacrevokeprivilegeinsert) + * 5.2.8.119 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter](#rqsrs-006rbacrevokeprivilegealter) + * 5.2.8.120 [RQ.SRS-006.RBAC.Revoke.Privilege.Create](#rqsrs-006rbacrevokeprivilegecreate) + * 5.2.8.121 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop](#rqsrs-006rbacrevokeprivilegedrop) + * 5.2.8.122 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate](#rqsrs-006rbacrevokeprivilegetruncate) + * 5.2.8.123 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize](#rqsrs-006rbacrevokeprivilegeoptimize) + * 5.2.8.124 [RQ.SRS-006.RBAC.Revoke.Privilege.Show](#rqsrs-006rbacrevokeprivilegeshow) + * 5.2.8.125 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery](#rqsrs-006rbacrevokeprivilegekillquery) + * 5.2.8.126 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement](#rqsrs-006rbacrevokeprivilegeaccessmanagement) + * 5.2.8.127 [RQ.SRS-006.RBAC.Revoke.Privilege.System](#rqsrs-006rbacrevokeprivilegesystem) + * 5.2.8.128 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection](#rqsrs-006rbacrevokeprivilegeintrospection) + * 5.2.8.129 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources](#rqsrs-006rbacrevokeprivilegesources) + * 5.2.8.130 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet](#rqsrs-006rbacrevokeprivilegedictget) + * 5.2.8.131 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns](#rqsrs-006rbacrevokeprivilegeprivelegecolumns) + * 5.2.8.132 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple](#rqsrs-006rbacrevokeprivilegemultiple) + * 5.2.8.133 [RQ.SRS-006.RBAC.Revoke.Privilege.All](#rqsrs-006rbacrevokeprivilegeall) + * 5.2.8.134 [RQ.SRS-006.RBAC.Revoke.Privilege.None](#rqsrs-006rbacrevokeprivilegenone) + * 5.2.8.135 [RQ.SRS-006.RBAC.Revoke.Privilege.On](#rqsrs-006rbacrevokeprivilegeon) + * 5.2.8.136 [RQ.SRS-006.RBAC.Revoke.Privilege.From](#rqsrs-006rbacrevokeprivilegefrom) + * 5.2.8.137 [RQ.SRS-006.RBAC.Revoke.Privilege.Syntax](#rqsrs-006rbacrevokeprivilegesyntax) + * 5.2.8.138 [RQ.SRS-006.RBAC.PartialRevoke.Syntax](#rqsrs-006rbacpartialrevokesyntax) + * 5.2.8.139 [RQ.SRS-006.RBAC.Grant.Role](#rqsrs-006rbacgrantrole) + * 5.2.8.140 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser](#rqsrs-006rbacgrantrolecurrentuser) + * 5.2.8.141 [RQ.SRS-006.RBAC.Grant.Role.AdminOption](#rqsrs-006rbacgrantroleadminoption) + * 5.2.8.142 [RQ.SRS-006.RBAC.Grant.Role.OnCluster](#rqsrs-006rbacgrantroleoncluster) + * 5.2.8.143 [RQ.SRS-006.RBAC.Grant.Role.Syntax](#rqsrs-006rbacgrantrolesyntax) + * 5.2.8.144 [RQ.SRS-006.RBAC.Revoke.Role](#rqsrs-006rbacrevokerole) + * 5.2.8.145 [RQ.SRS-006.RBAC.Revoke.Role.Keywords](#rqsrs-006rbacrevokerolekeywords) + * 5.2.8.146 [RQ.SRS-006.RBAC.Revoke.Role.Cluster](#rqsrs-006rbacrevokerolecluster) + * 5.2.8.147 [RQ.SRS-006.RBAC.Revoke.AdminOption](#rqsrs-006rbacrevokeadminoption) + * 5.2.8.148 [RQ.SRS-006.RBAC.Revoke.Role.Syntax](#rqsrs-006rbacrevokerolesyntax) + * 5.2.8.149 [RQ.SRS-006.RBAC.Show.Grants](#rqsrs-006rbacshowgrants) + * 5.2.8.150 [RQ.SRS-006.RBAC.Show.Grants.For](#rqsrs-006rbacshowgrantsfor) + * 5.2.8.151 [RQ.SRS-006.RBAC.Show.Grants.Syntax](#rqsrs-006rbacshowgrantssyntax) + * 5.2.8.152 [RQ.SRS-006.RBAC.SettingsProfile.Create](#rqsrs-006rbacsettingsprofilecreate) + * 5.2.8.153 [RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists](#rqsrs-006rbacsettingsprofilecreateifnotexists) + * 5.2.8.154 [RQ.SRS-006.RBAC.SettingsProfile.Create.Replace](#rqsrs-006rbacsettingsprofilecreatereplace) + * 5.2.8.155 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables](#rqsrs-006rbacsettingsprofilecreatevariables) + * 5.2.8.156 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value](#rqsrs-006rbacsettingsprofilecreatevariablesvalue) + * 5.2.8.157 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints](#rqsrs-006rbacsettingsprofilecreatevariablesconstraints) + * 5.2.8.158 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment](#rqsrs-006rbacsettingsprofilecreateassignment) + * 5.2.8.159 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None](#rqsrs-006rbacsettingsprofilecreateassignmentnone) + * 5.2.8.160 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All](#rqsrs-006rbacsettingsprofilecreateassignmentall) + * 5.2.8.161 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilecreateassignmentallexcept) + * 5.2.8.162 [RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit](#rqsrs-006rbacsettingsprofilecreateinherit) + * 5.2.8.163 [RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster](#rqsrs-006rbacsettingsprofilecreateoncluster) + * 5.2.8.164 [RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax](#rqsrs-006rbacsettingsprofilecreatesyntax) + * 5.2.8.165 [RQ.SRS-006.RBAC.SettingsProfile.Alter](#rqsrs-006rbacsettingsprofilealter) + * 5.2.8.166 [RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists](#rqsrs-006rbacsettingsprofilealterifexists) + * 5.2.8.167 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename](#rqsrs-006rbacsettingsprofilealterrename) + * 5.2.8.168 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables](#rqsrs-006rbacsettingsprofilealtervariables) + * 5.2.8.169 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value](#rqsrs-006rbacsettingsprofilealtervariablesvalue) + * 5.2.8.170 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints](#rqsrs-006rbacsettingsprofilealtervariablesconstraints) + * 5.2.8.171 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment](#rqsrs-006rbacsettingsprofilealterassignment) + * 5.2.8.172 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None](#rqsrs-006rbacsettingsprofilealterassignmentnone) + * 5.2.8.173 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All](#rqsrs-006rbacsettingsprofilealterassignmentall) + * 5.2.8.174 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilealterassignmentallexcept) + * 5.2.8.175 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit](#rqsrs-006rbacsettingsprofilealterassignmentinherit) + * 5.2.8.176 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster](#rqsrs-006rbacsettingsprofilealterassignmentoncluster) + * 5.2.8.177 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax](#rqsrs-006rbacsettingsprofilealtersyntax) + * 5.2.8.178 [RQ.SRS-006.RBAC.SettingsProfile.Drop](#rqsrs-006rbacsettingsprofiledrop) + * 5.2.8.179 [RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists](#rqsrs-006rbacsettingsprofiledropifexists) + * 5.2.8.180 [RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster](#rqsrs-006rbacsettingsprofiledroponcluster) + * 5.2.8.181 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax](#rqsrs-006rbacsettingsprofiledropsyntax) + * 5.2.8.182 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile](#rqsrs-006rbacsettingsprofileshowcreatesettingsprofile) + * 5.2.8.183 [RQ.SRS-006.RBAC.Quota.Create](#rqsrs-006rbacquotacreate) + * 5.2.8.184 [RQ.SRS-006.RBAC.Quota.Create.IfNotExists](#rqsrs-006rbacquotacreateifnotexists) + * 5.2.8.185 [RQ.SRS-006.RBAC.Quota.Create.Replace](#rqsrs-006rbacquotacreatereplace) + * 5.2.8.186 [RQ.SRS-006.RBAC.Quota.Create.Cluster](#rqsrs-006rbacquotacreatecluster) + * 5.2.8.187 [RQ.SRS-006.RBAC.Quota.Create.Interval](#rqsrs-006rbacquotacreateinterval) + * 5.2.8.188 [RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized](#rqsrs-006rbacquotacreateintervalrandomized) + * 5.2.8.189 [RQ.SRS-006.RBAC.Quota.Create.Queries](#rqsrs-006rbacquotacreatequeries) + * 5.2.8.190 [RQ.SRS-006.RBAC.Quota.Create.Errors](#rqsrs-006rbacquotacreateerrors) + * 5.2.8.191 [RQ.SRS-006.RBAC.Quota.Create.ResultRows](#rqsrs-006rbacquotacreateresultrows) + * 5.2.8.192 [RQ.SRS-006.RBAC.Quota.Create.ReadRows](#rqsrs-006rbacquotacreatereadrows) + * 5.2.8.193 [RQ.SRS-006.RBAC.Quota.Create.ResultBytes](#rqsrs-006rbacquotacreateresultbytes) + * 5.2.8.194 [RQ.SRS-006.RBAC.Quota.Create.ReadBytes](#rqsrs-006rbacquotacreatereadbytes) + * 5.2.8.195 [RQ.SRS-006.RBAC.Quota.Create.ExecutionTime](#rqsrs-006rbacquotacreateexecutiontime) + * 5.2.8.196 [RQ.SRS-006.RBAC.Quota.Create.NoLimits](#rqsrs-006rbacquotacreatenolimits) + * 5.2.8.197 [RQ.SRS-006.RBAC.Quota.Create.TrackingOnly](#rqsrs-006rbacquotacreatetrackingonly) + * 5.2.8.198 [RQ.SRS-006.RBAC.Quota.Create.KeyedBy](#rqsrs-006rbacquotacreatekeyedby) + * 5.2.8.199 [RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions](#rqsrs-006rbacquotacreatekeyedbyoptions) + * 5.2.8.200 [RQ.SRS-006.RBAC.Quota.Create.Assignment](#rqsrs-006rbacquotacreateassignment) + * 5.2.8.201 [RQ.SRS-006.RBAC.Quota.Create.Assignment.None](#rqsrs-006rbacquotacreateassignmentnone) + * 5.2.8.202 [RQ.SRS-006.RBAC.Quota.Create.Assignment.All](#rqsrs-006rbacquotacreateassignmentall) + * 5.2.8.203 [RQ.SRS-006.RBAC.Quota.Create.Assignment.Except](#rqsrs-006rbacquotacreateassignmentexcept) + * 5.2.8.204 [RQ.SRS-006.RBAC.Quota.Create.Syntax](#rqsrs-006rbacquotacreatesyntax) + * 5.2.8.205 [RQ.SRS-006.RBAC.Quota.Alter](#rqsrs-006rbacquotaalter) + * 5.2.8.206 [RQ.SRS-006.RBAC.Quota.Alter.IfExists](#rqsrs-006rbacquotaalterifexists) + * 5.2.8.207 [RQ.SRS-006.RBAC.Quota.Alter.Rename](#rqsrs-006rbacquotaalterrename) + * 5.2.8.208 [RQ.SRS-006.RBAC.Quota.Alter.Cluster](#rqsrs-006rbacquotaaltercluster) + * 5.2.8.209 [RQ.SRS-006.RBAC.Quota.Alter.Interval](#rqsrs-006rbacquotaalterinterval) + * 5.2.8.210 [RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized](#rqsrs-006rbacquotaalterintervalrandomized) + * 5.2.8.211 [RQ.SRS-006.RBAC.Quota.Alter.Queries](#rqsrs-006rbacquotaalterqueries) + * 5.2.8.212 [RQ.SRS-006.RBAC.Quota.Alter.Errors](#rqsrs-006rbacquotaaltererrors) + * 5.2.8.213 [RQ.SRS-006.RBAC.Quota.Alter.ResultRows](#rqsrs-006rbacquotaalterresultrows) + * 5.2.8.214 [RQ.SRS-006.RBAC.Quota.Alter.ReadRows](#rqsrs-006rbacquotaalterreadrows) + * 5.2.8.215 [RQ.SRS-006.RBAC.Quota.ALter.ResultBytes](#rqsrs-006rbacquotaalterresultbytes) + * 5.2.8.216 [RQ.SRS-006.RBAC.Quota.Alter.ReadBytes](#rqsrs-006rbacquotaalterreadbytes) + * 5.2.8.217 [RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime](#rqsrs-006rbacquotaalterexecutiontime) + * 5.2.8.218 [RQ.SRS-006.RBAC.Quota.Alter.NoLimits](#rqsrs-006rbacquotaalternolimits) + * 5.2.8.219 [RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly](#rqsrs-006rbacquotaaltertrackingonly) + * 5.2.8.220 [RQ.SRS-006.RBAC.Quota.Alter.KeyedBy](#rqsrs-006rbacquotaalterkeyedby) + * 5.2.8.221 [RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions](#rqsrs-006rbacquotaalterkeyedbyoptions) + * 5.2.8.222 [RQ.SRS-006.RBAC.Quota.Alter.Assignment](#rqsrs-006rbacquotaalterassignment) + * 5.2.8.223 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.None](#rqsrs-006rbacquotaalterassignmentnone) + * 5.2.8.224 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.All](#rqsrs-006rbacquotaalterassignmentall) + * 5.2.8.225 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except](#rqsrs-006rbacquotaalterassignmentexcept) + * 5.2.8.226 [RQ.SRS-006.RBAC.Quota.Alter.Syntax](#rqsrs-006rbacquotaaltersyntax) + * 5.2.8.227 [RQ.SRS-006.RBAC.Quota.Drop](#rqsrs-006rbacquotadrop) + * 5.2.8.228 [RQ.SRS-006.RBAC.Quota.Drop.IfExists](#rqsrs-006rbacquotadropifexists) + * 5.2.8.229 [RQ.SRS-006.RBAC.Quota.Drop.Cluster](#rqsrs-006rbacquotadropcluster) + * 5.2.8.230 [RQ.SRS-006.RBAC.Quota.Drop.Syntax](#rqsrs-006rbacquotadropsyntax) + * 5.2.8.231 [RQ.SRS-006.RBAC.Quota.ShowQuotas](#rqsrs-006rbacquotashowquotas) + * 5.2.8.232 [RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile](#rqsrs-006rbacquotashowquotasintooutfile) + * 5.2.8.233 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Format](#rqsrs-006rbacquotashowquotasformat) + * 5.2.8.234 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings](#rqsrs-006rbacquotashowquotassettings) + * 5.2.8.235 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax](#rqsrs-006rbacquotashowquotassyntax) + * 5.2.8.236 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name](#rqsrs-006rbacquotashowcreatequotaname) + * 5.2.8.237 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current](#rqsrs-006rbacquotashowcreatequotacurrent) + * 5.2.8.238 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax](#rqsrs-006rbacquotashowcreatequotasyntax) + * 5.2.8.239 [RQ.SRS-006.RBAC.RowPolicy.Create](#rqsrs-006rbacrowpolicycreate) + * 5.2.8.240 [RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists](#rqsrs-006rbacrowpolicycreateifnotexists) + * 5.2.8.241 [RQ.SRS-006.RBAC.RowPolicy.Create.Replace](#rqsrs-006rbacrowpolicycreatereplace) + * 5.2.8.242 [RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster](#rqsrs-006rbacrowpolicycreateoncluster) + * 5.2.8.243 [RQ.SRS-006.RBAC.RowPolicy.Create.On](#rqsrs-006rbacrowpolicycreateon) + * 5.2.8.244 [RQ.SRS-006.RBAC.RowPolicy.Create.Access](#rqsrs-006rbacrowpolicycreateaccess) + * 5.2.8.245 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive](#rqsrs-006rbacrowpolicycreateaccesspermissive) + * 5.2.8.246 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive](#rqsrs-006rbacrowpolicycreateaccessrestrictive) + * 5.2.8.247 [RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect](#rqsrs-006rbacrowpolicycreateforselect) + * 5.2.8.248 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition](#rqsrs-006rbacrowpolicycreatecondition) + * 5.2.8.249 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment](#rqsrs-006rbacrowpolicycreateassignment) + * 5.2.8.250 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None](#rqsrs-006rbacrowpolicycreateassignmentnone) + * 5.2.8.251 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All](#rqsrs-006rbacrowpolicycreateassignmentall) + * 5.2.8.252 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept](#rqsrs-006rbacrowpolicycreateassignmentallexcept) + * 5.2.8.253 [RQ.SRS-006.RBAC.RowPolicy.Create.Syntax](#rqsrs-006rbacrowpolicycreatesyntax) + * 5.2.8.254 [RQ.SRS-006.RBAC.RowPolicy.Alter](#rqsrs-006rbacrowpolicyalter) + * 5.2.8.255 [RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists](#rqsrs-006rbacrowpolicyalterifexists) + * 5.2.8.256 [RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect](#rqsrs-006rbacrowpolicyalterforselect) + * 5.2.8.257 [RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster](#rqsrs-006rbacrowpolicyalteroncluster) + * 5.2.8.258 [RQ.SRS-006.RBAC.RowPolicy.Alter.On](#rqsrs-006rbacrowpolicyalteron) + * 5.2.8.259 [RQ.SRS-006.RBAC.RowPolicy.Alter.Rename](#rqsrs-006rbacrowpolicyalterrename) + * 5.2.8.260 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access](#rqsrs-006rbacrowpolicyalteraccess) + * 5.2.8.261 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive](#rqsrs-006rbacrowpolicyalteraccesspermissive) + * 5.2.8.262 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive](#rqsrs-006rbacrowpolicyalteraccessrestrictive) + * 5.2.8.263 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition](#rqsrs-006rbacrowpolicyaltercondition) + * 5.2.8.264 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None](#rqsrs-006rbacrowpolicyalterconditionnone) + * 5.2.8.265 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment](#rqsrs-006rbacrowpolicyalterassignment) + * 5.2.8.266 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None](#rqsrs-006rbacrowpolicyalterassignmentnone) + * 5.2.8.267 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All](#rqsrs-006rbacrowpolicyalterassignmentall) + * 5.2.8.268 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept](#rqsrs-006rbacrowpolicyalterassignmentallexcept) + * 5.2.8.269 [RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax](#rqsrs-006rbacrowpolicyaltersyntax) + * 5.2.8.270 [RQ.SRS-006.RBAC.RowPolicy.Drop](#rqsrs-006rbacrowpolicydrop) + * 5.2.8.271 [RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists](#rqsrs-006rbacrowpolicydropifexists) + * 5.2.8.272 [RQ.SRS-006.RBAC.RowPolicy.Drop.On](#rqsrs-006rbacrowpolicydropon) + * 5.2.8.273 [RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster](#rqsrs-006rbacrowpolicydroponcluster) + * 5.2.8.274 [RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax](#rqsrs-006rbacrowpolicydropsyntax) + * 5.2.8.275 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy](#rqsrs-006rbacrowpolicyshowcreaterowpolicy) + * 5.2.8.276 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On](#rqsrs-006rbacrowpolicyshowcreaterowpolicyon) + * 5.2.8.277 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax](#rqsrs-006rbacrowpolicyshowcreaterowpolicysyntax) + * 5.2.8.278 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies](#rqsrs-006rbacrowpolicyshowrowpolicies) + * 5.2.8.279 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On](#rqsrs-006rbacrowpolicyshowrowpolicieson) + * 5.2.8.280 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) * 5.2.9 [Table Privileges](#table-privileges) * 5.2.9.1 [RQ.SRS-006.RBAC.Table.PublicTables](#rqsrs-006rbactablepublictables) - * 5.2.9.2 [RQ.SRS-006.RBAC.Table.QueryLog](#rqsrs-006rbactablequerylog) - * 5.2.9.3 [Distributed Tables](#distributed-tables) - * 5.2.9.3.1 [RQ.SRS-006.RBAC.Table.DistributedTable.Create](#rqsrs-006rbactabledistributedtablecreate) - * 5.2.9.3.2 [RQ.SRS-006.RBAC.Table.DistributedTable.Select](#rqsrs-006rbactabledistributedtableselect) - * 5.2.9.3.3 [RQ.SRS-006.RBAC.Table.DistributedTable.Insert](#rqsrs-006rbactabledistributedtableinsert) - * 5.2.9.3.4 [RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables](#rqsrs-006rbactabledistributedtablespecialtables) - * 5.2.9.3.5 [RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser](#rqsrs-006rbactabledistributedtablelocaluser) - * 5.2.9.3.6 [RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbactabledistributedtablesameuserdifferentnodesdifferentprivileges) - * 5.2.10 [Views](#views) - * 5.2.10.1 [View](#view) - * 5.2.10.1.1 [RQ.SRS-006.RBAC.View](#rqsrs-006rbacview) - * 5.2.10.1.2 [RQ.SRS-006.RBAC.View.Create](#rqsrs-006rbacviewcreate) - * 5.2.10.1.3 [RQ.SRS-006.RBAC.View.Select](#rqsrs-006rbacviewselect) - * 5.2.10.1.4 [RQ.SRS-006.RBAC.View.Drop](#rqsrs-006rbacviewdrop) - * 5.2.10.2 [Materialized View](#materialized-view) - * 5.2.10.2.1 [RQ.SRS-006.RBAC.MaterializedView](#rqsrs-006rbacmaterializedview) - * 5.2.10.2.2 [RQ.SRS-006.RBAC.MaterializedView.Create](#rqsrs-006rbacmaterializedviewcreate) - * 5.2.10.2.3 [RQ.SRS-006.RBAC.MaterializedView.Select](#rqsrs-006rbacmaterializedviewselect) - * 5.2.10.2.4 [RQ.SRS-006.RBAC.MaterializedView.Select.TargetTable](#rqsrs-006rbacmaterializedviewselecttargettable) - * 5.2.10.2.5 [RQ.SRS-006.RBAC.MaterializedView.Select.SourceTable](#rqsrs-006rbacmaterializedviewselectsourcetable) - * 5.2.10.2.6 [RQ.SRS-006.RBAC.MaterializedView.Drop](#rqsrs-006rbacmaterializedviewdrop) - * 5.2.10.2.7 [RQ.SRS-006.RBAC.MaterializedView.ModifyQuery](#rqsrs-006rbacmaterializedviewmodifyquery) - * 5.2.10.2.8 [RQ.SRS-006.RBAC.MaterializedView.Insert](#rqsrs-006rbacmaterializedviewinsert) - * 5.2.10.2.9 [RQ.SRS-006.RBAC.MaterializedView.Insert.SourceTable](#rqsrs-006rbacmaterializedviewinsertsourcetable) - * 5.2.10.2.10 [RQ.SRS-006.RBAC.MaterializedView.Insert.TargetTable](#rqsrs-006rbacmaterializedviewinserttargettable) - * 5.2.10.3 [Live View](#live-view) - * 5.2.10.3.1 [RQ.SRS-006.RBAC.LiveView](#rqsrs-006rbacliveview) - * 5.2.10.3.2 [RQ.SRS-006.RBAC.LiveView.Create](#rqsrs-006rbacliveviewcreate) - * 5.2.10.3.3 [RQ.SRS-006.RBAC.LiveView.Select](#rqsrs-006rbacliveviewselect) - * 5.2.10.3.4 [RQ.SRS-006.RBAC.LiveView.Drop](#rqsrs-006rbacliveviewdrop) - * 5.2.10.3.5 [RQ.SRS-006.RBAC.LiveView.Refresh](#rqsrs-006rbacliveviewrefresh) - * 5.2.11 [Privileges](#privileges) - * 5.2.11.1 [RQ.SRS-006.RBAC.Privileges.Usage](#rqsrs-006rbacprivilegesusage) - * 5.2.11.2 [Select](#select) - * 5.2.11.2.1 [RQ.SRS-006.RBAC.Privileges.Select](#rqsrs-006rbacprivilegesselect) - * 5.2.11.2.2 [RQ.SRS-006.RBAC.Privileges.Select.Column](#rqsrs-006rbacprivilegesselectcolumn) - * 5.2.11.2.3 [RQ.SRS-006.RBAC.Privileges.Select.Cluster](#rqsrs-006rbacprivilegesselectcluster) - * 5.2.11.2.4 [RQ.SRS-006.RBAC.Privileges.Select.TableEngines](#rqsrs-006rbacprivilegesselecttableengines) - * 5.2.11.3 [Insert](#insert) - * 5.2.11.3.1 [RQ.SRS-006.RBAC.Privileges.Insert](#rqsrs-006rbacprivilegesinsert) - * 5.2.11.3.2 [RQ.SRS-006.RBAC.Privileges.Insert.Column](#rqsrs-006rbacprivilegesinsertcolumn) - * 5.2.11.3.3 [RQ.SRS-006.RBAC.Privileges.Insert.Cluster](#rqsrs-006rbacprivilegesinsertcluster) - * 5.2.11.3.4 [RQ.SRS-006.RBAC.Privileges.Insert.TableEngines](#rqsrs-006rbacprivilegesinserttableengines) - * 5.2.11.4 [Alter Column](#alter-column) - * 5.2.11.4.1 [RQ.SRS-006.RBAC.Privileges.AlterColumn](#rqsrs-006rbacprivilegesaltercolumn) - * 5.2.11.4.2 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant](#rqsrs-006rbacprivilegesaltercolumngrant) - * 5.2.11.4.3 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke](#rqsrs-006rbacprivilegesaltercolumnrevoke) - * 5.2.11.4.4 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Column](#rqsrs-006rbacprivilegesaltercolumncolumn) - * 5.2.11.4.5 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster](#rqsrs-006rbacprivilegesaltercolumncluster) - * 5.2.11.4.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) - * 5.2.11.5 [Alter Index](#alter-index) - * 5.2.11.5.1 [RQ.SRS-006.RBAC.Privileges.AlterIndex](#rqsrs-006rbacprivilegesalterindex) - * 5.2.11.5.2 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant](#rqsrs-006rbacprivilegesalterindexgrant) - * 5.2.11.5.3 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke](#rqsrs-006rbacprivilegesalterindexrevoke) - * 5.2.11.5.4 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster](#rqsrs-006rbacprivilegesalterindexcluster) - * 5.2.11.5.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) - * 5.2.11.6 [Alter Constraint](#alter-constraint) - * 5.2.11.6.1 [RQ.SRS-006.RBAC.Privileges.AlterConstraint](#rqsrs-006rbacprivilegesalterconstraint) - * 5.2.11.6.2 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant](#rqsrs-006rbacprivilegesalterconstraintgrant) - * 5.2.11.6.3 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke](#rqsrs-006rbacprivilegesalterconstraintrevoke) - * 5.2.11.6.4 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster](#rqsrs-006rbacprivilegesalterconstraintcluster) - * 5.2.11.6.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) - * 5.2.11.7 [Alter TTL](#alter-ttl) - * 5.2.11.7.1 [RQ.SRS-006.RBAC.Privileges.AlterTTL](#rqsrs-006rbacprivilegesalterttl) - * 5.2.11.7.2 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant](#rqsrs-006rbacprivilegesalterttlgrant) - * 5.2.11.7.3 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke](#rqsrs-006rbacprivilegesalterttlrevoke) - * 5.2.11.7.4 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster](#rqsrs-006rbacprivilegesalterttlcluster) - * 5.2.11.7.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) - * 5.2.11.8 [Alter Settings](#alter-settings) - * 5.2.11.8.1 [RQ.SRS-006.RBAC.Privileges.AlterSettings](#rqsrs-006rbacprivilegesaltersettings) - * 5.2.11.8.2 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant](#rqsrs-006rbacprivilegesaltersettingsgrant) - * 5.2.11.8.3 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke](#rqsrs-006rbacprivilegesaltersettingsrevoke) - * 5.2.11.8.4 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster](#rqsrs-006rbacprivilegesaltersettingscluster) - * 5.2.11.8.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) - * 5.2.11.9 [Alter Update](#alter-update) - * 5.2.11.9.1 [RQ.SRS-006.RBAC.Privileges.AlterUpdate](#rqsrs-006rbacprivilegesalterupdate) - * 5.2.11.9.2 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Grant](#rqsrs-006rbacprivilegesalterupdategrant) - * 5.2.11.9.3 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Revoke](#rqsrs-006rbacprivilegesalterupdaterevoke) - * 5.2.11.9.4 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines](#rqsrs-006rbacprivilegesalterupdatetableengines) - * 5.2.11.10 [Alter Delete](#alter-delete) - * 5.2.11.10.1 [RQ.SRS-006.RBAC.Privileges.AlterDelete](#rqsrs-006rbacprivilegesalterdelete) - * 5.2.11.10.2 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Grant](#rqsrs-006rbacprivilegesalterdeletegrant) - * 5.2.11.10.3 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Revoke](#rqsrs-006rbacprivilegesalterdeleterevoke) - * 5.2.11.10.4 [RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines](#rqsrs-006rbacprivilegesalterdeletetableengines) - * 5.2.11.11 [Alter Freeze Partition](#alter-freeze-partition) - * 5.2.11.11.1 [RQ.SRS-006.RBAC.Privileges.AlterFreeze](#rqsrs-006rbacprivilegesalterfreeze) - * 5.2.11.11.2 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Grant](#rqsrs-006rbacprivilegesalterfreezegrant) - * 5.2.11.11.3 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Revoke](#rqsrs-006rbacprivilegesalterfreezerevoke) - * 5.2.11.11.4 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines](#rqsrs-006rbacprivilegesalterfreezetableengines) - * 5.2.11.12 [Alter Fetch Partition](#alter-fetch-partition) - * 5.2.11.12.1 [RQ.SRS-006.RBAC.Privileges.AlterFetch](#rqsrs-006rbacprivilegesalterfetch) - * 5.2.11.12.2 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Grant](#rqsrs-006rbacprivilegesalterfetchgrant) - * 5.2.11.12.3 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Revoke](#rqsrs-006rbacprivilegesalterfetchrevoke) - * 5.2.11.12.4 [RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines](#rqsrs-006rbacprivilegesalterfetchtableengines) - * 5.2.11.13 [Alter Move Partition](#alter-move-partition) - * 5.2.11.13.1 [RQ.SRS-006.RBAC.Privileges.AlterMove](#rqsrs-006rbacprivilegesaltermove) - * 5.2.11.13.2 [RQ.SRS-006.RBAC.Privileges.AlterMove.Grant](#rqsrs-006rbacprivilegesaltermovegrant) - * 5.2.11.13.3 [RQ.SRS-006.RBAC.Privileges.AlterMove.Revoke](#rqsrs-006rbacprivilegesaltermoverevoke) - * 5.2.11.13.4 [RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines](#rqsrs-006rbacprivilegesaltermovetableengines) - * 5.2.11.14 [RQ.SRS-006.RBAC.Privileges.CreateTable](#rqsrs-006rbacprivilegescreatetable) - * 5.2.11.15 [RQ.SRS-006.RBAC.Privileges.CreateDatabase](#rqsrs-006rbacprivilegescreatedatabase) - * 5.2.11.16 [RQ.SRS-006.RBAC.Privileges.CreateDictionary](#rqsrs-006rbacprivilegescreatedictionary) - * 5.2.11.17 [RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable](#rqsrs-006rbacprivilegescreatetemporarytable) - * 5.2.11.18 [RQ.SRS-006.RBAC.Privileges.AttachDatabase](#rqsrs-006rbacprivilegesattachdatabase) - * 5.2.11.19 [RQ.SRS-006.RBAC.Privileges.AttachDictionary](#rqsrs-006rbacprivilegesattachdictionary) - * 5.2.11.20 [RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable](#rqsrs-006rbacprivilegesattachtemporarytable) - * 5.2.11.21 [RQ.SRS-006.RBAC.Privileges.AttachTable](#rqsrs-006rbacprivilegesattachtable) - * 5.2.11.22 [RQ.SRS-006.RBAC.Privileges.DropTable](#rqsrs-006rbacprivilegesdroptable) - * 5.2.11.23 [RQ.SRS-006.RBAC.Privileges.DropDatabase](#rqsrs-006rbacprivilegesdropdatabase) - * 5.2.11.24 [RQ.SRS-006.RBAC.Privileges.DropDictionary](#rqsrs-006rbacprivilegesdropdictionary) - * 5.2.11.25 [RQ.SRS-006.RBAC.Privileges.DetachTable](#rqsrs-006rbacprivilegesdetachtable) - * 5.2.11.26 [RQ.SRS-006.RBAC.Privileges.DetachView](#rqsrs-006rbacprivilegesdetachview) - * 5.2.11.27 [RQ.SRS-006.RBAC.Privileges.DetachDatabase](#rqsrs-006rbacprivilegesdetachdatabase) - * 5.2.11.28 [RQ.SRS-006.RBAC.Privileges.DetachDictionary](#rqsrs-006rbacprivilegesdetachdictionary) - * 5.2.11.29 [RQ.SRS-006.RBAC.Privileges.Truncate](#rqsrs-006rbacprivilegestruncate) - * 5.2.11.30 [RQ.SRS-006.RBAC.Privileges.Optimize](#rqsrs-006rbacprivilegesoptimize) - * 5.2.11.31 [RQ.SRS-006.RBAC.Privileges.KillQuery](#rqsrs-006rbacprivilegeskillquery) - * 5.2.11.32 [RQ.SRS-006.RBAC.Privileges.KillMutation](#rqsrs-006rbacprivilegeskillmutation) - * 5.2.11.33 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterUpdate](#rqsrs-006rbacprivilegeskillmutationalterupdate) - * 5.2.11.34 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDelete](#rqsrs-006rbacprivilegeskillmutationalterdelete) - * 5.2.11.35 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDropColumn](#rqsrs-006rbacprivilegeskillmutationalterdropcolumn) - * 5.2.11.36 [RQ.SRS-006.RBAC.Privileges.ShowTables](#rqsrs-006rbacprivilegesshowtables) - * 5.2.11.37 [RQ.SRS-006.RBAC.Privileges.ShowTables.Query](#rqsrs-006rbacprivilegesshowtablesquery) - * 5.2.11.38 [RQ.SRS-006.RBAC.Privileges.ExistsTable](#rqsrs-006rbacprivilegesexiststable) - * 5.2.11.39 [RQ.SRS-006.RBAC.Privileges.CheckTable](#rqsrs-006rbacprivilegeschecktable) - * 5.2.11.40 [RQ.SRS-006.RBAC.Privileges.ShowDatabases](#rqsrs-006rbacprivilegesshowdatabases) - * 5.2.11.41 [RQ.SRS-006.RBAC.Privileges.ShowDatabases.Query](#rqsrs-006rbacprivilegesshowdatabasesquery) - * 5.2.11.42 [RQ.SRS-006.RBAC.Privileges.ShowCreateDatabase](#rqsrs-006rbacprivilegesshowcreatedatabase) - * 5.2.11.43 [RQ.SRS-006.RBAC.Privileges.UseDatabase](#rqsrs-006rbacprivilegesusedatabase) - * 5.2.11.44 [RQ.SRS-006.RBAC.Privileges.ShowColumns](#rqsrs-006rbacprivilegesshowcolumns) - * 5.2.11.45 [RQ.SRS-006.RBAC.Privileges.ShowCreateTable](#rqsrs-006rbacprivilegesshowcreatetable) - * 5.2.11.46 [RQ.SRS-006.RBAC.Privileges.DescribeTable](#rqsrs-006rbacprivilegesdescribetable) - * 5.2.11.47 [RQ.SRS-006.RBAC.Privileges.ShowDictionaries](#rqsrs-006rbacprivilegesshowdictionaries) - * 5.2.11.48 [RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query](#rqsrs-006rbacprivilegesshowdictionariesquery) - * 5.2.11.49 [RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary](#rqsrs-006rbacprivilegesshowcreatedictionary) - * 5.2.11.50 [RQ.SRS-006.RBAC.Privileges.ExistsDictionary](#rqsrs-006rbacprivilegesexistsdictionary) - * 5.2.11.51 [RQ.SRS-006.RBAC.Privileges.CreateUser](#rqsrs-006rbacprivilegescreateuser) - * 5.2.11.52 [RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole](#rqsrs-006rbacprivilegescreateuserdefaultrole) - * 5.2.11.53 [RQ.SRS-006.RBAC.Privileges.AlterUser](#rqsrs-006rbacprivilegesalteruser) - * 5.2.11.54 [RQ.SRS-006.RBAC.Privileges.DropUser](#rqsrs-006rbacprivilegesdropuser) - * 5.2.11.55 [RQ.SRS-006.RBAC.Privileges.CreateRole](#rqsrs-006rbacprivilegescreaterole) - * 5.2.11.56 [RQ.SRS-006.RBAC.Privileges.AlterRole](#rqsrs-006rbacprivilegesalterrole) - * 5.2.11.57 [RQ.SRS-006.RBAC.Privileges.DropRole](#rqsrs-006rbacprivilegesdroprole) - * 5.2.11.58 [RQ.SRS-006.RBAC.Privileges.CreateRowPolicy](#rqsrs-006rbacprivilegescreaterowpolicy) - * 5.2.11.59 [RQ.SRS-006.RBAC.Privileges.AlterRowPolicy](#rqsrs-006rbacprivilegesalterrowpolicy) - * 5.2.11.60 [RQ.SRS-006.RBAC.Privileges.DropRowPolicy](#rqsrs-006rbacprivilegesdroprowpolicy) - * 5.2.11.61 [RQ.SRS-006.RBAC.Privileges.CreateQuota](#rqsrs-006rbacprivilegescreatequota) - * 5.2.11.62 [RQ.SRS-006.RBAC.Privileges.AlterQuota](#rqsrs-006rbacprivilegesalterquota) - * 5.2.11.63 [RQ.SRS-006.RBAC.Privileges.DropQuota](#rqsrs-006rbacprivilegesdropquota) - * 5.2.11.64 [RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile](#rqsrs-006rbacprivilegescreatesettingsprofile) - * 5.2.11.65 [RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile](#rqsrs-006rbacprivilegesaltersettingsprofile) - * 5.2.11.66 [RQ.SRS-006.RBAC.Privileges.DropSettingsProfile](#rqsrs-006rbacprivilegesdropsettingsprofile) - * 5.2.11.67 [RQ.SRS-006.RBAC.Privileges.RoleAdmin](#rqsrs-006rbacprivilegesroleadmin) - * 5.2.11.68 [RQ.SRS-006.RBAC.Privileges.ShowUsers](#rqsrs-006rbacprivilegesshowusers) - * 5.2.11.69 [RQ.SRS-006.RBAC.Privileges.ShowUsers.Query](#rqsrs-006rbacprivilegesshowusersquery) - * 5.2.11.70 [RQ.SRS-006.RBAC.Privileges.ShowCreateUser](#rqsrs-006rbacprivilegesshowcreateuser) - * 5.2.11.71 [RQ.SRS-006.RBAC.Privileges.ShowRoles](#rqsrs-006rbacprivilegesshowroles) - * 5.2.11.72 [RQ.SRS-006.RBAC.Privileges.ShowRoles.Query](#rqsrs-006rbacprivilegesshowrolesquery) - * 5.2.11.73 [RQ.SRS-006.RBAC.Privileges.ShowCreateRole](#rqsrs-006rbacprivilegesshowcreaterole) - * 5.2.11.74 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies](#rqsrs-006rbacprivilegesshowrowpolicies) - * 5.2.11.75 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query](#rqsrs-006rbacprivilegesshowrowpoliciesquery) - * 5.2.11.76 [RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy](#rqsrs-006rbacprivilegesshowcreaterowpolicy) - * 5.2.11.77 [RQ.SRS-006.RBAC.Privileges.ShowQuotas](#rqsrs-006rbacprivilegesshowquotas) - * 5.2.11.78 [RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query](#rqsrs-006rbacprivilegesshowquotasquery) - * 5.2.11.79 [RQ.SRS-006.RBAC.Privileges.ShowCreateQuota](#rqsrs-006rbacprivilegesshowcreatequota) - * 5.2.11.80 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles](#rqsrs-006rbacprivilegesshowsettingsprofiles) - * 5.2.11.81 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query](#rqsrs-006rbacprivilegesshowsettingsprofilesquery) - * 5.2.11.82 [RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile](#rqsrs-006rbacprivilegesshowcreatesettingsprofile) - * 5.2.11.83 [Grant Option](#grant-option) - * 5.2.11.83.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) - * 5.2.11.84 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) - * 5.2.11.85 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) - * 5.2.11.86 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) - * 5.2.12 [Required Privileges](#required-privileges) - * 5.2.12.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) - * 5.2.12.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) - * 5.2.12.3 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop](#rqsrs-006rbacrequiredprivilegesdrop) - * 5.2.12.4 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table](#rqsrs-006rbacrequiredprivilegesdroptable) - * 5.2.12.5 [RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke](#rqsrs-006rbacrequiredprivilegesgrantrevoke) - * 5.2.12.6 [RQ.SRS-006.RBAC.RequiredPrivileges.Use](#rqsrs-006rbacrequiredprivilegesuse) - * 5.2.12.7 [RQ.SRS-006.RBAC.RequiredPrivileges.Admin](#rqsrs-006rbacrequiredprivilegesadmin) + * 5.2.9.2 [RQ.SRS-006.RBAC.Table.SensitiveTables](#rqsrs-006rbactablesensitivetables) + * 5.2.10 [Distributed Tables](#distributed-tables) + * 5.2.10.1 [RQ.SRS-006.RBAC.DistributedTable.Create](#rqsrs-006rbacdistributedtablecreate) + * 5.2.10.2 [RQ.SRS-006.RBAC.DistributedTable.Select](#rqsrs-006rbacdistributedtableselect) + * 5.2.10.3 [RQ.SRS-006.RBAC.DistributedTable.Insert](#rqsrs-006rbacdistributedtableinsert) + * 5.2.10.4 [RQ.SRS-006.RBAC.DistributedTable.SpecialTables](#rqsrs-006rbacdistributedtablespecialtables) + * 5.2.10.5 [RQ.SRS-006.RBAC.DistributedTable.LocalUser](#rqsrs-006rbacdistributedtablelocaluser) + * 5.2.10.6 [RQ.SRS-006.RBAC.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbacdistributedtablesameuserdifferentnodesdifferentprivileges) + * 5.2.11 [Views](#views) + * 5.2.11.1 [View](#view) + * 5.2.11.1.1 [RQ.SRS-006.RBAC.View](#rqsrs-006rbacview) + * 5.2.11.1.2 [RQ.SRS-006.RBAC.View.Create](#rqsrs-006rbacviewcreate) + * 5.2.11.1.3 [RQ.SRS-006.RBAC.View.Select](#rqsrs-006rbacviewselect) + * 5.2.11.1.4 [RQ.SRS-006.RBAC.View.Drop](#rqsrs-006rbacviewdrop) + * 5.2.11.2 [Materialized View](#materialized-view) + * 5.2.11.2.1 [RQ.SRS-006.RBAC.MaterializedView](#rqsrs-006rbacmaterializedview) + * 5.2.11.2.2 [RQ.SRS-006.RBAC.MaterializedView.Create](#rqsrs-006rbacmaterializedviewcreate) + * 5.2.11.2.3 [RQ.SRS-006.RBAC.MaterializedView.Select](#rqsrs-006rbacmaterializedviewselect) + * 5.2.11.2.4 [RQ.SRS-006.RBAC.MaterializedView.Select.TargetTable](#rqsrs-006rbacmaterializedviewselecttargettable) + * 5.2.11.2.5 [RQ.SRS-006.RBAC.MaterializedView.Select.SourceTable](#rqsrs-006rbacmaterializedviewselectsourcetable) + * 5.2.11.2.6 [RQ.SRS-006.RBAC.MaterializedView.Drop](#rqsrs-006rbacmaterializedviewdrop) + * 5.2.11.2.7 [RQ.SRS-006.RBAC.MaterializedView.ModifyQuery](#rqsrs-006rbacmaterializedviewmodifyquery) + * 5.2.11.2.8 [RQ.SRS-006.RBAC.MaterializedView.Insert](#rqsrs-006rbacmaterializedviewinsert) + * 5.2.11.2.9 [RQ.SRS-006.RBAC.MaterializedView.Insert.SourceTable](#rqsrs-006rbacmaterializedviewinsertsourcetable) + * 5.2.11.2.10 [RQ.SRS-006.RBAC.MaterializedView.Insert.TargetTable](#rqsrs-006rbacmaterializedviewinserttargettable) + * 5.2.11.3 [Live View](#live-view) + * 5.2.11.3.1 [RQ.SRS-006.RBAC.LiveView](#rqsrs-006rbacliveview) + * 5.2.11.3.2 [RQ.SRS-006.RBAC.LiveView.Create](#rqsrs-006rbacliveviewcreate) + * 5.2.11.3.3 [RQ.SRS-006.RBAC.LiveView.Select](#rqsrs-006rbacliveviewselect) + * 5.2.11.3.4 [RQ.SRS-006.RBAC.LiveView.Drop](#rqsrs-006rbacliveviewdrop) + * 5.2.11.3.5 [RQ.SRS-006.RBAC.LiveView.Refresh](#rqsrs-006rbacliveviewrefresh) + * 5.2.12 [Select](#select) + * 5.2.12.1 [RQ.SRS-006.RBAC.Select](#rqsrs-006rbacselect) + * 5.2.12.2 [RQ.SRS-006.RBAC.Select.Column](#rqsrs-006rbacselectcolumn) + * 5.2.12.3 [RQ.SRS-006.RBAC.Select.Cluster](#rqsrs-006rbacselectcluster) + * 5.2.12.4 [RQ.SRS-006.RBAC.Select.TableEngines](#rqsrs-006rbacselecttableengines) + * 5.2.13 [Insert](#insert) + * 5.2.13.1 [RQ.SRS-006.RBAC.Insert](#rqsrs-006rbacinsert) + * 5.2.13.2 [RQ.SRS-006.RBAC.Insert.Column](#rqsrs-006rbacinsertcolumn) + * 5.2.13.3 [RQ.SRS-006.RBAC.Insert.Cluster](#rqsrs-006rbacinsertcluster) + * 5.2.13.4 [RQ.SRS-006.RBAC.Insert.TableEngines](#rqsrs-006rbacinserttableengines) + * 5.2.14 [Alter](#alter) + * 5.2.14.1 [Alter Column](#alter-column) + * 5.2.14.1.1 [RQ.SRS-006.RBAC.Privileges.AlterColumn](#rqsrs-006rbacprivilegesaltercolumn) + * 5.2.14.1.2 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant](#rqsrs-006rbacprivilegesaltercolumngrant) + * 5.2.14.1.3 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke](#rqsrs-006rbacprivilegesaltercolumnrevoke) + * 5.2.14.1.4 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Column](#rqsrs-006rbacprivilegesaltercolumncolumn) + * 5.2.14.1.5 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster](#rqsrs-006rbacprivilegesaltercolumncluster) + * 5.2.14.1.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) + * 5.2.14.2 [Alter Index](#alter-index) + * 5.2.14.2.1 [RQ.SRS-006.RBAC.Privileges.AlterIndex](#rqsrs-006rbacprivilegesalterindex) + * 5.2.14.2.2 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant](#rqsrs-006rbacprivilegesalterindexgrant) + * 5.2.14.2.3 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke](#rqsrs-006rbacprivilegesalterindexrevoke) + * 5.2.14.2.4 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster](#rqsrs-006rbacprivilegesalterindexcluster) + * 5.2.14.2.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) + * 5.2.14.3 [Alter Constraint](#alter-constraint) + * 5.2.14.3.1 [RQ.SRS-006.RBAC.Privileges.AlterConstraint](#rqsrs-006rbacprivilegesalterconstraint) + * 5.2.14.3.2 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant](#rqsrs-006rbacprivilegesalterconstraintgrant) + * 5.2.14.3.3 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke](#rqsrs-006rbacprivilegesalterconstraintrevoke) + * 5.2.14.3.4 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster](#rqsrs-006rbacprivilegesalterconstraintcluster) + * 5.2.14.3.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) + * 5.2.14.4 [Alter TTL](#alter-ttl) + * 5.2.14.4.1 [RQ.SRS-006.RBAC.Privileges.AlterTTL](#rqsrs-006rbacprivilegesalterttl) + * 5.2.14.4.2 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant](#rqsrs-006rbacprivilegesalterttlgrant) + * 5.2.14.4.3 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke](#rqsrs-006rbacprivilegesalterttlrevoke) + * 5.2.14.4.4 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster](#rqsrs-006rbacprivilegesalterttlcluster) + * 5.2.14.4.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) + * 5.2.14.5 [Alter Settings](#alter-settings) + * 5.2.14.5.1 [RQ.SRS-006.RBAC.Privileges.AlterSettings](#rqsrs-006rbacprivilegesaltersettings) + * 5.2.14.5.2 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant](#rqsrs-006rbacprivilegesaltersettingsgrant) + * 5.2.14.5.3 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke](#rqsrs-006rbacprivilegesaltersettingsrevoke) + * 5.2.14.5.4 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster](#rqsrs-006rbacprivilegesaltersettingscluster) + * 5.2.14.5.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) + * 5.2.14.6 [Alter Update](#alter-update) + * 5.2.14.6.1 [RQ.SRS-006.RBAC.Privileges.AlterUpdate](#rqsrs-006rbacprivilegesalterupdate) + * 5.2.14.6.2 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Grant](#rqsrs-006rbacprivilegesalterupdategrant) + * 5.2.14.6.3 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Revoke](#rqsrs-006rbacprivilegesalterupdaterevoke) + * 5.2.14.6.4 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines](#rqsrs-006rbacprivilegesalterupdatetableengines) + * 5.2.14.7 [Alter Delete](#alter-delete) + * 5.2.14.7.1 [RQ.SRS-006.RBAC.Privileges.AlterDelete](#rqsrs-006rbacprivilegesalterdelete) + * 5.2.14.7.2 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Grant](#rqsrs-006rbacprivilegesalterdeletegrant) + * 5.2.14.7.3 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Revoke](#rqsrs-006rbacprivilegesalterdeleterevoke) + * 5.2.14.7.4 [RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines](#rqsrs-006rbacprivilegesalterdeletetableengines) + * 5.2.14.8 [Alter Freeze Partition](#alter-freeze-partition) + * 5.2.14.8.1 [RQ.SRS-006.RBAC.Privileges.AlterFreeze](#rqsrs-006rbacprivilegesalterfreeze) + * 5.2.14.8.2 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Grant](#rqsrs-006rbacprivilegesalterfreezegrant) + * 5.2.14.8.3 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Revoke](#rqsrs-006rbacprivilegesalterfreezerevoke) + * 5.2.14.8.4 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines](#rqsrs-006rbacprivilegesalterfreezetableengines) + * 5.2.14.9 [Alter Fetch Partition](#alter-fetch-partition) + * 5.2.14.9.1 [RQ.SRS-006.RBAC.Privileges.AlterFetch](#rqsrs-006rbacprivilegesalterfetch) + * 5.2.14.9.2 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Grant](#rqsrs-006rbacprivilegesalterfetchgrant) + * 5.2.14.9.3 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Revoke](#rqsrs-006rbacprivilegesalterfetchrevoke) + * 5.2.14.9.4 [RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines](#rqsrs-006rbacprivilegesalterfetchtableengines) + * 5.2.14.10 [Alter Move Partition](#alter-move-partition) + * 5.2.14.10.1 [RQ.SRS-006.RBAC.Privileges.AlterMove](#rqsrs-006rbacprivilegesaltermove) + * 5.2.14.10.2 [RQ.SRS-006.RBAC.Privileges.AlterMove.Grant](#rqsrs-006rbacprivilegesaltermovegrant) + * 5.2.14.10.3 [RQ.SRS-006.RBAC.Privileges.AlterMove.Revoke](#rqsrs-006rbacprivilegesaltermoverevoke) + * 5.2.14.10.4 [RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines](#rqsrs-006rbacprivilegesaltermovetableengines) + * 5.2.15 [RQ.SRS-006.RBAC.Privileges.CreateTable](#rqsrs-006rbacprivilegescreatetable) + * 5.2.16 [RQ.SRS-006.RBAC.Privileges.CreateDatabase](#rqsrs-006rbacprivilegescreatedatabase) + * 5.2.17 [RQ.SRS-006.RBAC.Privileges.CreateDictionary](#rqsrs-006rbacprivilegescreatedictionary) + * 5.2.18 [RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable](#rqsrs-006rbacprivilegescreatetemporarytable) + * 5.2.19 [RQ.SRS-006.RBAC.Privileges.AttachDatabase](#rqsrs-006rbacprivilegesattachdatabase) + * 5.2.20 [RQ.SRS-006.RBAC.Privileges.AttachDictionary](#rqsrs-006rbacprivilegesattachdictionary) + * 5.2.21 [RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable](#rqsrs-006rbacprivilegesattachtemporarytable) + * 5.2.22 [RQ.SRS-006.RBAC.Privileges.AttachTable](#rqsrs-006rbacprivilegesattachtable) + * 5.2.23 [RQ.SRS-006.RBAC.Privileges.DropTable](#rqsrs-006rbacprivilegesdroptable) + * 5.2.24 [RQ.SRS-006.RBAC.Privileges.DropDatabase](#rqsrs-006rbacprivilegesdropdatabase) + * 5.2.25 [RQ.SRS-006.RBAC.Privileges.DropDictionary](#rqsrs-006rbacprivilegesdropdictionary) + * 5.2.26 [RQ.SRS-006.RBAC.Privileges.DetachTable](#rqsrs-006rbacprivilegesdetachtable) + * 5.2.27 [RQ.SRS-006.RBAC.Privileges.DetachView](#rqsrs-006rbacprivilegesdetachview) + * 5.2.28 [RQ.SRS-006.RBAC.Privileges.DetachDatabase](#rqsrs-006rbacprivilegesdetachdatabase) + * 5.2.29 [RQ.SRS-006.RBAC.Privileges.DetachDictionary](#rqsrs-006rbacprivilegesdetachdictionary) + * 5.2.30 [RQ.SRS-006.RBAC.Privileges.Truncate](#rqsrs-006rbacprivilegestruncate) + * 5.2.31 [RQ.SRS-006.RBAC.Privileges.Optimize](#rqsrs-006rbacprivilegesoptimize) + * 5.2.32 [RQ.SRS-006.RBAC.Privileges.KillQuery](#rqsrs-006rbacprivilegeskillquery) + * 5.2.33 [Kill Mutation](#kill-mutation) + * 5.2.33.1 [RQ.SRS-006.RBAC.Privileges.KillMutation](#rqsrs-006rbacprivilegeskillmutation) + * 5.2.33.2 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterUpdate](#rqsrs-006rbacprivilegeskillmutationalterupdate) + * 5.2.33.3 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDelete](#rqsrs-006rbacprivilegeskillmutationalterdelete) + * 5.2.33.4 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDropColumn](#rqsrs-006rbacprivilegeskillmutationalterdropcolumn) + * 5.2.34 [Show](#show) + * 5.2.34.1 [RQ.SRS-006.RBAC.ShowTables.Privilege](#rqsrs-006rbacshowtablesprivilege) + * 5.2.34.2 [RQ.SRS-006.RBAC.ShowTables.RequiredPrivilege](#rqsrs-006rbacshowtablesrequiredprivilege) + * 5.2.34.3 [RQ.SRS-006.RBAC.ExistsTable.RequiredPrivilege](#rqsrs-006rbacexiststablerequiredprivilege) + * 5.2.34.4 [RQ.SRS-006.RBAC.CheckTable.RequiredPrivilege](#rqsrs-006rbacchecktablerequiredprivilege) + * 5.2.34.5 [RQ.SRS-006.RBAC.ShowDatabases.Privilege](#rqsrs-006rbacshowdatabasesprivilege) + * 5.2.34.6 [RQ.SRS-006.RBAC.ShowDatabases.RequiredPrivilege](#rqsrs-006rbacshowdatabasesrequiredprivilege) + * 5.2.34.7 [RQ.SRS-006.RBAC.ShowCreateDatabase.RequiredPrivilege](#rqsrs-006rbacshowcreatedatabaserequiredprivilege) + * 5.2.34.8 [RQ.SRS-006.RBAC.UseDatabase.RequiredPrivilege](#rqsrs-006rbacusedatabaserequiredprivilege) + * 5.2.34.9 [RQ.SRS-006.RBAC.ShowColumns.Privilege](#rqsrs-006rbacshowcolumnsprivilege) + * 5.2.34.10 [RQ.SRS-006.RBAC.ShowCreateTable.RequiredPrivilege](#rqsrs-006rbacshowcreatetablerequiredprivilege) + * 5.2.34.11 [RQ.SRS-006.RBAC.DescribeTable.RequiredPrivilege](#rqsrs-006rbacdescribetablerequiredprivilege) + * 5.2.34.12 [RQ.SRS-006.RBAC.ShowDictionaries.Privilege](#rqsrs-006rbacshowdictionariesprivilege) + * 5.2.34.13 [RQ.SRS-006.RBAC.ShowDictionaries.RequiredPrivilege](#rqsrs-006rbacshowdictionariesrequiredprivilege) + * 5.2.34.14 [RQ.SRS-006.RBAC.ShowCreateDictionary.RequiredPrivilege](#rqsrs-006rbacshowcreatedictionaryrequiredprivilege) + * 5.2.34.15 [RQ.SRS-006.RBAC.ExistsDictionary.RequiredPrivilege](#rqsrs-006rbacexistsdictionaryrequiredprivilege) + * 5.2.35 [Access Management](#access-management) + * 5.2.35.1 [RQ.SRS-006.RBAC.Privileges.CreateUser](#rqsrs-006rbacprivilegescreateuser) + * 5.2.35.2 [RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole](#rqsrs-006rbacprivilegescreateuserdefaultrole) + * 5.2.35.3 [RQ.SRS-006.RBAC.Privileges.AlterUser](#rqsrs-006rbacprivilegesalteruser) + * 5.2.35.4 [RQ.SRS-006.RBAC.Privileges.DropUser](#rqsrs-006rbacprivilegesdropuser) + * 5.2.35.5 [RQ.SRS-006.RBAC.Privileges.CreateRole](#rqsrs-006rbacprivilegescreaterole) + * 5.2.35.6 [RQ.SRS-006.RBAC.Privileges.AlterRole](#rqsrs-006rbacprivilegesalterrole) + * 5.2.35.7 [RQ.SRS-006.RBAC.Privileges.DropRole](#rqsrs-006rbacprivilegesdroprole) + * 5.2.35.8 [RQ.SRS-006.RBAC.Privileges.CreateRowPolicy](#rqsrs-006rbacprivilegescreaterowpolicy) + * 5.2.35.9 [RQ.SRS-006.RBAC.Privileges.AlterRowPolicy](#rqsrs-006rbacprivilegesalterrowpolicy) + * 5.2.35.10 [RQ.SRS-006.RBAC.Privileges.DropRowPolicy](#rqsrs-006rbacprivilegesdroprowpolicy) + * 5.2.35.11 [RQ.SRS-006.RBAC.Privileges.CreateQuota](#rqsrs-006rbacprivilegescreatequota) + * 5.2.35.12 [RQ.SRS-006.RBAC.Privileges.AlterQuota](#rqsrs-006rbacprivilegesalterquota) + * 5.2.35.13 [RQ.SRS-006.RBAC.Privileges.DropQuota](#rqsrs-006rbacprivilegesdropquota) + * 5.2.35.14 [RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile](#rqsrs-006rbacprivilegescreatesettingsprofile) + * 5.2.35.15 [RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile](#rqsrs-006rbacprivilegesaltersettingsprofile) + * 5.2.35.16 [RQ.SRS-006.RBAC.Privileges.DropSettingsProfile](#rqsrs-006rbacprivilegesdropsettingsprofile) + * 5.2.35.17 [RQ.SRS-006.RBAC.Privileges.RoleAdmin](#rqsrs-006rbacprivilegesroleadmin) + * 5.2.35.18 [Show Access](#show-access) + * 5.2.35.18.1 [RQ.SRS-006.RBAC.ShowUsers.Privilege](#rqsrs-006rbacshowusersprivilege) + * 5.2.35.18.2 [RQ.SRS-006.RBAC.ShowUsers.RequiredPrivilege](#rqsrs-006rbacshowusersrequiredprivilege) + * 5.2.35.18.3 [RQ.SRS-006.RBAC.ShowCreateUser.RequiredPrivilege](#rqsrs-006rbacshowcreateuserrequiredprivilege) + * 5.2.35.18.4 [RQ.SRS-006.RBAC.ShowRoles.Privilege](#rqsrs-006rbacshowrolesprivilege) + * 5.2.35.18.5 [RQ.SRS-006.RBAC.ShowRoles.RequiredPrivilege](#rqsrs-006rbacshowrolesrequiredprivilege) + * 5.2.35.18.6 [RQ.SRS-006.RBAC.ShowCreateRole.RequiredPrivilege](#rqsrs-006rbacshowcreaterolerequiredprivilege) + * 5.2.35.18.7 [RQ.SRS-006.RBAC.ShowRowPolicies.Privilege](#rqsrs-006rbacshowrowpoliciesprivilege) + * 5.2.35.18.8 [RQ.SRS-006.RBAC.ShowRowPolicies.RequiredPrivilege](#rqsrs-006rbacshowrowpoliciesrequiredprivilege) + * 5.2.35.18.9 [RQ.SRS-006.RBAC.ShowCreateRowPolicy.RequiredPrivilege](#rqsrs-006rbacshowcreaterowpolicyrequiredprivilege) + * 5.2.35.18.10 [RQ.SRS-006.RBAC.ShowQuotas.Privilege](#rqsrs-006rbacshowquotasprivilege) + * 5.2.35.18.11 [RQ.SRS-006.RBAC.ShowQuotas.RequiredPrivilege](#rqsrs-006rbacshowquotasrequiredprivilege) + * 5.2.35.18.12 [RQ.SRS-006.RBAC.ShowCreateQuota.RequiredPrivilege](#rqsrs-006rbacshowcreatequotarequiredprivilege) + * 5.2.35.18.13 [RQ.SRS-006.RBAC.ShowSettingsProfiles.Privilege](#rqsrs-006rbacshowsettingsprofilesprivilege) + * 5.2.35.18.14 [RQ.SRS-006.RBAC.ShowSettingsProfiles.RequiredPrivilege](#rqsrs-006rbacshowsettingsprofilesrequiredprivilege) + * 5.2.35.18.15 [RQ.SRS-006.RBAC.ShowCreateSettingsProfile.RequiredPrivilege](#rqsrs-006rbacshowcreatesettingsprofilerequiredprivilege) + * 5.2.36 [dictGet](#dictget) + * 5.2.36.1 [RQ.SRS-006.RBAC.dictGet.Privilege](#rqsrs-006rbacdictgetprivilege) + * 5.2.36.2 [RQ.SRS-006.RBAC.dictGet.RequiredPrivilege](#rqsrs-006rbacdictgetrequiredprivilege) + * 5.2.36.3 [RQ.SRS-006.RBAC.dictGet.Type.RequiredPrivilege](#rqsrs-006rbacdictgettyperequiredprivilege) + * 5.2.36.4 [RQ.SRS-006.RBAC.dictGet.OrDefault.RequiredPrivilege](#rqsrs-006rbacdictgetordefaultrequiredprivilege) + * 5.2.36.5 [RQ.SRS-006.RBAC.dictHas.RequiredPrivilege](#rqsrs-006rbacdicthasrequiredprivilege) + * 5.2.36.6 [RQ.SRS-006.RBAC.dictGetHierarchy.RequiredPrivilege](#rqsrs-006rbacdictgethierarchyrequiredprivilege) + * 5.2.36.7 [RQ.SRS-006.RBAC.dictIsIn.RequiredPrivilege](#rqsrs-006rbacdictisinrequiredprivilege) + * 5.2.37 [Introspection](#introspection) + * 5.2.37.1 [RQ.SRS-006.RBAC.Privileges.Introspection](#rqsrs-006rbacprivilegesintrospection) + * 5.2.37.2 [RQ.SRS-006.RBAC.Privileges.Introspection.addressToLine](#rqsrs-006rbacprivilegesintrospectionaddresstoline) + * 5.2.37.3 [RQ.SRS-006.RBAC.Privileges.Introspection.addressToSymbol](#rqsrs-006rbacprivilegesintrospectionaddresstosymbol) + * 5.2.37.4 [RQ.SRS-006.RBAC.Privileges.Introspection.demangle](#rqsrs-006rbacprivilegesintrospectiondemangle) + * 5.2.38 [System](#system) + * 5.2.38.1 [RQ.SRS-006.RBAC.Privileges.System.Shutdown](#rqsrs-006rbacprivilegessystemshutdown) + * 5.2.38.2 [RQ.SRS-006.RBAC.Privileges.System.DropCache](#rqsrs-006rbacprivilegessystemdropcache) + * 5.2.38.3 [RQ.SRS-006.RBAC.Privileges.System.DropCache.DNS](#rqsrs-006rbacprivilegessystemdropcachedns) + * 5.2.38.4 [RQ.SRS-006.RBAC.Privileges.System.DropCache.Mark](#rqsrs-006rbacprivilegessystemdropcachemark) + * 5.2.38.5 [RQ.SRS-006.RBAC.Privileges.System.DropCache.Uncompressed](#rqsrs-006rbacprivilegessystemdropcacheuncompressed) + * 5.2.38.6 [RQ.SRS-006.RBAC.Privileges.System.Reload](#rqsrs-006rbacprivilegessystemreload) + * 5.2.38.7 [RQ.SRS-006.RBAC.Privileges.System.Reload.Config](#rqsrs-006rbacprivilegessystemreloadconfig) + * 5.2.38.8 [RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionary](#rqsrs-006rbacprivilegessystemreloaddictionary) + * 5.2.38.9 [RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionaries](#rqsrs-006rbacprivilegessystemreloaddictionaries) + * 5.2.38.10 [RQ.SRS-006.RBAC.Privileges.System.Reload.EmbeddedDictionaries](#rqsrs-006rbacprivilegessystemreloadembeddeddictionaries) + * 5.2.38.11 [RQ.SRS-006.RBAC.Privileges.System.Merges](#rqsrs-006rbacprivilegessystemmerges) + * 5.2.38.12 [RQ.SRS-006.RBAC.Privileges.System.TTLMerges](#rqsrs-006rbacprivilegessystemttlmerges) + * 5.2.38.13 [RQ.SRS-006.RBAC.Privileges.System.Fetches](#rqsrs-006rbacprivilegessystemfetches) + * 5.2.38.14 [RQ.SRS-006.RBAC.Privileges.System.Moves](#rqsrs-006rbacprivilegessystemmoves) + * 5.2.38.15 [RQ.SRS-006.RBAC.Privileges.System.Sends](#rqsrs-006rbacprivilegessystemsends) + * 5.2.38.16 [RQ.SRS-006.RBAC.Privileges.System.Sends.Distributed](#rqsrs-006rbacprivilegessystemsendsdistributed) + * 5.2.38.17 [RQ.SRS-006.RBAC.Privileges.System.Sends.Replicated](#rqsrs-006rbacprivilegessystemsendsreplicated) + * 5.2.38.18 [RQ.SRS-006.RBAC.Privileges.System.ReplicationQueues](#rqsrs-006rbacprivilegessystemreplicationqueues) + * 5.2.38.19 [RQ.SRS-006.RBAC.Privileges.System.SyncReplica](#rqsrs-006rbacprivilegessystemsyncreplica) + * 5.2.38.20 [RQ.SRS-006.RBAC.Privileges.System.RestartReplica](#rqsrs-006rbacprivilegessystemrestartreplica) + * 5.2.38.21 [RQ.SRS-006.RBAC.Privileges.System.Flush](#rqsrs-006rbacprivilegessystemflush) + * 5.2.38.22 [RQ.SRS-006.RBAC.Privileges.System.Flush.Distributed](#rqsrs-006rbacprivilegessystemflushdistributed) + * 5.2.38.23 [RQ.SRS-006.RBAC.Privileges.System.Flush.Logs](#rqsrs-006rbacprivilegessystemflushlogs) + * 5.2.39 [Sources](#sources) + * 5.2.39.1 [RQ.SRS-006.RBAC.Privileges.Sources](#rqsrs-006rbacprivilegessources) + * 5.2.39.2 [RQ.SRS-006.RBAC.Privileges.Sources.File](#rqsrs-006rbacprivilegessourcesfile) + * 5.2.39.3 [RQ.SRS-006.RBAC.Privileges.Sources.URL](#rqsrs-006rbacprivilegessourcesurl) + * 5.2.39.4 [RQ.SRS-006.RBAC.Privileges.Sources.Remote](#rqsrs-006rbacprivilegessourcesremote) + * 5.2.39.5 [RQ.SRS-006.RBAC.Privileges.Sources.MySQL](#rqsrs-006rbacprivilegessourcesmysql) + * 5.2.39.6 [RQ.SRS-006.RBAC.Privileges.Sources.ODBC](#rqsrs-006rbacprivilegessourcesodbc) + * 5.2.39.7 [RQ.SRS-006.RBAC.Privileges.Sources.JDBC](#rqsrs-006rbacprivilegessourcesjdbc) + * 5.2.39.8 [RQ.SRS-006.RBAC.Privileges.Sources.HDFS](#rqsrs-006rbacprivilegessourceshdfs) + * 5.2.39.9 [RQ.SRS-006.RBAC.Privileges.Sources.S3](#rqsrs-006rbacprivilegessourcess3) + * 5.2.40 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) + * 5.2.41 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) + * 5.2.42 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) * 6 [References](#references) ## Revision History @@ -1496,60 +1520,12 @@ version: 1.0 [ClickHouse] SHALL support granting the **create** privilege to one or more users or roles using the `GRANT CREATE` statement. -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.Database -version: 1.0 - -[ClickHouse] SHALL support granting the **create database** privilege to one or more users or roles -for a database using the `GRANT CREATE DATABASE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.Dictionary -version: 1.0 - -[ClickHouse] SHALL support granting the **create dictionary** privilege to one or more users or roles -for a dictionary using the `GRANT CREATE DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.Table -version: 1.0 - -[ClickHouse] SHALL support granting the **create table** privilege to one or more users or roles -for a table using the `GRANT CREATE TABLE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support granting the **create temporary table** privilege to one or more users or roles -for a temporary table using the `GRANT CREATE TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Grant.Privilege.Drop version: 1.0 [ClickHouse] SHALL support granting the **drop** privilege to one or more users or roles using the `GRANT DROP` statement. -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.Database -version: 1.0 - -[ClickHouse] SHALL support granting the **drop database** privilege to one or more users or roles -for a database using the `GRANT DROP DATABASE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.Dictionary -version: 1.0 - -[ClickHouse] SHALL support granting the **drop dictionary** privilege to one or more users or roles -for a dictionary using the `GRANT DROP DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.Table -version: 1.0 - -[ClickHouse] SHALL support granting the **drop table** privilege to one or more users or roles -for a table using the `GRANT DROP TABLE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support granting the **drop temporary table** privilege to one or more users or roles -for a temporary table using the `GRANT DROP TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Grant.Privilege.Truncate version: 1.0 @@ -1701,60 +1677,12 @@ version: 1.0 [ClickHouse] SHALL support revoking the **create** privilege to one or more users or roles using the `REVOKE CREATE` statement. -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.Database -version: 1.0 - -[ClickHouse] SHALL support revoking the **create database** privilege to one or more users or roles -for a database using the `REVOKE CREATE DATABASE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.Dictionary -version: 1.0 - -[ClickHouse] SHALL support revoking the **create dictionary** privilege to one or more users or roles -for a dictionary using the `REVOKE CREATE DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.Table -version: 1.0 - -[ClickHouse] SHALL support revoking the **create table** privilege to one or more users or roles -for a table using the `REVOKE CREATE TABLE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support revoking the **create temporary table** privilege to one or more users or roles -for a temporary table using the `REVOKE CREATE TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop version: 1.0 [ClickHouse] SHALL support revoking the **drop** privilege to one or more users or roles using the `REVOKE DROP` statement. -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Database -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop database** privilege to one or more users or roles -for a database using the `REVOKE DROP DATABASE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Dictionary -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop dictionary** privilege to one or more users or roles -for a dictionary using the `REVOKE DROP DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Table -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop table** privilege to one or more users or roles -for a table using the `REVOKE DROP TABLE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop temporary table** privilege to one or more users or roles -for a temporary table using the `REVOKE DROP TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Revoke.Privilege.Truncate version: 1.0 @@ -2897,20 +2825,32 @@ version: 1.0 * system.contributors * system.functions -##### RQ.SRS-006.RBAC.Table.QueryLog +##### RQ.SRS-006.RBAC.Table.SensitiveTables version: 1.0 -[ClickHouse] SHALL return only queries executed by the user when the user is selecting from system.query_log. +[ClickHouse] SHALL not support a user with no privileges accessing the following `system` tables: -##### Distributed Tables +* processes +* query_log +* query_thread_log +* clusters +* events +* graphite_retentions +* stack_trace +* trace_log +* user_directories +* zookeeper +* macros -###### RQ.SRS-006.RBAC.Table.DistributedTable.Create +#### Distributed Tables + +##### RQ.SRS-006.RBAC.DistributedTable.Create version: 1.0 [ClickHouse] SHALL successfully `CREATE` a distributed table if and only if the user has **create table** privilege on the table and **remote** privilege on *.* -###### RQ.SRS-006.RBAC.Table.DistributedTable.Select +##### RQ.SRS-006.RBAC.DistributedTable.Select version: 1.0 [ClickHouse] SHALL successfully `SELECT` from a distributed table if and only if @@ -2918,7 +2858,7 @@ the user has **select** privilege on the table and on the remote table specified Does not require **select** privilege for the remote table if the remote table does not exist on the same server as the user. -###### RQ.SRS-006.RBAC.Table.DistributedTable.Insert +##### RQ.SRS-006.RBAC.DistributedTable.Insert version: 1.0 [ClickHouse] SHALL successfully `INSERT` into a distributed table if and only if @@ -2927,7 +2867,7 @@ the user has **insert** privilege on the table and on the remote table specified Does not require **insert** privilege for the remote table if the remote table does not exist on the same server as the user, insert executes into the remote table on a different server. -###### RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables +##### RQ.SRS-006.RBAC.DistributedTable.SpecialTables version: 1.0 [ClickHouse] SHALL successfully execute a query using a distributed table that uses one of the special tables if and only if @@ -2937,13 +2877,13 @@ Special tables include: * distributed table * source table of a materialized view -###### RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser +##### RQ.SRS-006.RBAC.DistributedTable.LocalUser version: 1.0 [ClickHouse] SHALL successfully execute a query using a distributed table from a user present locally, but not remotely. -###### RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges +##### RQ.SRS-006.RBAC.DistributedTable.SameUserDifferentNodesDifferentPrivileges version: 1.0 [ClickHouse] SHALL successfully execute a query using a distributed table by a user that exists on multiple nodes @@ -3170,41 +3110,33 @@ version: 1.0 [ClickHouse] SHALL only successfully execute an `ALTER LIVE VIEW REFRESH` command if and only if the user has **refresh** privilege on that view either explicitly or through a role. -#### Privileges +#### Select -##### RQ.SRS-006.RBAC.Privileges.Usage +##### RQ.SRS-006.RBAC.Select version: 1.0 -[ClickHouse] SHALL support granting or revoking **usage** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### Select - -###### RQ.SRS-006.RBAC.Privileges.Select -version: 1.0 - -[ClickHouse] SHALL not execute `SELECT INTO` if and only if the user +[ClickHouse] SHALL execute `SELECT` if and only if the user has the **select** privilege for the destination table either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Select.Column +##### RQ.SRS-006.RBAC.Select.Column version: 1.0 [ClickHouse] SHALL support granting or revoking **select** privilege for one or more specified columns in a table to one or more **users** or **roles**. -Any `SELECT INTO` statements SHALL not to be executed, unless the user +Any `SELECT` statements SHALL not to be executed, unless the user has the **select** privilege for the destination column either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Select.Cluster +##### RQ.SRS-006.RBAC.Select.Cluster version: 1.0 [ClickHouse] SHALL support granting or revoking **select** privilege on a specified cluster to one or more **users** or **roles**. -Any `SELECT INTO` statements SHALL succeed only on nodes where +Any `SELECT` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Select.TableEngines +##### RQ.SRS-006.RBAC.Select.TableEngines version: 1.0 [ClickHouse] SHALL support controlling access to the **select** privilege @@ -3225,16 +3157,16 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree -##### Insert +#### Insert -###### RQ.SRS-006.RBAC.Privileges.Insert +##### RQ.SRS-006.RBAC.Insert version: 1.0 [ClickHouse] SHALL execute `INSERT INTO` if and only if the user has the **insert** privilege for the destination table either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Insert.Column +##### RQ.SRS-006.RBAC.Insert.Column version: 1.0 [ClickHouse] SHALL support granting or revoking **insert** privilege @@ -3243,7 +3175,7 @@ Any `INSERT INTO` statements SHALL not to be executed, unless the user has the **insert** privilege for the destination column either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Insert.Cluster +##### RQ.SRS-006.RBAC.Insert.Cluster version: 1.0 [ClickHouse] SHALL support granting or revoking **insert** privilege @@ -3251,7 +3183,7 @@ on a specified cluster to one or more **users** or **roles**. Any `INSERT INTO` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Insert.TableEngines +##### RQ.SRS-006.RBAC.Insert.TableEngines version: 1.0 [ClickHouse] SHALL support controlling access to the **insert** privilege @@ -3272,6 +3204,8 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree +#### Alter + ##### Alter Column ###### RQ.SRS-006.RBAC.Privileges.AlterColumn @@ -3736,7 +3670,7 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree -##### RQ.SRS-006.RBAC.Privileges.CreateTable +#### RQ.SRS-006.RBAC.Privileges.CreateTable version: 1.0 [ClickHouse] SHALL only successfully execute a `CREATE TABLE` command if and only if @@ -3754,108 +3688,110 @@ CREATE TABLE table AS SELECT column FROM table0 JOIN table1 USING column UNION A CREATE TABLE table0 AS SELECT column FROM table1 UNION ALL SELECT column FROM table2 ``` -##### RQ.SRS-006.RBAC.Privileges.CreateDatabase +#### RQ.SRS-006.RBAC.Privileges.CreateDatabase version: 1.0 [ClickHouse] SHALL successfully execute `CREATE DATABASE` statement if and only if the user has **create database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.CreateDictionary +#### RQ.SRS-006.RBAC.Privileges.CreateDictionary version: 1.0 [ClickHouse] SHALL successfully execute `CREATE DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable +#### RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable version: 1.0 [ClickHouse] SHALL successfully execute `CREATE TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachDatabase +#### RQ.SRS-006.RBAC.Privileges.AttachDatabase version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH DATABASE` statement if and only if the user has **create database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachDictionary +#### RQ.SRS-006.RBAC.Privileges.AttachDictionary version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable +#### RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachTable +#### RQ.SRS-006.RBAC.Privileges.AttachTable version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH TABLE` statement if and only if the user has **create table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DropTable +#### RQ.SRS-006.RBAC.Privileges.DropTable version: 1.0 [ClickHouse] SHALL successfully execute `DROP TABLE` statement if and only if the user has **drop table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DropDatabase +#### RQ.SRS-006.RBAC.Privileges.DropDatabase version: 1.0 [ClickHouse] SHALL successfully execute `DROP DATABASE` statement if and only if the user has **drop database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DropDictionary +#### RQ.SRS-006.RBAC.Privileges.DropDictionary version: 1.0 [ClickHouse] SHALL successfully execute `DROP DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachTable +#### RQ.SRS-006.RBAC.Privileges.DetachTable version: 1.0 [ClickHouse] SHALL successfully execute `DETACH TABLE` statement if and only if the user has **drop table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachView +#### RQ.SRS-006.RBAC.Privileges.DetachView version: 1.0 [ClickHouse] SHALL successfully execute `DETACH VIEW` statement if and only if the user has **drop view** privilege on the view, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachDatabase +#### RQ.SRS-006.RBAC.Privileges.DetachDatabase version: 1.0 [ClickHouse] SHALL successfully execute `DETACH DATABASE` statement if and only if the user has **drop database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachDictionary +#### RQ.SRS-006.RBAC.Privileges.DetachDictionary version: 1.0 [ClickHouse] SHALL successfully execute `DETACH DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.Truncate +#### RQ.SRS-006.RBAC.Privileges.Truncate version: 1.0 [ClickHouse] SHALL successfully execute `TRUNCATE TABLE` statement if and only if the user has **truncate table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.Optimize +#### RQ.SRS-006.RBAC.Privileges.Optimize version: 1.0 [ClickHouse] SHALL successfully execute `OPTIMIZE TABLE` statement if and only if the user has **optimize table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.KillQuery +#### RQ.SRS-006.RBAC.Privileges.KillQuery version: 1.0 [ClickHouse] SHALL successfully execute `KILL QUERY` statement if and only if the user has **kill query** privilege, either directly or through a role. +#### Kill Mutation + ##### RQ.SRS-006.RBAC.Privileges.KillMutation version: 1.0 @@ -3881,95 +3817,99 @@ version: 1.0 [ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER DROP COLUMN` mutation if and only if the user has `ALTER DROP COLUMN` privilege on the table where the mutation was created, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowTables +#### Show + +##### RQ.SRS-006.RBAC.ShowTables.Privilege version: 1.0 [ClickHouse] SHALL grant **show tables** privilege on a table to a user if that user has recieved any grant, including `SHOW TABLES`, on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowTables.Query +##### RQ.SRS-006.RBAC.ShowTables.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW TABLES` statement if and only if the user has **show tables** privilege, or any privilege on the table either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ExistsTable +##### RQ.SRS-006.RBAC.ExistsTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `EXISTS table` statement if and only if the user has **show tables** privilege, or any privilege on the table either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.CheckTable +##### RQ.SRS-006.RBAC.CheckTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `CHECK table` statement if and only if the user has **show tables** privilege, or any privilege on the table either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDatabases +##### RQ.SRS-006.RBAC.ShowDatabases.Privilege version: 1.0 [ClickHouse] SHALL grant **show databases** privilege on a database to a user if that user has recieved any grant, including `SHOW DATABASES`, on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDatabases.Query +##### RQ.SRS-006.RBAC.ShowDatabases.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW DATABASES` statement if and only if the user has **show databases** privilege, or any privilege on the database either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateDatabase +##### RQ.SRS-006.RBAC.ShowCreateDatabase.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW CREATE DATABASE` statement if and only if the user has **show databases** privilege, or any privilege on the database either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.UseDatabase +##### RQ.SRS-006.RBAC.UseDatabase.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `USE database` statement if and only if the user has **show databases** privilege, or any privilege on the database either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowColumns +##### RQ.SRS-006.RBAC.ShowColumns.Privilege version: 1.0 -[ClickHouse] SHALL grant **show columns** privilege to a user if and only if it is granted directly or through a role. +[ClickHouse] SHALL support granting or revoking the `SHOW COLUMNS` privilege. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateTable +##### RQ.SRS-006.RBAC.ShowCreateTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW CREATE TABLE` statement if and only if the user has **show columns** privilege on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DescribeTable +##### RQ.SRS-006.RBAC.DescribeTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `DESCRIBE table` statement if and only if the user has **show columns** privilege on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDictionaries +##### RQ.SRS-006.RBAC.ShowDictionaries.Privilege version: 1.0 [ClickHouse] SHALL grant **show dictionaries** privilege on a dictionary to a user if that user has recieved any grant, including `SHOW DICTIONARIES`, on that dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query +##### RQ.SRS-006.RBAC.ShowDictionaries.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW DICTIONARIES` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary +##### RQ.SRS-006.RBAC.ShowCreateDictionary.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW CREATE DICTIONARY` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ExistsDictionary +##### RQ.SRS-006.RBAC.ExistsDictionary.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `EXISTS dictionary` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. +#### Access Management + ##### RQ.SRS-006.RBAC.Privileges.CreateUser version: 1.0 @@ -4071,215 +4011,404 @@ version: 1.0 [ClickHouse] SHALL successfully execute any role grant or revoke by a user with `ROLE ADMIN` privilege. -##### RQ.SRS-006.RBAC.Privileges.ShowUsers +##### Show Access + +###### RQ.SRS-006.RBAC.ShowUsers.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW USERS` privilege when the user is granted `SHOW USERS`, `SHOW CREATE USER`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowUsers.Query +###### RQ.SRS-006.RBAC.ShowUsers.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW USERS` statement if and only if the user has **show users** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateUser +###### RQ.SRS-006.RBAC.ShowCreateUser.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show create user** privilege, +[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show users** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowRoles +###### RQ.SRS-006.RBAC.ShowRoles.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW ROLES` privilege when the user is granted `SHOW ROLES`, `SHOW CREATE ROLE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowRoles.Query +###### RQ.SRS-006.RBAC.ShowRoles.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW ROLES` statement if and only if the user has **show roles** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateRole +###### RQ.SRS-006.RBAC.ShowCreateRole.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show create role** privilege, +[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show roles** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies +###### RQ.SRS-006.RBAC.ShowRowPolicies.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW ROW POLICIES` privilege when the user is granted `SHOW ROW POLICIES`, `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query +###### RQ.SRS-006.RBAC.ShowRowPolicies.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if the user has **show row policies** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if +the user has **show row policies** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy +###### RQ.SRS-006.RBAC.ShowCreateRowPolicy.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement if and only if the user has **show create row policy** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement +if and only if the user has **show row policies** privilege,either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowQuotas +###### RQ.SRS-006.RBAC.ShowQuotas.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW QUOTAS` privilege when the user is granted `SHOW QUOTAS`, `SHOW CREATE QUOTA`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query +###### RQ.SRS-006.RBAC.ShowQuotas.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW QUOTAS` statement if and only if the user has **show quotas** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateQuota +###### RQ.SRS-006.RBAC.ShowCreateQuota.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if the user has **show create quota** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if +the user has **show quotas** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles +###### RQ.SRS-006.RBAC.ShowSettingsProfiles.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW SETTINGS PROFILES` privilege when the user is granted `SHOW SETTINGS PROFILES`, `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW SETTINGS PROFILE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query +###### RQ.SRS-006.RBAC.ShowSettingsProfiles.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement if and only if the user has **show settings profiles** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement +if and only if the user has **show settings profiles** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile +###### RQ.SRS-006.RBAC.ShowCreateSettingsProfile.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement if and only if the user has **show create settings profile** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement +if and only if the user has **show settings profiles** privilege, either directly or through a role. -##### Grant Option +#### dictGet -###### RQ.SRS-006.RBAC.Privileges.GrantOption +##### RQ.SRS-006.RBAC.dictGet.Privilege +version: 1.0 + +[ClickHouse] SHALL successfully grant `dictGet` privilege when +the user is granted `dictGet`, `dictHas`, `dictGetHierarchy`, or `dictIsIn`. + +##### RQ.SRS-006.RBAC.dictGet.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGet` statement +if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictGet.Type.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGet[TYPE]` statement +if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role. +Available types: + +* Int8 +* Int16 +* Int32 +* Int64 +* UInt8 +* UInt16 +* UInt32 +* UInt64 +* Float32 +* Float64 +* Date +* DateTime +* UUID +* String + +##### RQ.SRS-006.RBAC.dictGet.OrDefault.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGetOrDefault` statement +if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictHas.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictHas` statement +if and only if the user has **dictGet** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictGetHierarchy.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGetHierarchy` statement +if and only if the user has **dictGet** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictIsIn.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictIsIn` statement +if and only if the user has **dictGet** privilege, either directly or through a role. + +#### Introspection + +##### RQ.SRS-006.RBAC.Privileges.Introspection +version: 1.0 + +[ClickHouse] SHALL successfully grant `INTROSPECTION` privilege when +the user is granted `INTROSPECTION` or `INTROSPECTION FUNCTIONS`. + +##### RQ.SRS-006.RBAC.Privileges.Introspection.addressToLine +version: 1.0 + +[ClickHouse] SHALL successfully execute `addressToLine` statement if and only if +the user has **introspection** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Introspection.addressToSymbol +version: 1.0 + +[ClickHouse] SHALL successfully execute `addressToSymbol` statement if and only if +the user has **introspection** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Introspection.demangle +version: 1.0 + +[ClickHouse] SHALL successfully execute `demangle` statement if and only if +the user has **introspection** privilege, either directly or through a role. + +#### System + +##### RQ.SRS-006.RBAC.Privileges.System.Shutdown +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM SHUTDOWN` privilege when +the user is granted `SYSTEM`, `SYSTEM SHUTDOWN`, `SHUTDOWN`,or `SYSTEM KILL`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, or `DROP CACHE`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache.DNS +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP DNS CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP DNS CACHE`, +`SYSTEM DROP DNS`, `DROP DNS CACHE`, or `DROP DNS`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache.Mark +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP MARK CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP MARK CACHE`, +`SYSTEM DROP MARK`, `DROP MARK CACHE`, or `DROP MARKS`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache.Uncompressed +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP UNCOMPRESSED CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP UNCOMPRESSED CACHE`, +`SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, or `DROP UNCOMPRESSED`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD` privilege when +the user is granted `SYSTEM` or `SYSTEM RELOAD`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.Config +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD CONFIG` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD CONFIG`, or `RELOAD CONFIG`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionary +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD DICTIONARY` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARIES`, or `RELOAD DICTIONARY`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionaries +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD DICTIONARIES` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARIES`, or `RELOAD DICTIONARY`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.EmbeddedDictionaries +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARY ON *.*`, or `SYSTEM RELOAD EMBEDDED DICTIONARIES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Merges +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM MERGES` privilege when +the user is granted `SYSTEM`, `SYSTEM MERGES`, `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, or `START MERGES`. + +##### RQ.SRS-006.RBAC.Privileges.System.TTLMerges +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM TTL MERGES` privilege when +the user is granted `SYSTEM`, `SYSTEM TTL MERGES`, `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, or `START TTL MERGES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Fetches +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FETCHES` privilege when +the user is granted `SYSTEM`, `SYSTEM FETCHES`, `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, or `START FETCHES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Moves +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM MOVES` privilege when +the user is granted `SYSTEM`, `SYSTEM MOVES`, `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, or `START MOVES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Sends +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM SENDS` privilege when +the user is granted `SYSTEM`, `SYSTEM SENDS`, `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, or `START SENDS`. + +##### RQ.SRS-006.RBAC.Privileges.System.Sends.Distributed +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DISTRIBUTED SENDS` privilege when +the user is granted `SYSTEM`, `SYSTEM DISTRIBUTED SENDS`, `SYSTEM STOP DISTRIBUTED SENDS`, +`SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, or `START DISTRIBUTED SENDS`. + +##### RQ.SRS-006.RBAC.Privileges.System.Sends.Replicated +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM REPLICATED SENDS` privilege when +the user is granted `SYSTEM`, `SYSTEM REPLICATED SENDS`, `SYSTEM STOP REPLICATED SENDS`, +`SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, or `START REPLICATED SENDS`. + +##### RQ.SRS-006.RBAC.Privileges.System.ReplicationQueues +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM REPLICATION QUEUES` privilege when +the user is granted `SYSTEM`, `SYSTEM REPLICATION QUEUES`, `SYSTEM STOP REPLICATION QUEUES`, +`SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, or `START REPLICATION QUEUES`. + +##### RQ.SRS-006.RBAC.Privileges.System.SyncReplica +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM SYNC REPLICA` privilege when +the user is granted `SYSTEM`, `SYSTEM SYNC REPLICA`, or `SYNC REPLICA`. + +##### RQ.SRS-006.RBAC.Privileges.System.RestartReplica +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RESTART REPLICA` privilege when +the user is granted `SYSTEM`, `SYSTEM RESTART REPLICA`, or `RESTART REPLICA`. + +##### RQ.SRS-006.RBAC.Privileges.System.Flush +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FLUSH` privilege when +the user is granted `SYSTEM` or `SYSTEM FLUSH`. + +##### RQ.SRS-006.RBAC.Privileges.System.Flush.Distributed +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FLUSH DISTRIBUTED` privilege when +the user is granted `SYSTEM`, `SYSTEM FLUSH DISTRIBUTED`, or `FLUSH DISTRIBUTED`. + +##### RQ.SRS-006.RBAC.Privileges.System.Flush.Logs +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FLUSH LOGS` privilege when +the user is granted `SYSTEM`, `SYSTEM FLUSH LOGS`, or `FLUSH LOGS`. + +#### Sources + +##### RQ.SRS-006.RBAC.Privileges.Sources +version: 1.0 + +[ClickHouse] SHALL support granting or revoking `SOURCES` privilege from +the user, either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.File +version: 1.0 + +[ClickHouse] SHALL support the use of `FILE` source by a user if and only if +the user has `FILE` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.URL +version: 1.0 + +[ClickHouse] SHALL support the use of `URL` source by a user if and only if +the user has `URL` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.Remote +version: 1.0 + +[ClickHouse] SHALL support the use of `REMOTE` source by a user if and only if +the user has `REMOTE` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.MySQL +version: 1.0 + +[ClickHouse] SHALL support the use of `MySQL` source by a user if and only if +the user has `MySQL` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.ODBC +version: 1.0 + +[ClickHouse] SHALL support the use of `ODBC` source by a user if and only if +the user has `ODBC` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.JDBC +version: 1.0 + +[ClickHouse] SHALL support the use of `JDBC` source by a user if and only if +the user has `JDBC` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.HDFS +version: 1.0 + +[ClickHouse] SHALL support the use of `HDFS` source by a user if and only if +the user has `HDFS` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.S3 +version: 1.0 + +[ClickHouse] SHALL support the use of `S3` source by a user if and only if +the user has `S3` or `SOURCES` privileges granted to them directly or through a role. + +#### RQ.SRS-006.RBAC.Privileges.GrantOption version: 1.0 [ClickHouse] SHALL successfully execute `GRANT` or `REVOKE` privilege statements by a user if and only if the user has that privilege with `GRANT OPTION`, either directly or through a role. -`GRANT OPTION` is supported by the following privileges - -* `ALTER MOVE PARTITION` -* `ALTER FETCH PARTITION` -* `ALTER FREEZE PARTITION` -* `ALTER DELETE` -* `ALTER UPDATE` -* `ALTER SETTINGS` -* `ALTER TTL` -* `ALTER CONSTRAINT` -* `ALTER COLUMN` -* `ALTER INDEX` -* `INSERT` -* `SELECT` -* `CREATE TABLE` -* `CREATE VIEW` -* `CREATE DATABASE` -* `CREATE DICTIONARY` -* `CREATE TEMPORARY TABLE` -* `DROP TABLE` -* `DROP VIEW` -* `DROP DATABASE` -* `DROP DICTIONARY` - -##### RQ.SRS-006.RBAC.Privileges.All +#### RQ.SRS-006.RBAC.Privileges.All version: 1.0 -[ClickHouse] SHALL include in the **all** privilege the same rights -as provided by **usage**, **select**, **select columns**, -**insert**, **delete**, **alter**, **create**, and **drop** privileges. +[ClickHouse] SHALL support granting or revoking `ALL` privilege. -##### RQ.SRS-006.RBAC.Privileges.All.GrantRevoke +#### RQ.SRS-006.RBAC.Privileges.AdminOption version: 1.0 -[ClickHouse] SHALL support granting or revoking **all** privileges -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.AdminOption -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **admin option** privilege -to one or more **users** or **roles**. - -#### Required Privileges - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Create -version: 1.0 - -[ClickHouse] SHALL not allow any `CREATE` statements -to be executed unless the user has the **create** privilege for the destination database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Alter -version: 1.0 - -[ClickHouse] SHALL not allow any `ALTER` statements -to be executed unless the user has the **alter** privilege for the destination table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop -version: 1.0 - -[ClickHouse] SHALL not allow any `DROP` statements -to be executed unless the user has the **drop** privilege for the destination database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table -version: 1.0 - -[ClickHouse] SHALL not allow any `DROP TABLE` statements -to be executed unless the user has the **drop** privilege for the destination database or the table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke -version: 1.0 - -[ClickHouse] SHALL not allow any `GRANT` or `REVOKE` statements -to be executed unless the user has the **grant option** privilege -for the privilege of the destination table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Use -version: 1.0 - -[ClickHouse] SHALL not allow the `USE` statement to be executed -unless the user has at least one of the privileges for the database -or the table inside that database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Admin -version: 1.0 - -[ClickHouse] SHALL not allow any of the following statements - -* `SYSTEM` -* `SHOW` -* `ATTACH` -* `CHECK TABLE` -* `DESCRIBE TABLE` -* `DETACH` -* `EXISTS` -* `KILL QUERY` -* `KILL MUTATION` -* `OPTIMIZE` -* `RENAME` -* `TRUNCATE` - -to be executed unless the user has the **admin option** privilege -through one of the roles with **admin option** privilege assigned to the user. +[ClickHouse] SHALL support a user granting or revoking a role if and only if +the user has that role with `ADMIN OPTION` privilege. ## References diff --git a/tests/testflows/rbac/requirements/requirements.py b/tests/testflows/rbac/requirements/requirements.py index 1e16f3fee13..f276d811f19 100755 --- a/tests/testflows/rbac/requirements/requirements.py +++ b/tests/testflows/rbac/requirements/requirements.py @@ -1,9667 +1,28 @@ # These requirements were auto generated # from software requirements specification (SRS) -# document by TestFlows v1.6.201216.1172002. +# document by TestFlows v1.6.201124.1002350. # Do not edit by hand but re-generate instead # using 'tfs requirements generate' command. from testflows.core import Specification from testflows.core import Requirement -Heading = Specification.Heading - -RQ_SRS_006_RBAC = Requirement( - name='RQ.SRS-006.RBAC', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support role based access control.\n' - '\n' - ), - link=None, - level=3, - num='5.1.1') - -RQ_SRS_006_RBAC_Login = Requirement( - name='RQ.SRS-006.RBAC.Login', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only allow access to the server for a given\n' - 'user only when correct username and password are used during\n' - 'the connection to the server.\n' - '\n' - ), - link=None, - level=4, - num='5.1.2.1') - -RQ_SRS_006_RBAC_Login_DefaultUser = Requirement( - name='RQ.SRS-006.RBAC.Login.DefaultUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL use the **default user** when no username and password\n' - 'are specified during the connection to the server.\n' - '\n' - ), - link=None, - level=4, - num='5.1.2.2') - -RQ_SRS_006_RBAC_User = Requirement( - name='RQ.SRS-006.RBAC.User', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creation and manipulation of\n' - 'one or more **user** accounts to which roles, privileges,\n' - 'settings profile, quotas and row policies can be assigned.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.1') - -RQ_SRS_006_RBAC_User_Roles = Requirement( - name='RQ.SRS-006.RBAC.User.Roles', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more **roles**\n' - 'to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.2') - -RQ_SRS_006_RBAC_User_Privileges = Requirement( - name='RQ.SRS-006.RBAC.User.Privileges', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more privileges to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.3') - -RQ_SRS_006_RBAC_User_Variables = Requirement( - name='RQ.SRS-006.RBAC.User.Variables', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more variables to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.4') - -RQ_SRS_006_RBAC_User_Variables_Constraints = Requirement( - name='RQ.SRS-006.RBAC.User.Variables.Constraints', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning min, max and read-only constraints\n' - 'for the variables that can be set and read by the **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.5') - -RQ_SRS_006_RBAC_User_SettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.User.SettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more **settings profiles**\n' - 'to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.6') - -RQ_SRS_006_RBAC_User_Quotas = Requirement( - name='RQ.SRS-006.RBAC.User.Quotas', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more **quotas** to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.7') - -RQ_SRS_006_RBAC_User_RowPolicies = Requirement( - name='RQ.SRS-006.RBAC.User.RowPolicies', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more **row policies** to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.8') - -RQ_SRS_006_RBAC_User_AccountLock = Requirement( - name='RQ.SRS-006.RBAC.User.AccountLock', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support locking and unlocking of **user** accounts.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.9') - -RQ_SRS_006_RBAC_User_AccountLock_DenyAccess = Requirement( - name='RQ.SRS-006.RBAC.User.AccountLock.DenyAccess', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL deny access to the user whose account is locked.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.10') - -RQ_SRS_006_RBAC_User_DefaultRole = Requirement( - name='RQ.SRS-006.RBAC.User.DefaultRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning a default role to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.11') - -RQ_SRS_006_RBAC_User_RoleSelection = Requirement( - name='RQ.SRS-006.RBAC.User.RoleSelection', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support selection of one or more **roles** from the available roles\n' - 'that are assigned to a **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.12') - -RQ_SRS_006_RBAC_User_ShowCreate = Requirement( - name='RQ.SRS-006.RBAC.User.ShowCreate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the command of how **user** account was created.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.13') - -RQ_SRS_006_RBAC_User_ShowPrivileges = Requirement( - name='RQ.SRS-006.RBAC.User.ShowPrivileges', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support listing the privileges of the **user**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.3.14') - -RQ_SRS_006_RBAC_Role = Requirement( - name='RQ.SRS-006.RBAC.Role', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClikHouse] SHALL support creation and manipulation of **roles**\n' - 'to which privileges, settings profile, quotas and row policies can be\n' - 'assigned.\n' - '\n' - ), - link=None, - level=4, - num='5.1.4.1') - -RQ_SRS_006_RBAC_Role_Privileges = Requirement( - name='RQ.SRS-006.RBAC.Role.Privileges', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more privileges to a **role**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.4.2') - -RQ_SRS_006_RBAC_Role_Variables = Requirement( - name='RQ.SRS-006.RBAC.Role.Variables', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more variables to a **role**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.4.3') - -RQ_SRS_006_RBAC_Role_SettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.Role.SettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more **settings profiles**\n' - 'to a **role**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.4.4') - -RQ_SRS_006_RBAC_Role_Quotas = Requirement( - name='RQ.SRS-006.RBAC.Role.Quotas', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more **quotas** to a **role**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.4.5') - -RQ_SRS_006_RBAC_Role_RowPolicies = Requirement( - name='RQ.SRS-006.RBAC.Role.RowPolicies', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning one or more **row policies** to a **role**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.4.6') - -RQ_SRS_006_RBAC_PartialRevokes = Requirement( - name='RQ.SRS-006.RBAC.PartialRevokes', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support partial revoking of privileges granted\n' - 'to a **user** or a **role**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.5.1') - -RQ_SRS_006_RBAC_SettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creation and manipulation of **settings profiles**\n' - 'that can include value definition for one or more variables and can\n' - 'can be assigned to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.6.1') - -RQ_SRS_006_RBAC_SettingsProfile_Constraints = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Constraints', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning min, max and read-only constraints\n' - 'for the variables specified in the **settings profile**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.6.2') - -RQ_SRS_006_RBAC_SettingsProfile_ShowCreate = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.ShowCreate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the command of how **setting profile** was created.\n' - '\n' - ), - link=None, - level=4, - num='5.1.6.3') - -RQ_SRS_006_RBAC_Quotas = Requirement( - name='RQ.SRS-006.RBAC.Quotas', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creation and manipulation of **quotas**\n' - 'that can be used to limit resource usage by a **user** or a **role**\n' - 'over a period of time.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.1') - -RQ_SRS_006_RBAC_Quotas_Keyed = Requirement( - name='RQ.SRS-006.RBAC.Quotas.Keyed', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creating **quotas** that are keyed\n' - 'so that a quota is tracked separately for each key value.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.2') - -RQ_SRS_006_RBAC_Quotas_Queries = Requirement( - name='RQ.SRS-006.RBAC.Quotas.Queries', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting **queries** quota to limit the total number of requests.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.3') - -RQ_SRS_006_RBAC_Quotas_Errors = Requirement( - name='RQ.SRS-006.RBAC.Quotas.Errors', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting **errors** quota to limit the number of queries that threw an exception.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.4') - -RQ_SRS_006_RBAC_Quotas_ResultRows = Requirement( - name='RQ.SRS-006.RBAC.Quotas.ResultRows', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting **result rows** quota to limit the\n' - 'the total number of rows given as the result.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.5') - -RQ_SRS_006_RBAC_Quotas_ReadRows = Requirement( - name='RQ.SRS-006.RBAC.Quotas.ReadRows', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting **read rows** quota to limit the total\n' - 'number of source rows read from tables for running the query on all remote servers.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.6') - -RQ_SRS_006_RBAC_Quotas_ResultBytes = Requirement( - name='RQ.SRS-006.RBAC.Quotas.ResultBytes', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting **result bytes** quota to limit the total number\n' - 'of bytes that can be returned as the result.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.7') - -RQ_SRS_006_RBAC_Quotas_ReadBytes = Requirement( - name='RQ.SRS-006.RBAC.Quotas.ReadBytes', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting **read bytes** quota to limit the total number\n' - 'of source bytes read from tables for running the query on all remote servers.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.8') - -RQ_SRS_006_RBAC_Quotas_ExecutionTime = Requirement( - name='RQ.SRS-006.RBAC.Quotas.ExecutionTime', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting **execution time** quota to limit the maximum\n' - 'query execution time.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.9') - -RQ_SRS_006_RBAC_Quotas_ShowCreate = Requirement( - name='RQ.SRS-006.RBAC.Quotas.ShowCreate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the command of how **quota** was created.\n' - '\n' - ), - link=None, - level=4, - num='5.1.7.10') - -RQ_SRS_006_RBAC_RowPolicy = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creation and manipulation of table **row policies**\n' - 'that can be used to limit access to the table contents for a **user** or a **role**\n' - 'using a specified **condition**.\n' - '\n' - ), - link=None, - level=4, - num='5.1.8.1') - -RQ_SRS_006_RBAC_RowPolicy_Condition = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Condition', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support row policy **conditions** that can be any SQL\n' - 'expression that returns a boolean.\n' - '\n' - ), - link=None, - level=4, - num='5.1.8.2') - -RQ_SRS_006_RBAC_RowPolicy_ShowCreate = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.ShowCreate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the command of how **row policy** was created.\n' - '\n' - ), - link=None, - level=4, - num='5.1.8.3') - -RQ_SRS_006_RBAC_User_Use_DefaultRole = Requirement( - name='RQ.SRS-006.RBAC.User.Use.DefaultRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL by default use default role or roles assigned\n' - 'to the user if specified.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.1') - -RQ_SRS_006_RBAC_User_Use_AllRolesWhenNoDefaultRole = Requirement( - name='RQ.SRS-006.RBAC.User.Use.AllRolesWhenNoDefaultRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL by default use all the roles assigned to the user\n' - 'if no default role or roles are specified for the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.2') - -RQ_SRS_006_RBAC_User_Create = Requirement( - name='RQ.SRS-006.RBAC.User.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creating **user** accounts using `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.3') - -RQ_SRS_006_RBAC_User_Create_IfNotExists = Requirement( - name='RQ.SRS-006.RBAC.User.Create.IfNotExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE USER` statement\n' - 'to skip raising an exception if a user with the same **name** already exists.\n' - 'If the `IF NOT EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if a user with the same **name** already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.4') - -RQ_SRS_006_RBAC_User_Create_Replace = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Replace', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE USER` statement\n' - 'to replace existing user account if already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.5') - -RQ_SRS_006_RBAC_User_Create_Password_NoPassword = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.NoPassword', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying no password when creating\n' - 'user account using `IDENTIFIED WITH NO_PASSWORD` clause .\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.6') - -RQ_SRS_006_RBAC_User_Create_Password_NoPassword_Login = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.NoPassword.Login', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL use no password for the user when connecting to the server\n' - 'when an account was created with `IDENTIFIED WITH NO_PASSWORD` clause.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.7') - -RQ_SRS_006_RBAC_User_Create_Password_PlainText = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.PlainText', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying plaintext password when creating\n' - 'user account using `IDENTIFIED WITH PLAINTEXT_PASSWORD BY` clause.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.8') - -RQ_SRS_006_RBAC_User_Create_Password_PlainText_Login = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.PlainText.Login', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL use the plaintext password passed by the user when connecting to the server\n' - 'when an account was created with `IDENTIFIED WITH PLAINTEXT_PASSWORD` clause\n' - 'and compare the password with the one used in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.9') - -RQ_SRS_006_RBAC_User_Create_Password_Sha256Password = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Password', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the result of applying SHA256\n' - 'to some password when creating user account using `IDENTIFIED WITH SHA256_PASSWORD BY` or `IDENTIFIED BY`\n' - 'clause.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.10') - -RQ_SRS_006_RBAC_User_Create_Password_Sha256Password_Login = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Password.Login', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL calculate `SHA256` of the password passed by the user when connecting to the server\n' - "when an account was created with `IDENTIFIED WITH SHA256_PASSWORD` or with 'IDENTIFIED BY' clause\n" - 'and compare the calculated hash to the one used in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.11') - -RQ_SRS_006_RBAC_User_Create_Password_Sha256Hash = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the result of applying SHA256\n' - 'to some already calculated hash when creating user account using `IDENTIFIED WITH SHA256_HASH`\n' - 'clause.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.12') - -RQ_SRS_006_RBAC_User_Create_Password_Sha256Hash_Login = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash.Login', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL calculate `SHA256` of the already calculated hash passed by\n' - 'the user when connecting to the server\n' - 'when an account was created with `IDENTIFIED WITH SHA256_HASH` clause\n' - 'and compare the calculated hash to the one used in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.13') - -RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Password = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the result of applying SHA1 two times\n' - 'to a password when creating user account using `IDENTIFIED WITH DOUBLE_SHA1_PASSWORD`\n' - 'clause.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.14') - -RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Password_Login = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password.Login', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL calculate `SHA1` two times over the password passed by\n' - 'the user when connecting to the server\n' - 'when an account was created with `IDENTIFIED WITH DOUBLE_SHA1_PASSWORD` clause\n' - 'and compare the calculated value to the one used in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.15') - -RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Hash = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the result of applying SHA1 two times\n' - 'to a hash when creating user account using `IDENTIFIED WITH DOUBLE_SHA1_HASH`\n' - 'clause.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.16') - -RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Hash_Login = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash.Login', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL calculate `SHA1` two times over the hash passed by\n' - 'the user when connecting to the server\n' - 'when an account was created with `IDENTIFIED WITH DOUBLE_SHA1_HASH` clause\n' - 'and compare the calculated value to the one used in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.17') - -RQ_SRS_006_RBAC_User_Create_Host_Name = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.Name', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more hostnames from\n' - 'which user can access the server using the `HOST NAME` clause\n' - 'in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.18') - -RQ_SRS_006_RBAC_User_Create_Host_Regexp = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.Regexp', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more regular expressions\n' - 'to match hostnames from which user can access the server\n' - 'using the `HOST REGEXP` clause in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.19') - -RQ_SRS_006_RBAC_User_Create_Host_IP = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.IP', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more IP address or subnet from\n' - 'which user can access the server using the `HOST IP` clause in the\n' - '`CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.20') - -RQ_SRS_006_RBAC_User_Create_Host_Any = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.Any', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying `HOST ANY` clause in the `CREATE USER` statement\n' - 'to indicate that user can access the server from any host.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.21') - -RQ_SRS_006_RBAC_User_Create_Host_None = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support fobidding access from any host using `HOST NONE` clause in the\n' - '`CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.22') - -RQ_SRS_006_RBAC_User_Create_Host_Local = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.Local', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting user access to local only using `HOST LOCAL` clause in the\n' - '`CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.23') - -RQ_SRS_006_RBAC_User_Create_Host_Like = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.Like', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying host using `LIKE` command syntax using the\n' - '`HOST LIKE` clause in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.24') - -RQ_SRS_006_RBAC_User_Create_Host_Default = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Host.Default', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support user access to server from any host\n' - 'if no `HOST` clause is specified in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.25') - -RQ_SRS_006_RBAC_User_Create_DefaultRole = Requirement( - name='RQ.SRS-006.RBAC.User.Create.DefaultRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more default roles\n' - 'using `DEFAULT ROLE` clause in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.26') - -RQ_SRS_006_RBAC_User_Create_DefaultRole_None = Requirement( - name='RQ.SRS-006.RBAC.User.Create.DefaultRole.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying no default roles\n' - 'using `DEFAULT ROLE NONE` clause in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.27') - -RQ_SRS_006_RBAC_User_Create_DefaultRole_All = Requirement( - name='RQ.SRS-006.RBAC.User.Create.DefaultRole.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying all roles to be used as default\n' - 'using `DEFAULT ROLE ALL` clause in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.28') - -RQ_SRS_006_RBAC_User_Create_Settings = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Settings', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying settings and profile\n' - 'using `SETTINGS` clause in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.29') - -RQ_SRS_006_RBAC_User_Create_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.User.Create.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying cluster on which the user\n' - 'will be created using `ON CLUSTER` clause in the `CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.30') - -RQ_SRS_006_RBAC_User_Create_Syntax = Requirement( - name='RQ.SRS-006.RBAC.User.Create.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for `CREATE USER` statement.\n' - '\n' - '```sql\n' - 'CREATE USER [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]\n' - " [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}]\n" - " [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]\n" - ' [DEFAULT ROLE role [,...]]\n' - " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.31') - -RQ_SRS_006_RBAC_User_Alter = Requirement( - name='RQ.SRS-006.RBAC.User.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering **user** accounts using `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.32') - -RQ_SRS_006_RBAC_User_Alter_OrderOfEvaluation = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.OrderOfEvaluation', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support evaluating `ALTER USER` statement from left to right\n' - 'where things defined on the right override anything that was previously defined on\n' - 'the left.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.33') - -RQ_SRS_006_RBAC_User_Alter_IfExists = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF EXISTS` clause in the `ALTER USER` statement\n' - 'to skip raising an exception (producing a warning instead) if a user with the specified **name** does not exist. If the `IF EXISTS` clause is not specified then an exception SHALL be raised if a user with the **name** does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.34') - -RQ_SRS_006_RBAC_User_Alter_Cluster = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the cluster the user is on\n' - 'when altering user account using `ON CLUSTER` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.35') - -RQ_SRS_006_RBAC_User_Alter_Rename = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Rename', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying a new name for the user when\n' - 'altering user account using `RENAME` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.36') - -RQ_SRS_006_RBAC_User_Alter_Password_PlainText = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Password.PlainText', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying plaintext password when altering\n' - 'user account using `IDENTIFIED WITH PLAINTEXT_PASSWORD BY` or\n' - 'using shorthand `IDENTIFIED BY` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.37') - -RQ_SRS_006_RBAC_User_Alter_Password_Sha256Password = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Password.Sha256Password', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the result of applying SHA256\n' - 'to some password as identification when altering user account using\n' - '`IDENTIFIED WITH SHA256_PASSWORD` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.38') - -RQ_SRS_006_RBAC_User_Alter_Password_DoubleSha1Password = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Password.DoubleSha1Password', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the result of applying Double SHA1\n' - 'to some password as identification when altering user account using\n' - '`IDENTIFIED WITH DOUBLE_SHA1_PASSWORD` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.39') - -RQ_SRS_006_RBAC_User_Alter_Host_AddDrop = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.AddDrop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering user by adding and dropping access to hosts with the `ADD HOST` or the `DROP HOST`in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.40') - -RQ_SRS_006_RBAC_User_Alter_Host_Local = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.Local', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting user access to local only using `HOST LOCAL` clause in the\n' - '`ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.41') - -RQ_SRS_006_RBAC_User_Alter_Host_Name = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.Name', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more hostnames from\n' - 'which user can access the server using the `HOST NAME` clause\n' - 'in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.42') - -RQ_SRS_006_RBAC_User_Alter_Host_Regexp = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.Regexp', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more regular expressions\n' - 'to match hostnames from which user can access the server\n' - 'using the `HOST REGEXP` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.43') - -RQ_SRS_006_RBAC_User_Alter_Host_IP = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.IP', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more IP address or subnet from\n' - 'which user can access the server using the `HOST IP` clause in the\n' - '`ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.44') - -RQ_SRS_006_RBAC_User_Alter_Host_Like = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.Like', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying sone or more similar hosts using `LIKE` command syntax using the `HOST LIKE` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.45') - -RQ_SRS_006_RBAC_User_Alter_Host_Any = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.Any', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying `HOST ANY` clause in the `ALTER USER` statement\n' - 'to indicate that user can access the server from any host.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.46') - -RQ_SRS_006_RBAC_User_Alter_Host_None = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Host.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support fobidding access from any host using `HOST NONE` clause in the\n' - '`ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.47') - -RQ_SRS_006_RBAC_User_Alter_DefaultRole = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.DefaultRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more default roles\n' - 'using `DEFAULT ROLE` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.48') - -RQ_SRS_006_RBAC_User_Alter_DefaultRole_All = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.DefaultRole.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying all roles to be used as default\n' - 'using `DEFAULT ROLE ALL` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.49') - -RQ_SRS_006_RBAC_User_Alter_DefaultRole_AllExcept = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.DefaultRole.AllExcept', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more roles which will not be used as default\n' - 'using `DEFAULT ROLE ALL EXCEPT` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.50') - -RQ_SRS_006_RBAC_User_Alter_Settings = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Settings', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying one or more variables\n' - 'using `SETTINGS` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.51') - -RQ_SRS_006_RBAC_User_Alter_Settings_Min = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Settings.Min', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying a minimum value for the variable specifed using `SETTINGS` with `MIN` clause in the `ALTER USER` statement.\n' - '\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.52') - -RQ_SRS_006_RBAC_User_Alter_Settings_Max = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Settings.Max', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying a maximum value for the variable specifed using `SETTINGS` with `MAX` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.53') - -RQ_SRS_006_RBAC_User_Alter_Settings_Profile = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Settings.Profile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying the name of a profile for the variable specifed using `SETTINGS` with `PROFILE` clause in the `ALTER USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.54') - -RQ_SRS_006_RBAC_User_Alter_Syntax = Requirement( - name='RQ.SRS-006.RBAC.User.Alter.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `ALTER USER` statement.\n' - '\n' - '```sql\n' - 'ALTER USER [IF EXISTS] name [ON CLUSTER cluster_name]\n' - ' [RENAME TO new_name]\n' - " [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}]\n" - " [[ADD|DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]\n" - ' [DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ]\n' - " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.55') - -RQ_SRS_006_RBAC_SetDefaultRole = Requirement( - name='RQ.SRS-006.RBAC.SetDefaultRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting or changing granted roles to default for one or more\n' - 'users using `SET DEFAULT ROLE` statement which\n' - 'SHALL permanently change the default roles for the user or users if successful.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.56') - -RQ_SRS_006_RBAC_SetDefaultRole_CurrentUser = Requirement( - name='RQ.SRS-006.RBAC.SetDefaultRole.CurrentUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting or changing granted roles to default for\n' - 'the current user using `CURRENT_USER` clause in the `SET DEFAULT ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.57') - -RQ_SRS_006_RBAC_SetDefaultRole_All = Requirement( - name='RQ.SRS-006.RBAC.SetDefaultRole.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting or changing all granted roles to default\n' - 'for one or more users using `ALL` clause in the `SET DEFAULT ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.58') - -RQ_SRS_006_RBAC_SetDefaultRole_AllExcept = Requirement( - name='RQ.SRS-006.RBAC.SetDefaultRole.AllExcept', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting or changing all granted roles except those specified\n' - 'to default for one or more users using `ALL EXCEPT` clause in the `SET DEFAULT ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.59') - -RQ_SRS_006_RBAC_SetDefaultRole_None = Requirement( - name='RQ.SRS-006.RBAC.SetDefaultRole.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing all granted roles from default\n' - 'for one or more users using `NONE` clause in the `SET DEFAULT ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.60') - -RQ_SRS_006_RBAC_SetDefaultRole_Syntax = Requirement( - name='RQ.SRS-006.RBAC.SetDefaultRole.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `SET DEFAULT ROLE` statement.\n' - '\n' - '```sql\n' - 'SET DEFAULT ROLE\n' - ' {NONE | role [,...] | ALL | ALL EXCEPT role [,...]}\n' - ' TO {user|CURRENT_USER} [,...]\n' - '\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.61') - -RQ_SRS_006_RBAC_SetRole = Requirement( - name='RQ.SRS-006.RBAC.SetRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support activating role or roles for the current user\n' - 'using `SET ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.62') - -RQ_SRS_006_RBAC_SetRole_Default = Requirement( - name='RQ.SRS-006.RBAC.SetRole.Default', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support activating default roles for the current user\n' - 'using `DEFAULT` clause in the `SET ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.63') - -RQ_SRS_006_RBAC_SetRole_None = Requirement( - name='RQ.SRS-006.RBAC.SetRole.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support activating no roles for the current user\n' - 'using `NONE` clause in the `SET ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.64') - -RQ_SRS_006_RBAC_SetRole_All = Requirement( - name='RQ.SRS-006.RBAC.SetRole.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support activating all roles for the current user\n' - 'using `ALL` clause in the `SET ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.65') - -RQ_SRS_006_RBAC_SetRole_AllExcept = Requirement( - name='RQ.SRS-006.RBAC.SetRole.AllExcept', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support activating all roles except those specified\n' - 'for the current user using `ALL EXCEPT` clause in the `SET ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.66') - -RQ_SRS_006_RBAC_SetRole_Syntax = Requirement( - name='RQ.SRS-006.RBAC.SetRole.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '```sql\n' - 'SET ROLE {DEFAULT | NONE | role [,...] | ALL | ALL EXCEPT role [,...]}\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.67') - -RQ_SRS_006_RBAC_User_ShowCreateUser = Requirement( - name='RQ.SRS-006.RBAC.User.ShowCreateUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the `CREATE USER` statement used to create the current user object\n' - 'using the `SHOW CREATE USER` statement with `CURRENT_USER` or no argument.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.68') - -RQ_SRS_006_RBAC_User_ShowCreateUser_For = Requirement( - name='RQ.SRS-006.RBAC.User.ShowCreateUser.For', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the `CREATE USER` statement used to create the specified user object\n' - 'using the `FOR` clause in the `SHOW CREATE USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.69') - -RQ_SRS_006_RBAC_User_ShowCreateUser_Syntax = Requirement( - name='RQ.SRS-006.RBAC.User.ShowCreateUser.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the following syntax for `SHOW CREATE USER` statement.\n' - '\n' - '```sql\n' - 'SHOW CREATE USER [name | CURRENT_USER]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.70') - -RQ_SRS_006_RBAC_User_Drop = Requirement( - name='RQ.SRS-006.RBAC.User.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing a user account using `DROP USER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.71') - -RQ_SRS_006_RBAC_User_Drop_IfExists = Requirement( - name='RQ.SRS-006.RBAC.User.Drop.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP USER` statement\n' - 'to skip raising an exception if the user account does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if a user does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.72') - -RQ_SRS_006_RBAC_User_Drop_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.User.Drop.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using `ON CLUSTER` clause in the `DROP USER` statement\n' - 'to specify the name of the cluster the user should be dropped from.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.73') - -RQ_SRS_006_RBAC_User_Drop_Syntax = Requirement( - name='RQ.SRS-006.RBAC.User.Drop.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for `DROP USER` statement\n' - '\n' - '```sql\n' - 'DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.74') - -RQ_SRS_006_RBAC_Role_Create = Requirement( - name='RQ.SRS-006.RBAC.Role.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creating a **role** using `CREATE ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.75') - -RQ_SRS_006_RBAC_Role_Create_IfNotExists = Requirement( - name='RQ.SRS-006.RBAC.Role.Create.IfNotExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE ROLE` statement\n' - 'to raising an exception if a role with the same **name** already exists.\n' - 'If the `IF NOT EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if a role with the same **name** already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.76') - -RQ_SRS_006_RBAC_Role_Create_Replace = Requirement( - name='RQ.SRS-006.RBAC.Role.Create.Replace', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE ROLE` statement\n' - 'to replace existing role if it already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.77') - -RQ_SRS_006_RBAC_Role_Create_Settings = Requirement( - name='RQ.SRS-006.RBAC.Role.Create.Settings', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying settings and profile using `SETTINGS`\n' - 'clause in the `CREATE ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.78') - -RQ_SRS_006_RBAC_Role_Create_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Role.Create.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `CREATE ROLE` statement\n' - '\n' - '``` sql\n' - 'CREATE ROLE [IF NOT EXISTS | OR REPLACE] name\n' - " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.79') - -RQ_SRS_006_RBAC_Role_Alter = Requirement( - name='RQ.SRS-006.RBAC.Role.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.80') - -RQ_SRS_006_RBAC_Role_Alter_IfExists = Requirement( - name='RQ.SRS-006.RBAC.Role.Alter.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE IF EXISTS` statement, where no exception\n' - 'will be thrown if the role does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.81') - -RQ_SRS_006_RBAC_Role_Alter_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Role.Alter.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE role ON CLUSTER` statement to specify the\n' - 'cluster location of the specified role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.82') - -RQ_SRS_006_RBAC_Role_Alter_Rename = Requirement( - name='RQ.SRS-006.RBAC.Role.Alter.Rename', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE role RENAME TO` statement which renames the\n' - 'role to a specified new name. If the new name already exists, that an exception SHALL be raised unless the\n' - '`IF EXISTS` clause is specified, by which no exception will be raised and nothing will change.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.83') - -RQ_SRS_006_RBAC_Role_Alter_Settings = Requirement( - name='RQ.SRS-006.RBAC.Role.Alter.Settings', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the settings of one **role** using `ALTER ROLE role SETTINGS ...` statement.\n' - 'Altering variable values, creating max and min values, specifying readonly or writable, and specifying the\n' - 'profiles for which this alter change shall be applied to, are all supported, using the following syntax.\n' - '\n' - '```sql\n' - "[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" - '```\n' - '\n' - 'One or more variables and profiles may be specified as shown above.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.84') - -RQ_SRS_006_RBAC_Role_Alter_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Role.Alter.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '```sql\n' - 'ALTER ROLE [IF EXISTS] name [ON CLUSTER cluster_name]\n' - ' [RENAME TO new_name]\n' - " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.85') - -RQ_SRS_006_RBAC_Role_Drop = Requirement( - name='RQ.SRS-006.RBAC.Role.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing one or more roles using `DROP ROLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.86') - -RQ_SRS_006_RBAC_Role_Drop_IfExists = Requirement( - name='RQ.SRS-006.RBAC.Role.Drop.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP ROLE` statement\n' - 'to skip raising an exception if the role does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if a role does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.87') - -RQ_SRS_006_RBAC_Role_Drop_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Role.Drop.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using `ON CLUSTER` clause in the `DROP ROLE` statement to specify the cluster from which to drop the specified role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.88') - -RQ_SRS_006_RBAC_Role_Drop_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Role.Drop.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `DROP ROLE` statement\n' - '\n' - '``` sql\n' - 'DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.89') - -RQ_SRS_006_RBAC_Role_ShowCreate = Requirement( - name='RQ.SRS-006.RBAC.Role.ShowCreate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support viewing the settings for a role upon creation with the `SHOW CREATE ROLE`\n' - 'statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.90') - -RQ_SRS_006_RBAC_Role_ShowCreate_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Role.ShowCreate.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `SHOW CREATE ROLE` command.\n' - '\n' - '```sql\n' - 'SHOW CREATE ROLE name\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.91') - -RQ_SRS_006_RBAC_Grant_Privilege_To = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.To', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting privileges to one or more users or roles using `TO` clause\n' - 'in the `GRANT PRIVILEGE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.92') - -RQ_SRS_006_RBAC_Grant_Privilege_ToCurrentUser = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.ToCurrentUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting privileges to current user using `TO CURRENT_USER` clause\n' - 'in the `GRANT PRIVILEGE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.93') - -RQ_SRS_006_RBAC_Grant_Privilege_Select = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **select** privilege to one or more users or roles\n' - 'for a database or a table using the `GRANT SELECT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.94') - -RQ_SRS_006_RBAC_Grant_Privilege_Insert = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Insert', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **insert** privilege to one or more users or roles\n' - 'for a database or a table using the `GRANT INSERT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.95') - -RQ_SRS_006_RBAC_Grant_Privilege_Alter = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **alter** privilege to one or more users or roles\n' - 'for a database or a table using the `GRANT ALTER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.96') - -RQ_SRS_006_RBAC_Grant_Privilege_Create = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **create** privilege to one or more users or roles\n' - 'using the `GRANT CREATE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.97') - -RQ_SRS_006_RBAC_Grant_Privilege_Create_Database = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Create.Database', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **create database** privilege to one or more users or roles\n' - 'for a database using the `GRANT CREATE DATABASE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.98') - -RQ_SRS_006_RBAC_Grant_Privilege_Create_Dictionary = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Create.Dictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **create dictionary** privilege to one or more users or roles\n' - 'for a dictionary using the `GRANT CREATE DICTIONARY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.99') - -RQ_SRS_006_RBAC_Grant_Privilege_Create_Table = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Create.Table', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **create table** privilege to one or more users or roles\n' - 'for a table using the `GRANT CREATE TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.100') - -RQ_SRS_006_RBAC_Grant_Privilege_Create_TemporaryTable = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Create.TemporaryTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **create temporary table** privilege to one or more users or roles\n' - 'for a temporary table using the `GRANT CREATE TEMPORARY TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.101') - -RQ_SRS_006_RBAC_Grant_Privilege_Drop = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **drop** privilege to one or more users or roles\n' - 'using the `GRANT DROP` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.102') - -RQ_SRS_006_RBAC_Grant_Privilege_Drop_Database = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.Database', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **drop database** privilege to one or more users or roles\n' - 'for a database using the `GRANT DROP DATABASE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.103') - -RQ_SRS_006_RBAC_Grant_Privilege_Drop_Dictionary = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.Dictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **drop dictionary** privilege to one or more users or roles\n' - 'for a dictionary using the `GRANT DROP DICTIONARY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.104') - -RQ_SRS_006_RBAC_Grant_Privilege_Drop_Table = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.Table', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **drop table** privilege to one or more users or roles\n' - 'for a table using the `GRANT DROP TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.105') - -RQ_SRS_006_RBAC_Grant_Privilege_Drop_TemporaryTable = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.TemporaryTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **drop temporary table** privilege to one or more users or roles\n' - 'for a temporary table using the `GRANT DROP TEMPORARY TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.106') - -RQ_SRS_006_RBAC_Grant_Privilege_Truncate = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Truncate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **truncate** privilege to one or more users or roles\n' - 'for a database or a table using `GRANT TRUNCATE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.107') - -RQ_SRS_006_RBAC_Grant_Privilege_Optimize = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Optimize', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **optimize** privilege to one or more users or roles\n' - 'for a database or a table using `GRANT OPTIMIZE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.108') - -RQ_SRS_006_RBAC_Grant_Privilege_Show = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Show', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **show** privilege to one or more users or roles\n' - 'for a database or a table using `GRANT SHOW` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.109') - -RQ_SRS_006_RBAC_Grant_Privilege_KillQuery = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.KillQuery', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **kill query** privilege to one or more users or roles\n' - 'for a database or a table using `GRANT KILL QUERY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.110') - -RQ_SRS_006_RBAC_Grant_Privilege_AccessManagement = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **access management** privileges to one or more users or roles\n' - 'for a database or a table using `GRANT ACCESS MANAGEMENT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.111') - -RQ_SRS_006_RBAC_Grant_Privilege_System = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.System', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **system** privileges to one or more users or roles\n' - 'for a database or a table using `GRANT SYSTEM` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.112') - -RQ_SRS_006_RBAC_Grant_Privilege_Introspection = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Introspection', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **introspection** privileges to one or more users or roles\n' - 'for a database or a table using `GRANT INTROSPECTION` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.113') - -RQ_SRS_006_RBAC_Grant_Privilege_Sources = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Sources', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **sources** privileges to one or more users or roles\n' - 'for a database or a table using `GRANT SOURCES` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.114') - -RQ_SRS_006_RBAC_Grant_Privilege_DictGet = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.DictGet', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **dictGet** privilege to one or more users or roles\n' - 'for a database or a table using `GRANT dictGet` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.115') - -RQ_SRS_006_RBAC_Grant_Privilege_None = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting no privileges to one or more users or roles\n' - 'for a database or a table using `GRANT NONE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.116') - -RQ_SRS_006_RBAC_Grant_Privilege_All = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **all** privileges to one or more users or roles\n' - 'for a database or a table using the `GRANT ALL` or `GRANT ALL PRIVILEGES` statements.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.117') - -RQ_SRS_006_RBAC_Grant_Privilege_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the **grant option** privilege to one or more users or roles\n' - 'for a database or a table using the `WITH GRANT OPTION` clause in the `GRANT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.118') - -RQ_SRS_006_RBAC_Grant_Privilege_On = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.On', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the `ON` clause in the `GRANT` privilege statement\n' - 'which SHALL allow to specify one or more tables to which the privilege SHALL\n' - 'be granted using the following patterns\n' - '\n' - '* `*.*` any table in any database\n' - '* `database.*` any table in the specified database\n' - '* `database.table` specific table in the specified database\n' - '* `*` any table in the current database\n' - '* `table` specific table in the current database\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.119') - -RQ_SRS_006_RBAC_Grant_Privilege_PrivilegeColumns = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting the privilege **some_privilege** to one or more users or roles\n' - 'for a database or a table using the `GRANT some_privilege(column)` statement for one column.\n' - 'Multiple columns will be supported with `GRANT some_privilege(column1, column2...)` statement.\n' - 'The privileges will be granted for only the specified columns.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.120') - -RQ_SRS_006_RBAC_Grant_Privilege_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying cluster on which to grant privileges using the `ON CLUSTER`\n' - 'clause in the `GRANT PRIVILEGE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.121') - -RQ_SRS_006_RBAC_Grant_Privilege_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Grant.Privilege.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `GRANT` statement that\n' - 'grants explicit privileges to a user or a role.\n' - '\n' - '```sql\n' - 'GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...]\n' - ' ON {db.table|db.*|*.*|table|*}\n' - ' TO {user | role | CURRENT_USER} [,...]\n' - ' [WITH GRANT OPTION]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.122') - -RQ_SRS_006_RBAC_Revoke_Privilege_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking privileges to one or more users or roles\n' - 'for a database or a table on some specific cluster using the `REVOKE ON CLUSTER cluster_name` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.123') - -RQ_SRS_006_RBAC_Revoke_Privilege_Any = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Any', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking ANY privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE some_privilege` statement.\n' - '**some_privilege** refers to any Clickhouse defined privilege, whose hierarchy includes\n' - 'SELECT, INSERT, ALTER, CREATE, DROP, TRUNCATE, OPTIMIZE, SHOW, KILL QUERY, ACCESS MANAGEMENT,\n' - 'SYSTEM, INTROSPECTION, SOURCES, dictGet and all of their sub-privileges.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.124') - -RQ_SRS_006_RBAC_Revoke_Privilege_Select = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **select** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE SELECT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.125') - -RQ_SRS_006_RBAC_Revoke_Privilege_Insert = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Insert', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **insert** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE INSERT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.126') - -RQ_SRS_006_RBAC_Revoke_Privilege_Alter = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **alter** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE ALTER` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.127') - -RQ_SRS_006_RBAC_Revoke_Privilege_Create = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **create** privilege to one or more users or roles\n' - 'using the `REVOKE CREATE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.128') - -RQ_SRS_006_RBAC_Revoke_Privilege_Create_Database = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.Database', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **create database** privilege to one or more users or roles\n' - 'for a database using the `REVOKE CREATE DATABASE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.129') - -RQ_SRS_006_RBAC_Revoke_Privilege_Create_Dictionary = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.Dictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **create dictionary** privilege to one or more users or roles\n' - 'for a dictionary using the `REVOKE CREATE DICTIONARY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.130') - -RQ_SRS_006_RBAC_Revoke_Privilege_Create_Table = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.Table', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **create table** privilege to one or more users or roles\n' - 'for a table using the `REVOKE CREATE TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.131') - -RQ_SRS_006_RBAC_Revoke_Privilege_Create_TemporaryTable = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.TemporaryTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **create temporary table** privilege to one or more users or roles\n' - 'for a temporary table using the `REVOKE CREATE TEMPORARY TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.132') - -RQ_SRS_006_RBAC_Revoke_Privilege_Drop = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **drop** privilege to one or more users or roles\n' - 'using the `REVOKE DROP` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.133') - -RQ_SRS_006_RBAC_Revoke_Privilege_Drop_Database = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Database', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **drop database** privilege to one or more users or roles\n' - 'for a database using the `REVOKE DROP DATABASE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.134') - -RQ_SRS_006_RBAC_Revoke_Privilege_Drop_Dictionary = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Dictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **drop dictionary** privilege to one or more users or roles\n' - 'for a dictionary using the `REVOKE DROP DICTIONARY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.135') - -RQ_SRS_006_RBAC_Revoke_Privilege_Drop_Table = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Table', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **drop table** privilege to one or more users or roles\n' - 'for a table using the `REVOKE DROP TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.136') - -RQ_SRS_006_RBAC_Revoke_Privilege_Drop_TemporaryTable = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.TemporaryTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **drop temporary table** privilege to one or more users or roles\n' - 'for a temporary table using the `REVOKE DROP TEMPORARY TABLE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.137') - -RQ_SRS_006_RBAC_Revoke_Privilege_Truncate = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Truncate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **truncate** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE TRUNCATE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.138') - -RQ_SRS_006_RBAC_Revoke_Privilege_Optimize = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Optimize', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **optimize** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE OPTIMIZE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.139') - -RQ_SRS_006_RBAC_Revoke_Privilege_Show = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Show', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **show** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE SHOW` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.140') - -RQ_SRS_006_RBAC_Revoke_Privilege_KillQuery = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **kill query** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE KILL QUERY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.141') - -RQ_SRS_006_RBAC_Revoke_Privilege_AccessManagement = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **access management** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE ACCESS MANAGEMENT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.142') - -RQ_SRS_006_RBAC_Revoke_Privilege_System = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.System', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **system** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE SYSTEM` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.143') - -RQ_SRS_006_RBAC_Revoke_Privilege_Introspection = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Introspection', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **introspection** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE INTROSPECTION` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.144') - -RQ_SRS_006_RBAC_Revoke_Privilege_Sources = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Sources', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **sources** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE SOURCES` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.145') - -RQ_SRS_006_RBAC_Revoke_Privilege_DictGet = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.DictGet', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the **dictGet** privilege to one or more users or roles\n' - 'for a database or a table using the `REVOKE dictGet` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.146') - -RQ_SRS_006_RBAC_Revoke_Privilege_PrivelegeColumns = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking the privilege **some_privilege** to one or more users or roles\n' - 'for a database or a table using the `REVOKE some_privilege(column)` statement for one column.\n' - 'Multiple columns will be supported with `REVOKE some_privilege(column1, column2...)` statement.\n' - 'The privileges will be revoked for only the specified columns.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.147') - -RQ_SRS_006_RBAC_Revoke_Privilege_Multiple = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Multiple', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking MULTIPLE **privileges** to one or more users or roles\n' - 'for a database or a table using the `REVOKE privilege1, privilege2...` statement.\n' - '**privileges** refers to any set of Clickhouse defined privilege, whose hierarchy includes\n' - 'SELECT, INSERT, ALTER, CREATE, DROP, TRUNCATE, OPTIMIZE, SHOW, KILL QUERY, ACCESS MANAGEMENT,\n' - 'SYSTEM, INTROSPECTION, SOURCES, dictGet and all of their sub-privileges.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.148') - -RQ_SRS_006_RBAC_Revoke_Privilege_All = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **all** privileges to one or more users or roles\n' - 'for a database or a table using the `REVOKE ALL` or `REVOKE ALL PRIVILEGES` statements.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.149') - -RQ_SRS_006_RBAC_Revoke_Privilege_None = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **no** privileges to one or more users or roles\n' - 'for a database or a table using the `REVOKE NONE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.150') - -RQ_SRS_006_RBAC_Revoke_Privilege_On = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.On', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the `ON` clause in the `REVOKE` privilege statement\n' - 'which SHALL allow to specify one or more tables to which the privilege SHALL\n' - 'be revoked using the following patterns\n' - '\n' - '* `db.table` specific table in the specified database\n' - '* `db.*` any table in the specified database\n' - '* `*.*` any table in any database\n' - '* `table` specific table in the current database\n' - '* `*` any table in the current database\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.151') - -RQ_SRS_006_RBAC_Revoke_Privilege_From = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.From', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the `FROM` clause in the `REVOKE` privilege statement\n' - 'which SHALL allow to specify one or more users to which the privilege SHALL\n' - 'be revoked using the following patterns\n' - '\n' - '* `{user | CURRENT_USER} [,...]` some combination of users by name, which may include the current user\n' - '* `ALL` all users\n' - '* `ALL EXCEPT {user | CURRENT_USER} [,...]` the logical reverse of the first pattern\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.152') - -RQ_SRS_006_RBAC_Revoke_Privilege_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Privilege.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `REVOKE` statement that\n' - 'revokes explicit privileges of a user or a role.\n' - '\n' - '```sql\n' - 'REVOKE [ON CLUSTER cluster_name] privilege\n' - ' [(column_name [,...])] [,...]\n' - ' ON {db.table|db.*|*.*|table|*}\n' - ' FROM {user | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user | CURRENT_USER} [,...]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.153') - -RQ_SRS_006_RBAC_PartialRevoke_Syntax = Requirement( - name='RQ.SRS-006.RBAC.PartialRevoke.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support partial revokes by using `partial_revokes` variable\n' - 'that can be set or unset using the following syntax.\n' - '\n' - 'To disable partial revokes the `partial_revokes` variable SHALL be set to `0`\n' - '\n' - '```sql\n' - 'SET partial_revokes = 0\n' - '```\n' - '\n' - 'To enable partial revokes the `partial revokes` variable SHALL be set to `1`\n' - '\n' - '```sql\n' - 'SET partial_revokes = 1\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.154') - -RQ_SRS_006_RBAC_Grant_Role = Requirement( - name='RQ.SRS-006.RBAC.Grant.Role', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting one or more roles to\n' - 'one or more users or roles using the `GRANT` role statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.155') - -RQ_SRS_006_RBAC_Grant_Role_CurrentUser = Requirement( - name='RQ.SRS-006.RBAC.Grant.Role.CurrentUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting one or more roles to current user using\n' - '`TO CURRENT_USER` clause in the `GRANT` role statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.156') - -RQ_SRS_006_RBAC_Grant_Role_AdminOption = Requirement( - name='RQ.SRS-006.RBAC.Grant.Role.AdminOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting `admin option` privilege\n' - 'to one or more users or roles using the `WITH ADMIN OPTION` clause\n' - 'in the `GRANT` role statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.157') - -RQ_SRS_006_RBAC_Grant_Role_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.Grant.Role.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying cluster on which the user is to be granted one or more roles\n' - 'using `ON CLUSTER` clause in the `GRANT` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.158') - -RQ_SRS_006_RBAC_Grant_Role_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Grant.Role.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for `GRANT` role statement\n' - '\n' - '``` sql\n' - 'GRANT\n' - ' ON CLUSTER cluster_name\n' - ' role [, role ...]\n' - ' TO {user | role | CURRENT_USER} [,...]\n' - ' [WITH ADMIN OPTION]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.159') - -RQ_SRS_006_RBAC_Revoke_Role = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Role', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking one or more roles from\n' - 'one or more users or roles using the `REVOKE` role statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.160') - -RQ_SRS_006_RBAC_Revoke_Role_Keywords = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Role.Keywords', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking one or more roles from\n' - 'special groupings of one or more users or roles with the `ALL`, `ALL EXCEPT`,\n' - 'and `CURRENT_USER` keywords.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.161') - -RQ_SRS_006_RBAC_Revoke_Role_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Role.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking one or more roles from\n' - 'one or more users or roles from one or more clusters\n' - 'using the `REVOKE ON CLUSTER` role statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.162') - -RQ_SRS_006_RBAC_Revoke_AdminOption = Requirement( - name='RQ.SRS-006.RBAC.Revoke.AdminOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking `admin option` privilege\n' - 'in one or more users or roles using the `ADMIN OPTION FOR` clause\n' - 'in the `REVOKE` role statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.163') - -RQ_SRS_006_RBAC_Revoke_Role_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Revoke.Role.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `REVOKE` role statement\n' - '\n' - '```sql\n' - 'REVOKE [ON CLUSTER cluster_name] [ADMIN OPTION FOR]\n' - ' role [,...]\n' - ' FROM {user | role | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.164') - -RQ_SRS_006_RBAC_Show_Grants = Requirement( - name='RQ.SRS-006.RBAC.Show.Grants', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support listing all the privileges granted to current user and role\n' - 'using the `SHOW GRANTS` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.165') - -RQ_SRS_006_RBAC_Show_Grants_For = Requirement( - name='RQ.SRS-006.RBAC.Show.Grants.For', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support listing all the privileges granted to a user or a role\n' - 'using the `FOR` clause in the `SHOW GRANTS` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.166') - -RQ_SRS_006_RBAC_Show_Grants_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Show.Grants.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[Clickhouse] SHALL use the following syntax for the `SHOW GRANTS` statement\n' - '\n' - '``` sql\n' - 'SHOW GRANTS [FOR user_or_role]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.167') - -RQ_SRS_006_RBAC_SettingsProfile_Create = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creating settings profile using the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.168') - -RQ_SRS_006_RBAC_SettingsProfile_Create_IfNotExists = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE SETTINGS PROFILE` statement\n' - 'to skip raising an exception if a settings profile with the same **name** already exists.\n' - 'If `IF NOT EXISTS` clause is not specified then an exception SHALL be raised if\n' - 'a settings profile with the same **name** already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.169') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Replace = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Replace', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE SETTINGS PROFILE` statement\n' - 'to replace existing settings profile if it already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.170') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Variables = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning values and constraints to one or more\n' - 'variables in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.171') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Value = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning variable value in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.172') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Constraints = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support setting `MIN`, `MAX`, `READONLY`, and `WRITABLE`\n' - 'constraints for the variables in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.173') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning settings profile to one or more users\n' - 'or roles in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.174') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_None = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning settings profile to no users or roles using\n' - '`TO NONE` clause in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.175') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_All = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning settings profile to all current users and roles\n' - 'using `TO ALL` clause in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.176') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_AllExcept = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support excluding assignment to one or more users or roles using\n' - 'the `ALL EXCEPT` clause in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.177') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Inherit = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support inheriting profile settings from indicated profile using\n' - 'the `INHERIT` clause in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.178') - -RQ_SRS_006_RBAC_SettingsProfile_Create_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying what cluster to create settings profile on\n' - 'using `ON CLUSTER` clause in the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.179') - -RQ_SRS_006_RBAC_SettingsProfile_Create_Syntax = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `CREATE SETTINGS PROFILE` statement.\n' - '\n' - '``` sql\n' - 'CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name\n' - ' [ON CLUSTER cluster_name]\n' - " [SET varname [= value] [MIN min] [MAX max] [READONLY|WRITABLE] | [INHERIT 'profile_name'] [,...]]\n" - ' [TO {user_or_role [,...] | NONE | ALL | ALL EXCEPT user_or_role [,...]}]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.180') - -RQ_SRS_006_RBAC_SettingsProfile_Alter = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering settings profile using the `ALTER STETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.181') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_IfExists = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF EXISTS` clause in the `ALTER SETTINGS PROFILE` statement\n' - 'to not raise exception if a settings profile does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if a settings profile does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.182') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Rename = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support renaming settings profile using the `RANAME TO` clause\n' - 'in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.183') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering values and constraints of one or more\n' - 'variables in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.184') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering value of the variable in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.185') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Constraints = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering `MIN`, `MAX`, `READONLY`, and `WRITABLE`\n' - 'constraints for the variables in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.186') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning settings profile to one or more users\n' - 'or roles using the `TO` clause in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.187') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_None = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning settings profile to no users or roles using the\n' - '`TO NONE` clause in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.188') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_All = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning settings profile to all current users and roles\n' - 'using the `TO ALL` clause in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.189') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_AllExcept = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support excluding assignment to one or more users or roles using\n' - 'the `TO ALL EXCEPT` clause in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.190') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_Inherit = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the settings profile by inheriting settings from\n' - 'specified profile using `INHERIT` clause in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.191') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the settings profile on a specified cluster using\n' - '`ON CLUSTER` clause in the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.192') - -RQ_SRS_006_RBAC_SettingsProfile_Alter_Syntax = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `ALTER SETTINGS PROFILE` statement.\n' - '\n' - '``` sql\n' - 'ALTER SETTINGS PROFILE [IF EXISTS] name\n' - ' [ON CLUSTER cluster_name]\n' - ' [RENAME TO new_name]\n' - " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...]\n" - ' [TO {user_or_role [,...] | NONE | ALL | ALL EXCEPT user_or_role [,...]]}\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.193') - -RQ_SRS_006_RBAC_SettingsProfile_Drop = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing one or more settings profiles using the `DROP SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.194') - -RQ_SRS_006_RBAC_SettingsProfile_Drop_IfExists = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP SETTINGS PROFILE` statement\n' - 'to skip raising an exception if the settings profile does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if a settings profile does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.195') - -RQ_SRS_006_RBAC_SettingsProfile_Drop_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support dropping one or more settings profiles on specified cluster using\n' - '`ON CLUSTER` clause in the `DROP SETTINGS PROFILE` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.196') - -RQ_SRS_006_RBAC_SettingsProfile_Drop_Syntax = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `DROP SETTINGS PROFILE` statement\n' - '\n' - '``` sql\n' - 'DROP SETTINGS PROFILE [IF EXISTS] name [,name,...]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.197') - -RQ_SRS_006_RBAC_SettingsProfile_ShowCreateSettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the `CREATE SETTINGS PROFILE` statement used to create the settings profile\n' - 'using the `SHOW CREATE SETTINGS PROFILE` statement with the following syntax\n' - '\n' - '``` sql\n' - 'SHOW CREATE SETTINGS PROFILE name\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.198') - -RQ_SRS_006_RBAC_Quota_Create = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creating quotas using the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.199') - -RQ_SRS_006_RBAC_Quota_Create_IfNotExists = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.IfNotExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE QUOTA` statement\n' - 'to skip raising an exception if a quota with the same **name** already exists.\n' - 'If `IF NOT EXISTS` clause is not specified then an exception SHALL be raised if\n' - 'a quota with the same **name** already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.200') - -RQ_SRS_006_RBAC_Quota_Create_Replace = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Replace', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE QUOTA` statement\n' - 'to replace existing quota if it already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.201') - -RQ_SRS_006_RBAC_Quota_Create_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creating quotas on a specific cluster with the\n' - '`ON CLUSTER` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.202') - -RQ_SRS_006_RBAC_Quota_Create_Interval = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Interval', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support defining the quota interval that specifies\n' - 'a period of time over for which the quota SHALL apply using the\n' - '`FOR INTERVAL` clause in the `CREATE QUOTA` statement.\n' - '\n' - 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' - '\n' - '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number\n' - 'to define the interval.\n' - '\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.203') - -RQ_SRS_006_RBAC_Quota_Create_Interval_Randomized = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support defining the quota randomized interval that specifies\n' - 'a period of time over for which the quota SHALL apply using the\n' - '`FOR RANDOMIZED INTERVAL` clause in the `CREATE QUOTA` statement.\n' - '\n' - 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' - '\n' - '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some\n' - 'real number to define the interval.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.204') - -RQ_SRS_006_RBAC_Quota_Create_Queries = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Queries', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting number of requests over a period of time\n' - 'using the `QUERIES` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.205') - -RQ_SRS_006_RBAC_Quota_Create_Errors = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Errors', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting number of queries that threw an exception\n' - 'using the `ERRORS` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.206') - -RQ_SRS_006_RBAC_Quota_Create_ResultRows = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.ResultRows', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the total number of rows given as the result\n' - 'using the `RESULT ROWS` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.207') - -RQ_SRS_006_RBAC_Quota_Create_ReadRows = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.ReadRows', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the total number of source rows read from tables\n' - 'for running the query on all remote servers\n' - 'using the `READ ROWS` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.208') - -RQ_SRS_006_RBAC_Quota_Create_ResultBytes = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.ResultBytes', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the total number of bytes that can be returned as the result\n' - 'using the `RESULT BYTES` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.209') - -RQ_SRS_006_RBAC_Quota_Create_ReadBytes = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.ReadBytes', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the total number of source bytes read from tables\n' - 'for running the query on all remote servers\n' - 'using the `READ BYTES` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.210') - -RQ_SRS_006_RBAC_Quota_Create_ExecutionTime = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.ExecutionTime', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the maximum query execution time\n' - 'using the `EXECUTION TIME` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.211') - -RQ_SRS_006_RBAC_Quota_Create_NoLimits = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.NoLimits', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the maximum query execution time\n' - 'using the `NO LIMITS` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.212') - -RQ_SRS_006_RBAC_Quota_Create_TrackingOnly = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.TrackingOnly', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the maximum query execution time\n' - 'using the `TRACKING ONLY` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.213') - -RQ_SRS_006_RBAC_Quota_Create_KeyedBy = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.KeyedBy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support to track quota for some key\n' - 'following the `KEYED BY` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.214') - -RQ_SRS_006_RBAC_Quota_Create_KeyedByOptions = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support to track quota separately for some parameter\n' - "using the `KEYED BY 'parameter'` clause in the `CREATE QUOTA` statement.\n" - '\n' - "'parameter' can be one of:\n" - "`{'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}`\n" - '\n' - ), - link=None, - level=4, - num='5.2.8.215') - -RQ_SRS_006_RBAC_Quota_Create_Assignment = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Assignment', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning quota to one or more users\n' - 'or roles using the `TO` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.216') - -RQ_SRS_006_RBAC_Quota_Create_Assignment_None = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Assignment.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning quota to no users or roles using\n' - '`TO NONE` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.217') - -RQ_SRS_006_RBAC_Quota_Create_Assignment_All = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Assignment.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning quota to all current users and roles\n' - 'using `TO ALL` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.218') - -RQ_SRS_006_RBAC_Quota_Create_Assignment_Except = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Assignment.Except', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support excluding assignment of quota to one or more users or roles using\n' - 'the `EXCEPT` clause in the `CREATE QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.219') - -RQ_SRS_006_RBAC_Quota_Create_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Quota.Create.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `CREATE QUOTA` statement\n' - '\n' - '```sql\n' - 'CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]\n' - " [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]\n" - ' [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}\n' - ' {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |\n' - ' NO LIMITS | TRACKING ONLY} [,...]]\n' - ' [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.220') - -RQ_SRS_006_RBAC_Quota_Alter = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering quotas using the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.221') - -RQ_SRS_006_RBAC_Quota_Alter_IfExists = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF EXISTS` clause in the `ALTER QUOTA` statement\n' - 'to skip raising an exception if a quota does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be raised if\n' - 'a quota does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.222') - -RQ_SRS_006_RBAC_Quota_Alter_Rename = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Rename', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `RENAME TO` clause in the `ALTER QUOTA` statement\n' - 'to rename the quota to the specified name.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.223') - -RQ_SRS_006_RBAC_Quota_Alter_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering quotas on a specific cluster with the\n' - '`ON CLUSTER` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.224') - -RQ_SRS_006_RBAC_Quota_Alter_Interval = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Interval', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support redefining the quota interval that specifies\n' - 'a period of time over for which the quota SHALL apply using the\n' - '`FOR INTERVAL` clause in the `ALTER QUOTA` statement.\n' - '\n' - 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' - '\n' - '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number\n' - 'to define the interval.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.225') - -RQ_SRS_006_RBAC_Quota_Alter_Interval_Randomized = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support redefining the quota randomized interval that specifies\n' - 'a period of time over for which the quota SHALL apply using the\n' - '`FOR RANDOMIZED INTERVAL` clause in the `ALTER QUOTA` statement.\n' - '\n' - 'This statement SHALL also support a number and a time period which will be one\n' - 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' - '\n' - '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some\n' - 'real number to define the interval.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.226') - -RQ_SRS_006_RBAC_Quota_Alter_Queries = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Queries', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the limit of number of requests over a period of time\n' - 'using the `QUERIES` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.227') - -RQ_SRS_006_RBAC_Quota_Alter_Errors = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Errors', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the limit of number of queries that threw an exception\n' - 'using the `ERRORS` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.228') - -RQ_SRS_006_RBAC_Quota_Alter_ResultRows = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.ResultRows', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the limit of the total number of rows given as the result\n' - 'using the `RESULT ROWS` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.229') - -RQ_SRS_006_RBAC_Quota_Alter_ReadRows = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.ReadRows', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the limit of the total number of source rows read from tables\n' - 'for running the query on all remote servers\n' - 'using the `READ ROWS` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.230') - -RQ_SRS_006_RBAC_Quota_ALter_ResultBytes = Requirement( - name='RQ.SRS-006.RBAC.Quota.ALter.ResultBytes', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the limit of the total number of bytes that can be returned as the result\n' - 'using the `RESULT BYTES` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.231') - -RQ_SRS_006_RBAC_Quota_Alter_ReadBytes = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.ReadBytes', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the limit of the total number of source bytes read from tables\n' - 'for running the query on all remote servers\n' - 'using the `READ BYTES` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.232') - -RQ_SRS_006_RBAC_Quota_Alter_ExecutionTime = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering the limit of the maximum query execution time\n' - 'using the `EXECUTION TIME` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.233') - -RQ_SRS_006_RBAC_Quota_Alter_NoLimits = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.NoLimits', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the maximum query execution time\n' - 'using the `NO LIMITS` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.234') - -RQ_SRS_006_RBAC_Quota_Alter_TrackingOnly = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support limiting the maximum query execution time\n' - 'using the `TRACKING ONLY` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.235') - -RQ_SRS_006_RBAC_Quota_Alter_KeyedBy = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.KeyedBy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering quota to track quota separately for some key\n' - 'following the `KEYED BY` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.236') - -RQ_SRS_006_RBAC_Quota_Alter_KeyedByOptions = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering quota to track quota separately for some parameter\n' - "using the `KEYED BY 'parameter'` clause in the `ALTER QUOTA` statement.\n" - '\n' - "'parameter' can be one of:\n" - "`{'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}`\n" - '\n' - ), - link=None, - level=4, - num='5.2.8.237') - -RQ_SRS_006_RBAC_Quota_Alter_Assignment = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Assignment', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning quota to one or more users\n' - 'or roles using the `TO` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.238') - -RQ_SRS_006_RBAC_Quota_Alter_Assignment_None = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning quota to no users or roles using\n' - '`TO NONE` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.239') - -RQ_SRS_006_RBAC_Quota_Alter_Assignment_All = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning quota to all current users and roles\n' - 'using `TO ALL` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.240') - -RQ_SRS_006_RBAC_Quota_Alter_Assignment_Except = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support excluding assignment of quota to one or more users or roles using\n' - 'the `EXCEPT` clause in the `ALTER QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.241') - -RQ_SRS_006_RBAC_Quota_Alter_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Quota.Alter.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `ALTER QUOTA` statement\n' - '\n' - '``` sql\n' - 'ALTER QUOTA [IF EXIST] name\n' - ' {{{QUERIES | ERRORS | RESULT ROWS | READ ROWS | RESULT BYTES | READ BYTES | EXECUTION TIME} number} [, ...] FOR INTERVAL number time_unit} [, ...]\n' - ' [KEYED BY USERNAME | KEYED BY IP | NOT KEYED] [ALLOW CUSTOM KEY | DISALLOW CUSTOM KEY]\n' - ' [TO {user_or_role [,...] | NONE | ALL} [EXCEPT user_or_role [,...]]]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.242') - -RQ_SRS_006_RBAC_Quota_Drop = Requirement( - name='RQ.SRS-006.RBAC.Quota.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing one or more quotas using the `DROP QUOTA` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.243') - -RQ_SRS_006_RBAC_Quota_Drop_IfExists = Requirement( - name='RQ.SRS-006.RBAC.Quota.Drop.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP QUOTA` statement\n' - 'to skip raising an exception when the quota does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if the quota does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.244') - -RQ_SRS_006_RBAC_Quota_Drop_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Quota.Drop.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using `ON CLUSTER` clause in the `DROP QUOTA` statement\n' - 'to indicate the cluster the quota to be dropped is located on.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.245') - -RQ_SRS_006_RBAC_Quota_Drop_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Quota.Drop.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `DROP QUOTA` statement\n' - '\n' - '``` sql\n' - 'DROP QUOTA [IF EXISTS] name [,name...]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.246') - -RQ_SRS_006_RBAC_Quota_ShowQuotas = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowQuotas', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing all of the current quotas\n' - 'using the `SHOW QUOTAS` statement with the following syntax\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.247') - -RQ_SRS_006_RBAC_Quota_ShowQuotas_IntoOutfile = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the `INTO OUTFILE` clause in the `SHOW QUOTAS` statement to define an outfile by some given string literal.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.248') - -RQ_SRS_006_RBAC_Quota_ShowQuotas_Format = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Format', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the `FORMAT` clause in the `SHOW QUOTAS` statement to define a format for the output quota list.\n' - '\n' - 'The types of valid formats are many, listed in output column:\n' - 'https://clickhouse.tech/docs/en/interfaces/formats/\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.249') - -RQ_SRS_006_RBAC_Quota_ShowQuotas_Settings = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the `SETTINGS` clause in the `SHOW QUOTAS` statement to define settings in the showing of all quotas.\n' - '\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.250') - -RQ_SRS_006_RBAC_Quota_ShowQuotas_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using the `SHOW QUOTAS` statement\n' - 'with the following syntax\n' - '``` sql\n' - 'SHOW QUOTAS\n' - '```\n' - ), - link=None, - level=4, - num='5.2.8.251') - -RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Name = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the `CREATE QUOTA` statement used to create the quota with some given name\n' - 'using the `SHOW CREATE QUOTA` statement with the following syntax\n' - '\n' - '``` sql\n' - 'SHOW CREATE QUOTA name\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.252') - -RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Current = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the `CREATE QUOTA` statement used to create the CURRENT quota\n' - 'using the `SHOW CREATE QUOTA CURRENT` statement or the shorthand form\n' - '`SHOW CREATE QUOTA`\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.253') - -RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Syntax = Requirement( - name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax when\n' - 'using the `SHOW CREATE QUOTA` statement.\n' - '\n' - '```sql\n' - 'SHOW CREATE QUOTA [name | CURRENT]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.254') - -RQ_SRS_006_RBAC_RowPolicy_Create = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support creating row policy using the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.255') - -RQ_SRS_006_RBAC_RowPolicy_Create_IfNotExists = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE ROW POLICY` statement\n' - 'to skip raising an exception if a row policy with the same **name** already exists.\n' - 'If the `IF NOT EXISTS` clause is not specified then an exception SHALL be raised if\n' - 'a row policy with the same **name** already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.256') - -RQ_SRS_006_RBAC_RowPolicy_Create_Replace = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Replace', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE ROW POLICY` statement\n' - 'to replace existing row policy if it already exists.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.257') - -RQ_SRS_006_RBAC_RowPolicy_Create_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying cluster on which to create the role policy\n' - 'using the `ON CLUSTER` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.258') - -RQ_SRS_006_RBAC_RowPolicy_Create_On = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.On', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying table on which to create the role policy\n' - 'using the `ON` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.259') - -RQ_SRS_006_RBAC_RowPolicy_Create_Access = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Access', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support allowing or restricting access to rows using the\n' - '`AS` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.260') - -RQ_SRS_006_RBAC_RowPolicy_Create_Access_Permissive = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support allowing access to rows using the\n' - '`AS PERMISSIVE` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.261') - -RQ_SRS_006_RBAC_RowPolicy_Create_Access_Restrictive = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support restricting access to rows using the\n' - '`AS RESTRICTIVE` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.262') - -RQ_SRS_006_RBAC_RowPolicy_Create_ForSelect = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying which rows are affected\n' - 'using the `FOR SELECT` clause in the `CREATE ROW POLICY` statement.\n' - 'REQUIRES CONFIRMATION\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.263') - -RQ_SRS_006_RBAC_RowPolicy_Create_Condition = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Condition', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying a condition that\n' - 'that can be any SQL expression which returns a boolean using the `USING`\n' - 'clause in the `CREATE ROW POLOCY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.264') - -RQ_SRS_006_RBAC_RowPolicy_Create_Assignment = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning row policy to one or more users\n' - 'or roles using the `TO` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.265') - -RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_None = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning row policy to no users or roles using\n' - 'the `TO NONE` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.266') - -RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_All = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support assigning row policy to all current users and roles\n' - 'using `TO ALL` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.267') - -RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_AllExcept = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support excluding assignment of row policy to one or more users or roles using\n' - 'the `ALL EXCEPT` clause in the `CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.268') - -RQ_SRS_006_RBAC_RowPolicy_Create_Syntax = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Create.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `CRETE ROW POLICY` statement\n' - '\n' - '``` sql\n' - 'CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name [ON CLUSTER cluster_name] ON [db.]table\n' - ' [AS {PERMISSIVE | RESTRICTIVE}]\n' - ' [FOR SELECT]\n' - ' [USING condition]\n' - ' [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.269') - -RQ_SRS_006_RBAC_RowPolicy_Alter = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering row policy using the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.270') - -RQ_SRS_006_RBAC_RowPolicy_Alter_IfExists = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the `IF EXISTS` clause in the `ALTER ROW POLICY` statement\n' - 'to skip raising an exception if a row policy does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be raised if\n' - 'a row policy does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.271') - -RQ_SRS_006_RBAC_RowPolicy_Alter_ForSelect = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support modifying rows on which to apply the row policy\n' - 'using the `FOR SELECT` clause in the `ALTER ROW POLICY` statement.\n' - 'REQUIRES FUNCTION CONFIRMATION.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.272') - -RQ_SRS_006_RBAC_RowPolicy_Alter_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying cluster on which to alter the row policy\n' - 'using the `ON CLUSTER` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.273') - -RQ_SRS_006_RBAC_RowPolicy_Alter_On = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.On', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support specifying table on which to alter the row policy\n' - 'using the `ON` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.274') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Rename = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Rename', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support renaming the row policy using the `RENAME` clause\n' - 'in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.275') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Access = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support altering access to rows using the\n' - '`AS` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.276') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Access_Permissive = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support permitting access to rows using the\n' - '`AS PERMISSIVE` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.277') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Access_Restrictive = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support restricting access to rows using the\n' - '`AS RESTRICTIVE` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.278') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Condition = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Condition', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support re-specifying the row policy condition\n' - 'using the `USING` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.279') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Condition_None = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing the row policy condition\n' - 'using the `USING NONE` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.280') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning row policy to one or more users\n' - 'or roles using the `TO` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.281') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_None = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning row policy to no users or roles using\n' - 'the `TO NONE` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.282') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_All = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support reassigning row policy to all current users and roles\n' - 'using the `TO ALL` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.283') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_AllExcept = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support excluding assignment of row policy to one or more users or roles using\n' - 'the `ALL EXCEPT` clause in the `ALTER ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.284') - -RQ_SRS_006_RBAC_RowPolicy_Alter_Syntax = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `ALTER ROW POLICY` statement\n' - '\n' - '``` sql\n' - 'ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]table\n' - ' [RENAME TO new_name]\n' - ' [AS {PERMISSIVE | RESTRICTIVE}]\n' - ' [FOR SELECT]\n' - ' [USING {condition | NONE}][,...]\n' - ' [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.285') - -RQ_SRS_006_RBAC_RowPolicy_Drop = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing one or more row policies using the `DROP ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.286') - -RQ_SRS_006_RBAC_RowPolicy_Drop_IfExists = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support using the `IF EXISTS` clause in the `DROP ROW POLICY` statement\n' - 'to skip raising an exception when the row policy does not exist.\n' - 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' - 'raised if the row policy does not exist.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.287') - -RQ_SRS_006_RBAC_RowPolicy_Drop_On = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Drop.On', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing row policy from one or more specified tables\n' - 'using the `ON` clause in the `DROP ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.288') - -RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support removing row policy from specified cluster\n' - 'using the `ON CLUSTER` clause in the `DROP ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.289') - -RQ_SRS_006_RBAC_RowPolicy_Drop_Syntax = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for the `DROP ROW POLICY` statement.\n' - '\n' - '``` sql\n' - 'DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.290') - -RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing the `CREATE ROW POLICY` statement used to create the row policy\n' - 'using the `SHOW CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.291') - -RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy_On = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing statement used to create row policy on specific table\n' - 'using the `ON` in the `SHOW CREATE ROW POLICY` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.292') - -RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy_Syntax = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for `SHOW CREATE ROW POLICY`.\n' - '\n' - '``` sql\n' - 'SHOW CREATE [ROW] POLICY name ON [database.]table\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.293') - -RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing row policies using the `SHOW ROW POLICIES` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.294') - -RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies_On = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support showing row policies on a specific table\n' - 'using the `ON` clause in the `SHOW ROW POLICIES` statement.\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.295') - -RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies_Syntax = Requirement( - name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support the following syntax for `SHOW ROW POLICIES`.\n' - '\n' - '```sql\n' - 'SHOW [ROW] POLICIES [ON [database.]table]\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.8.296') - -RQ_SRS_006_RBAC_Table_PublicTables = Requirement( - name='RQ.SRS-006.RBAC.Table.PublicTables', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support that a user without any privileges will be able to access the following tables\n' - '\n' - '* system.one\n' - '* system.numbers\n' - '* system.contributors\n' - '* system.functions\n' - '\n' - ), - link=None, - level=4, - num='5.2.9.1') - -RQ_SRS_006_RBAC_Table_QueryLog = Requirement( - name='RQ.SRS-006.RBAC.Table.QueryLog', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL return only queries executed by the user when the user is selecting from system.query_log.\n' - '\n' - ), - link=None, - level=4, - num='5.2.9.2') - -RQ_SRS_006_RBAC_Table_DistributedTable_Create = Requirement( - name='RQ.SRS-006.RBAC.Table.DistributedTable.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully `CREATE` a distributed table if and only if\n' - 'the user has **create table** privilege on the table and **remote** privilege on *.*\n' - '\n' - ), - link=None, - level=5, - num='5.2.9.3.1') - -RQ_SRS_006_RBAC_Table_DistributedTable_Select = Requirement( - name='RQ.SRS-006.RBAC.Table.DistributedTable.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully `SELECT` from a distributed table if and only if\n' - 'the user has **select** privilege on the table and on the remote table specified in the `CREATE` query of the distributed table.\n' - '\n' - 'Does not require **select** privilege for the remote table if the remote table does not exist on the same server as the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.9.3.2') - -RQ_SRS_006_RBAC_Table_DistributedTable_Insert = Requirement( - name='RQ.SRS-006.RBAC.Table.DistributedTable.Insert', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully `INSERT` into a distributed table if and only if\n' - 'the user has **insert** privilege on the table and on the remote table specified in the `CREATE` query of the distributed table.\n' - '\n' - 'Does not require **insert** privilege for the remote table if the remote table does not exist on the same server as the user,\n' - 'insert executes into the remote table on a different server.\n' - '\n' - ), - link=None, - level=5, - num='5.2.9.3.3') - -RQ_SRS_006_RBAC_Table_DistributedTable_SpecialTables = Requirement( - name='RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute a query using a distributed table that uses one of the special tables if and only if\n' - 'the user has the necessary privileges to interact with that special table, either granted directly or through a role.\n' - 'Special tables include:\n' - '* materialized view\n' - '* distributed table\n' - '* source table of a materialized view\n' - '\n' - ), - link=None, - level=5, - num='5.2.9.3.4') - -RQ_SRS_006_RBAC_Table_DistributedTable_LocalUser = Requirement( - name='RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute a query using a distributed table from\n' - 'a user present locally, but not remotely.\n' - '\n' - ), - link=None, - level=5, - num='5.2.9.3.5') - -RQ_SRS_006_RBAC_Table_DistributedTable_SameUserDifferentNodesDifferentPrivileges = Requirement( - name='RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute a query using a distributed table by a user that exists on multiple nodes\n' - 'if and only if the user has the required privileges on the node the query is being executed from.\n' - '\n' - ), - link=None, - level=5, - num='5.2.9.3.6') - -RQ_SRS_006_RBAC_View = Requirement( - name='RQ.SRS-006.RBAC.View', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to **create**, **select** and **drop**\n' - 'privileges for a view for users or roles.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.1.1') - -RQ_SRS_006_RBAC_View_Create = Requirement( - name='RQ.SRS-006.RBAC.View.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `CREATE VIEW` command if and only if\n' - 'the user has **create view** privilege either explicitly or through roles.\n' - '\n' - 'If the stored query includes one or more source tables, the user must have **select** privilege\n' - 'on all the source tables either explicitly or through a role.\n' - 'For example,\n' - '```sql\n' - 'CREATE VIEW view AS SELECT * FROM source_table\n' - 'CREATE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' - 'CREATE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' - 'CREATE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' - 'CREATE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' - 'CREATE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.1.2') - -RQ_SRS_006_RBAC_View_Select = Requirement( - name='RQ.SRS-006.RBAC.View.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully `SELECT` from a view if and only if\n' - 'the user has **select** privilege for that view either explicitly or through a role.\n' - '\n' - 'If the stored query includes one or more source tables, the user must have **select** privilege\n' - 'on all the source tables either explicitly or through a role.\n' - 'For example,\n' - '```sql\n' - 'CREATE VIEW view AS SELECT * FROM source_table\n' - 'CREATE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' - 'CREATE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' - 'CREATE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' - 'CREATE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' - 'CREATE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' - '\n' - 'SELECT * FROM view\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.1.3') - -RQ_SRS_006_RBAC_View_Drop = Requirement( - name='RQ.SRS-006.RBAC.View.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `DROP VIEW` command if and only if\n' - 'the user has **drop view** privilege on that view either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.1.4') - -RQ_SRS_006_RBAC_MaterializedView = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to **create**, **select**, **alter** and **drop**\n' - 'privileges for a materialized view for users or roles.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.1') - -RQ_SRS_006_RBAC_MaterializedView_Create = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `CREATE MATERIALIZED VIEW` command if and only if\n' - 'the user has **create view** privilege either explicitly or through roles.\n' - '\n' - 'If `POPULATE` is specified, the user must have `INSERT` privilege on the view,\n' - 'either explicitly or through roles.\n' - 'For example,\n' - '```sql\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory POPULATE AS SELECT * FROM source_table\n' - '```\n' - '\n' - 'If the stored query includes one or more source tables, the user must have **select** privilege\n' - 'on all the source tables either explicitly or through a role.\n' - 'For example,\n' - '```sql\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM source_table\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 JOIN table1 USING column\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' - 'CREATE MATERIALIZED VIEW view0 ENGINE = Memory AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' - '```\n' - '\n' - 'If the materialized view has a target table explicitly declared in the `TO` clause, the user must have\n' - '**insert** and **select** privilege on the target table.\n' - 'For example,\n' - '```sql\n' - 'CREATE MATERIALIZED VIEW view TO target_table AS SELECT * FROM source_table\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.2') - -RQ_SRS_006_RBAC_MaterializedView_Select = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully `SELECT` from a materialized view if and only if\n' - 'the user has **select** privilege for that view either explicitly or through a role.\n' - '\n' - 'If the stored query includes one or more source tables, the user must have **select** privilege\n' - 'on all the source tables either explicitly or through a role.\n' - 'For example,\n' - '```sql\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM source_table\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 JOIN table1 USING column\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' - 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' - 'CREATE MATERIALIZED VIEW view0 ENGINE = Memory AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' - '\n' - 'SELECT * FROM view\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.3') - -RQ_SRS_006_RBAC_MaterializedView_Select_TargetTable = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Select.TargetTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully `SELECT` from the target table, implicit or explicit, of a materialized view if and only if\n' - 'the user has `SELECT` privilege for the table, either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.4') - -RQ_SRS_006_RBAC_MaterializedView_Select_SourceTable = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Select.SourceTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully `SELECT` from the source table of a materialized view if and only if\n' - 'the user has `SELECT` privilege for the table, either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.5') - -RQ_SRS_006_RBAC_MaterializedView_Drop = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `DROP VIEW` command if and only if\n' - 'the user has **drop view** privilege on that view either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.6') - -RQ_SRS_006_RBAC_MaterializedView_ModifyQuery = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.ModifyQuery', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `MODIFY QUERY` command if and only if\n' - 'the user has **modify query** privilege on that view either explicitly or through a role.\n' - '\n' - 'If the new query includes one or more source tables, the user must have **select** privilege\n' - 'on all the source tables either explicitly or through a role.\n' - 'For example,\n' - '```sql\n' - 'ALTER TABLE view MODIFY QUERY SELECT * FROM source_table\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.7') - -RQ_SRS_006_RBAC_MaterializedView_Insert = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Insert', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only succesfully `INSERT` into a materialized view if and only if\n' - 'the user has `INSERT` privilege on the view, either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.8') - -RQ_SRS_006_RBAC_MaterializedView_Insert_SourceTable = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Insert.SourceTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only succesfully `INSERT` into a source table of a materialized view if and only if\n' - 'the user has `INSERT` privilege on the source table, either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.9') - -RQ_SRS_006_RBAC_MaterializedView_Insert_TargetTable = Requirement( - name='RQ.SRS-006.RBAC.MaterializedView.Insert.TargetTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only succesfully `INSERT` into a target table of a materialized view if and only if\n' - 'the user has `INSERT` privelege on the target table, either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.2.10') - -RQ_SRS_006_RBAC_LiveView = Requirement( - name='RQ.SRS-006.RBAC.LiveView', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to **create**, **select**, **alter** and **drop**\n' - 'privileges for a live view for users or roles.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.3.1') - -RQ_SRS_006_RBAC_LiveView_Create = Requirement( - name='RQ.SRS-006.RBAC.LiveView.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `CREATE LIVE VIEW` command if and only if\n' - 'the user has **create view** privilege either explicitly or through roles.\n' - '\n' - 'If the stored query includes one or more source tables, the user must have **select** privilege\n' - 'on all the source tables either explicitly or through a role.\n' - 'For example,\n' - '```sql\n' - 'CREATE LIVE VIEW view AS SELECT * FROM source_table\n' - 'CREATE LIVE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' - 'CREATE LIVE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' - 'CREATE LIVE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' - 'CREATE LIVE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' - 'CREATE LIVE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.3.2') - -RQ_SRS_006_RBAC_LiveView_Select = Requirement( - name='RQ.SRS-006.RBAC.LiveView.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully `SELECT` from a live view if and only if\n' - 'the user has **select** privilege for that view either explicitly or through a role.\n' - '\n' - 'If the stored query includes one or more source tables, the user must have **select** privilege\n' - 'on all the source tables either explicitly or through a role.\n' - 'For example,\n' - '```sql\n' - 'CREATE LIVE VIEW view AS SELECT * FROM source_table\n' - 'CREATE LIVE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' - 'CREATE LIVE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' - 'CREATE LIVE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' - 'CREATE LIVE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' - 'CREATE LIVE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' - '\n' - 'SELECT * FROM view\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.3.3') - -RQ_SRS_006_RBAC_LiveView_Drop = Requirement( - name='RQ.SRS-006.RBAC.LiveView.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `DROP VIEW` command if and only if\n' - 'the user has **drop view** privilege on that view either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.3.4') - -RQ_SRS_006_RBAC_LiveView_Refresh = Requirement( - name='RQ.SRS-006.RBAC.LiveView.Refresh', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute an `ALTER LIVE VIEW REFRESH` command if and only if\n' - 'the user has **refresh** privilege on that view either explicitly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.10.3.5') - -RQ_SRS_006_RBAC_Privileges_Usage = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Usage', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **usage** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.1') - -RQ_SRS_006_RBAC_Privileges_Select = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not execute `SELECT INTO` if and only if the user\n' - 'has the **select** privilege for the destination table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.2.1') - -RQ_SRS_006_RBAC_Privileges_Select_Column = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select.Column', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **select** privilege\n' - 'for one or more specified columns in a table to one or more **users** or **roles**.\n' - 'Any `SELECT INTO` statements SHALL not to be executed, unless the user\n' - 'has the **select** privilege for the destination column\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.2.2') - -RQ_SRS_006_RBAC_Privileges_Select_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **select** privilege\n' - 'on a specified cluster to one or more **users** or **roles**.\n' - 'Any `SELECT INTO` statements SHALL succeed only on nodes where\n' - 'the table exists and privilege was granted.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.2.3') - -RQ_SRS_006_RBAC_Privileges_Select_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Select.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **select** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.2.4') - -RQ_SRS_006_RBAC_Privileges_Insert = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL execute `INSERT INTO` if and only if the user\n' - 'has the **insert** privilege for the destination table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.3.1') - -RQ_SRS_006_RBAC_Privileges_Insert_Column = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert.Column', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **insert** privilege\n' - 'for one or more specified columns in a table to one or more **users** or **roles**.\n' - 'Any `INSERT INTO` statements SHALL not to be executed, unless the user\n' - 'has the **insert** privilege for the destination column\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.3.2') - -RQ_SRS_006_RBAC_Privileges_Insert_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **insert** privilege\n' - 'on a specified cluster to one or more **users** or **roles**.\n' - 'Any `INSERT INTO` statements SHALL succeed only on nodes where\n' - 'the table exists and privilege was granted.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.3.3') - -RQ_SRS_006_RBAC_Privileges_Insert_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Insert.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **insert** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.3.4') - -RQ_SRS_006_RBAC_Privileges_AlterColumn = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter column** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL\n' - 'return an error, unless the user has the **alter column** privilege for\n' - 'the destination table either because of the explicit grant or through one of\n' - 'the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.4.1') - -RQ_SRS_006_RBAC_Privileges_AlterColumn_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter column** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.4.2') - -RQ_SRS_006_RBAC_Privileges_AlterColumn_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter column** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.4.3') - -RQ_SRS_006_RBAC_Privileges_AlterColumn_Column = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Column', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **alter column** privilege\n' - 'for one or more specified columns in a table to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL return an error,\n' - 'unless the user has the **alter column** privilege for the destination column\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.4.4') - -RQ_SRS_006_RBAC_Privileges_AlterColumn_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **alter column** privilege\n' - 'on a specified cluster to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN`\n' - 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.4.5') - -RQ_SRS_006_RBAC_Privileges_AlterColumn_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter column** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.4.6') - -RQ_SRS_006_RBAC_Privileges_AlterIndex = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter index** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements SHALL\n' - 'return an error, unless the user has the **alter index** privilege for\n' - 'the destination table either because of the explicit grant or through one of\n' - 'the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.5.1') - -RQ_SRS_006_RBAC_Privileges_AlterIndex_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter index** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.5.2') - -RQ_SRS_006_RBAC_Privileges_AlterIndex_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter index** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.5.3') - -RQ_SRS_006_RBAC_Privileges_AlterIndex_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **alter index** privilege\n' - 'on a specified cluster to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX`\n' - 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.5.4') - -RQ_SRS_006_RBAC_Privileges_AlterIndex_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter index** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.5.5') - -RQ_SRS_006_RBAC_Privileges_AlterConstraint = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter constraint** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ADD|CREATE CONSTRAINT` statements SHALL\n' - 'return an error, unless the user has the **alter constraint** privilege for\n' - 'the destination table either because of the explicit grant or through one of\n' - 'the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.6.1') - -RQ_SRS_006_RBAC_Privileges_AlterConstraint_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter constraint** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.6.2') - -RQ_SRS_006_RBAC_Privileges_AlterConstraint_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter constraint** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.6.3') - -RQ_SRS_006_RBAC_Privileges_AlterConstraint_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **alter constraint** privilege\n' - 'on a specified cluster to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ADD|DROP CONSTRAINT`\n' - 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.6.4') - -RQ_SRS_006_RBAC_Privileges_AlterConstraint_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter constraint** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.6.5') - -RQ_SRS_006_RBAC_Privileges_AlterTTL = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter ttl** or **alter materialize ttl** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements SHALL\n' - 'return an error, unless the user has the **alter ttl** or **alter materialize ttl** privilege for\n' - 'the destination table either because of the explicit grant or through one of\n' - 'the roles assigned to the user.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.7.1') - -RQ_SRS_006_RBAC_Privileges_AlterTTL_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter ttl** or **alter materialize ttl** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.7.2') - -RQ_SRS_006_RBAC_Privileges_AlterTTL_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter ttl** or **alter materialize ttl** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.7.3') - -RQ_SRS_006_RBAC_Privileges_AlterTTL_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **alter ttl** or **alter materialize ttl** privilege\n' - 'on a specified cluster to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL`\n' - 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.7.4') - -RQ_SRS_006_RBAC_Privileges_AlterTTL_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter ttl** or **alter materialize ttl** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.7.5') - -RQ_SRS_006_RBAC_Privileges_AlterSettings = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter settings** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... MODIFY SETTING setting` statements SHALL\n' - 'return an error, unless the user has the **alter settings** privilege for\n' - 'the destination table either because of the explicit grant or through one of\n' - 'the roles assigned to the user. The **alter settings** privilege allows\n' - 'modifying table engine settings. It doesn’t affect settings or server configuration parameters.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.8.1') - -RQ_SRS_006_RBAC_Privileges_AlterSettings_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter settings** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.8.2') - -RQ_SRS_006_RBAC_Privileges_AlterSettings_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter settings** privilege\n' - 'for a database or a specific table to one or more **users** or **roles**\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.8.3') - -RQ_SRS_006_RBAC_Privileges_AlterSettings_Cluster = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **alter settings** privilege\n' - 'on a specified cluster to one or more **users** or **roles**.\n' - 'Any `ALTER TABLE ... MODIFY SETTING setting`\n' - 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.8.4') - -RQ_SRS_006_RBAC_Privileges_AlterSettings_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter settings** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.8.5') - -RQ_SRS_006_RBAC_Privileges_AlterUpdate = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterUpdate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER UPDATE` statement if and only if the user has **alter update** privilege for that column,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.9.1') - -RQ_SRS_006_RBAC_Privileges_AlterUpdate_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter update** privilege on a column level\n' - 'to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.9.2') - -RQ_SRS_006_RBAC_Privileges_AlterUpdate_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter update** privilege on a column level\n' - 'from one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.9.3') - -RQ_SRS_006_RBAC_Privileges_AlterUpdate_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter update** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.9.4') - -RQ_SRS_006_RBAC_Privileges_AlterDelete = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterDelete', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER DELETE` statement if and only if the user has **alter delete** privilege for that table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.10.1') - -RQ_SRS_006_RBAC_Privileges_AlterDelete_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterDelete.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter delete** privilege on a column level\n' - 'to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.10.2') - -RQ_SRS_006_RBAC_Privileges_AlterDelete_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterDelete.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter delete** privilege on a column level\n' - 'from one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.10.3') - -RQ_SRS_006_RBAC_Privileges_AlterDelete_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter delete** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.10.4') - -RQ_SRS_006_RBAC_Privileges_AlterFreeze = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFreeze', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER FREEZE` statement if and only if the user has **alter freeze** privilege for that table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.11.1') - -RQ_SRS_006_RBAC_Privileges_AlterFreeze_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter freeze** privilege on a column level\n' - 'to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.11.2') - -RQ_SRS_006_RBAC_Privileges_AlterFreeze_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter freeze** privilege on a column level\n' - 'from one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.11.3') - -RQ_SRS_006_RBAC_Privileges_AlterFreeze_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter freeze** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.11.4') - -RQ_SRS_006_RBAC_Privileges_AlterFetch = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFetch', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER FETCH` statement if and only if the user has **alter fetch** privilege for that table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.12.1') - -RQ_SRS_006_RBAC_Privileges_AlterFetch_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFetch.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter fetch** privilege on a column level\n' - 'to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.12.2') - -RQ_SRS_006_RBAC_Privileges_AlterFetch_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFetch.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter fetch** privilege on a column level\n' - 'from one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.12.3') - -RQ_SRS_006_RBAC_Privileges_AlterFetch_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter fetch** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.12.4') - -RQ_SRS_006_RBAC_Privileges_AlterMove = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterMove', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER MOVE` statement if and only if the user has **alter move**, **select**, and **alter delete** privilege on the source table\n' - 'and **insert** privilege on the target table, either directly or through a role.\n' - 'For example,\n' - '```sql\n' - 'ALTER TABLE source_table MOVE PARTITION 1 TO target_table\n' - '```\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.13.1') - -RQ_SRS_006_RBAC_Privileges_AlterMove_Grant = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterMove.Grant', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting **alter move** privilege on a column level\n' - 'to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.13.2') - -RQ_SRS_006_RBAC_Privileges_AlterMove_Revoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterMove.Revoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support revoking **alter move** privilege on a column level\n' - 'from one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.13.3') - -RQ_SRS_006_RBAC_Privileges_AlterMove_TableEngines = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support controlling access to the **alter move** privilege\n' - 'on tables created using the following engines\n' - '\n' - '* MergeTree\n' - '* ReplacingMergeTree\n' - '* SummingMergeTree\n' - '* AggregatingMergeTree\n' - '* CollapsingMergeTree\n' - '* VersionedCollapsingMergeTree\n' - '* GraphiteMergeTree\n' - '* ReplicatedMergeTree\n' - '* ReplicatedSummingMergeTree\n' - '* ReplicatedReplacingMergeTree\n' - '* ReplicatedAggregatingMergeTree\n' - '* ReplicatedCollapsingMergeTree\n' - '* ReplicatedVersionedCollapsingMergeTree\n' - '* ReplicatedGraphiteMergeTree\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.13.4') - -RQ_SRS_006_RBAC_Privileges_CreateTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL only successfully execute a `CREATE TABLE` command if and only if\n' - 'the user has **create table** privilege either explicitly or through roles.\n' - '\n' - 'If the stored query includes one or more source tables, the user must have **select** privilege\n' - "on all the source tables and **insert** for the table they're trying to create either explicitly or through a role.\n" - 'For example,\n' - '```sql\n' - 'CREATE TABLE table AS SELECT * FROM source_table\n' - 'CREATE TABLE table AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' - 'CREATE TABLE table AS SELECT * FROM table0 JOIN table1 USING column\n' - 'CREATE TABLE table AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' - 'CREATE TABLE table AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' - 'CREATE TABLE table0 AS SELECT column FROM table1 UNION ALL SELECT column FROM table2\n' - '```\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.14') - -RQ_SRS_006_RBAC_Privileges_CreateDatabase = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateDatabase', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE DATABASE` statement if and only if the user has **create database** privilege on the database,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.15') - -RQ_SRS_006_RBAC_Privileges_CreateDictionary = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateDictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.16') - -RQ_SRS_006_RBAC_Privileges_CreateTemporaryTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.17') - -RQ_SRS_006_RBAC_Privileges_AttachDatabase = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AttachDatabase', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ATTACH DATABASE` statement if and only if the user has **create database** privilege on the database,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.18') - -RQ_SRS_006_RBAC_Privileges_AttachDictionary = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AttachDictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ATTACH DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.19') - -RQ_SRS_006_RBAC_Privileges_AttachTemporaryTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ATTACH TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.20') - -RQ_SRS_006_RBAC_Privileges_AttachTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AttachTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ATTACH TABLE` statement if and only if the user has **create table** privilege on the table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.21') - -RQ_SRS_006_RBAC_Privileges_DropTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP TABLE` statement if and only if the user has **drop table** privilege on the table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.22') - -RQ_SRS_006_RBAC_Privileges_DropDatabase = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropDatabase', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP DATABASE` statement if and only if the user has **drop database** privilege on the database,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.23') - -RQ_SRS_006_RBAC_Privileges_DropDictionary = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropDictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.24') - -RQ_SRS_006_RBAC_Privileges_DetachTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DetachTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DETACH TABLE` statement if and only if the user has **drop table** privilege on the table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.25') - -RQ_SRS_006_RBAC_Privileges_DetachView = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DetachView', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DETACH VIEW` statement if and only if the user has **drop view** privilege on the view,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.26') - -RQ_SRS_006_RBAC_Privileges_DetachDatabase = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DetachDatabase', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DETACH DATABASE` statement if and only if the user has **drop database** privilege on the database,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.27') - -RQ_SRS_006_RBAC_Privileges_DetachDictionary = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DetachDictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DETACH DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.28') - -RQ_SRS_006_RBAC_Privileges_Truncate = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Truncate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `TRUNCATE TABLE` statement if and only if the user has **truncate table** privilege on the table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.29') - -RQ_SRS_006_RBAC_Privileges_Optimize = Requirement( - name='RQ.SRS-006.RBAC.Privileges.Optimize', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `OPTIMIZE TABLE` statement if and only if the user has **optimize table** privilege on the table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.30') - -RQ_SRS_006_RBAC_Privileges_KillQuery = Requirement( - name='RQ.SRS-006.RBAC.Privileges.KillQuery', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `KILL QUERY` statement if and only if the user has **kill query** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.31') - -RQ_SRS_006_RBAC_Privileges_KillMutation = Requirement( - name='RQ.SRS-006.RBAC.Privileges.KillMutation', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `KILL MUTATION` statement if and only if\n' - 'the user has the privilege that created the mutation, either directly or through a role.\n' - 'For example, to `KILL MUTATION` after `ALTER UPDATE` query, the user needs `ALTER UPDATE` privilege.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.32') - -RQ_SRS_006_RBAC_Privileges_KillMutation_AlterUpdate = Requirement( - name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterUpdate', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER UPDATE` mutation if and only if\n' - 'the user has `ALTER UPDATE` privilege on the table where the mutation was created, either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.33') - -RQ_SRS_006_RBAC_Privileges_KillMutation_AlterDelete = Requirement( - name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDelete', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER DELETE` mutation if and only if\n' - 'the user has `ALTER DELETE` privilege on the table where the mutation was created, either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.34') - -RQ_SRS_006_RBAC_Privileges_KillMutation_AlterDropColumn = Requirement( - name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDropColumn', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER DROP COLUMN` mutation if and only if\n' - 'the user has `ALTER DROP COLUMN` privilege on the table where the mutation was created, either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.35') - -RQ_SRS_006_RBAC_Privileges_ShowTables = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowTables', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL grant **show tables** privilege on a table to a user if that user has recieved any grant,\n' - 'including `SHOW TABLES`, on that table, either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.36') - -RQ_SRS_006_RBAC_Privileges_ShowTables_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowTables.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW TABLES` statement if and only if the user has **show tables** privilege,\n' - 'or any privilege on the table either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.37') - -RQ_SRS_006_RBAC_Privileges_ExistsTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ExistsTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `EXISTS table` statement if and only if the user has **show tables** privilege,\n' - 'or any privilege on the table either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.38') - -RQ_SRS_006_RBAC_Privileges_CheckTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CheckTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CHECK table` statement if and only if the user has **show tables** privilege,\n' - 'or any privilege on the table either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.39') - -RQ_SRS_006_RBAC_Privileges_ShowDatabases = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowDatabases', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL grant **show databases** privilege on a database to a user if that user has recieved any grant,\n' - 'including `SHOW DATABASES`, on that table, either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.40') - -RQ_SRS_006_RBAC_Privileges_ShowDatabases_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowDatabases.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW DATABASES` statement if and only if the user has **show databases** privilege,\n' - 'or any privilege on the database either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.41') - -RQ_SRS_006_RBAC_Privileges_ShowCreateDatabase = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateDatabase', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE DATABASE` statement if and only if the user has **show databases** privilege,\n' - 'or any privilege on the database either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.42') - -RQ_SRS_006_RBAC_Privileges_UseDatabase = Requirement( - name='RQ.SRS-006.RBAC.Privileges.UseDatabase', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `USE database` statement if and only if the user has **show databases** privilege,\n' - 'or any privilege on the database either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.43') - -RQ_SRS_006_RBAC_Privileges_ShowColumns = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowColumns', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL grant **show columns** privilege to a user if and only if it is granted directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.44') - -RQ_SRS_006_RBAC_Privileges_ShowCreateTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE TABLE` statement if and only if the user has **show columns** privilege on that table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.45') - -RQ_SRS_006_RBAC_Privileges_DescribeTable = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DescribeTable', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DESCRIBE table` statement if and only if the user has **show columns** privilege on that table,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.46') - -RQ_SRS_006_RBAC_Privileges_ShowDictionaries = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowDictionaries', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL grant **show dictionaries** privilege on a dictionary to a user if that user has recieved any grant,\n' - 'including `SHOW DICTIONARIES`, on that dictionary, either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.47') - -RQ_SRS_006_RBAC_Privileges_ShowDictionaries_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW DICTIONARIES` statement if and only if the user has **show dictionaries** privilege,\n' - 'or any privilege on the dictionary either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.48') - -RQ_SRS_006_RBAC_Privileges_ShowCreateDictionary = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE DICTIONARY` statement if and only if the user has **show dictionaries** privilege,\n' - 'or any privilege on the dictionary either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.49') - -RQ_SRS_006_RBAC_Privileges_ExistsDictionary = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ExistsDictionary', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `EXISTS dictionary` statement if and only if the user has **show dictionaries** privilege,\n' - 'or any privilege on the dictionary either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.50') - -RQ_SRS_006_RBAC_Privileges_CreateUser = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE USER` statement if and only if the user has **create user** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.51') - -RQ_SRS_006_RBAC_Privileges_CreateUser_DefaultRole = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE USER` statement with `DEFAULT ROLE ` clause if and only if\n' - 'the user has **create user** privilege and the role with **admin option**, or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.52') - -RQ_SRS_006_RBAC_Privileges_AlterUser = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER USER` statement if and only if the user has **alter user** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.53') - -RQ_SRS_006_RBAC_Privileges_DropUser = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP USER` statement if and only if the user has **drop user** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.54') - -RQ_SRS_006_RBAC_Privileges_CreateRole = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE ROLE` statement if and only if the user has **create role** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.55') - -RQ_SRS_006_RBAC_Privileges_AlterRole = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER ROLE` statement if and only if the user has **alter role** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.56') - -RQ_SRS_006_RBAC_Privileges_DropRole = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP ROLE` statement if and only if the user has **drop role** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.57') - -RQ_SRS_006_RBAC_Privileges_CreateRowPolicy = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateRowPolicy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE ROW POLICY` statement if and only if the user has **create row policy** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.58') - -RQ_SRS_006_RBAC_Privileges_AlterRowPolicy = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterRowPolicy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER ROW POLICY` statement if and only if the user has **alter row policy** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.59') - -RQ_SRS_006_RBAC_Privileges_DropRowPolicy = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropRowPolicy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP ROW POLICY` statement if and only if the user has **drop row policy** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.60') - -RQ_SRS_006_RBAC_Privileges_CreateQuota = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateQuota', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE QUOTA` statement if and only if the user has **create quota** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.61') - -RQ_SRS_006_RBAC_Privileges_AlterQuota = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterQuota', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER QUOTA` statement if and only if the user has **alter quota** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.62') - -RQ_SRS_006_RBAC_Privileges_DropQuota = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropQuota', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP QUOTA` statement if and only if the user has **drop quota** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.63') - -RQ_SRS_006_RBAC_Privileges_CreateSettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `CREATE SETTINGS PROFILE` statement if and only if the user has **create settings profile** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.64') - -RQ_SRS_006_RBAC_Privileges_AlterSettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `ALTER SETTINGS PROFILE` statement if and only if the user has **alter settings profile** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.65') - -RQ_SRS_006_RBAC_Privileges_DropSettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.Privileges.DropSettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `DROP SETTINGS PROFILE` statement if and only if the user has **drop settings profile** privilege,\n' - 'or either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.66') - -RQ_SRS_006_RBAC_Privileges_RoleAdmin = Requirement( - name='RQ.SRS-006.RBAC.Privileges.RoleAdmin', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute any role grant or revoke by a user with `ROLE ADMIN` privilege.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.67') - -RQ_SRS_006_RBAC_Privileges_ShowUsers = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowUsers', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully grant `SHOW USERS` privilege when\n' - 'the user is granted `SHOW USERS`, `SHOW CREATE USER`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.68') - -RQ_SRS_006_RBAC_Privileges_ShowUsers_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowUsers.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW USERS` statement if and only if the user has **show users** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.69') - -RQ_SRS_006_RBAC_Privileges_ShowCreateUser = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateUser', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show create user** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.70') - -RQ_SRS_006_RBAC_Privileges_ShowRoles = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowRoles', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully grant `SHOW ROLES` privilege when\n' - 'the user is granted `SHOW ROLES`, `SHOW CREATE ROLE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.71') - -RQ_SRS_006_RBAC_Privileges_ShowRoles_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowRoles.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW ROLES` statement if and only if the user has **show roles** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.72') - -RQ_SRS_006_RBAC_Privileges_ShowCreateRole = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateRole', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show create role** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.73') - -RQ_SRS_006_RBAC_Privileges_ShowRowPolicies = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowRowPolicies', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully grant `SHOW ROW POLICIES` privilege when\n' - 'the user is granted `SHOW ROW POLICIES`, `SHOW POLICIES`, `SHOW CREATE ROW POLICY`,\n' - '`SHOW CREATE POLICY`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.74') - -RQ_SRS_006_RBAC_Privileges_ShowRowPolicies_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if the user has **show row policies** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.75') - -RQ_SRS_006_RBAC_Privileges_ShowCreateRowPolicy = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement if and only if the user has **show create row policy** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.76') - -RQ_SRS_006_RBAC_Privileges_ShowQuotas = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowQuotas', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully grant `SHOW QUOTAS` privilege when\n' - 'the user is granted `SHOW QUOTAS`, `SHOW CREATE QUOTA`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.77') - -RQ_SRS_006_RBAC_Privileges_ShowQuotas_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW QUOTAS` statement if and only if the user has **show quotas** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.78') - -RQ_SRS_006_RBAC_Privileges_ShowCreateQuota = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateQuota', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if the user has **show create quota** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.79') - -RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully grant `SHOW SETTINGS PROFILES` privilege when\n' - 'the user is granted `SHOW SETTINGS PROFILES`, `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`,\n' - '`SHOW SETTINGS PROFILE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.80') - -RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles_Query = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement if and only if the user has **show settings profiles** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.81') - -RQ_SRS_006_RBAC_Privileges_ShowCreateSettingsProfile = Requirement( - name='RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement if and only if the user has **show create settings profile** privilege,\n' - 'either directly or through a role.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.82') - -RQ_SRS_006_RBAC_Privileges_GrantOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.GrantOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL successfully execute `GRANT` or `REVOKE` privilege statements by a user if and only if\n' - 'the user has that privilege with `GRANT OPTION`, either directly or through a role.\n' - '\n' - '`GRANT OPTION` is supported by the following privileges\n' - '\n' - '* `ALTER MOVE PARTITION`\n' - '* `ALTER FETCH PARTITION`\n' - '* `ALTER FREEZE PARTITION`\n' - '* `ALTER DELETE`\n' - '* `ALTER UPDATE`\n' - '* `ALTER SETTINGS`\n' - '* `ALTER TTL`\n' - '* `ALTER CONSTRAINT`\n' - '* `ALTER COLUMN`\n' - '* `ALTER INDEX`\n' - '* `INSERT`\n' - '* `SELECT`\n' - '* `CREATE TABLE`\n' - '* `CREATE VIEW`\n' - '* `CREATE DATABASE`\n' - '* `CREATE DICTIONARY`\n' - '* `CREATE TEMPORARY TABLE`\n' - '* `DROP TABLE`\n' - '* `DROP VIEW`\n' - '* `DROP DATABASE`\n' - '* `DROP DICTIONARY`\n' - '\n' - ), - link=None, - level=5, - num='5.2.11.83.1') - -RQ_SRS_006_RBAC_Privileges_All = Requirement( - name='RQ.SRS-006.RBAC.Privileges.All', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL include in the **all** privilege the same rights\n' - 'as provided by **usage**, **select**, **select columns**,\n' - '**insert**, **delete**, **alter**, **create**, and **drop** privileges.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.84') - -RQ_SRS_006_RBAC_Privileges_All_GrantRevoke = Requirement( - name='RQ.SRS-006.RBAC.Privileges.All.GrantRevoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **all** privileges\n' - 'for a database or a specific table to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.85') - -RQ_SRS_006_RBAC_Privileges_AdminOption = Requirement( - name='RQ.SRS-006.RBAC.Privileges.AdminOption', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL support granting or revoking **admin option** privilege\n' - 'to one or more **users** or **roles**.\n' - '\n' - ), - link=None, - level=4, - num='5.2.11.86') - -RQ_SRS_006_RBAC_RequiredPrivileges_Create = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Create', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `CREATE` statements\n' - 'to be executed unless the user has the **create** privilege for the destination database\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.12.1') - -RQ_SRS_006_RBAC_RequiredPrivileges_Alter = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Alter', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `ALTER` statements\n' - 'to be executed unless the user has the **alter** privilege for the destination table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.12.2') - -RQ_SRS_006_RBAC_RequiredPrivileges_Drop = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `DROP` statements\n' - 'to be executed unless the user has the **drop** privilege for the destination database\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.12.3') - -RQ_SRS_006_RBAC_RequiredPrivileges_Drop_Table = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `DROP TABLE` statements\n' - 'to be executed unless the user has the **drop** privilege for the destination database or the table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.12.4') - -RQ_SRS_006_RBAC_RequiredPrivileges_GrantRevoke = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any `GRANT` or `REVOKE` statements\n' - 'to be executed unless the user has the **grant option** privilege\n' - 'for the privilege of the destination table\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.12.5') - -RQ_SRS_006_RBAC_RequiredPrivileges_Use = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Use', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow the `USE` statement to be executed\n' - 'unless the user has at least one of the privileges for the database\n' - 'or the table inside that database\n' - 'either because of the explicit grant or through one of the roles assigned to the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.12.6') - -RQ_SRS_006_RBAC_RequiredPrivileges_Admin = Requirement( - name='RQ.SRS-006.RBAC.RequiredPrivileges.Admin', - version='1.0', - priority=None, - group=None, - type=None, - uid=None, - description=( - '[ClickHouse] SHALL not allow any of the following statements\n' - '\n' - '* `SYSTEM`\n' - '* `SHOW`\n' - '* `ATTACH`\n' - '* `CHECK TABLE`\n' - '* `DESCRIBE TABLE`\n' - '* `DETACH`\n' - '* `EXISTS`\n' - '* `KILL QUERY`\n' - '* `KILL MUTATION`\n' - '* `OPTIMIZE`\n' - '* `RENAME`\n' - '* `TRUNCATE`\n' - '\n' - 'to be executed unless the user has the **admin option** privilege\n' - 'through one of the roles with **admin option** privilege assigned to the user.\n' - '\n' - ), - link=None, - level=4, - num='5.2.12.7') - SRS_006_ClickHouse_Role_Based_Access_Control = Specification( - name='SRS-006 ClickHouse Role Based Access Control', - description=None, - author=None, - date=None, - status=None, - approved_by=None, - approved_date=None, - approved_version=None, - version=None, - group=None, - type=None, - link=None, - uid=None, - parent=None, - children=None, - headings=( - Heading(name='Revision History', level=1, num='1'), - Heading(name='Introduction', level=1, num='2'), - Heading(name='Terminology', level=1, num='3'), - Heading(name='Privilege Definitions', level=1, num='4'), - Heading(name='Requirements', level=1, num='5'), - Heading(name='Generic', level=2, num='5.1'), - Heading(name='RQ.SRS-006.RBAC', level=3, num='5.1.1'), - Heading(name='Login', level=3, num='5.1.2'), - Heading(name='RQ.SRS-006.RBAC.Login', level=4, num='5.1.2.1'), - Heading(name='RQ.SRS-006.RBAC.Login.DefaultUser', level=4, num='5.1.2.2'), - Heading(name='User', level=3, num='5.1.3'), - Heading(name='RQ.SRS-006.RBAC.User', level=4, num='5.1.3.1'), - Heading(name='RQ.SRS-006.RBAC.User.Roles', level=4, num='5.1.3.2'), - Heading(name='RQ.SRS-006.RBAC.User.Privileges', level=4, num='5.1.3.3'), - Heading(name='RQ.SRS-006.RBAC.User.Variables', level=4, num='5.1.3.4'), - Heading(name='RQ.SRS-006.RBAC.User.Variables.Constraints', level=4, num='5.1.3.5'), - Heading(name='RQ.SRS-006.RBAC.User.SettingsProfile', level=4, num='5.1.3.6'), - Heading(name='RQ.SRS-006.RBAC.User.Quotas', level=4, num='5.1.3.7'), - Heading(name='RQ.SRS-006.RBAC.User.RowPolicies', level=4, num='5.1.3.8'), - Heading(name='RQ.SRS-006.RBAC.User.AccountLock', level=4, num='5.1.3.9'), - Heading(name='RQ.SRS-006.RBAC.User.AccountLock.DenyAccess', level=4, num='5.1.3.10'), - Heading(name='RQ.SRS-006.RBAC.User.DefaultRole', level=4, num='5.1.3.11'), - Heading(name='RQ.SRS-006.RBAC.User.RoleSelection', level=4, num='5.1.3.12'), - Heading(name='RQ.SRS-006.RBAC.User.ShowCreate', level=4, num='5.1.3.13'), - Heading(name='RQ.SRS-006.RBAC.User.ShowPrivileges', level=4, num='5.1.3.14'), - Heading(name='Role', level=3, num='5.1.4'), - Heading(name='RQ.SRS-006.RBAC.Role', level=4, num='5.1.4.1'), - Heading(name='RQ.SRS-006.RBAC.Role.Privileges', level=4, num='5.1.4.2'), - Heading(name='RQ.SRS-006.RBAC.Role.Variables', level=4, num='5.1.4.3'), - Heading(name='RQ.SRS-006.RBAC.Role.SettingsProfile', level=4, num='5.1.4.4'), - Heading(name='RQ.SRS-006.RBAC.Role.Quotas', level=4, num='5.1.4.5'), - Heading(name='RQ.SRS-006.RBAC.Role.RowPolicies', level=4, num='5.1.4.6'), - Heading(name='Partial Revokes', level=3, num='5.1.5'), - Heading(name='RQ.SRS-006.RBAC.PartialRevokes', level=4, num='5.1.5.1'), - Heading(name='Settings Profile', level=3, num='5.1.6'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile', level=4, num='5.1.6.1'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Constraints', level=4, num='5.1.6.2'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.ShowCreate', level=4, num='5.1.6.3'), - Heading(name='Quotas', level=3, num='5.1.7'), - Heading(name='RQ.SRS-006.RBAC.Quotas', level=4, num='5.1.7.1'), - Heading(name='RQ.SRS-006.RBAC.Quotas.Keyed', level=4, num='5.1.7.2'), - Heading(name='RQ.SRS-006.RBAC.Quotas.Queries', level=4, num='5.1.7.3'), - Heading(name='RQ.SRS-006.RBAC.Quotas.Errors', level=4, num='5.1.7.4'), - Heading(name='RQ.SRS-006.RBAC.Quotas.ResultRows', level=4, num='5.1.7.5'), - Heading(name='RQ.SRS-006.RBAC.Quotas.ReadRows', level=4, num='5.1.7.6'), - Heading(name='RQ.SRS-006.RBAC.Quotas.ResultBytes', level=4, num='5.1.7.7'), - Heading(name='RQ.SRS-006.RBAC.Quotas.ReadBytes', level=4, num='5.1.7.8'), - Heading(name='RQ.SRS-006.RBAC.Quotas.ExecutionTime', level=4, num='5.1.7.9'), - Heading(name='RQ.SRS-006.RBAC.Quotas.ShowCreate', level=4, num='5.1.7.10'), - Heading(name='Row Policy', level=3, num='5.1.8'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy', level=4, num='5.1.8.1'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Condition', level=4, num='5.1.8.2'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.ShowCreate', level=4, num='5.1.8.3'), - Heading(name='Specific', level=2, num='5.2'), - Heading(name='RQ.SRS-006.RBAC.User.Use.DefaultRole', level=4, num='5.2.8.1'), - Heading(name='RQ.SRS-006.RBAC.User.Use.AllRolesWhenNoDefaultRole', level=4, num='5.2.8.2'), - Heading(name='RQ.SRS-006.RBAC.User.Create', level=4, num='5.2.8.3'), - Heading(name='RQ.SRS-006.RBAC.User.Create.IfNotExists', level=4, num='5.2.8.4'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Replace', level=4, num='5.2.8.5'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.NoPassword', level=4, num='5.2.8.6'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.NoPassword.Login', level=4, num='5.2.8.7'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.PlainText', level=4, num='5.2.8.8'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.PlainText.Login', level=4, num='5.2.8.9'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Password', level=4, num='5.2.8.10'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Password.Login', level=4, num='5.2.8.11'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash', level=4, num='5.2.8.12'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash.Login', level=4, num='5.2.8.13'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password', level=4, num='5.2.8.14'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password.Login', level=4, num='5.2.8.15'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash', level=4, num='5.2.8.16'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash.Login', level=4, num='5.2.8.17'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.Name', level=4, num='5.2.8.18'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.Regexp', level=4, num='5.2.8.19'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.IP', level=4, num='5.2.8.20'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.Any', level=4, num='5.2.8.21'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.None', level=4, num='5.2.8.22'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.Local', level=4, num='5.2.8.23'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.Like', level=4, num='5.2.8.24'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Host.Default', level=4, num='5.2.8.25'), - Heading(name='RQ.SRS-006.RBAC.User.Create.DefaultRole', level=4, num='5.2.8.26'), - Heading(name='RQ.SRS-006.RBAC.User.Create.DefaultRole.None', level=4, num='5.2.8.27'), - Heading(name='RQ.SRS-006.RBAC.User.Create.DefaultRole.All', level=4, num='5.2.8.28'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Settings', level=4, num='5.2.8.29'), - Heading(name='RQ.SRS-006.RBAC.User.Create.OnCluster', level=4, num='5.2.8.30'), - Heading(name='RQ.SRS-006.RBAC.User.Create.Syntax', level=4, num='5.2.8.31'), - Heading(name='RQ.SRS-006.RBAC.User.Alter', level=4, num='5.2.8.32'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.OrderOfEvaluation', level=4, num='5.2.8.33'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.IfExists', level=4, num='5.2.8.34'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Cluster', level=4, num='5.2.8.35'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Rename', level=4, num='5.2.8.36'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Password.PlainText', level=4, num='5.2.8.37'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Password.Sha256Password', level=4, num='5.2.8.38'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Password.DoubleSha1Password', level=4, num='5.2.8.39'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.AddDrop', level=4, num='5.2.8.40'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.Local', level=4, num='5.2.8.41'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.Name', level=4, num='5.2.8.42'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.Regexp', level=4, num='5.2.8.43'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.IP', level=4, num='5.2.8.44'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.Like', level=4, num='5.2.8.45'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.Any', level=4, num='5.2.8.46'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Host.None', level=4, num='5.2.8.47'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.DefaultRole', level=4, num='5.2.8.48'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.DefaultRole.All', level=4, num='5.2.8.49'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.DefaultRole.AllExcept', level=4, num='5.2.8.50'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Settings', level=4, num='5.2.8.51'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Settings.Min', level=4, num='5.2.8.52'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Settings.Max', level=4, num='5.2.8.53'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Settings.Profile', level=4, num='5.2.8.54'), - Heading(name='RQ.SRS-006.RBAC.User.Alter.Syntax', level=4, num='5.2.8.55'), - Heading(name='RQ.SRS-006.RBAC.SetDefaultRole', level=4, num='5.2.8.56'), - Heading(name='RQ.SRS-006.RBAC.SetDefaultRole.CurrentUser', level=4, num='5.2.8.57'), - Heading(name='RQ.SRS-006.RBAC.SetDefaultRole.All', level=4, num='5.2.8.58'), - Heading(name='RQ.SRS-006.RBAC.SetDefaultRole.AllExcept', level=4, num='5.2.8.59'), - Heading(name='RQ.SRS-006.RBAC.SetDefaultRole.None', level=4, num='5.2.8.60'), - Heading(name='RQ.SRS-006.RBAC.SetDefaultRole.Syntax', level=4, num='5.2.8.61'), - Heading(name='RQ.SRS-006.RBAC.SetRole', level=4, num='5.2.8.62'), - Heading(name='RQ.SRS-006.RBAC.SetRole.Default', level=4, num='5.2.8.63'), - Heading(name='RQ.SRS-006.RBAC.SetRole.None', level=4, num='5.2.8.64'), - Heading(name='RQ.SRS-006.RBAC.SetRole.All', level=4, num='5.2.8.65'), - Heading(name='RQ.SRS-006.RBAC.SetRole.AllExcept', level=4, num='5.2.8.66'), - Heading(name='RQ.SRS-006.RBAC.SetRole.Syntax', level=4, num='5.2.8.67'), - Heading(name='RQ.SRS-006.RBAC.User.ShowCreateUser', level=4, num='5.2.8.68'), - Heading(name='RQ.SRS-006.RBAC.User.ShowCreateUser.For', level=4, num='5.2.8.69'), - Heading(name='RQ.SRS-006.RBAC.User.ShowCreateUser.Syntax', level=4, num='5.2.8.70'), - Heading(name='RQ.SRS-006.RBAC.User.Drop', level=4, num='5.2.8.71'), - Heading(name='RQ.SRS-006.RBAC.User.Drop.IfExists', level=4, num='5.2.8.72'), - Heading(name='RQ.SRS-006.RBAC.User.Drop.OnCluster', level=4, num='5.2.8.73'), - Heading(name='RQ.SRS-006.RBAC.User.Drop.Syntax', level=4, num='5.2.8.74'), - Heading(name='RQ.SRS-006.RBAC.Role.Create', level=4, num='5.2.8.75'), - Heading(name='RQ.SRS-006.RBAC.Role.Create.IfNotExists', level=4, num='5.2.8.76'), - Heading(name='RQ.SRS-006.RBAC.Role.Create.Replace', level=4, num='5.2.8.77'), - Heading(name='RQ.SRS-006.RBAC.Role.Create.Settings', level=4, num='5.2.8.78'), - Heading(name='RQ.SRS-006.RBAC.Role.Create.Syntax', level=4, num='5.2.8.79'), - Heading(name='RQ.SRS-006.RBAC.Role.Alter', level=4, num='5.2.8.80'), - Heading(name='RQ.SRS-006.RBAC.Role.Alter.IfExists', level=4, num='5.2.8.81'), - Heading(name='RQ.SRS-006.RBAC.Role.Alter.Cluster', level=4, num='5.2.8.82'), - Heading(name='RQ.SRS-006.RBAC.Role.Alter.Rename', level=4, num='5.2.8.83'), - Heading(name='RQ.SRS-006.RBAC.Role.Alter.Settings', level=4, num='5.2.8.84'), - Heading(name='RQ.SRS-006.RBAC.Role.Alter.Syntax', level=4, num='5.2.8.85'), - Heading(name='RQ.SRS-006.RBAC.Role.Drop', level=4, num='5.2.8.86'), - Heading(name='RQ.SRS-006.RBAC.Role.Drop.IfExists', level=4, num='5.2.8.87'), - Heading(name='RQ.SRS-006.RBAC.Role.Drop.Cluster', level=4, num='5.2.8.88'), - Heading(name='RQ.SRS-006.RBAC.Role.Drop.Syntax', level=4, num='5.2.8.89'), - Heading(name='RQ.SRS-006.RBAC.Role.ShowCreate', level=4, num='5.2.8.90'), - Heading(name='RQ.SRS-006.RBAC.Role.ShowCreate.Syntax', level=4, num='5.2.8.91'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.To', level=4, num='5.2.8.92'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.ToCurrentUser', level=4, num='5.2.8.93'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Select', level=4, num='5.2.8.94'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Insert', level=4, num='5.2.8.95'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Alter', level=4, num='5.2.8.96'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Create', level=4, num='5.2.8.97'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Create.Database', level=4, num='5.2.8.98'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Create.Dictionary', level=4, num='5.2.8.99'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Create.Table', level=4, num='5.2.8.100'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Create.TemporaryTable', level=4, num='5.2.8.101'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Drop', level=4, num='5.2.8.102'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.Database', level=4, num='5.2.8.103'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.Dictionary', level=4, num='5.2.8.104'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.Table', level=4, num='5.2.8.105'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Drop.TemporaryTable', level=4, num='5.2.8.106'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Truncate', level=4, num='5.2.8.107'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Optimize', level=4, num='5.2.8.108'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Show', level=4, num='5.2.8.109'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.KillQuery', level=4, num='5.2.8.110'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement', level=4, num='5.2.8.111'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.System', level=4, num='5.2.8.112'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Introspection', level=4, num='5.2.8.113'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Sources', level=4, num='5.2.8.114'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.DictGet', level=4, num='5.2.8.115'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.None', level=4, num='5.2.8.116'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.All', level=4, num='5.2.8.117'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.GrantOption', level=4, num='5.2.8.118'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.On', level=4, num='5.2.8.119'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns', level=4, num='5.2.8.120'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.OnCluster', level=4, num='5.2.8.121'), - Heading(name='RQ.SRS-006.RBAC.Grant.Privilege.Syntax', level=4, num='5.2.8.122'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Cluster', level=4, num='5.2.8.123'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Any', level=4, num='5.2.8.124'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Select', level=4, num='5.2.8.125'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Insert', level=4, num='5.2.8.126'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Alter', level=4, num='5.2.8.127'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Create', level=4, num='5.2.8.128'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.Database', level=4, num='5.2.8.129'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.Dictionary', level=4, num='5.2.8.130'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.Table', level=4, num='5.2.8.131'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Create.TemporaryTable', level=4, num='5.2.8.132'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop', level=4, num='5.2.8.133'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Database', level=4, num='5.2.8.134'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Dictionary', level=4, num='5.2.8.135'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Table', level=4, num='5.2.8.136'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop.TemporaryTable', level=4, num='5.2.8.137'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Truncate', level=4, num='5.2.8.138'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Optimize', level=4, num='5.2.8.139'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Show', level=4, num='5.2.8.140'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery', level=4, num='5.2.8.141'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement', level=4, num='5.2.8.142'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.System', level=4, num='5.2.8.143'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Introspection', level=4, num='5.2.8.144'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Sources', level=4, num='5.2.8.145'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.DictGet', level=4, num='5.2.8.146'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns', level=4, num='5.2.8.147'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Multiple', level=4, num='5.2.8.148'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.All', level=4, num='5.2.8.149'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.None', level=4, num='5.2.8.150'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.On', level=4, num='5.2.8.151'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.From', level=4, num='5.2.8.152'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Privilege.Syntax', level=4, num='5.2.8.153'), - Heading(name='RQ.SRS-006.RBAC.PartialRevoke.Syntax', level=4, num='5.2.8.154'), - Heading(name='RQ.SRS-006.RBAC.Grant.Role', level=4, num='5.2.8.155'), - Heading(name='RQ.SRS-006.RBAC.Grant.Role.CurrentUser', level=4, num='5.2.8.156'), - Heading(name='RQ.SRS-006.RBAC.Grant.Role.AdminOption', level=4, num='5.2.8.157'), - Heading(name='RQ.SRS-006.RBAC.Grant.Role.OnCluster', level=4, num='5.2.8.158'), - Heading(name='RQ.SRS-006.RBAC.Grant.Role.Syntax', level=4, num='5.2.8.159'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Role', level=4, num='5.2.8.160'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Role.Keywords', level=4, num='5.2.8.161'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Role.Cluster', level=4, num='5.2.8.162'), - Heading(name='RQ.SRS-006.RBAC.Revoke.AdminOption', level=4, num='5.2.8.163'), - Heading(name='RQ.SRS-006.RBAC.Revoke.Role.Syntax', level=4, num='5.2.8.164'), - Heading(name='RQ.SRS-006.RBAC.Show.Grants', level=4, num='5.2.8.165'), - Heading(name='RQ.SRS-006.RBAC.Show.Grants.For', level=4, num='5.2.8.166'), - Heading(name='RQ.SRS-006.RBAC.Show.Grants.Syntax', level=4, num='5.2.8.167'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create', level=4, num='5.2.8.168'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists', level=4, num='5.2.8.169'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Replace', level=4, num='5.2.8.170'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables', level=4, num='5.2.8.171'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value', level=4, num='5.2.8.172'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints', level=4, num='5.2.8.173'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment', level=4, num='5.2.8.174'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None', level=4, num='5.2.8.175'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All', level=4, num='5.2.8.176'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept', level=4, num='5.2.8.177'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit', level=4, num='5.2.8.178'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster', level=4, num='5.2.8.179'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax', level=4, num='5.2.8.180'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter', level=4, num='5.2.8.181'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists', level=4, num='5.2.8.182'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename', level=4, num='5.2.8.183'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables', level=4, num='5.2.8.184'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value', level=4, num='5.2.8.185'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints', level=4, num='5.2.8.186'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment', level=4, num='5.2.8.187'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None', level=4, num='5.2.8.188'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All', level=4, num='5.2.8.189'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept', level=4, num='5.2.8.190'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit', level=4, num='5.2.8.191'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster', level=4, num='5.2.8.192'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax', level=4, num='5.2.8.193'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Drop', level=4, num='5.2.8.194'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists', level=4, num='5.2.8.195'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster', level=4, num='5.2.8.196'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax', level=4, num='5.2.8.197'), - Heading(name='RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile', level=4, num='5.2.8.198'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create', level=4, num='5.2.8.199'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.IfNotExists', level=4, num='5.2.8.200'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Replace', level=4, num='5.2.8.201'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Cluster', level=4, num='5.2.8.202'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Interval', level=4, num='5.2.8.203'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized', level=4, num='5.2.8.204'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Queries', level=4, num='5.2.8.205'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Errors', level=4, num='5.2.8.206'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.ResultRows', level=4, num='5.2.8.207'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.ReadRows', level=4, num='5.2.8.208'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.ResultBytes', level=4, num='5.2.8.209'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.ReadBytes', level=4, num='5.2.8.210'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.ExecutionTime', level=4, num='5.2.8.211'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.NoLimits', level=4, num='5.2.8.212'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.TrackingOnly', level=4, num='5.2.8.213'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.KeyedBy', level=4, num='5.2.8.214'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions', level=4, num='5.2.8.215'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Assignment', level=4, num='5.2.8.216'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Assignment.None', level=4, num='5.2.8.217'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Assignment.All', level=4, num='5.2.8.218'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Assignment.Except', level=4, num='5.2.8.219'), - Heading(name='RQ.SRS-006.RBAC.Quota.Create.Syntax', level=4, num='5.2.8.220'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter', level=4, num='5.2.8.221'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.IfExists', level=4, num='5.2.8.222'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Rename', level=4, num='5.2.8.223'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Cluster', level=4, num='5.2.8.224'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Interval', level=4, num='5.2.8.225'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized', level=4, num='5.2.8.226'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Queries', level=4, num='5.2.8.227'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Errors', level=4, num='5.2.8.228'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.ResultRows', level=4, num='5.2.8.229'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.ReadRows', level=4, num='5.2.8.230'), - Heading(name='RQ.SRS-006.RBAC.Quota.ALter.ResultBytes', level=4, num='5.2.8.231'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.ReadBytes', level=4, num='5.2.8.232'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime', level=4, num='5.2.8.233'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.NoLimits', level=4, num='5.2.8.234'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly', level=4, num='5.2.8.235'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.KeyedBy', level=4, num='5.2.8.236'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions', level=4, num='5.2.8.237'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Assignment', level=4, num='5.2.8.238'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.None', level=4, num='5.2.8.239'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.All', level=4, num='5.2.8.240'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except', level=4, num='5.2.8.241'), - Heading(name='RQ.SRS-006.RBAC.Quota.Alter.Syntax', level=4, num='5.2.8.242'), - Heading(name='RQ.SRS-006.RBAC.Quota.Drop', level=4, num='5.2.8.243'), - Heading(name='RQ.SRS-006.RBAC.Quota.Drop.IfExists', level=4, num='5.2.8.244'), - Heading(name='RQ.SRS-006.RBAC.Quota.Drop.Cluster', level=4, num='5.2.8.245'), - Heading(name='RQ.SRS-006.RBAC.Quota.Drop.Syntax', level=4, num='5.2.8.246'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowQuotas', level=4, num='5.2.8.247'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile', level=4, num='5.2.8.248'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Format', level=4, num='5.2.8.249'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings', level=4, num='5.2.8.250'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax', level=4, num='5.2.8.251'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name', level=4, num='5.2.8.252'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current', level=4, num='5.2.8.253'), - Heading(name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax', level=4, num='5.2.8.254'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create', level=4, num='5.2.8.255'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists', level=4, num='5.2.8.256'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Replace', level=4, num='5.2.8.257'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster', level=4, num='5.2.8.258'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.On', level=4, num='5.2.8.259'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Access', level=4, num='5.2.8.260'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive', level=4, num='5.2.8.261'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive', level=4, num='5.2.8.262'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect', level=4, num='5.2.8.263'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Condition', level=4, num='5.2.8.264'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment', level=4, num='5.2.8.265'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None', level=4, num='5.2.8.266'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All', level=4, num='5.2.8.267'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept', level=4, num='5.2.8.268'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Create.Syntax', level=4, num='5.2.8.269'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter', level=4, num='5.2.8.270'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists', level=4, num='5.2.8.271'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect', level=4, num='5.2.8.272'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster', level=4, num='5.2.8.273'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.On', level=4, num='5.2.8.274'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Rename', level=4, num='5.2.8.275'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access', level=4, num='5.2.8.276'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive', level=4, num='5.2.8.277'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive', level=4, num='5.2.8.278'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Condition', level=4, num='5.2.8.279'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None', level=4, num='5.2.8.280'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment', level=4, num='5.2.8.281'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None', level=4, num='5.2.8.282'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All', level=4, num='5.2.8.283'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept', level=4, num='5.2.8.284'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax', level=4, num='5.2.8.285'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Drop', level=4, num='5.2.8.286'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists', level=4, num='5.2.8.287'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Drop.On', level=4, num='5.2.8.288'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster', level=4, num='5.2.8.289'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax', level=4, num='5.2.8.290'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy', level=4, num='5.2.8.291'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On', level=4, num='5.2.8.292'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax', level=4, num='5.2.8.293'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies', level=4, num='5.2.8.294'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On', level=4, num='5.2.8.295'), - Heading(name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax', level=4, num='5.2.8.296'), - Heading(name='Table Privileges', level=3, num='5.2.9'), - Heading(name='RQ.SRS-006.RBAC.Table.PublicTables', level=4, num='5.2.9.1'), - Heading(name='RQ.SRS-006.RBAC.Table.QueryLog', level=4, num='5.2.9.2'), - Heading(name='Distributed Tables', level=4, num='5.2.9.3'), - Heading(name='RQ.SRS-006.RBAC.Table.DistributedTable.Create', level=5, num='5.2.9.3.1'), - Heading(name='RQ.SRS-006.RBAC.Table.DistributedTable.Select', level=5, num='5.2.9.3.2'), - Heading(name='RQ.SRS-006.RBAC.Table.DistributedTable.Insert', level=5, num='5.2.9.3.3'), - Heading(name='RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables', level=5, num='5.2.9.3.4'), - Heading(name='RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser', level=5, num='5.2.9.3.5'), - Heading(name='RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges', level=5, num='5.2.9.3.6'), - Heading(name='Views', level=3, num='5.2.10'), - Heading(name='View', level=4, num='5.2.10.1'), - Heading(name='RQ.SRS-006.RBAC.View', level=5, num='5.2.10.1.1'), - Heading(name='RQ.SRS-006.RBAC.View.Create', level=5, num='5.2.10.1.2'), - Heading(name='RQ.SRS-006.RBAC.View.Select', level=5, num='5.2.10.1.3'), - Heading(name='RQ.SRS-006.RBAC.View.Drop', level=5, num='5.2.10.1.4'), - Heading(name='Materialized View', level=4, num='5.2.10.2'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView', level=5, num='5.2.10.2.1'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Create', level=5, num='5.2.10.2.2'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Select', level=5, num='5.2.10.2.3'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Select.TargetTable', level=5, num='5.2.10.2.4'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Select.SourceTable', level=5, num='5.2.10.2.5'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Drop', level=5, num='5.2.10.2.6'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.ModifyQuery', level=5, num='5.2.10.2.7'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Insert', level=5, num='5.2.10.2.8'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Insert.SourceTable', level=5, num='5.2.10.2.9'), - Heading(name='RQ.SRS-006.RBAC.MaterializedView.Insert.TargetTable', level=5, num='5.2.10.2.10'), - Heading(name='Live View', level=4, num='5.2.10.3'), - Heading(name='RQ.SRS-006.RBAC.LiveView', level=5, num='5.2.10.3.1'), - Heading(name='RQ.SRS-006.RBAC.LiveView.Create', level=5, num='5.2.10.3.2'), - Heading(name='RQ.SRS-006.RBAC.LiveView.Select', level=5, num='5.2.10.3.3'), - Heading(name='RQ.SRS-006.RBAC.LiveView.Drop', level=5, num='5.2.10.3.4'), - Heading(name='RQ.SRS-006.RBAC.LiveView.Refresh', level=5, num='5.2.10.3.5'), - Heading(name='Privileges', level=3, num='5.2.11'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Usage', level=4, num='5.2.11.1'), - Heading(name='Select', level=4, num='5.2.11.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Select', level=5, num='5.2.11.2.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Select.Column', level=5, num='5.2.11.2.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Select.Cluster', level=5, num='5.2.11.2.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Select.TableEngines', level=5, num='5.2.11.2.4'), - Heading(name='Insert', level=4, num='5.2.11.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Insert', level=5, num='5.2.11.3.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Insert.Column', level=5, num='5.2.11.3.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Insert.Cluster', level=5, num='5.2.11.3.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Insert.TableEngines', level=5, num='5.2.11.3.4'), - Heading(name='Alter Column', level=4, num='5.2.11.4'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterColumn', level=5, num='5.2.11.4.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant', level=5, num='5.2.11.4.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke', level=5, num='5.2.11.4.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Column', level=5, num='5.2.11.4.4'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster', level=5, num='5.2.11.4.5'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines', level=5, num='5.2.11.4.6'), - Heading(name='Alter Index', level=4, num='5.2.11.5'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterIndex', level=5, num='5.2.11.5.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant', level=5, num='5.2.11.5.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke', level=5, num='5.2.11.5.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster', level=5, num='5.2.11.5.4'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines', level=5, num='5.2.11.5.5'), - Heading(name='Alter Constraint', level=4, num='5.2.11.6'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterConstraint', level=5, num='5.2.11.6.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant', level=5, num='5.2.11.6.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke', level=5, num='5.2.11.6.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster', level=5, num='5.2.11.6.4'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines', level=5, num='5.2.11.6.5'), - Heading(name='Alter TTL', level=4, num='5.2.11.7'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterTTL', level=5, num='5.2.11.7.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant', level=5, num='5.2.11.7.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke', level=5, num='5.2.11.7.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster', level=5, num='5.2.11.7.4'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines', level=5, num='5.2.11.7.5'), - Heading(name='Alter Settings', level=4, num='5.2.11.8'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterSettings', level=5, num='5.2.11.8.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant', level=5, num='5.2.11.8.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke', level=5, num='5.2.11.8.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster', level=5, num='5.2.11.8.4'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines', level=5, num='5.2.11.8.5'), - Heading(name='Alter Update', level=4, num='5.2.11.9'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterUpdate', level=5, num='5.2.11.9.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.Grant', level=5, num='5.2.11.9.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.Revoke', level=5, num='5.2.11.9.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines', level=5, num='5.2.11.9.4'), - Heading(name='Alter Delete', level=4, num='5.2.11.10'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterDelete', level=5, num='5.2.11.10.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterDelete.Grant', level=5, num='5.2.11.10.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterDelete.Revoke', level=5, num='5.2.11.10.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines', level=5, num='5.2.11.10.4'), - Heading(name='Alter Freeze Partition', level=4, num='5.2.11.11'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFreeze', level=5, num='5.2.11.11.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.Grant', level=5, num='5.2.11.11.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.Revoke', level=5, num='5.2.11.11.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines', level=5, num='5.2.11.11.4'), - Heading(name='Alter Fetch Partition', level=4, num='5.2.11.12'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFetch', level=5, num='5.2.11.12.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFetch.Grant', level=5, num='5.2.11.12.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFetch.Revoke', level=5, num='5.2.11.12.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines', level=5, num='5.2.11.12.4'), - Heading(name='Alter Move Partition', level=4, num='5.2.11.13'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterMove', level=5, num='5.2.11.13.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterMove.Grant', level=5, num='5.2.11.13.2'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterMove.Revoke', level=5, num='5.2.11.13.3'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines', level=5, num='5.2.11.13.4'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateTable', level=4, num='5.2.11.14'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateDatabase', level=4, num='5.2.11.15'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateDictionary', level=4, num='5.2.11.16'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable', level=4, num='5.2.11.17'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AttachDatabase', level=4, num='5.2.11.18'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AttachDictionary', level=4, num='5.2.11.19'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable', level=4, num='5.2.11.20'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AttachTable', level=4, num='5.2.11.21'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropTable', level=4, num='5.2.11.22'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropDatabase', level=4, num='5.2.11.23'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropDictionary', level=4, num='5.2.11.24'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DetachTable', level=4, num='5.2.11.25'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DetachView', level=4, num='5.2.11.26'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DetachDatabase', level=4, num='5.2.11.27'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DetachDictionary', level=4, num='5.2.11.28'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Truncate', level=4, num='5.2.11.29'), - Heading(name='RQ.SRS-006.RBAC.Privileges.Optimize', level=4, num='5.2.11.30'), - Heading(name='RQ.SRS-006.RBAC.Privileges.KillQuery', level=4, num='5.2.11.31'), - Heading(name='RQ.SRS-006.RBAC.Privileges.KillMutation', level=4, num='5.2.11.32'), - Heading(name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterUpdate', level=4, num='5.2.11.33'), - Heading(name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDelete', level=4, num='5.2.11.34'), - Heading(name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDropColumn', level=4, num='5.2.11.35'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowTables', level=4, num='5.2.11.36'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowTables.Query', level=4, num='5.2.11.37'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ExistsTable', level=4, num='5.2.11.38'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CheckTable', level=4, num='5.2.11.39'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowDatabases', level=4, num='5.2.11.40'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowDatabases.Query', level=4, num='5.2.11.41'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateDatabase', level=4, num='5.2.11.42'), - Heading(name='RQ.SRS-006.RBAC.Privileges.UseDatabase', level=4, num='5.2.11.43'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowColumns', level=4, num='5.2.11.44'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateTable', level=4, num='5.2.11.45'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DescribeTable', level=4, num='5.2.11.46'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowDictionaries', level=4, num='5.2.11.47'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query', level=4, num='5.2.11.48'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary', level=4, num='5.2.11.49'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ExistsDictionary', level=4, num='5.2.11.50'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateUser', level=4, num='5.2.11.51'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole', level=4, num='5.2.11.52'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterUser', level=4, num='5.2.11.53'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropUser', level=4, num='5.2.11.54'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateRole', level=4, num='5.2.11.55'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterRole', level=4, num='5.2.11.56'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropRole', level=4, num='5.2.11.57'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateRowPolicy', level=4, num='5.2.11.58'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterRowPolicy', level=4, num='5.2.11.59'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropRowPolicy', level=4, num='5.2.11.60'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateQuota', level=4, num='5.2.11.61'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterQuota', level=4, num='5.2.11.62'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropQuota', level=4, num='5.2.11.63'), - Heading(name='RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile', level=4, num='5.2.11.64'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile', level=4, num='5.2.11.65'), - Heading(name='RQ.SRS-006.RBAC.Privileges.DropSettingsProfile', level=4, num='5.2.11.66'), - Heading(name='RQ.SRS-006.RBAC.Privileges.RoleAdmin', level=4, num='5.2.11.67'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowUsers', level=4, num='5.2.11.68'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowUsers.Query', level=4, num='5.2.11.69'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateUser', level=4, num='5.2.11.70'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowRoles', level=4, num='5.2.11.71'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowRoles.Query', level=4, num='5.2.11.72'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateRole', level=4, num='5.2.11.73'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowRowPolicies', level=4, num='5.2.11.74'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query', level=4, num='5.2.11.75'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy', level=4, num='5.2.11.76'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowQuotas', level=4, num='5.2.11.77'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query', level=4, num='5.2.11.78'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateQuota', level=4, num='5.2.11.79'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles', level=4, num='5.2.11.80'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query', level=4, num='5.2.11.81'), - Heading(name='RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile', level=4, num='5.2.11.82'), - Heading(name='Grant Option', level=4, num='5.2.11.83'), - Heading(name='RQ.SRS-006.RBAC.Privileges.GrantOption', level=5, num='5.2.11.83.1'), - Heading(name='RQ.SRS-006.RBAC.Privileges.All', level=4, num='5.2.11.84'), - Heading(name='RQ.SRS-006.RBAC.Privileges.All.GrantRevoke', level=4, num='5.2.11.85'), - Heading(name='RQ.SRS-006.RBAC.Privileges.AdminOption', level=4, num='5.2.11.86'), - Heading(name='Required Privileges', level=3, num='5.2.12'), - Heading(name='RQ.SRS-006.RBAC.RequiredPrivileges.Create', level=4, num='5.2.12.1'), - Heading(name='RQ.SRS-006.RBAC.RequiredPrivileges.Alter', level=4, num='5.2.12.2'), - Heading(name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop', level=4, num='5.2.12.3'), - Heading(name='RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table', level=4, num='5.2.12.4'), - Heading(name='RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke', level=4, num='5.2.12.5'), - Heading(name='RQ.SRS-006.RBAC.RequiredPrivileges.Use', level=4, num='5.2.12.6'), - Heading(name='RQ.SRS-006.RBAC.RequiredPrivileges.Admin', level=4, num='5.2.12.7'), - Heading(name='References', level=1, num='6'), - ), - requirements=( - RQ_SRS_006_RBAC, - RQ_SRS_006_RBAC_Login, - RQ_SRS_006_RBAC_Login_DefaultUser, - RQ_SRS_006_RBAC_User, - RQ_SRS_006_RBAC_User_Roles, - RQ_SRS_006_RBAC_User_Privileges, - RQ_SRS_006_RBAC_User_Variables, - RQ_SRS_006_RBAC_User_Variables_Constraints, - RQ_SRS_006_RBAC_User_SettingsProfile, - RQ_SRS_006_RBAC_User_Quotas, - RQ_SRS_006_RBAC_User_RowPolicies, - RQ_SRS_006_RBAC_User_AccountLock, - RQ_SRS_006_RBAC_User_AccountLock_DenyAccess, - RQ_SRS_006_RBAC_User_DefaultRole, - RQ_SRS_006_RBAC_User_RoleSelection, - RQ_SRS_006_RBAC_User_ShowCreate, - RQ_SRS_006_RBAC_User_ShowPrivileges, - RQ_SRS_006_RBAC_Role, - RQ_SRS_006_RBAC_Role_Privileges, - RQ_SRS_006_RBAC_Role_Variables, - RQ_SRS_006_RBAC_Role_SettingsProfile, - RQ_SRS_006_RBAC_Role_Quotas, - RQ_SRS_006_RBAC_Role_RowPolicies, - RQ_SRS_006_RBAC_PartialRevokes, - RQ_SRS_006_RBAC_SettingsProfile, - RQ_SRS_006_RBAC_SettingsProfile_Constraints, - RQ_SRS_006_RBAC_SettingsProfile_ShowCreate, - RQ_SRS_006_RBAC_Quotas, - RQ_SRS_006_RBAC_Quotas_Keyed, - RQ_SRS_006_RBAC_Quotas_Queries, - RQ_SRS_006_RBAC_Quotas_Errors, - RQ_SRS_006_RBAC_Quotas_ResultRows, - RQ_SRS_006_RBAC_Quotas_ReadRows, - RQ_SRS_006_RBAC_Quotas_ResultBytes, - RQ_SRS_006_RBAC_Quotas_ReadBytes, - RQ_SRS_006_RBAC_Quotas_ExecutionTime, - RQ_SRS_006_RBAC_Quotas_ShowCreate, - RQ_SRS_006_RBAC_RowPolicy, - RQ_SRS_006_RBAC_RowPolicy_Condition, - RQ_SRS_006_RBAC_RowPolicy_ShowCreate, - RQ_SRS_006_RBAC_User_Use_DefaultRole, - RQ_SRS_006_RBAC_User_Use_AllRolesWhenNoDefaultRole, - RQ_SRS_006_RBAC_User_Create, - RQ_SRS_006_RBAC_User_Create_IfNotExists, - RQ_SRS_006_RBAC_User_Create_Replace, - RQ_SRS_006_RBAC_User_Create_Password_NoPassword, - RQ_SRS_006_RBAC_User_Create_Password_NoPassword_Login, - RQ_SRS_006_RBAC_User_Create_Password_PlainText, - RQ_SRS_006_RBAC_User_Create_Password_PlainText_Login, - RQ_SRS_006_RBAC_User_Create_Password_Sha256Password, - RQ_SRS_006_RBAC_User_Create_Password_Sha256Password_Login, - RQ_SRS_006_RBAC_User_Create_Password_Sha256Hash, - RQ_SRS_006_RBAC_User_Create_Password_Sha256Hash_Login, - RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Password, - RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Password_Login, - RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Hash, - RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Hash_Login, - RQ_SRS_006_RBAC_User_Create_Host_Name, - RQ_SRS_006_RBAC_User_Create_Host_Regexp, - RQ_SRS_006_RBAC_User_Create_Host_IP, - RQ_SRS_006_RBAC_User_Create_Host_Any, - RQ_SRS_006_RBAC_User_Create_Host_None, - RQ_SRS_006_RBAC_User_Create_Host_Local, - RQ_SRS_006_RBAC_User_Create_Host_Like, - RQ_SRS_006_RBAC_User_Create_Host_Default, - RQ_SRS_006_RBAC_User_Create_DefaultRole, - RQ_SRS_006_RBAC_User_Create_DefaultRole_None, - RQ_SRS_006_RBAC_User_Create_DefaultRole_All, - RQ_SRS_006_RBAC_User_Create_Settings, - RQ_SRS_006_RBAC_User_Create_OnCluster, - RQ_SRS_006_RBAC_User_Create_Syntax, - RQ_SRS_006_RBAC_User_Alter, - RQ_SRS_006_RBAC_User_Alter_OrderOfEvaluation, - RQ_SRS_006_RBAC_User_Alter_IfExists, - RQ_SRS_006_RBAC_User_Alter_Cluster, - RQ_SRS_006_RBAC_User_Alter_Rename, - RQ_SRS_006_RBAC_User_Alter_Password_PlainText, - RQ_SRS_006_RBAC_User_Alter_Password_Sha256Password, - RQ_SRS_006_RBAC_User_Alter_Password_DoubleSha1Password, - RQ_SRS_006_RBAC_User_Alter_Host_AddDrop, - RQ_SRS_006_RBAC_User_Alter_Host_Local, - RQ_SRS_006_RBAC_User_Alter_Host_Name, - RQ_SRS_006_RBAC_User_Alter_Host_Regexp, - RQ_SRS_006_RBAC_User_Alter_Host_IP, - RQ_SRS_006_RBAC_User_Alter_Host_Like, - RQ_SRS_006_RBAC_User_Alter_Host_Any, - RQ_SRS_006_RBAC_User_Alter_Host_None, - RQ_SRS_006_RBAC_User_Alter_DefaultRole, - RQ_SRS_006_RBAC_User_Alter_DefaultRole_All, - RQ_SRS_006_RBAC_User_Alter_DefaultRole_AllExcept, - RQ_SRS_006_RBAC_User_Alter_Settings, - RQ_SRS_006_RBAC_User_Alter_Settings_Min, - RQ_SRS_006_RBAC_User_Alter_Settings_Max, - RQ_SRS_006_RBAC_User_Alter_Settings_Profile, - RQ_SRS_006_RBAC_User_Alter_Syntax, - RQ_SRS_006_RBAC_SetDefaultRole, - RQ_SRS_006_RBAC_SetDefaultRole_CurrentUser, - RQ_SRS_006_RBAC_SetDefaultRole_All, - RQ_SRS_006_RBAC_SetDefaultRole_AllExcept, - RQ_SRS_006_RBAC_SetDefaultRole_None, - RQ_SRS_006_RBAC_SetDefaultRole_Syntax, - RQ_SRS_006_RBAC_SetRole, - RQ_SRS_006_RBAC_SetRole_Default, - RQ_SRS_006_RBAC_SetRole_None, - RQ_SRS_006_RBAC_SetRole_All, - RQ_SRS_006_RBAC_SetRole_AllExcept, - RQ_SRS_006_RBAC_SetRole_Syntax, - RQ_SRS_006_RBAC_User_ShowCreateUser, - RQ_SRS_006_RBAC_User_ShowCreateUser_For, - RQ_SRS_006_RBAC_User_ShowCreateUser_Syntax, - RQ_SRS_006_RBAC_User_Drop, - RQ_SRS_006_RBAC_User_Drop_IfExists, - RQ_SRS_006_RBAC_User_Drop_OnCluster, - RQ_SRS_006_RBAC_User_Drop_Syntax, - RQ_SRS_006_RBAC_Role_Create, - RQ_SRS_006_RBAC_Role_Create_IfNotExists, - RQ_SRS_006_RBAC_Role_Create_Replace, - RQ_SRS_006_RBAC_Role_Create_Settings, - RQ_SRS_006_RBAC_Role_Create_Syntax, - RQ_SRS_006_RBAC_Role_Alter, - RQ_SRS_006_RBAC_Role_Alter_IfExists, - RQ_SRS_006_RBAC_Role_Alter_Cluster, - RQ_SRS_006_RBAC_Role_Alter_Rename, - RQ_SRS_006_RBAC_Role_Alter_Settings, - RQ_SRS_006_RBAC_Role_Alter_Syntax, - RQ_SRS_006_RBAC_Role_Drop, - RQ_SRS_006_RBAC_Role_Drop_IfExists, - RQ_SRS_006_RBAC_Role_Drop_Cluster, - RQ_SRS_006_RBAC_Role_Drop_Syntax, - RQ_SRS_006_RBAC_Role_ShowCreate, - RQ_SRS_006_RBAC_Role_ShowCreate_Syntax, - RQ_SRS_006_RBAC_Grant_Privilege_To, - RQ_SRS_006_RBAC_Grant_Privilege_ToCurrentUser, - RQ_SRS_006_RBAC_Grant_Privilege_Select, - RQ_SRS_006_RBAC_Grant_Privilege_Insert, - RQ_SRS_006_RBAC_Grant_Privilege_Alter, - RQ_SRS_006_RBAC_Grant_Privilege_Create, - RQ_SRS_006_RBAC_Grant_Privilege_Create_Database, - RQ_SRS_006_RBAC_Grant_Privilege_Create_Dictionary, - RQ_SRS_006_RBAC_Grant_Privilege_Create_Table, - RQ_SRS_006_RBAC_Grant_Privilege_Create_TemporaryTable, - RQ_SRS_006_RBAC_Grant_Privilege_Drop, - RQ_SRS_006_RBAC_Grant_Privilege_Drop_Database, - RQ_SRS_006_RBAC_Grant_Privilege_Drop_Dictionary, - RQ_SRS_006_RBAC_Grant_Privilege_Drop_Table, - RQ_SRS_006_RBAC_Grant_Privilege_Drop_TemporaryTable, - RQ_SRS_006_RBAC_Grant_Privilege_Truncate, - RQ_SRS_006_RBAC_Grant_Privilege_Optimize, - RQ_SRS_006_RBAC_Grant_Privilege_Show, - RQ_SRS_006_RBAC_Grant_Privilege_KillQuery, - RQ_SRS_006_RBAC_Grant_Privilege_AccessManagement, - RQ_SRS_006_RBAC_Grant_Privilege_System, - RQ_SRS_006_RBAC_Grant_Privilege_Introspection, - RQ_SRS_006_RBAC_Grant_Privilege_Sources, - RQ_SRS_006_RBAC_Grant_Privilege_DictGet, - RQ_SRS_006_RBAC_Grant_Privilege_None, - RQ_SRS_006_RBAC_Grant_Privilege_All, - RQ_SRS_006_RBAC_Grant_Privilege_GrantOption, - RQ_SRS_006_RBAC_Grant_Privilege_On, - RQ_SRS_006_RBAC_Grant_Privilege_PrivilegeColumns, - RQ_SRS_006_RBAC_Grant_Privilege_OnCluster, - RQ_SRS_006_RBAC_Grant_Privilege_Syntax, - RQ_SRS_006_RBAC_Revoke_Privilege_Cluster, - RQ_SRS_006_RBAC_Revoke_Privilege_Any, - RQ_SRS_006_RBAC_Revoke_Privilege_Select, - RQ_SRS_006_RBAC_Revoke_Privilege_Insert, - RQ_SRS_006_RBAC_Revoke_Privilege_Alter, - RQ_SRS_006_RBAC_Revoke_Privilege_Create, - RQ_SRS_006_RBAC_Revoke_Privilege_Create_Database, - RQ_SRS_006_RBAC_Revoke_Privilege_Create_Dictionary, - RQ_SRS_006_RBAC_Revoke_Privilege_Create_Table, - RQ_SRS_006_RBAC_Revoke_Privilege_Create_TemporaryTable, - RQ_SRS_006_RBAC_Revoke_Privilege_Drop, - RQ_SRS_006_RBAC_Revoke_Privilege_Drop_Database, - RQ_SRS_006_RBAC_Revoke_Privilege_Drop_Dictionary, - RQ_SRS_006_RBAC_Revoke_Privilege_Drop_Table, - RQ_SRS_006_RBAC_Revoke_Privilege_Drop_TemporaryTable, - RQ_SRS_006_RBAC_Revoke_Privilege_Truncate, - RQ_SRS_006_RBAC_Revoke_Privilege_Optimize, - RQ_SRS_006_RBAC_Revoke_Privilege_Show, - RQ_SRS_006_RBAC_Revoke_Privilege_KillQuery, - RQ_SRS_006_RBAC_Revoke_Privilege_AccessManagement, - RQ_SRS_006_RBAC_Revoke_Privilege_System, - RQ_SRS_006_RBAC_Revoke_Privilege_Introspection, - RQ_SRS_006_RBAC_Revoke_Privilege_Sources, - RQ_SRS_006_RBAC_Revoke_Privilege_DictGet, - RQ_SRS_006_RBAC_Revoke_Privilege_PrivelegeColumns, - RQ_SRS_006_RBAC_Revoke_Privilege_Multiple, - RQ_SRS_006_RBAC_Revoke_Privilege_All, - RQ_SRS_006_RBAC_Revoke_Privilege_None, - RQ_SRS_006_RBAC_Revoke_Privilege_On, - RQ_SRS_006_RBAC_Revoke_Privilege_From, - RQ_SRS_006_RBAC_Revoke_Privilege_Syntax, - RQ_SRS_006_RBAC_PartialRevoke_Syntax, - RQ_SRS_006_RBAC_Grant_Role, - RQ_SRS_006_RBAC_Grant_Role_CurrentUser, - RQ_SRS_006_RBAC_Grant_Role_AdminOption, - RQ_SRS_006_RBAC_Grant_Role_OnCluster, - RQ_SRS_006_RBAC_Grant_Role_Syntax, - RQ_SRS_006_RBAC_Revoke_Role, - RQ_SRS_006_RBAC_Revoke_Role_Keywords, - RQ_SRS_006_RBAC_Revoke_Role_Cluster, - RQ_SRS_006_RBAC_Revoke_AdminOption, - RQ_SRS_006_RBAC_Revoke_Role_Syntax, - RQ_SRS_006_RBAC_Show_Grants, - RQ_SRS_006_RBAC_Show_Grants_For, - RQ_SRS_006_RBAC_Show_Grants_Syntax, - RQ_SRS_006_RBAC_SettingsProfile_Create, - RQ_SRS_006_RBAC_SettingsProfile_Create_IfNotExists, - RQ_SRS_006_RBAC_SettingsProfile_Create_Replace, - RQ_SRS_006_RBAC_SettingsProfile_Create_Variables, - RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Value, - RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Constraints, - RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment, - RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_None, - RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_All, - RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_AllExcept, - RQ_SRS_006_RBAC_SettingsProfile_Create_Inherit, - RQ_SRS_006_RBAC_SettingsProfile_Create_OnCluster, - RQ_SRS_006_RBAC_SettingsProfile_Create_Syntax, - RQ_SRS_006_RBAC_SettingsProfile_Alter, - RQ_SRS_006_RBAC_SettingsProfile_Alter_IfExists, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Rename, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Constraints, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_None, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_All, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_AllExcept, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_Inherit, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_OnCluster, - RQ_SRS_006_RBAC_SettingsProfile_Alter_Syntax, - RQ_SRS_006_RBAC_SettingsProfile_Drop, - RQ_SRS_006_RBAC_SettingsProfile_Drop_IfExists, - RQ_SRS_006_RBAC_SettingsProfile_Drop_OnCluster, - RQ_SRS_006_RBAC_SettingsProfile_Drop_Syntax, - RQ_SRS_006_RBAC_SettingsProfile_ShowCreateSettingsProfile, - RQ_SRS_006_RBAC_Quota_Create, - RQ_SRS_006_RBAC_Quota_Create_IfNotExists, - RQ_SRS_006_RBAC_Quota_Create_Replace, - RQ_SRS_006_RBAC_Quota_Create_Cluster, - RQ_SRS_006_RBAC_Quota_Create_Interval, - RQ_SRS_006_RBAC_Quota_Create_Interval_Randomized, - RQ_SRS_006_RBAC_Quota_Create_Queries, - RQ_SRS_006_RBAC_Quota_Create_Errors, - RQ_SRS_006_RBAC_Quota_Create_ResultRows, - RQ_SRS_006_RBAC_Quota_Create_ReadRows, - RQ_SRS_006_RBAC_Quota_Create_ResultBytes, - RQ_SRS_006_RBAC_Quota_Create_ReadBytes, - RQ_SRS_006_RBAC_Quota_Create_ExecutionTime, - RQ_SRS_006_RBAC_Quota_Create_NoLimits, - RQ_SRS_006_RBAC_Quota_Create_TrackingOnly, - RQ_SRS_006_RBAC_Quota_Create_KeyedBy, - RQ_SRS_006_RBAC_Quota_Create_KeyedByOptions, - RQ_SRS_006_RBAC_Quota_Create_Assignment, - RQ_SRS_006_RBAC_Quota_Create_Assignment_None, - RQ_SRS_006_RBAC_Quota_Create_Assignment_All, - RQ_SRS_006_RBAC_Quota_Create_Assignment_Except, - RQ_SRS_006_RBAC_Quota_Create_Syntax, - RQ_SRS_006_RBAC_Quota_Alter, - RQ_SRS_006_RBAC_Quota_Alter_IfExists, - RQ_SRS_006_RBAC_Quota_Alter_Rename, - RQ_SRS_006_RBAC_Quota_Alter_Cluster, - RQ_SRS_006_RBAC_Quota_Alter_Interval, - RQ_SRS_006_RBAC_Quota_Alter_Interval_Randomized, - RQ_SRS_006_RBAC_Quota_Alter_Queries, - RQ_SRS_006_RBAC_Quota_Alter_Errors, - RQ_SRS_006_RBAC_Quota_Alter_ResultRows, - RQ_SRS_006_RBAC_Quota_Alter_ReadRows, - RQ_SRS_006_RBAC_Quota_ALter_ResultBytes, - RQ_SRS_006_RBAC_Quota_Alter_ReadBytes, - RQ_SRS_006_RBAC_Quota_Alter_ExecutionTime, - RQ_SRS_006_RBAC_Quota_Alter_NoLimits, - RQ_SRS_006_RBAC_Quota_Alter_TrackingOnly, - RQ_SRS_006_RBAC_Quota_Alter_KeyedBy, - RQ_SRS_006_RBAC_Quota_Alter_KeyedByOptions, - RQ_SRS_006_RBAC_Quota_Alter_Assignment, - RQ_SRS_006_RBAC_Quota_Alter_Assignment_None, - RQ_SRS_006_RBAC_Quota_Alter_Assignment_All, - RQ_SRS_006_RBAC_Quota_Alter_Assignment_Except, - RQ_SRS_006_RBAC_Quota_Alter_Syntax, - RQ_SRS_006_RBAC_Quota_Drop, - RQ_SRS_006_RBAC_Quota_Drop_IfExists, - RQ_SRS_006_RBAC_Quota_Drop_Cluster, - RQ_SRS_006_RBAC_Quota_Drop_Syntax, - RQ_SRS_006_RBAC_Quota_ShowQuotas, - RQ_SRS_006_RBAC_Quota_ShowQuotas_IntoOutfile, - RQ_SRS_006_RBAC_Quota_ShowQuotas_Format, - RQ_SRS_006_RBAC_Quota_ShowQuotas_Settings, - RQ_SRS_006_RBAC_Quota_ShowQuotas_Syntax, - RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Name, - RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Current, - RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Syntax, - RQ_SRS_006_RBAC_RowPolicy_Create, - RQ_SRS_006_RBAC_RowPolicy_Create_IfNotExists, - RQ_SRS_006_RBAC_RowPolicy_Create_Replace, - RQ_SRS_006_RBAC_RowPolicy_Create_OnCluster, - RQ_SRS_006_RBAC_RowPolicy_Create_On, - RQ_SRS_006_RBAC_RowPolicy_Create_Access, - RQ_SRS_006_RBAC_RowPolicy_Create_Access_Permissive, - RQ_SRS_006_RBAC_RowPolicy_Create_Access_Restrictive, - RQ_SRS_006_RBAC_RowPolicy_Create_ForSelect, - RQ_SRS_006_RBAC_RowPolicy_Create_Condition, - RQ_SRS_006_RBAC_RowPolicy_Create_Assignment, - RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_None, - RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_All, - RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_AllExcept, - RQ_SRS_006_RBAC_RowPolicy_Create_Syntax, - RQ_SRS_006_RBAC_RowPolicy_Alter, - RQ_SRS_006_RBAC_RowPolicy_Alter_IfExists, - RQ_SRS_006_RBAC_RowPolicy_Alter_ForSelect, - RQ_SRS_006_RBAC_RowPolicy_Alter_OnCluster, - RQ_SRS_006_RBAC_RowPolicy_Alter_On, - RQ_SRS_006_RBAC_RowPolicy_Alter_Rename, - RQ_SRS_006_RBAC_RowPolicy_Alter_Access, - RQ_SRS_006_RBAC_RowPolicy_Alter_Access_Permissive, - RQ_SRS_006_RBAC_RowPolicy_Alter_Access_Restrictive, - RQ_SRS_006_RBAC_RowPolicy_Alter_Condition, - RQ_SRS_006_RBAC_RowPolicy_Alter_Condition_None, - RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment, - RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_None, - RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_All, - RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_AllExcept, - RQ_SRS_006_RBAC_RowPolicy_Alter_Syntax, - RQ_SRS_006_RBAC_RowPolicy_Drop, - RQ_SRS_006_RBAC_RowPolicy_Drop_IfExists, - RQ_SRS_006_RBAC_RowPolicy_Drop_On, - RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster, - RQ_SRS_006_RBAC_RowPolicy_Drop_Syntax, - RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy, - RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy_On, - RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy_Syntax, - RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies, - RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies_On, - RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies_Syntax, - RQ_SRS_006_RBAC_Table_PublicTables, - RQ_SRS_006_RBAC_Table_QueryLog, - RQ_SRS_006_RBAC_Table_DistributedTable_Create, - RQ_SRS_006_RBAC_Table_DistributedTable_Select, - RQ_SRS_006_RBAC_Table_DistributedTable_Insert, - RQ_SRS_006_RBAC_Table_DistributedTable_SpecialTables, - RQ_SRS_006_RBAC_Table_DistributedTable_LocalUser, - RQ_SRS_006_RBAC_Table_DistributedTable_SameUserDifferentNodesDifferentPrivileges, - RQ_SRS_006_RBAC_View, - RQ_SRS_006_RBAC_View_Create, - RQ_SRS_006_RBAC_View_Select, - RQ_SRS_006_RBAC_View_Drop, - RQ_SRS_006_RBAC_MaterializedView, - RQ_SRS_006_RBAC_MaterializedView_Create, - RQ_SRS_006_RBAC_MaterializedView_Select, - RQ_SRS_006_RBAC_MaterializedView_Select_TargetTable, - RQ_SRS_006_RBAC_MaterializedView_Select_SourceTable, - RQ_SRS_006_RBAC_MaterializedView_Drop, - RQ_SRS_006_RBAC_MaterializedView_ModifyQuery, - RQ_SRS_006_RBAC_MaterializedView_Insert, - RQ_SRS_006_RBAC_MaterializedView_Insert_SourceTable, - RQ_SRS_006_RBAC_MaterializedView_Insert_TargetTable, - RQ_SRS_006_RBAC_LiveView, - RQ_SRS_006_RBAC_LiveView_Create, - RQ_SRS_006_RBAC_LiveView_Select, - RQ_SRS_006_RBAC_LiveView_Drop, - RQ_SRS_006_RBAC_LiveView_Refresh, - RQ_SRS_006_RBAC_Privileges_Usage, - RQ_SRS_006_RBAC_Privileges_Select, - RQ_SRS_006_RBAC_Privileges_Select_Column, - RQ_SRS_006_RBAC_Privileges_Select_Cluster, - RQ_SRS_006_RBAC_Privileges_Select_TableEngines, - RQ_SRS_006_RBAC_Privileges_Insert, - RQ_SRS_006_RBAC_Privileges_Insert_Column, - RQ_SRS_006_RBAC_Privileges_Insert_Cluster, - RQ_SRS_006_RBAC_Privileges_Insert_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterColumn, - RQ_SRS_006_RBAC_Privileges_AlterColumn_Grant, - RQ_SRS_006_RBAC_Privileges_AlterColumn_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterColumn_Column, - RQ_SRS_006_RBAC_Privileges_AlterColumn_Cluster, - RQ_SRS_006_RBAC_Privileges_AlterColumn_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterIndex, - RQ_SRS_006_RBAC_Privileges_AlterIndex_Grant, - RQ_SRS_006_RBAC_Privileges_AlterIndex_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterIndex_Cluster, - RQ_SRS_006_RBAC_Privileges_AlterIndex_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterConstraint, - RQ_SRS_006_RBAC_Privileges_AlterConstraint_Grant, - RQ_SRS_006_RBAC_Privileges_AlterConstraint_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterConstraint_Cluster, - RQ_SRS_006_RBAC_Privileges_AlterConstraint_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterTTL, - RQ_SRS_006_RBAC_Privileges_AlterTTL_Grant, - RQ_SRS_006_RBAC_Privileges_AlterTTL_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterTTL_Cluster, - RQ_SRS_006_RBAC_Privileges_AlterTTL_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterSettings, - RQ_SRS_006_RBAC_Privileges_AlterSettings_Grant, - RQ_SRS_006_RBAC_Privileges_AlterSettings_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterSettings_Cluster, - RQ_SRS_006_RBAC_Privileges_AlterSettings_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterUpdate, - RQ_SRS_006_RBAC_Privileges_AlterUpdate_Grant, - RQ_SRS_006_RBAC_Privileges_AlterUpdate_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterUpdate_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterDelete, - RQ_SRS_006_RBAC_Privileges_AlterDelete_Grant, - RQ_SRS_006_RBAC_Privileges_AlterDelete_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterDelete_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterFreeze, - RQ_SRS_006_RBAC_Privileges_AlterFreeze_Grant, - RQ_SRS_006_RBAC_Privileges_AlterFreeze_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterFreeze_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterFetch, - RQ_SRS_006_RBAC_Privileges_AlterFetch_Grant, - RQ_SRS_006_RBAC_Privileges_AlterFetch_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterFetch_TableEngines, - RQ_SRS_006_RBAC_Privileges_AlterMove, - RQ_SRS_006_RBAC_Privileges_AlterMove_Grant, - RQ_SRS_006_RBAC_Privileges_AlterMove_Revoke, - RQ_SRS_006_RBAC_Privileges_AlterMove_TableEngines, - RQ_SRS_006_RBAC_Privileges_CreateTable, - RQ_SRS_006_RBAC_Privileges_CreateDatabase, - RQ_SRS_006_RBAC_Privileges_CreateDictionary, - RQ_SRS_006_RBAC_Privileges_CreateTemporaryTable, - RQ_SRS_006_RBAC_Privileges_AttachDatabase, - RQ_SRS_006_RBAC_Privileges_AttachDictionary, - RQ_SRS_006_RBAC_Privileges_AttachTemporaryTable, - RQ_SRS_006_RBAC_Privileges_AttachTable, - RQ_SRS_006_RBAC_Privileges_DropTable, - RQ_SRS_006_RBAC_Privileges_DropDatabase, - RQ_SRS_006_RBAC_Privileges_DropDictionary, - RQ_SRS_006_RBAC_Privileges_DetachTable, - RQ_SRS_006_RBAC_Privileges_DetachView, - RQ_SRS_006_RBAC_Privileges_DetachDatabase, - RQ_SRS_006_RBAC_Privileges_DetachDictionary, - RQ_SRS_006_RBAC_Privileges_Truncate, - RQ_SRS_006_RBAC_Privileges_Optimize, - RQ_SRS_006_RBAC_Privileges_KillQuery, - RQ_SRS_006_RBAC_Privileges_KillMutation, - RQ_SRS_006_RBAC_Privileges_KillMutation_AlterUpdate, - RQ_SRS_006_RBAC_Privileges_KillMutation_AlterDelete, - RQ_SRS_006_RBAC_Privileges_KillMutation_AlterDropColumn, - RQ_SRS_006_RBAC_Privileges_ShowTables, - RQ_SRS_006_RBAC_Privileges_ShowTables_Query, - RQ_SRS_006_RBAC_Privileges_ExistsTable, - RQ_SRS_006_RBAC_Privileges_CheckTable, - RQ_SRS_006_RBAC_Privileges_ShowDatabases, - RQ_SRS_006_RBAC_Privileges_ShowDatabases_Query, - RQ_SRS_006_RBAC_Privileges_ShowCreateDatabase, - RQ_SRS_006_RBAC_Privileges_UseDatabase, - RQ_SRS_006_RBAC_Privileges_ShowColumns, - RQ_SRS_006_RBAC_Privileges_ShowCreateTable, - RQ_SRS_006_RBAC_Privileges_DescribeTable, - RQ_SRS_006_RBAC_Privileges_ShowDictionaries, - RQ_SRS_006_RBAC_Privileges_ShowDictionaries_Query, - RQ_SRS_006_RBAC_Privileges_ShowCreateDictionary, - RQ_SRS_006_RBAC_Privileges_ExistsDictionary, - RQ_SRS_006_RBAC_Privileges_CreateUser, - RQ_SRS_006_RBAC_Privileges_CreateUser_DefaultRole, - RQ_SRS_006_RBAC_Privileges_AlterUser, - RQ_SRS_006_RBAC_Privileges_DropUser, - RQ_SRS_006_RBAC_Privileges_CreateRole, - RQ_SRS_006_RBAC_Privileges_AlterRole, - RQ_SRS_006_RBAC_Privileges_DropRole, - RQ_SRS_006_RBAC_Privileges_CreateRowPolicy, - RQ_SRS_006_RBAC_Privileges_AlterRowPolicy, - RQ_SRS_006_RBAC_Privileges_DropRowPolicy, - RQ_SRS_006_RBAC_Privileges_CreateQuota, - RQ_SRS_006_RBAC_Privileges_AlterQuota, - RQ_SRS_006_RBAC_Privileges_DropQuota, - RQ_SRS_006_RBAC_Privileges_CreateSettingsProfile, - RQ_SRS_006_RBAC_Privileges_AlterSettingsProfile, - RQ_SRS_006_RBAC_Privileges_DropSettingsProfile, - RQ_SRS_006_RBAC_Privileges_RoleAdmin, - RQ_SRS_006_RBAC_Privileges_ShowUsers, - RQ_SRS_006_RBAC_Privileges_ShowUsers_Query, - RQ_SRS_006_RBAC_Privileges_ShowCreateUser, - RQ_SRS_006_RBAC_Privileges_ShowRoles, - RQ_SRS_006_RBAC_Privileges_ShowRoles_Query, - RQ_SRS_006_RBAC_Privileges_ShowCreateRole, - RQ_SRS_006_RBAC_Privileges_ShowRowPolicies, - RQ_SRS_006_RBAC_Privileges_ShowRowPolicies_Query, - RQ_SRS_006_RBAC_Privileges_ShowCreateRowPolicy, - RQ_SRS_006_RBAC_Privileges_ShowQuotas, - RQ_SRS_006_RBAC_Privileges_ShowQuotas_Query, - RQ_SRS_006_RBAC_Privileges_ShowCreateQuota, - RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles, - RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles_Query, - RQ_SRS_006_RBAC_Privileges_ShowCreateSettingsProfile, - RQ_SRS_006_RBAC_Privileges_GrantOption, - RQ_SRS_006_RBAC_Privileges_All, - RQ_SRS_006_RBAC_Privileges_All_GrantRevoke, - RQ_SRS_006_RBAC_Privileges_AdminOption, - RQ_SRS_006_RBAC_RequiredPrivileges_Create, - RQ_SRS_006_RBAC_RequiredPrivileges_Alter, - RQ_SRS_006_RBAC_RequiredPrivileges_Drop, - RQ_SRS_006_RBAC_RequiredPrivileges_Drop_Table, - RQ_SRS_006_RBAC_RequiredPrivileges_GrantRevoke, - RQ_SRS_006_RBAC_RequiredPrivileges_Use, - RQ_SRS_006_RBAC_RequiredPrivileges_Admin, - ), - content=''' + name='SRS-006 ClickHouse Role Based Access Control', + description=None, + author=None, + date=None, + status=None, + approved_by=None, + approved_date=None, + approved_version=None, + version=None, + group=None, + type=None, + link=None, + uid=None, + parent=None, + children=None, + content=''' # SRS-006 ClickHouse Role Based Access Control # Software Requirements Specification @@ -9817,388 +178,412 @@ SRS_006_ClickHouse_Role_Based_Access_Control = Specification( * 5.2.8.95 [RQ.SRS-006.RBAC.Grant.Privilege.Insert](#rqsrs-006rbacgrantprivilegeinsert) * 5.2.8.96 [RQ.SRS-006.RBAC.Grant.Privilege.Alter](#rqsrs-006rbacgrantprivilegealter) * 5.2.8.97 [RQ.SRS-006.RBAC.Grant.Privilege.Create](#rqsrs-006rbacgrantprivilegecreate) - * 5.2.8.98 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Database](#rqsrs-006rbacgrantprivilegecreatedatabase) - * 5.2.8.99 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Dictionary](#rqsrs-006rbacgrantprivilegecreatedictionary) - * 5.2.8.100 [RQ.SRS-006.RBAC.Grant.Privilege.Create.Table](#rqsrs-006rbacgrantprivilegecreatetable) - * 5.2.8.101 [RQ.SRS-006.RBAC.Grant.Privilege.Create.TemporaryTable](#rqsrs-006rbacgrantprivilegecreatetemporarytable) - * 5.2.8.102 [RQ.SRS-006.RBAC.Grant.Privilege.Drop](#rqsrs-006rbacgrantprivilegedrop) - * 5.2.8.103 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Database](#rqsrs-006rbacgrantprivilegedropdatabase) - * 5.2.8.104 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Dictionary](#rqsrs-006rbacgrantprivilegedropdictionary) - * 5.2.8.105 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.Table](#rqsrs-006rbacgrantprivilegedroptable) - * 5.2.8.106 [RQ.SRS-006.RBAC.Grant.Privilege.Drop.TemporaryTable](#rqsrs-006rbacgrantprivilegedroptemporarytable) - * 5.2.8.107 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate](#rqsrs-006rbacgrantprivilegetruncate) - * 5.2.8.108 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize](#rqsrs-006rbacgrantprivilegeoptimize) - * 5.2.8.109 [RQ.SRS-006.RBAC.Grant.Privilege.Show](#rqsrs-006rbacgrantprivilegeshow) - * 5.2.8.110 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery](#rqsrs-006rbacgrantprivilegekillquery) - * 5.2.8.111 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement](#rqsrs-006rbacgrantprivilegeaccessmanagement) - * 5.2.8.112 [RQ.SRS-006.RBAC.Grant.Privilege.System](#rqsrs-006rbacgrantprivilegesystem) - * 5.2.8.113 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection](#rqsrs-006rbacgrantprivilegeintrospection) - * 5.2.8.114 [RQ.SRS-006.RBAC.Grant.Privilege.Sources](#rqsrs-006rbacgrantprivilegesources) - * 5.2.8.115 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet](#rqsrs-006rbacgrantprivilegedictget) - * 5.2.8.116 [RQ.SRS-006.RBAC.Grant.Privilege.None](#rqsrs-006rbacgrantprivilegenone) - * 5.2.8.117 [RQ.SRS-006.RBAC.Grant.Privilege.All](#rqsrs-006rbacgrantprivilegeall) - * 5.2.8.118 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption](#rqsrs-006rbacgrantprivilegegrantoption) - * 5.2.8.119 [RQ.SRS-006.RBAC.Grant.Privilege.On](#rqsrs-006rbacgrantprivilegeon) - * 5.2.8.120 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns](#rqsrs-006rbacgrantprivilegeprivilegecolumns) - * 5.2.8.121 [RQ.SRS-006.RBAC.Grant.Privilege.OnCluster](#rqsrs-006rbacgrantprivilegeoncluster) - * 5.2.8.122 [RQ.SRS-006.RBAC.Grant.Privilege.Syntax](#rqsrs-006rbacgrantprivilegesyntax) - * 5.2.8.123 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster](#rqsrs-006rbacrevokeprivilegecluster) - * 5.2.8.124 [RQ.SRS-006.RBAC.Revoke.Privilege.Any](#rqsrs-006rbacrevokeprivilegeany) - * 5.2.8.125 [RQ.SRS-006.RBAC.Revoke.Privilege.Select](#rqsrs-006rbacrevokeprivilegeselect) - * 5.2.8.126 [RQ.SRS-006.RBAC.Revoke.Privilege.Insert](#rqsrs-006rbacrevokeprivilegeinsert) - * 5.2.8.127 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter](#rqsrs-006rbacrevokeprivilegealter) - * 5.2.8.128 [RQ.SRS-006.RBAC.Revoke.Privilege.Create](#rqsrs-006rbacrevokeprivilegecreate) - * 5.2.8.129 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Database](#rqsrs-006rbacrevokeprivilegecreatedatabase) - * 5.2.8.130 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Dictionary](#rqsrs-006rbacrevokeprivilegecreatedictionary) - * 5.2.8.131 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.Table](#rqsrs-006rbacrevokeprivilegecreatetable) - * 5.2.8.132 [RQ.SRS-006.RBAC.Revoke.Privilege.Create.TemporaryTable](#rqsrs-006rbacrevokeprivilegecreatetemporarytable) - * 5.2.8.133 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop](#rqsrs-006rbacrevokeprivilegedrop) - * 5.2.8.134 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Database](#rqsrs-006rbacrevokeprivilegedropdatabase) - * 5.2.8.135 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Dictionary](#rqsrs-006rbacrevokeprivilegedropdictionary) - * 5.2.8.136 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Table](#rqsrs-006rbacrevokeprivilegedroptable) - * 5.2.8.137 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop.TemporaryTable](#rqsrs-006rbacrevokeprivilegedroptemporarytable) - * 5.2.8.138 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate](#rqsrs-006rbacrevokeprivilegetruncate) - * 5.2.8.139 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize](#rqsrs-006rbacrevokeprivilegeoptimize) - * 5.2.8.140 [RQ.SRS-006.RBAC.Revoke.Privilege.Show](#rqsrs-006rbacrevokeprivilegeshow) - * 5.2.8.141 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery](#rqsrs-006rbacrevokeprivilegekillquery) - * 5.2.8.142 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement](#rqsrs-006rbacrevokeprivilegeaccessmanagement) - * 5.2.8.143 [RQ.SRS-006.RBAC.Revoke.Privilege.System](#rqsrs-006rbacrevokeprivilegesystem) - * 5.2.8.144 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection](#rqsrs-006rbacrevokeprivilegeintrospection) - * 5.2.8.145 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources](#rqsrs-006rbacrevokeprivilegesources) - * 5.2.8.146 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet](#rqsrs-006rbacrevokeprivilegedictget) - * 5.2.8.147 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns](#rqsrs-006rbacrevokeprivilegeprivelegecolumns) - * 5.2.8.148 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple](#rqsrs-006rbacrevokeprivilegemultiple) - * 5.2.8.149 [RQ.SRS-006.RBAC.Revoke.Privilege.All](#rqsrs-006rbacrevokeprivilegeall) - * 5.2.8.150 [RQ.SRS-006.RBAC.Revoke.Privilege.None](#rqsrs-006rbacrevokeprivilegenone) - * 5.2.8.151 [RQ.SRS-006.RBAC.Revoke.Privilege.On](#rqsrs-006rbacrevokeprivilegeon) - * 5.2.8.152 [RQ.SRS-006.RBAC.Revoke.Privilege.From](#rqsrs-006rbacrevokeprivilegefrom) - * 5.2.8.153 [RQ.SRS-006.RBAC.Revoke.Privilege.Syntax](#rqsrs-006rbacrevokeprivilegesyntax) - * 5.2.8.154 [RQ.SRS-006.RBAC.PartialRevoke.Syntax](#rqsrs-006rbacpartialrevokesyntax) - * 5.2.8.155 [RQ.SRS-006.RBAC.Grant.Role](#rqsrs-006rbacgrantrole) - * 5.2.8.156 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser](#rqsrs-006rbacgrantrolecurrentuser) - * 5.2.8.157 [RQ.SRS-006.RBAC.Grant.Role.AdminOption](#rqsrs-006rbacgrantroleadminoption) - * 5.2.8.158 [RQ.SRS-006.RBAC.Grant.Role.OnCluster](#rqsrs-006rbacgrantroleoncluster) - * 5.2.8.159 [RQ.SRS-006.RBAC.Grant.Role.Syntax](#rqsrs-006rbacgrantrolesyntax) - * 5.2.8.160 [RQ.SRS-006.RBAC.Revoke.Role](#rqsrs-006rbacrevokerole) - * 5.2.8.161 [RQ.SRS-006.RBAC.Revoke.Role.Keywords](#rqsrs-006rbacrevokerolekeywords) - * 5.2.8.162 [RQ.SRS-006.RBAC.Revoke.Role.Cluster](#rqsrs-006rbacrevokerolecluster) - * 5.2.8.163 [RQ.SRS-006.RBAC.Revoke.AdminOption](#rqsrs-006rbacrevokeadminoption) - * 5.2.8.164 [RQ.SRS-006.RBAC.Revoke.Role.Syntax](#rqsrs-006rbacrevokerolesyntax) - * 5.2.8.165 [RQ.SRS-006.RBAC.Show.Grants](#rqsrs-006rbacshowgrants) - * 5.2.8.166 [RQ.SRS-006.RBAC.Show.Grants.For](#rqsrs-006rbacshowgrantsfor) - * 5.2.8.167 [RQ.SRS-006.RBAC.Show.Grants.Syntax](#rqsrs-006rbacshowgrantssyntax) - * 5.2.8.168 [RQ.SRS-006.RBAC.SettingsProfile.Create](#rqsrs-006rbacsettingsprofilecreate) - * 5.2.8.169 [RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists](#rqsrs-006rbacsettingsprofilecreateifnotexists) - * 5.2.8.170 [RQ.SRS-006.RBAC.SettingsProfile.Create.Replace](#rqsrs-006rbacsettingsprofilecreatereplace) - * 5.2.8.171 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables](#rqsrs-006rbacsettingsprofilecreatevariables) - * 5.2.8.172 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value](#rqsrs-006rbacsettingsprofilecreatevariablesvalue) - * 5.2.8.173 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints](#rqsrs-006rbacsettingsprofilecreatevariablesconstraints) - * 5.2.8.174 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment](#rqsrs-006rbacsettingsprofilecreateassignment) - * 5.2.8.175 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None](#rqsrs-006rbacsettingsprofilecreateassignmentnone) - * 5.2.8.176 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All](#rqsrs-006rbacsettingsprofilecreateassignmentall) - * 5.2.8.177 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilecreateassignmentallexcept) - * 5.2.8.178 [RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit](#rqsrs-006rbacsettingsprofilecreateinherit) - * 5.2.8.179 [RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster](#rqsrs-006rbacsettingsprofilecreateoncluster) - * 5.2.8.180 [RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax](#rqsrs-006rbacsettingsprofilecreatesyntax) - * 5.2.8.181 [RQ.SRS-006.RBAC.SettingsProfile.Alter](#rqsrs-006rbacsettingsprofilealter) - * 5.2.8.182 [RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists](#rqsrs-006rbacsettingsprofilealterifexists) - * 5.2.8.183 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename](#rqsrs-006rbacsettingsprofilealterrename) - * 5.2.8.184 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables](#rqsrs-006rbacsettingsprofilealtervariables) - * 5.2.8.185 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value](#rqsrs-006rbacsettingsprofilealtervariablesvalue) - * 5.2.8.186 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints](#rqsrs-006rbacsettingsprofilealtervariablesconstraints) - * 5.2.8.187 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment](#rqsrs-006rbacsettingsprofilealterassignment) - * 5.2.8.188 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None](#rqsrs-006rbacsettingsprofilealterassignmentnone) - * 5.2.8.189 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All](#rqsrs-006rbacsettingsprofilealterassignmentall) - * 5.2.8.190 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilealterassignmentallexcept) - * 5.2.8.191 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit](#rqsrs-006rbacsettingsprofilealterassignmentinherit) - * 5.2.8.192 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster](#rqsrs-006rbacsettingsprofilealterassignmentoncluster) - * 5.2.8.193 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax](#rqsrs-006rbacsettingsprofilealtersyntax) - * 5.2.8.194 [RQ.SRS-006.RBAC.SettingsProfile.Drop](#rqsrs-006rbacsettingsprofiledrop) - * 5.2.8.195 [RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists](#rqsrs-006rbacsettingsprofiledropifexists) - * 5.2.8.196 [RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster](#rqsrs-006rbacsettingsprofiledroponcluster) - * 5.2.8.197 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax](#rqsrs-006rbacsettingsprofiledropsyntax) - * 5.2.8.198 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile](#rqsrs-006rbacsettingsprofileshowcreatesettingsprofile) - * 5.2.8.199 [RQ.SRS-006.RBAC.Quota.Create](#rqsrs-006rbacquotacreate) - * 5.2.8.200 [RQ.SRS-006.RBAC.Quota.Create.IfNotExists](#rqsrs-006rbacquotacreateifnotexists) - * 5.2.8.201 [RQ.SRS-006.RBAC.Quota.Create.Replace](#rqsrs-006rbacquotacreatereplace) - * 5.2.8.202 [RQ.SRS-006.RBAC.Quota.Create.Cluster](#rqsrs-006rbacquotacreatecluster) - * 5.2.8.203 [RQ.SRS-006.RBAC.Quota.Create.Interval](#rqsrs-006rbacquotacreateinterval) - * 5.2.8.204 [RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized](#rqsrs-006rbacquotacreateintervalrandomized) - * 5.2.8.205 [RQ.SRS-006.RBAC.Quota.Create.Queries](#rqsrs-006rbacquotacreatequeries) - * 5.2.8.206 [RQ.SRS-006.RBAC.Quota.Create.Errors](#rqsrs-006rbacquotacreateerrors) - * 5.2.8.207 [RQ.SRS-006.RBAC.Quota.Create.ResultRows](#rqsrs-006rbacquotacreateresultrows) - * 5.2.8.208 [RQ.SRS-006.RBAC.Quota.Create.ReadRows](#rqsrs-006rbacquotacreatereadrows) - * 5.2.8.209 [RQ.SRS-006.RBAC.Quota.Create.ResultBytes](#rqsrs-006rbacquotacreateresultbytes) - * 5.2.8.210 [RQ.SRS-006.RBAC.Quota.Create.ReadBytes](#rqsrs-006rbacquotacreatereadbytes) - * 5.2.8.211 [RQ.SRS-006.RBAC.Quota.Create.ExecutionTime](#rqsrs-006rbacquotacreateexecutiontime) - * 5.2.8.212 [RQ.SRS-006.RBAC.Quota.Create.NoLimits](#rqsrs-006rbacquotacreatenolimits) - * 5.2.8.213 [RQ.SRS-006.RBAC.Quota.Create.TrackingOnly](#rqsrs-006rbacquotacreatetrackingonly) - * 5.2.8.214 [RQ.SRS-006.RBAC.Quota.Create.KeyedBy](#rqsrs-006rbacquotacreatekeyedby) - * 5.2.8.215 [RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions](#rqsrs-006rbacquotacreatekeyedbyoptions) - * 5.2.8.216 [RQ.SRS-006.RBAC.Quota.Create.Assignment](#rqsrs-006rbacquotacreateassignment) - * 5.2.8.217 [RQ.SRS-006.RBAC.Quota.Create.Assignment.None](#rqsrs-006rbacquotacreateassignmentnone) - * 5.2.8.218 [RQ.SRS-006.RBAC.Quota.Create.Assignment.All](#rqsrs-006rbacquotacreateassignmentall) - * 5.2.8.219 [RQ.SRS-006.RBAC.Quota.Create.Assignment.Except](#rqsrs-006rbacquotacreateassignmentexcept) - * 5.2.8.220 [RQ.SRS-006.RBAC.Quota.Create.Syntax](#rqsrs-006rbacquotacreatesyntax) - * 5.2.8.221 [RQ.SRS-006.RBAC.Quota.Alter](#rqsrs-006rbacquotaalter) - * 5.2.8.222 [RQ.SRS-006.RBAC.Quota.Alter.IfExists](#rqsrs-006rbacquotaalterifexists) - * 5.2.8.223 [RQ.SRS-006.RBAC.Quota.Alter.Rename](#rqsrs-006rbacquotaalterrename) - * 5.2.8.224 [RQ.SRS-006.RBAC.Quota.Alter.Cluster](#rqsrs-006rbacquotaaltercluster) - * 5.2.8.225 [RQ.SRS-006.RBAC.Quota.Alter.Interval](#rqsrs-006rbacquotaalterinterval) - * 5.2.8.226 [RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized](#rqsrs-006rbacquotaalterintervalrandomized) - * 5.2.8.227 [RQ.SRS-006.RBAC.Quota.Alter.Queries](#rqsrs-006rbacquotaalterqueries) - * 5.2.8.228 [RQ.SRS-006.RBAC.Quota.Alter.Errors](#rqsrs-006rbacquotaaltererrors) - * 5.2.8.229 [RQ.SRS-006.RBAC.Quota.Alter.ResultRows](#rqsrs-006rbacquotaalterresultrows) - * 5.2.8.230 [RQ.SRS-006.RBAC.Quota.Alter.ReadRows](#rqsrs-006rbacquotaalterreadrows) - * 5.2.8.231 [RQ.SRS-006.RBAC.Quota.ALter.ResultBytes](#rqsrs-006rbacquotaalterresultbytes) - * 5.2.8.232 [RQ.SRS-006.RBAC.Quota.Alter.ReadBytes](#rqsrs-006rbacquotaalterreadbytes) - * 5.2.8.233 [RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime](#rqsrs-006rbacquotaalterexecutiontime) - * 5.2.8.234 [RQ.SRS-006.RBAC.Quota.Alter.NoLimits](#rqsrs-006rbacquotaalternolimits) - * 5.2.8.235 [RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly](#rqsrs-006rbacquotaaltertrackingonly) - * 5.2.8.236 [RQ.SRS-006.RBAC.Quota.Alter.KeyedBy](#rqsrs-006rbacquotaalterkeyedby) - * 5.2.8.237 [RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions](#rqsrs-006rbacquotaalterkeyedbyoptions) - * 5.2.8.238 [RQ.SRS-006.RBAC.Quota.Alter.Assignment](#rqsrs-006rbacquotaalterassignment) - * 5.2.8.239 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.None](#rqsrs-006rbacquotaalterassignmentnone) - * 5.2.8.240 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.All](#rqsrs-006rbacquotaalterassignmentall) - * 5.2.8.241 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except](#rqsrs-006rbacquotaalterassignmentexcept) - * 5.2.8.242 [RQ.SRS-006.RBAC.Quota.Alter.Syntax](#rqsrs-006rbacquotaaltersyntax) - * 5.2.8.243 [RQ.SRS-006.RBAC.Quota.Drop](#rqsrs-006rbacquotadrop) - * 5.2.8.244 [RQ.SRS-006.RBAC.Quota.Drop.IfExists](#rqsrs-006rbacquotadropifexists) - * 5.2.8.245 [RQ.SRS-006.RBAC.Quota.Drop.Cluster](#rqsrs-006rbacquotadropcluster) - * 5.2.8.246 [RQ.SRS-006.RBAC.Quota.Drop.Syntax](#rqsrs-006rbacquotadropsyntax) - * 5.2.8.247 [RQ.SRS-006.RBAC.Quota.ShowQuotas](#rqsrs-006rbacquotashowquotas) - * 5.2.8.248 [RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile](#rqsrs-006rbacquotashowquotasintooutfile) - * 5.2.8.249 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Format](#rqsrs-006rbacquotashowquotasformat) - * 5.2.8.250 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings](#rqsrs-006rbacquotashowquotassettings) - * 5.2.8.251 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax](#rqsrs-006rbacquotashowquotassyntax) - * 5.2.8.252 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name](#rqsrs-006rbacquotashowcreatequotaname) - * 5.2.8.253 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current](#rqsrs-006rbacquotashowcreatequotacurrent) - * 5.2.8.254 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax](#rqsrs-006rbacquotashowcreatequotasyntax) - * 5.2.8.255 [RQ.SRS-006.RBAC.RowPolicy.Create](#rqsrs-006rbacrowpolicycreate) - * 5.2.8.256 [RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists](#rqsrs-006rbacrowpolicycreateifnotexists) - * 5.2.8.257 [RQ.SRS-006.RBAC.RowPolicy.Create.Replace](#rqsrs-006rbacrowpolicycreatereplace) - * 5.2.8.258 [RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster](#rqsrs-006rbacrowpolicycreateoncluster) - * 5.2.8.259 [RQ.SRS-006.RBAC.RowPolicy.Create.On](#rqsrs-006rbacrowpolicycreateon) - * 5.2.8.260 [RQ.SRS-006.RBAC.RowPolicy.Create.Access](#rqsrs-006rbacrowpolicycreateaccess) - * 5.2.8.261 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive](#rqsrs-006rbacrowpolicycreateaccesspermissive) - * 5.2.8.262 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive](#rqsrs-006rbacrowpolicycreateaccessrestrictive) - * 5.2.8.263 [RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect](#rqsrs-006rbacrowpolicycreateforselect) - * 5.2.8.264 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition](#rqsrs-006rbacrowpolicycreatecondition) - * 5.2.8.265 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment](#rqsrs-006rbacrowpolicycreateassignment) - * 5.2.8.266 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None](#rqsrs-006rbacrowpolicycreateassignmentnone) - * 5.2.8.267 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All](#rqsrs-006rbacrowpolicycreateassignmentall) - * 5.2.8.268 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept](#rqsrs-006rbacrowpolicycreateassignmentallexcept) - * 5.2.8.269 [RQ.SRS-006.RBAC.RowPolicy.Create.Syntax](#rqsrs-006rbacrowpolicycreatesyntax) - * 5.2.8.270 [RQ.SRS-006.RBAC.RowPolicy.Alter](#rqsrs-006rbacrowpolicyalter) - * 5.2.8.271 [RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists](#rqsrs-006rbacrowpolicyalterifexists) - * 5.2.8.272 [RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect](#rqsrs-006rbacrowpolicyalterforselect) - * 5.2.8.273 [RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster](#rqsrs-006rbacrowpolicyalteroncluster) - * 5.2.8.274 [RQ.SRS-006.RBAC.RowPolicy.Alter.On](#rqsrs-006rbacrowpolicyalteron) - * 5.2.8.275 [RQ.SRS-006.RBAC.RowPolicy.Alter.Rename](#rqsrs-006rbacrowpolicyalterrename) - * 5.2.8.276 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access](#rqsrs-006rbacrowpolicyalteraccess) - * 5.2.8.277 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive](#rqsrs-006rbacrowpolicyalteraccesspermissive) - * 5.2.8.278 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive](#rqsrs-006rbacrowpolicyalteraccessrestrictive) - * 5.2.8.279 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition](#rqsrs-006rbacrowpolicyaltercondition) - * 5.2.8.280 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None](#rqsrs-006rbacrowpolicyalterconditionnone) - * 5.2.8.281 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment](#rqsrs-006rbacrowpolicyalterassignment) - * 5.2.8.282 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None](#rqsrs-006rbacrowpolicyalterassignmentnone) - * 5.2.8.283 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All](#rqsrs-006rbacrowpolicyalterassignmentall) - * 5.2.8.284 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept](#rqsrs-006rbacrowpolicyalterassignmentallexcept) - * 5.2.8.285 [RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax](#rqsrs-006rbacrowpolicyaltersyntax) - * 5.2.8.286 [RQ.SRS-006.RBAC.RowPolicy.Drop](#rqsrs-006rbacrowpolicydrop) - * 5.2.8.287 [RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists](#rqsrs-006rbacrowpolicydropifexists) - * 5.2.8.288 [RQ.SRS-006.RBAC.RowPolicy.Drop.On](#rqsrs-006rbacrowpolicydropon) - * 5.2.8.289 [RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster](#rqsrs-006rbacrowpolicydroponcluster) - * 5.2.8.290 [RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax](#rqsrs-006rbacrowpolicydropsyntax) - * 5.2.8.291 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy](#rqsrs-006rbacrowpolicyshowcreaterowpolicy) - * 5.2.8.292 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On](#rqsrs-006rbacrowpolicyshowcreaterowpolicyon) - * 5.2.8.293 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax](#rqsrs-006rbacrowpolicyshowcreaterowpolicysyntax) - * 5.2.8.294 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies](#rqsrs-006rbacrowpolicyshowrowpolicies) - * 5.2.8.295 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On](#rqsrs-006rbacrowpolicyshowrowpolicieson) - * 5.2.8.296 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) + * 5.2.8.98 [RQ.SRS-006.RBAC.Grant.Privilege.Drop](#rqsrs-006rbacgrantprivilegedrop) + * 5.2.8.99 [RQ.SRS-006.RBAC.Grant.Privilege.Truncate](#rqsrs-006rbacgrantprivilegetruncate) + * 5.2.8.100 [RQ.SRS-006.RBAC.Grant.Privilege.Optimize](#rqsrs-006rbacgrantprivilegeoptimize) + * 5.2.8.101 [RQ.SRS-006.RBAC.Grant.Privilege.Show](#rqsrs-006rbacgrantprivilegeshow) + * 5.2.8.102 [RQ.SRS-006.RBAC.Grant.Privilege.KillQuery](#rqsrs-006rbacgrantprivilegekillquery) + * 5.2.8.103 [RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement](#rqsrs-006rbacgrantprivilegeaccessmanagement) + * 5.2.8.104 [RQ.SRS-006.RBAC.Grant.Privilege.System](#rqsrs-006rbacgrantprivilegesystem) + * 5.2.8.105 [RQ.SRS-006.RBAC.Grant.Privilege.Introspection](#rqsrs-006rbacgrantprivilegeintrospection) + * 5.2.8.106 [RQ.SRS-006.RBAC.Grant.Privilege.Sources](#rqsrs-006rbacgrantprivilegesources) + * 5.2.8.107 [RQ.SRS-006.RBAC.Grant.Privilege.DictGet](#rqsrs-006rbacgrantprivilegedictget) + * 5.2.8.108 [RQ.SRS-006.RBAC.Grant.Privilege.None](#rqsrs-006rbacgrantprivilegenone) + * 5.2.8.109 [RQ.SRS-006.RBAC.Grant.Privilege.All](#rqsrs-006rbacgrantprivilegeall) + * 5.2.8.110 [RQ.SRS-006.RBAC.Grant.Privilege.GrantOption](#rqsrs-006rbacgrantprivilegegrantoption) + * 5.2.8.111 [RQ.SRS-006.RBAC.Grant.Privilege.On](#rqsrs-006rbacgrantprivilegeon) + * 5.2.8.112 [RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns](#rqsrs-006rbacgrantprivilegeprivilegecolumns) + * 5.2.8.113 [RQ.SRS-006.RBAC.Grant.Privilege.OnCluster](#rqsrs-006rbacgrantprivilegeoncluster) + * 5.2.8.114 [RQ.SRS-006.RBAC.Grant.Privilege.Syntax](#rqsrs-006rbacgrantprivilegesyntax) + * 5.2.8.115 [RQ.SRS-006.RBAC.Revoke.Privilege.Cluster](#rqsrs-006rbacrevokeprivilegecluster) + * 5.2.8.116 [RQ.SRS-006.RBAC.Revoke.Privilege.Any](#rqsrs-006rbacrevokeprivilegeany) + * 5.2.8.117 [RQ.SRS-006.RBAC.Revoke.Privilege.Select](#rqsrs-006rbacrevokeprivilegeselect) + * 5.2.8.118 [RQ.SRS-006.RBAC.Revoke.Privilege.Insert](#rqsrs-006rbacrevokeprivilegeinsert) + * 5.2.8.119 [RQ.SRS-006.RBAC.Revoke.Privilege.Alter](#rqsrs-006rbacrevokeprivilegealter) + * 5.2.8.120 [RQ.SRS-006.RBAC.Revoke.Privilege.Create](#rqsrs-006rbacrevokeprivilegecreate) + * 5.2.8.121 [RQ.SRS-006.RBAC.Revoke.Privilege.Drop](#rqsrs-006rbacrevokeprivilegedrop) + * 5.2.8.122 [RQ.SRS-006.RBAC.Revoke.Privilege.Truncate](#rqsrs-006rbacrevokeprivilegetruncate) + * 5.2.8.123 [RQ.SRS-006.RBAC.Revoke.Privilege.Optimize](#rqsrs-006rbacrevokeprivilegeoptimize) + * 5.2.8.124 [RQ.SRS-006.RBAC.Revoke.Privilege.Show](#rqsrs-006rbacrevokeprivilegeshow) + * 5.2.8.125 [RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery](#rqsrs-006rbacrevokeprivilegekillquery) + * 5.2.8.126 [RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement](#rqsrs-006rbacrevokeprivilegeaccessmanagement) + * 5.2.8.127 [RQ.SRS-006.RBAC.Revoke.Privilege.System](#rqsrs-006rbacrevokeprivilegesystem) + * 5.2.8.128 [RQ.SRS-006.RBAC.Revoke.Privilege.Introspection](#rqsrs-006rbacrevokeprivilegeintrospection) + * 5.2.8.129 [RQ.SRS-006.RBAC.Revoke.Privilege.Sources](#rqsrs-006rbacrevokeprivilegesources) + * 5.2.8.130 [RQ.SRS-006.RBAC.Revoke.Privilege.DictGet](#rqsrs-006rbacrevokeprivilegedictget) + * 5.2.8.131 [RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns](#rqsrs-006rbacrevokeprivilegeprivelegecolumns) + * 5.2.8.132 [RQ.SRS-006.RBAC.Revoke.Privilege.Multiple](#rqsrs-006rbacrevokeprivilegemultiple) + * 5.2.8.133 [RQ.SRS-006.RBAC.Revoke.Privilege.All](#rqsrs-006rbacrevokeprivilegeall) + * 5.2.8.134 [RQ.SRS-006.RBAC.Revoke.Privilege.None](#rqsrs-006rbacrevokeprivilegenone) + * 5.2.8.135 [RQ.SRS-006.RBAC.Revoke.Privilege.On](#rqsrs-006rbacrevokeprivilegeon) + * 5.2.8.136 [RQ.SRS-006.RBAC.Revoke.Privilege.From](#rqsrs-006rbacrevokeprivilegefrom) + * 5.2.8.137 [RQ.SRS-006.RBAC.Revoke.Privilege.Syntax](#rqsrs-006rbacrevokeprivilegesyntax) + * 5.2.8.138 [RQ.SRS-006.RBAC.PartialRevoke.Syntax](#rqsrs-006rbacpartialrevokesyntax) + * 5.2.8.139 [RQ.SRS-006.RBAC.Grant.Role](#rqsrs-006rbacgrantrole) + * 5.2.8.140 [RQ.SRS-006.RBAC.Grant.Role.CurrentUser](#rqsrs-006rbacgrantrolecurrentuser) + * 5.2.8.141 [RQ.SRS-006.RBAC.Grant.Role.AdminOption](#rqsrs-006rbacgrantroleadminoption) + * 5.2.8.142 [RQ.SRS-006.RBAC.Grant.Role.OnCluster](#rqsrs-006rbacgrantroleoncluster) + * 5.2.8.143 [RQ.SRS-006.RBAC.Grant.Role.Syntax](#rqsrs-006rbacgrantrolesyntax) + * 5.2.8.144 [RQ.SRS-006.RBAC.Revoke.Role](#rqsrs-006rbacrevokerole) + * 5.2.8.145 [RQ.SRS-006.RBAC.Revoke.Role.Keywords](#rqsrs-006rbacrevokerolekeywords) + * 5.2.8.146 [RQ.SRS-006.RBAC.Revoke.Role.Cluster](#rqsrs-006rbacrevokerolecluster) + * 5.2.8.147 [RQ.SRS-006.RBAC.Revoke.AdminOption](#rqsrs-006rbacrevokeadminoption) + * 5.2.8.148 [RQ.SRS-006.RBAC.Revoke.Role.Syntax](#rqsrs-006rbacrevokerolesyntax) + * 5.2.8.149 [RQ.SRS-006.RBAC.Show.Grants](#rqsrs-006rbacshowgrants) + * 5.2.8.150 [RQ.SRS-006.RBAC.Show.Grants.For](#rqsrs-006rbacshowgrantsfor) + * 5.2.8.151 [RQ.SRS-006.RBAC.Show.Grants.Syntax](#rqsrs-006rbacshowgrantssyntax) + * 5.2.8.152 [RQ.SRS-006.RBAC.SettingsProfile.Create](#rqsrs-006rbacsettingsprofilecreate) + * 5.2.8.153 [RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists](#rqsrs-006rbacsettingsprofilecreateifnotexists) + * 5.2.8.154 [RQ.SRS-006.RBAC.SettingsProfile.Create.Replace](#rqsrs-006rbacsettingsprofilecreatereplace) + * 5.2.8.155 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables](#rqsrs-006rbacsettingsprofilecreatevariables) + * 5.2.8.156 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value](#rqsrs-006rbacsettingsprofilecreatevariablesvalue) + * 5.2.8.157 [RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints](#rqsrs-006rbacsettingsprofilecreatevariablesconstraints) + * 5.2.8.158 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment](#rqsrs-006rbacsettingsprofilecreateassignment) + * 5.2.8.159 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None](#rqsrs-006rbacsettingsprofilecreateassignmentnone) + * 5.2.8.160 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All](#rqsrs-006rbacsettingsprofilecreateassignmentall) + * 5.2.8.161 [RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilecreateassignmentallexcept) + * 5.2.8.162 [RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit](#rqsrs-006rbacsettingsprofilecreateinherit) + * 5.2.8.163 [RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster](#rqsrs-006rbacsettingsprofilecreateoncluster) + * 5.2.8.164 [RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax](#rqsrs-006rbacsettingsprofilecreatesyntax) + * 5.2.8.165 [RQ.SRS-006.RBAC.SettingsProfile.Alter](#rqsrs-006rbacsettingsprofilealter) + * 5.2.8.166 [RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists](#rqsrs-006rbacsettingsprofilealterifexists) + * 5.2.8.167 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename](#rqsrs-006rbacsettingsprofilealterrename) + * 5.2.8.168 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables](#rqsrs-006rbacsettingsprofilealtervariables) + * 5.2.8.169 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value](#rqsrs-006rbacsettingsprofilealtervariablesvalue) + * 5.2.8.170 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints](#rqsrs-006rbacsettingsprofilealtervariablesconstraints) + * 5.2.8.171 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment](#rqsrs-006rbacsettingsprofilealterassignment) + * 5.2.8.172 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None](#rqsrs-006rbacsettingsprofilealterassignmentnone) + * 5.2.8.173 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All](#rqsrs-006rbacsettingsprofilealterassignmentall) + * 5.2.8.174 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept](#rqsrs-006rbacsettingsprofilealterassignmentallexcept) + * 5.2.8.175 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit](#rqsrs-006rbacsettingsprofilealterassignmentinherit) + * 5.2.8.176 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster](#rqsrs-006rbacsettingsprofilealterassignmentoncluster) + * 5.2.8.177 [RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax](#rqsrs-006rbacsettingsprofilealtersyntax) + * 5.2.8.178 [RQ.SRS-006.RBAC.SettingsProfile.Drop](#rqsrs-006rbacsettingsprofiledrop) + * 5.2.8.179 [RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists](#rqsrs-006rbacsettingsprofiledropifexists) + * 5.2.8.180 [RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster](#rqsrs-006rbacsettingsprofiledroponcluster) + * 5.2.8.181 [RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax](#rqsrs-006rbacsettingsprofiledropsyntax) + * 5.2.8.182 [RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile](#rqsrs-006rbacsettingsprofileshowcreatesettingsprofile) + * 5.2.8.183 [RQ.SRS-006.RBAC.Quota.Create](#rqsrs-006rbacquotacreate) + * 5.2.8.184 [RQ.SRS-006.RBAC.Quota.Create.IfNotExists](#rqsrs-006rbacquotacreateifnotexists) + * 5.2.8.185 [RQ.SRS-006.RBAC.Quota.Create.Replace](#rqsrs-006rbacquotacreatereplace) + * 5.2.8.186 [RQ.SRS-006.RBAC.Quota.Create.Cluster](#rqsrs-006rbacquotacreatecluster) + * 5.2.8.187 [RQ.SRS-006.RBAC.Quota.Create.Interval](#rqsrs-006rbacquotacreateinterval) + * 5.2.8.188 [RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized](#rqsrs-006rbacquotacreateintervalrandomized) + * 5.2.8.189 [RQ.SRS-006.RBAC.Quota.Create.Queries](#rqsrs-006rbacquotacreatequeries) + * 5.2.8.190 [RQ.SRS-006.RBAC.Quota.Create.Errors](#rqsrs-006rbacquotacreateerrors) + * 5.2.8.191 [RQ.SRS-006.RBAC.Quota.Create.ResultRows](#rqsrs-006rbacquotacreateresultrows) + * 5.2.8.192 [RQ.SRS-006.RBAC.Quota.Create.ReadRows](#rqsrs-006rbacquotacreatereadrows) + * 5.2.8.193 [RQ.SRS-006.RBAC.Quota.Create.ResultBytes](#rqsrs-006rbacquotacreateresultbytes) + * 5.2.8.194 [RQ.SRS-006.RBAC.Quota.Create.ReadBytes](#rqsrs-006rbacquotacreatereadbytes) + * 5.2.8.195 [RQ.SRS-006.RBAC.Quota.Create.ExecutionTime](#rqsrs-006rbacquotacreateexecutiontime) + * 5.2.8.196 [RQ.SRS-006.RBAC.Quota.Create.NoLimits](#rqsrs-006rbacquotacreatenolimits) + * 5.2.8.197 [RQ.SRS-006.RBAC.Quota.Create.TrackingOnly](#rqsrs-006rbacquotacreatetrackingonly) + * 5.2.8.198 [RQ.SRS-006.RBAC.Quota.Create.KeyedBy](#rqsrs-006rbacquotacreatekeyedby) + * 5.2.8.199 [RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions](#rqsrs-006rbacquotacreatekeyedbyoptions) + * 5.2.8.200 [RQ.SRS-006.RBAC.Quota.Create.Assignment](#rqsrs-006rbacquotacreateassignment) + * 5.2.8.201 [RQ.SRS-006.RBAC.Quota.Create.Assignment.None](#rqsrs-006rbacquotacreateassignmentnone) + * 5.2.8.202 [RQ.SRS-006.RBAC.Quota.Create.Assignment.All](#rqsrs-006rbacquotacreateassignmentall) + * 5.2.8.203 [RQ.SRS-006.RBAC.Quota.Create.Assignment.Except](#rqsrs-006rbacquotacreateassignmentexcept) + * 5.2.8.204 [RQ.SRS-006.RBAC.Quota.Create.Syntax](#rqsrs-006rbacquotacreatesyntax) + * 5.2.8.205 [RQ.SRS-006.RBAC.Quota.Alter](#rqsrs-006rbacquotaalter) + * 5.2.8.206 [RQ.SRS-006.RBAC.Quota.Alter.IfExists](#rqsrs-006rbacquotaalterifexists) + * 5.2.8.207 [RQ.SRS-006.RBAC.Quota.Alter.Rename](#rqsrs-006rbacquotaalterrename) + * 5.2.8.208 [RQ.SRS-006.RBAC.Quota.Alter.Cluster](#rqsrs-006rbacquotaaltercluster) + * 5.2.8.209 [RQ.SRS-006.RBAC.Quota.Alter.Interval](#rqsrs-006rbacquotaalterinterval) + * 5.2.8.210 [RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized](#rqsrs-006rbacquotaalterintervalrandomized) + * 5.2.8.211 [RQ.SRS-006.RBAC.Quota.Alter.Queries](#rqsrs-006rbacquotaalterqueries) + * 5.2.8.212 [RQ.SRS-006.RBAC.Quota.Alter.Errors](#rqsrs-006rbacquotaaltererrors) + * 5.2.8.213 [RQ.SRS-006.RBAC.Quota.Alter.ResultRows](#rqsrs-006rbacquotaalterresultrows) + * 5.2.8.214 [RQ.SRS-006.RBAC.Quota.Alter.ReadRows](#rqsrs-006rbacquotaalterreadrows) + * 5.2.8.215 [RQ.SRS-006.RBAC.Quota.ALter.ResultBytes](#rqsrs-006rbacquotaalterresultbytes) + * 5.2.8.216 [RQ.SRS-006.RBAC.Quota.Alter.ReadBytes](#rqsrs-006rbacquotaalterreadbytes) + * 5.2.8.217 [RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime](#rqsrs-006rbacquotaalterexecutiontime) + * 5.2.8.218 [RQ.SRS-006.RBAC.Quota.Alter.NoLimits](#rqsrs-006rbacquotaalternolimits) + * 5.2.8.219 [RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly](#rqsrs-006rbacquotaaltertrackingonly) + * 5.2.8.220 [RQ.SRS-006.RBAC.Quota.Alter.KeyedBy](#rqsrs-006rbacquotaalterkeyedby) + * 5.2.8.221 [RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions](#rqsrs-006rbacquotaalterkeyedbyoptions) + * 5.2.8.222 [RQ.SRS-006.RBAC.Quota.Alter.Assignment](#rqsrs-006rbacquotaalterassignment) + * 5.2.8.223 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.None](#rqsrs-006rbacquotaalterassignmentnone) + * 5.2.8.224 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.All](#rqsrs-006rbacquotaalterassignmentall) + * 5.2.8.225 [RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except](#rqsrs-006rbacquotaalterassignmentexcept) + * 5.2.8.226 [RQ.SRS-006.RBAC.Quota.Alter.Syntax](#rqsrs-006rbacquotaaltersyntax) + * 5.2.8.227 [RQ.SRS-006.RBAC.Quota.Drop](#rqsrs-006rbacquotadrop) + * 5.2.8.228 [RQ.SRS-006.RBAC.Quota.Drop.IfExists](#rqsrs-006rbacquotadropifexists) + * 5.2.8.229 [RQ.SRS-006.RBAC.Quota.Drop.Cluster](#rqsrs-006rbacquotadropcluster) + * 5.2.8.230 [RQ.SRS-006.RBAC.Quota.Drop.Syntax](#rqsrs-006rbacquotadropsyntax) + * 5.2.8.231 [RQ.SRS-006.RBAC.Quota.ShowQuotas](#rqsrs-006rbacquotashowquotas) + * 5.2.8.232 [RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile](#rqsrs-006rbacquotashowquotasintooutfile) + * 5.2.8.233 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Format](#rqsrs-006rbacquotashowquotasformat) + * 5.2.8.234 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings](#rqsrs-006rbacquotashowquotassettings) + * 5.2.8.235 [RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax](#rqsrs-006rbacquotashowquotassyntax) + * 5.2.8.236 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name](#rqsrs-006rbacquotashowcreatequotaname) + * 5.2.8.237 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current](#rqsrs-006rbacquotashowcreatequotacurrent) + * 5.2.8.238 [RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax](#rqsrs-006rbacquotashowcreatequotasyntax) + * 5.2.8.239 [RQ.SRS-006.RBAC.RowPolicy.Create](#rqsrs-006rbacrowpolicycreate) + * 5.2.8.240 [RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists](#rqsrs-006rbacrowpolicycreateifnotexists) + * 5.2.8.241 [RQ.SRS-006.RBAC.RowPolicy.Create.Replace](#rqsrs-006rbacrowpolicycreatereplace) + * 5.2.8.242 [RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster](#rqsrs-006rbacrowpolicycreateoncluster) + * 5.2.8.243 [RQ.SRS-006.RBAC.RowPolicy.Create.On](#rqsrs-006rbacrowpolicycreateon) + * 5.2.8.244 [RQ.SRS-006.RBAC.RowPolicy.Create.Access](#rqsrs-006rbacrowpolicycreateaccess) + * 5.2.8.245 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive](#rqsrs-006rbacrowpolicycreateaccesspermissive) + * 5.2.8.246 [RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive](#rqsrs-006rbacrowpolicycreateaccessrestrictive) + * 5.2.8.247 [RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect](#rqsrs-006rbacrowpolicycreateforselect) + * 5.2.8.248 [RQ.SRS-006.RBAC.RowPolicy.Create.Condition](#rqsrs-006rbacrowpolicycreatecondition) + * 5.2.8.249 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment](#rqsrs-006rbacrowpolicycreateassignment) + * 5.2.8.250 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None](#rqsrs-006rbacrowpolicycreateassignmentnone) + * 5.2.8.251 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All](#rqsrs-006rbacrowpolicycreateassignmentall) + * 5.2.8.252 [RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept](#rqsrs-006rbacrowpolicycreateassignmentallexcept) + * 5.2.8.253 [RQ.SRS-006.RBAC.RowPolicy.Create.Syntax](#rqsrs-006rbacrowpolicycreatesyntax) + * 5.2.8.254 [RQ.SRS-006.RBAC.RowPolicy.Alter](#rqsrs-006rbacrowpolicyalter) + * 5.2.8.255 [RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists](#rqsrs-006rbacrowpolicyalterifexists) + * 5.2.8.256 [RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect](#rqsrs-006rbacrowpolicyalterforselect) + * 5.2.8.257 [RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster](#rqsrs-006rbacrowpolicyalteroncluster) + * 5.2.8.258 [RQ.SRS-006.RBAC.RowPolicy.Alter.On](#rqsrs-006rbacrowpolicyalteron) + * 5.2.8.259 [RQ.SRS-006.RBAC.RowPolicy.Alter.Rename](#rqsrs-006rbacrowpolicyalterrename) + * 5.2.8.260 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access](#rqsrs-006rbacrowpolicyalteraccess) + * 5.2.8.261 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive](#rqsrs-006rbacrowpolicyalteraccesspermissive) + * 5.2.8.262 [RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive](#rqsrs-006rbacrowpolicyalteraccessrestrictive) + * 5.2.8.263 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition](#rqsrs-006rbacrowpolicyaltercondition) + * 5.2.8.264 [RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None](#rqsrs-006rbacrowpolicyalterconditionnone) + * 5.2.8.265 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment](#rqsrs-006rbacrowpolicyalterassignment) + * 5.2.8.266 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None](#rqsrs-006rbacrowpolicyalterassignmentnone) + * 5.2.8.267 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All](#rqsrs-006rbacrowpolicyalterassignmentall) + * 5.2.8.268 [RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept](#rqsrs-006rbacrowpolicyalterassignmentallexcept) + * 5.2.8.269 [RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax](#rqsrs-006rbacrowpolicyaltersyntax) + * 5.2.8.270 [RQ.SRS-006.RBAC.RowPolicy.Drop](#rqsrs-006rbacrowpolicydrop) + * 5.2.8.271 [RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists](#rqsrs-006rbacrowpolicydropifexists) + * 5.2.8.272 [RQ.SRS-006.RBAC.RowPolicy.Drop.On](#rqsrs-006rbacrowpolicydropon) + * 5.2.8.273 [RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster](#rqsrs-006rbacrowpolicydroponcluster) + * 5.2.8.274 [RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax](#rqsrs-006rbacrowpolicydropsyntax) + * 5.2.8.275 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy](#rqsrs-006rbacrowpolicyshowcreaterowpolicy) + * 5.2.8.276 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On](#rqsrs-006rbacrowpolicyshowcreaterowpolicyon) + * 5.2.8.277 [RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax](#rqsrs-006rbacrowpolicyshowcreaterowpolicysyntax) + * 5.2.8.278 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies](#rqsrs-006rbacrowpolicyshowrowpolicies) + * 5.2.8.279 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On](#rqsrs-006rbacrowpolicyshowrowpolicieson) + * 5.2.8.280 [RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax](#rqsrs-006rbacrowpolicyshowrowpoliciessyntax) * 5.2.9 [Table Privileges](#table-privileges) * 5.2.9.1 [RQ.SRS-006.RBAC.Table.PublicTables](#rqsrs-006rbactablepublictables) - * 5.2.9.2 [RQ.SRS-006.RBAC.Table.QueryLog](#rqsrs-006rbactablequerylog) - * 5.2.9.3 [Distributed Tables](#distributed-tables) - * 5.2.9.3.1 [RQ.SRS-006.RBAC.Table.DistributedTable.Create](#rqsrs-006rbactabledistributedtablecreate) - * 5.2.9.3.2 [RQ.SRS-006.RBAC.Table.DistributedTable.Select](#rqsrs-006rbactabledistributedtableselect) - * 5.2.9.3.3 [RQ.SRS-006.RBAC.Table.DistributedTable.Insert](#rqsrs-006rbactabledistributedtableinsert) - * 5.2.9.3.4 [RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables](#rqsrs-006rbactabledistributedtablespecialtables) - * 5.2.9.3.5 [RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser](#rqsrs-006rbactabledistributedtablelocaluser) - * 5.2.9.3.6 [RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbactabledistributedtablesameuserdifferentnodesdifferentprivileges) - * 5.2.10 [Views](#views) - * 5.2.10.1 [View](#view) - * 5.2.10.1.1 [RQ.SRS-006.RBAC.View](#rqsrs-006rbacview) - * 5.2.10.1.2 [RQ.SRS-006.RBAC.View.Create](#rqsrs-006rbacviewcreate) - * 5.2.10.1.3 [RQ.SRS-006.RBAC.View.Select](#rqsrs-006rbacviewselect) - * 5.2.10.1.4 [RQ.SRS-006.RBAC.View.Drop](#rqsrs-006rbacviewdrop) - * 5.2.10.2 [Materialized View](#materialized-view) - * 5.2.10.2.1 [RQ.SRS-006.RBAC.MaterializedView](#rqsrs-006rbacmaterializedview) - * 5.2.10.2.2 [RQ.SRS-006.RBAC.MaterializedView.Create](#rqsrs-006rbacmaterializedviewcreate) - * 5.2.10.2.3 [RQ.SRS-006.RBAC.MaterializedView.Select](#rqsrs-006rbacmaterializedviewselect) - * 5.2.10.2.4 [RQ.SRS-006.RBAC.MaterializedView.Select.TargetTable](#rqsrs-006rbacmaterializedviewselecttargettable) - * 5.2.10.2.5 [RQ.SRS-006.RBAC.MaterializedView.Select.SourceTable](#rqsrs-006rbacmaterializedviewselectsourcetable) - * 5.2.10.2.6 [RQ.SRS-006.RBAC.MaterializedView.Drop](#rqsrs-006rbacmaterializedviewdrop) - * 5.2.10.2.7 [RQ.SRS-006.RBAC.MaterializedView.ModifyQuery](#rqsrs-006rbacmaterializedviewmodifyquery) - * 5.2.10.2.8 [RQ.SRS-006.RBAC.MaterializedView.Insert](#rqsrs-006rbacmaterializedviewinsert) - * 5.2.10.2.9 [RQ.SRS-006.RBAC.MaterializedView.Insert.SourceTable](#rqsrs-006rbacmaterializedviewinsertsourcetable) - * 5.2.10.2.10 [RQ.SRS-006.RBAC.MaterializedView.Insert.TargetTable](#rqsrs-006rbacmaterializedviewinserttargettable) - * 5.2.10.3 [Live View](#live-view) - * 5.2.10.3.1 [RQ.SRS-006.RBAC.LiveView](#rqsrs-006rbacliveview) - * 5.2.10.3.2 [RQ.SRS-006.RBAC.LiveView.Create](#rqsrs-006rbacliveviewcreate) - * 5.2.10.3.3 [RQ.SRS-006.RBAC.LiveView.Select](#rqsrs-006rbacliveviewselect) - * 5.2.10.3.4 [RQ.SRS-006.RBAC.LiveView.Drop](#rqsrs-006rbacliveviewdrop) - * 5.2.10.3.5 [RQ.SRS-006.RBAC.LiveView.Refresh](#rqsrs-006rbacliveviewrefresh) - * 5.2.11 [Privileges](#privileges) - * 5.2.11.1 [RQ.SRS-006.RBAC.Privileges.Usage](#rqsrs-006rbacprivilegesusage) - * 5.2.11.2 [Select](#select) - * 5.2.11.2.1 [RQ.SRS-006.RBAC.Privileges.Select](#rqsrs-006rbacprivilegesselect) - * 5.2.11.2.2 [RQ.SRS-006.RBAC.Privileges.Select.Column](#rqsrs-006rbacprivilegesselectcolumn) - * 5.2.11.2.3 [RQ.SRS-006.RBAC.Privileges.Select.Cluster](#rqsrs-006rbacprivilegesselectcluster) - * 5.2.11.2.4 [RQ.SRS-006.RBAC.Privileges.Select.TableEngines](#rqsrs-006rbacprivilegesselecttableengines) - * 5.2.11.3 [Insert](#insert) - * 5.2.11.3.1 [RQ.SRS-006.RBAC.Privileges.Insert](#rqsrs-006rbacprivilegesinsert) - * 5.2.11.3.2 [RQ.SRS-006.RBAC.Privileges.Insert.Column](#rqsrs-006rbacprivilegesinsertcolumn) - * 5.2.11.3.3 [RQ.SRS-006.RBAC.Privileges.Insert.Cluster](#rqsrs-006rbacprivilegesinsertcluster) - * 5.2.11.3.4 [RQ.SRS-006.RBAC.Privileges.Insert.TableEngines](#rqsrs-006rbacprivilegesinserttableengines) - * 5.2.11.4 [Alter Column](#alter-column) - * 5.2.11.4.1 [RQ.SRS-006.RBAC.Privileges.AlterColumn](#rqsrs-006rbacprivilegesaltercolumn) - * 5.2.11.4.2 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant](#rqsrs-006rbacprivilegesaltercolumngrant) - * 5.2.11.4.3 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke](#rqsrs-006rbacprivilegesaltercolumnrevoke) - * 5.2.11.4.4 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Column](#rqsrs-006rbacprivilegesaltercolumncolumn) - * 5.2.11.4.5 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster](#rqsrs-006rbacprivilegesaltercolumncluster) - * 5.2.11.4.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) - * 5.2.11.5 [Alter Index](#alter-index) - * 5.2.11.5.1 [RQ.SRS-006.RBAC.Privileges.AlterIndex](#rqsrs-006rbacprivilegesalterindex) - * 5.2.11.5.2 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant](#rqsrs-006rbacprivilegesalterindexgrant) - * 5.2.11.5.3 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke](#rqsrs-006rbacprivilegesalterindexrevoke) - * 5.2.11.5.4 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster](#rqsrs-006rbacprivilegesalterindexcluster) - * 5.2.11.5.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) - * 5.2.11.6 [Alter Constraint](#alter-constraint) - * 5.2.11.6.1 [RQ.SRS-006.RBAC.Privileges.AlterConstraint](#rqsrs-006rbacprivilegesalterconstraint) - * 5.2.11.6.2 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant](#rqsrs-006rbacprivilegesalterconstraintgrant) - * 5.2.11.6.3 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke](#rqsrs-006rbacprivilegesalterconstraintrevoke) - * 5.2.11.6.4 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster](#rqsrs-006rbacprivilegesalterconstraintcluster) - * 5.2.11.6.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) - * 5.2.11.7 [Alter TTL](#alter-ttl) - * 5.2.11.7.1 [RQ.SRS-006.RBAC.Privileges.AlterTTL](#rqsrs-006rbacprivilegesalterttl) - * 5.2.11.7.2 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant](#rqsrs-006rbacprivilegesalterttlgrant) - * 5.2.11.7.3 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke](#rqsrs-006rbacprivilegesalterttlrevoke) - * 5.2.11.7.4 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster](#rqsrs-006rbacprivilegesalterttlcluster) - * 5.2.11.7.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) - * 5.2.11.8 [Alter Settings](#alter-settings) - * 5.2.11.8.1 [RQ.SRS-006.RBAC.Privileges.AlterSettings](#rqsrs-006rbacprivilegesaltersettings) - * 5.2.11.8.2 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant](#rqsrs-006rbacprivilegesaltersettingsgrant) - * 5.2.11.8.3 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke](#rqsrs-006rbacprivilegesaltersettingsrevoke) - * 5.2.11.8.4 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster](#rqsrs-006rbacprivilegesaltersettingscluster) - * 5.2.11.8.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) - * 5.2.11.9 [Alter Update](#alter-update) - * 5.2.11.9.1 [RQ.SRS-006.RBAC.Privileges.AlterUpdate](#rqsrs-006rbacprivilegesalterupdate) - * 5.2.11.9.2 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Grant](#rqsrs-006rbacprivilegesalterupdategrant) - * 5.2.11.9.3 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Revoke](#rqsrs-006rbacprivilegesalterupdaterevoke) - * 5.2.11.9.4 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines](#rqsrs-006rbacprivilegesalterupdatetableengines) - * 5.2.11.10 [Alter Delete](#alter-delete) - * 5.2.11.10.1 [RQ.SRS-006.RBAC.Privileges.AlterDelete](#rqsrs-006rbacprivilegesalterdelete) - * 5.2.11.10.2 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Grant](#rqsrs-006rbacprivilegesalterdeletegrant) - * 5.2.11.10.3 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Revoke](#rqsrs-006rbacprivilegesalterdeleterevoke) - * 5.2.11.10.4 [RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines](#rqsrs-006rbacprivilegesalterdeletetableengines) - * 5.2.11.11 [Alter Freeze Partition](#alter-freeze-partition) - * 5.2.11.11.1 [RQ.SRS-006.RBAC.Privileges.AlterFreeze](#rqsrs-006rbacprivilegesalterfreeze) - * 5.2.11.11.2 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Grant](#rqsrs-006rbacprivilegesalterfreezegrant) - * 5.2.11.11.3 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Revoke](#rqsrs-006rbacprivilegesalterfreezerevoke) - * 5.2.11.11.4 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines](#rqsrs-006rbacprivilegesalterfreezetableengines) - * 5.2.11.12 [Alter Fetch Partition](#alter-fetch-partition) - * 5.2.11.12.1 [RQ.SRS-006.RBAC.Privileges.AlterFetch](#rqsrs-006rbacprivilegesalterfetch) - * 5.2.11.12.2 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Grant](#rqsrs-006rbacprivilegesalterfetchgrant) - * 5.2.11.12.3 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Revoke](#rqsrs-006rbacprivilegesalterfetchrevoke) - * 5.2.11.12.4 [RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines](#rqsrs-006rbacprivilegesalterfetchtableengines) - * 5.2.11.13 [Alter Move Partition](#alter-move-partition) - * 5.2.11.13.1 [RQ.SRS-006.RBAC.Privileges.AlterMove](#rqsrs-006rbacprivilegesaltermove) - * 5.2.11.13.2 [RQ.SRS-006.RBAC.Privileges.AlterMove.Grant](#rqsrs-006rbacprivilegesaltermovegrant) - * 5.2.11.13.3 [RQ.SRS-006.RBAC.Privileges.AlterMove.Revoke](#rqsrs-006rbacprivilegesaltermoverevoke) - * 5.2.11.13.4 [RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines](#rqsrs-006rbacprivilegesaltermovetableengines) - * 5.2.11.14 [RQ.SRS-006.RBAC.Privileges.CreateTable](#rqsrs-006rbacprivilegescreatetable) - * 5.2.11.15 [RQ.SRS-006.RBAC.Privileges.CreateDatabase](#rqsrs-006rbacprivilegescreatedatabase) - * 5.2.11.16 [RQ.SRS-006.RBAC.Privileges.CreateDictionary](#rqsrs-006rbacprivilegescreatedictionary) - * 5.2.11.17 [RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable](#rqsrs-006rbacprivilegescreatetemporarytable) - * 5.2.11.18 [RQ.SRS-006.RBAC.Privileges.AttachDatabase](#rqsrs-006rbacprivilegesattachdatabase) - * 5.2.11.19 [RQ.SRS-006.RBAC.Privileges.AttachDictionary](#rqsrs-006rbacprivilegesattachdictionary) - * 5.2.11.20 [RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable](#rqsrs-006rbacprivilegesattachtemporarytable) - * 5.2.11.21 [RQ.SRS-006.RBAC.Privileges.AttachTable](#rqsrs-006rbacprivilegesattachtable) - * 5.2.11.22 [RQ.SRS-006.RBAC.Privileges.DropTable](#rqsrs-006rbacprivilegesdroptable) - * 5.2.11.23 [RQ.SRS-006.RBAC.Privileges.DropDatabase](#rqsrs-006rbacprivilegesdropdatabase) - * 5.2.11.24 [RQ.SRS-006.RBAC.Privileges.DropDictionary](#rqsrs-006rbacprivilegesdropdictionary) - * 5.2.11.25 [RQ.SRS-006.RBAC.Privileges.DetachTable](#rqsrs-006rbacprivilegesdetachtable) - * 5.2.11.26 [RQ.SRS-006.RBAC.Privileges.DetachView](#rqsrs-006rbacprivilegesdetachview) - * 5.2.11.27 [RQ.SRS-006.RBAC.Privileges.DetachDatabase](#rqsrs-006rbacprivilegesdetachdatabase) - * 5.2.11.28 [RQ.SRS-006.RBAC.Privileges.DetachDictionary](#rqsrs-006rbacprivilegesdetachdictionary) - * 5.2.11.29 [RQ.SRS-006.RBAC.Privileges.Truncate](#rqsrs-006rbacprivilegestruncate) - * 5.2.11.30 [RQ.SRS-006.RBAC.Privileges.Optimize](#rqsrs-006rbacprivilegesoptimize) - * 5.2.11.31 [RQ.SRS-006.RBAC.Privileges.KillQuery](#rqsrs-006rbacprivilegeskillquery) - * 5.2.11.32 [RQ.SRS-006.RBAC.Privileges.KillMutation](#rqsrs-006rbacprivilegeskillmutation) - * 5.2.11.33 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterUpdate](#rqsrs-006rbacprivilegeskillmutationalterupdate) - * 5.2.11.34 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDelete](#rqsrs-006rbacprivilegeskillmutationalterdelete) - * 5.2.11.35 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDropColumn](#rqsrs-006rbacprivilegeskillmutationalterdropcolumn) - * 5.2.11.36 [RQ.SRS-006.RBAC.Privileges.ShowTables](#rqsrs-006rbacprivilegesshowtables) - * 5.2.11.37 [RQ.SRS-006.RBAC.Privileges.ShowTables.Query](#rqsrs-006rbacprivilegesshowtablesquery) - * 5.2.11.38 [RQ.SRS-006.RBAC.Privileges.ExistsTable](#rqsrs-006rbacprivilegesexiststable) - * 5.2.11.39 [RQ.SRS-006.RBAC.Privileges.CheckTable](#rqsrs-006rbacprivilegeschecktable) - * 5.2.11.40 [RQ.SRS-006.RBAC.Privileges.ShowDatabases](#rqsrs-006rbacprivilegesshowdatabases) - * 5.2.11.41 [RQ.SRS-006.RBAC.Privileges.ShowDatabases.Query](#rqsrs-006rbacprivilegesshowdatabasesquery) - * 5.2.11.42 [RQ.SRS-006.RBAC.Privileges.ShowCreateDatabase](#rqsrs-006rbacprivilegesshowcreatedatabase) - * 5.2.11.43 [RQ.SRS-006.RBAC.Privileges.UseDatabase](#rqsrs-006rbacprivilegesusedatabase) - * 5.2.11.44 [RQ.SRS-006.RBAC.Privileges.ShowColumns](#rqsrs-006rbacprivilegesshowcolumns) - * 5.2.11.45 [RQ.SRS-006.RBAC.Privileges.ShowCreateTable](#rqsrs-006rbacprivilegesshowcreatetable) - * 5.2.11.46 [RQ.SRS-006.RBAC.Privileges.DescribeTable](#rqsrs-006rbacprivilegesdescribetable) - * 5.2.11.47 [RQ.SRS-006.RBAC.Privileges.ShowDictionaries](#rqsrs-006rbacprivilegesshowdictionaries) - * 5.2.11.48 [RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query](#rqsrs-006rbacprivilegesshowdictionariesquery) - * 5.2.11.49 [RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary](#rqsrs-006rbacprivilegesshowcreatedictionary) - * 5.2.11.50 [RQ.SRS-006.RBAC.Privileges.ExistsDictionary](#rqsrs-006rbacprivilegesexistsdictionary) - * 5.2.11.51 [RQ.SRS-006.RBAC.Privileges.CreateUser](#rqsrs-006rbacprivilegescreateuser) - * 5.2.11.52 [RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole](#rqsrs-006rbacprivilegescreateuserdefaultrole) - * 5.2.11.53 [RQ.SRS-006.RBAC.Privileges.AlterUser](#rqsrs-006rbacprivilegesalteruser) - * 5.2.11.54 [RQ.SRS-006.RBAC.Privileges.DropUser](#rqsrs-006rbacprivilegesdropuser) - * 5.2.11.55 [RQ.SRS-006.RBAC.Privileges.CreateRole](#rqsrs-006rbacprivilegescreaterole) - * 5.2.11.56 [RQ.SRS-006.RBAC.Privileges.AlterRole](#rqsrs-006rbacprivilegesalterrole) - * 5.2.11.57 [RQ.SRS-006.RBAC.Privileges.DropRole](#rqsrs-006rbacprivilegesdroprole) - * 5.2.11.58 [RQ.SRS-006.RBAC.Privileges.CreateRowPolicy](#rqsrs-006rbacprivilegescreaterowpolicy) - * 5.2.11.59 [RQ.SRS-006.RBAC.Privileges.AlterRowPolicy](#rqsrs-006rbacprivilegesalterrowpolicy) - * 5.2.11.60 [RQ.SRS-006.RBAC.Privileges.DropRowPolicy](#rqsrs-006rbacprivilegesdroprowpolicy) - * 5.2.11.61 [RQ.SRS-006.RBAC.Privileges.CreateQuota](#rqsrs-006rbacprivilegescreatequota) - * 5.2.11.62 [RQ.SRS-006.RBAC.Privileges.AlterQuota](#rqsrs-006rbacprivilegesalterquota) - * 5.2.11.63 [RQ.SRS-006.RBAC.Privileges.DropQuota](#rqsrs-006rbacprivilegesdropquota) - * 5.2.11.64 [RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile](#rqsrs-006rbacprivilegescreatesettingsprofile) - * 5.2.11.65 [RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile](#rqsrs-006rbacprivilegesaltersettingsprofile) - * 5.2.11.66 [RQ.SRS-006.RBAC.Privileges.DropSettingsProfile](#rqsrs-006rbacprivilegesdropsettingsprofile) - * 5.2.11.67 [RQ.SRS-006.RBAC.Privileges.RoleAdmin](#rqsrs-006rbacprivilegesroleadmin) - * 5.2.11.68 [RQ.SRS-006.RBAC.Privileges.ShowUsers](#rqsrs-006rbacprivilegesshowusers) - * 5.2.11.69 [RQ.SRS-006.RBAC.Privileges.ShowUsers.Query](#rqsrs-006rbacprivilegesshowusersquery) - * 5.2.11.70 [RQ.SRS-006.RBAC.Privileges.ShowCreateUser](#rqsrs-006rbacprivilegesshowcreateuser) - * 5.2.11.71 [RQ.SRS-006.RBAC.Privileges.ShowRoles](#rqsrs-006rbacprivilegesshowroles) - * 5.2.11.72 [RQ.SRS-006.RBAC.Privileges.ShowRoles.Query](#rqsrs-006rbacprivilegesshowrolesquery) - * 5.2.11.73 [RQ.SRS-006.RBAC.Privileges.ShowCreateRole](#rqsrs-006rbacprivilegesshowcreaterole) - * 5.2.11.74 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies](#rqsrs-006rbacprivilegesshowrowpolicies) - * 5.2.11.75 [RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query](#rqsrs-006rbacprivilegesshowrowpoliciesquery) - * 5.2.11.76 [RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy](#rqsrs-006rbacprivilegesshowcreaterowpolicy) - * 5.2.11.77 [RQ.SRS-006.RBAC.Privileges.ShowQuotas](#rqsrs-006rbacprivilegesshowquotas) - * 5.2.11.78 [RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query](#rqsrs-006rbacprivilegesshowquotasquery) - * 5.2.11.79 [RQ.SRS-006.RBAC.Privileges.ShowCreateQuota](#rqsrs-006rbacprivilegesshowcreatequota) - * 5.2.11.80 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles](#rqsrs-006rbacprivilegesshowsettingsprofiles) - * 5.2.11.81 [RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query](#rqsrs-006rbacprivilegesshowsettingsprofilesquery) - * 5.2.11.82 [RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile](#rqsrs-006rbacprivilegesshowcreatesettingsprofile) - * 5.2.11.83 [Grant Option](#grant-option) - * 5.2.11.83.1 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) - * 5.2.11.84 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) - * 5.2.11.85 [RQ.SRS-006.RBAC.Privileges.All.GrantRevoke](#rqsrs-006rbacprivilegesallgrantrevoke) - * 5.2.11.86 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) - * 5.2.12 [Required Privileges](#required-privileges) - * 5.2.12.1 [RQ.SRS-006.RBAC.RequiredPrivileges.Create](#rqsrs-006rbacrequiredprivilegescreate) - * 5.2.12.2 [RQ.SRS-006.RBAC.RequiredPrivileges.Alter](#rqsrs-006rbacrequiredprivilegesalter) - * 5.2.12.3 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop](#rqsrs-006rbacrequiredprivilegesdrop) - * 5.2.12.4 [RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table](#rqsrs-006rbacrequiredprivilegesdroptable) - * 5.2.12.5 [RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke](#rqsrs-006rbacrequiredprivilegesgrantrevoke) - * 5.2.12.6 [RQ.SRS-006.RBAC.RequiredPrivileges.Use](#rqsrs-006rbacrequiredprivilegesuse) - * 5.2.12.7 [RQ.SRS-006.RBAC.RequiredPrivileges.Admin](#rqsrs-006rbacrequiredprivilegesadmin) + * 5.2.9.2 [RQ.SRS-006.RBAC.Table.SensitiveTables](#rqsrs-006rbactablesensitivetables) + * 5.2.10 [Distributed Tables](#distributed-tables) + * 5.2.10.1 [RQ.SRS-006.RBAC.DistributedTable.Create](#rqsrs-006rbacdistributedtablecreate) + * 5.2.10.2 [RQ.SRS-006.RBAC.DistributedTable.Select](#rqsrs-006rbacdistributedtableselect) + * 5.2.10.3 [RQ.SRS-006.RBAC.DistributedTable.Insert](#rqsrs-006rbacdistributedtableinsert) + * 5.2.10.4 [RQ.SRS-006.RBAC.DistributedTable.SpecialTables](#rqsrs-006rbacdistributedtablespecialtables) + * 5.2.10.5 [RQ.SRS-006.RBAC.DistributedTable.LocalUser](#rqsrs-006rbacdistributedtablelocaluser) + * 5.2.10.6 [RQ.SRS-006.RBAC.DistributedTable.SameUserDifferentNodesDifferentPrivileges](#rqsrs-006rbacdistributedtablesameuserdifferentnodesdifferentprivileges) + * 5.2.11 [Views](#views) + * 5.2.11.1 [View](#view) + * 5.2.11.1.1 [RQ.SRS-006.RBAC.View](#rqsrs-006rbacview) + * 5.2.11.1.2 [RQ.SRS-006.RBAC.View.Create](#rqsrs-006rbacviewcreate) + * 5.2.11.1.3 [RQ.SRS-006.RBAC.View.Select](#rqsrs-006rbacviewselect) + * 5.2.11.1.4 [RQ.SRS-006.RBAC.View.Drop](#rqsrs-006rbacviewdrop) + * 5.2.11.2 [Materialized View](#materialized-view) + * 5.2.11.2.1 [RQ.SRS-006.RBAC.MaterializedView](#rqsrs-006rbacmaterializedview) + * 5.2.11.2.2 [RQ.SRS-006.RBAC.MaterializedView.Create](#rqsrs-006rbacmaterializedviewcreate) + * 5.2.11.2.3 [RQ.SRS-006.RBAC.MaterializedView.Select](#rqsrs-006rbacmaterializedviewselect) + * 5.2.11.2.4 [RQ.SRS-006.RBAC.MaterializedView.Select.TargetTable](#rqsrs-006rbacmaterializedviewselecttargettable) + * 5.2.11.2.5 [RQ.SRS-006.RBAC.MaterializedView.Select.SourceTable](#rqsrs-006rbacmaterializedviewselectsourcetable) + * 5.2.11.2.6 [RQ.SRS-006.RBAC.MaterializedView.Drop](#rqsrs-006rbacmaterializedviewdrop) + * 5.2.11.2.7 [RQ.SRS-006.RBAC.MaterializedView.ModifyQuery](#rqsrs-006rbacmaterializedviewmodifyquery) + * 5.2.11.2.8 [RQ.SRS-006.RBAC.MaterializedView.Insert](#rqsrs-006rbacmaterializedviewinsert) + * 5.2.11.2.9 [RQ.SRS-006.RBAC.MaterializedView.Insert.SourceTable](#rqsrs-006rbacmaterializedviewinsertsourcetable) + * 5.2.11.2.10 [RQ.SRS-006.RBAC.MaterializedView.Insert.TargetTable](#rqsrs-006rbacmaterializedviewinserttargettable) + * 5.2.11.3 [Live View](#live-view) + * 5.2.11.3.1 [RQ.SRS-006.RBAC.LiveView](#rqsrs-006rbacliveview) + * 5.2.11.3.2 [RQ.SRS-006.RBAC.LiveView.Create](#rqsrs-006rbacliveviewcreate) + * 5.2.11.3.3 [RQ.SRS-006.RBAC.LiveView.Select](#rqsrs-006rbacliveviewselect) + * 5.2.11.3.4 [RQ.SRS-006.RBAC.LiveView.Drop](#rqsrs-006rbacliveviewdrop) + * 5.2.11.3.5 [RQ.SRS-006.RBAC.LiveView.Refresh](#rqsrs-006rbacliveviewrefresh) + * 5.2.12 [Select](#select) + * 5.2.12.1 [RQ.SRS-006.RBAC.Select](#rqsrs-006rbacselect) + * 5.2.12.2 [RQ.SRS-006.RBAC.Select.Column](#rqsrs-006rbacselectcolumn) + * 5.2.12.3 [RQ.SRS-006.RBAC.Select.Cluster](#rqsrs-006rbacselectcluster) + * 5.2.12.4 [RQ.SRS-006.RBAC.Select.TableEngines](#rqsrs-006rbacselecttableengines) + * 5.2.13 [Insert](#insert) + * 5.2.13.1 [RQ.SRS-006.RBAC.Insert](#rqsrs-006rbacinsert) + * 5.2.13.2 [RQ.SRS-006.RBAC.Insert.Column](#rqsrs-006rbacinsertcolumn) + * 5.2.13.3 [RQ.SRS-006.RBAC.Insert.Cluster](#rqsrs-006rbacinsertcluster) + * 5.2.13.4 [RQ.SRS-006.RBAC.Insert.TableEngines](#rqsrs-006rbacinserttableengines) + * 5.2.14 [Alter](#alter) + * 5.2.14.1 [Alter Column](#alter-column) + * 5.2.14.1.1 [RQ.SRS-006.RBAC.Privileges.AlterColumn](#rqsrs-006rbacprivilegesaltercolumn) + * 5.2.14.1.2 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant](#rqsrs-006rbacprivilegesaltercolumngrant) + * 5.2.14.1.3 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke](#rqsrs-006rbacprivilegesaltercolumnrevoke) + * 5.2.14.1.4 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Column](#rqsrs-006rbacprivilegesaltercolumncolumn) + * 5.2.14.1.5 [RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster](#rqsrs-006rbacprivilegesaltercolumncluster) + * 5.2.14.1.6 [RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines](#rqsrs-006rbacprivilegesaltercolumntableengines) + * 5.2.14.2 [Alter Index](#alter-index) + * 5.2.14.2.1 [RQ.SRS-006.RBAC.Privileges.AlterIndex](#rqsrs-006rbacprivilegesalterindex) + * 5.2.14.2.2 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant](#rqsrs-006rbacprivilegesalterindexgrant) + * 5.2.14.2.3 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke](#rqsrs-006rbacprivilegesalterindexrevoke) + * 5.2.14.2.4 [RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster](#rqsrs-006rbacprivilegesalterindexcluster) + * 5.2.14.2.5 [RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines](#rqsrs-006rbacprivilegesalterindextableengines) + * 5.2.14.3 [Alter Constraint](#alter-constraint) + * 5.2.14.3.1 [RQ.SRS-006.RBAC.Privileges.AlterConstraint](#rqsrs-006rbacprivilegesalterconstraint) + * 5.2.14.3.2 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant](#rqsrs-006rbacprivilegesalterconstraintgrant) + * 5.2.14.3.3 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke](#rqsrs-006rbacprivilegesalterconstraintrevoke) + * 5.2.14.3.4 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster](#rqsrs-006rbacprivilegesalterconstraintcluster) + * 5.2.14.3.5 [RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines](#rqsrs-006rbacprivilegesalterconstrainttableengines) + * 5.2.14.4 [Alter TTL](#alter-ttl) + * 5.2.14.4.1 [RQ.SRS-006.RBAC.Privileges.AlterTTL](#rqsrs-006rbacprivilegesalterttl) + * 5.2.14.4.2 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant](#rqsrs-006rbacprivilegesalterttlgrant) + * 5.2.14.4.3 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke](#rqsrs-006rbacprivilegesalterttlrevoke) + * 5.2.14.4.4 [RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster](#rqsrs-006rbacprivilegesalterttlcluster) + * 5.2.14.4.5 [RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines](#rqsrs-006rbacprivilegesalterttltableengines) + * 5.2.14.5 [Alter Settings](#alter-settings) + * 5.2.14.5.1 [RQ.SRS-006.RBAC.Privileges.AlterSettings](#rqsrs-006rbacprivilegesaltersettings) + * 5.2.14.5.2 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant](#rqsrs-006rbacprivilegesaltersettingsgrant) + * 5.2.14.5.3 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke](#rqsrs-006rbacprivilegesaltersettingsrevoke) + * 5.2.14.5.4 [RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster](#rqsrs-006rbacprivilegesaltersettingscluster) + * 5.2.14.5.5 [RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines](#rqsrs-006rbacprivilegesaltersettingstableengines) + * 5.2.14.6 [Alter Update](#alter-update) + * 5.2.14.6.1 [RQ.SRS-006.RBAC.Privileges.AlterUpdate](#rqsrs-006rbacprivilegesalterupdate) + * 5.2.14.6.2 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Grant](#rqsrs-006rbacprivilegesalterupdategrant) + * 5.2.14.6.3 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.Revoke](#rqsrs-006rbacprivilegesalterupdaterevoke) + * 5.2.14.6.4 [RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines](#rqsrs-006rbacprivilegesalterupdatetableengines) + * 5.2.14.7 [Alter Delete](#alter-delete) + * 5.2.14.7.1 [RQ.SRS-006.RBAC.Privileges.AlterDelete](#rqsrs-006rbacprivilegesalterdelete) + * 5.2.14.7.2 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Grant](#rqsrs-006rbacprivilegesalterdeletegrant) + * 5.2.14.7.3 [RQ.SRS-006.RBAC.Privileges.AlterDelete.Revoke](#rqsrs-006rbacprivilegesalterdeleterevoke) + * 5.2.14.7.4 [RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines](#rqsrs-006rbacprivilegesalterdeletetableengines) + * 5.2.14.8 [Alter Freeze Partition](#alter-freeze-partition) + * 5.2.14.8.1 [RQ.SRS-006.RBAC.Privileges.AlterFreeze](#rqsrs-006rbacprivilegesalterfreeze) + * 5.2.14.8.2 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Grant](#rqsrs-006rbacprivilegesalterfreezegrant) + * 5.2.14.8.3 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.Revoke](#rqsrs-006rbacprivilegesalterfreezerevoke) + * 5.2.14.8.4 [RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines](#rqsrs-006rbacprivilegesalterfreezetableengines) + * 5.2.14.9 [Alter Fetch Partition](#alter-fetch-partition) + * 5.2.14.9.1 [RQ.SRS-006.RBAC.Privileges.AlterFetch](#rqsrs-006rbacprivilegesalterfetch) + * 5.2.14.9.2 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Grant](#rqsrs-006rbacprivilegesalterfetchgrant) + * 5.2.14.9.3 [RQ.SRS-006.RBAC.Privileges.AlterFetch.Revoke](#rqsrs-006rbacprivilegesalterfetchrevoke) + * 5.2.14.9.4 [RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines](#rqsrs-006rbacprivilegesalterfetchtableengines) + * 5.2.14.10 [Alter Move Partition](#alter-move-partition) + * 5.2.14.10.1 [RQ.SRS-006.RBAC.Privileges.AlterMove](#rqsrs-006rbacprivilegesaltermove) + * 5.2.14.10.2 [RQ.SRS-006.RBAC.Privileges.AlterMove.Grant](#rqsrs-006rbacprivilegesaltermovegrant) + * 5.2.14.10.3 [RQ.SRS-006.RBAC.Privileges.AlterMove.Revoke](#rqsrs-006rbacprivilegesaltermoverevoke) + * 5.2.14.10.4 [RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines](#rqsrs-006rbacprivilegesaltermovetableengines) + * 5.2.15 [RQ.SRS-006.RBAC.Privileges.CreateTable](#rqsrs-006rbacprivilegescreatetable) + * 5.2.16 [RQ.SRS-006.RBAC.Privileges.CreateDatabase](#rqsrs-006rbacprivilegescreatedatabase) + * 5.2.17 [RQ.SRS-006.RBAC.Privileges.CreateDictionary](#rqsrs-006rbacprivilegescreatedictionary) + * 5.2.18 [RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable](#rqsrs-006rbacprivilegescreatetemporarytable) + * 5.2.19 [RQ.SRS-006.RBAC.Privileges.AttachDatabase](#rqsrs-006rbacprivilegesattachdatabase) + * 5.2.20 [RQ.SRS-006.RBAC.Privileges.AttachDictionary](#rqsrs-006rbacprivilegesattachdictionary) + * 5.2.21 [RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable](#rqsrs-006rbacprivilegesattachtemporarytable) + * 5.2.22 [RQ.SRS-006.RBAC.Privileges.AttachTable](#rqsrs-006rbacprivilegesattachtable) + * 5.2.23 [RQ.SRS-006.RBAC.Privileges.DropTable](#rqsrs-006rbacprivilegesdroptable) + * 5.2.24 [RQ.SRS-006.RBAC.Privileges.DropDatabase](#rqsrs-006rbacprivilegesdropdatabase) + * 5.2.25 [RQ.SRS-006.RBAC.Privileges.DropDictionary](#rqsrs-006rbacprivilegesdropdictionary) + * 5.2.26 [RQ.SRS-006.RBAC.Privileges.DetachTable](#rqsrs-006rbacprivilegesdetachtable) + * 5.2.27 [RQ.SRS-006.RBAC.Privileges.DetachView](#rqsrs-006rbacprivilegesdetachview) + * 5.2.28 [RQ.SRS-006.RBAC.Privileges.DetachDatabase](#rqsrs-006rbacprivilegesdetachdatabase) + * 5.2.29 [RQ.SRS-006.RBAC.Privileges.DetachDictionary](#rqsrs-006rbacprivilegesdetachdictionary) + * 5.2.30 [RQ.SRS-006.RBAC.Privileges.Truncate](#rqsrs-006rbacprivilegestruncate) + * 5.2.31 [RQ.SRS-006.RBAC.Privileges.Optimize](#rqsrs-006rbacprivilegesoptimize) + * 5.2.32 [RQ.SRS-006.RBAC.Privileges.KillQuery](#rqsrs-006rbacprivilegeskillquery) + * 5.2.33 [Kill Mutation](#kill-mutation) + * 5.2.33.1 [RQ.SRS-006.RBAC.Privileges.KillMutation](#rqsrs-006rbacprivilegeskillmutation) + * 5.2.33.2 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterUpdate](#rqsrs-006rbacprivilegeskillmutationalterupdate) + * 5.2.33.3 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDelete](#rqsrs-006rbacprivilegeskillmutationalterdelete) + * 5.2.33.4 [RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDropColumn](#rqsrs-006rbacprivilegeskillmutationalterdropcolumn) + * 5.2.34 [Show](#show) + * 5.2.34.1 [RQ.SRS-006.RBAC.ShowTables.Privilege](#rqsrs-006rbacshowtablesprivilege) + * 5.2.34.2 [RQ.SRS-006.RBAC.ShowTables.RequiredPrivilege](#rqsrs-006rbacshowtablesrequiredprivilege) + * 5.2.34.3 [RQ.SRS-006.RBAC.ExistsTable.RequiredPrivilege](#rqsrs-006rbacexiststablerequiredprivilege) + * 5.2.34.4 [RQ.SRS-006.RBAC.CheckTable.RequiredPrivilege](#rqsrs-006rbacchecktablerequiredprivilege) + * 5.2.34.5 [RQ.SRS-006.RBAC.ShowDatabases.Privilege](#rqsrs-006rbacshowdatabasesprivilege) + * 5.2.34.6 [RQ.SRS-006.RBAC.ShowDatabases.RequiredPrivilege](#rqsrs-006rbacshowdatabasesrequiredprivilege) + * 5.2.34.7 [RQ.SRS-006.RBAC.ShowCreateDatabase.RequiredPrivilege](#rqsrs-006rbacshowcreatedatabaserequiredprivilege) + * 5.2.34.8 [RQ.SRS-006.RBAC.UseDatabase.RequiredPrivilege](#rqsrs-006rbacusedatabaserequiredprivilege) + * 5.2.34.9 [RQ.SRS-006.RBAC.ShowColumns.Privilege](#rqsrs-006rbacshowcolumnsprivilege) + * 5.2.34.10 [RQ.SRS-006.RBAC.ShowCreateTable.RequiredPrivilege](#rqsrs-006rbacshowcreatetablerequiredprivilege) + * 5.2.34.11 [RQ.SRS-006.RBAC.DescribeTable.RequiredPrivilege](#rqsrs-006rbacdescribetablerequiredprivilege) + * 5.2.34.12 [RQ.SRS-006.RBAC.ShowDictionaries.Privilege](#rqsrs-006rbacshowdictionariesprivilege) + * 5.2.34.13 [RQ.SRS-006.RBAC.ShowDictionaries.RequiredPrivilege](#rqsrs-006rbacshowdictionariesrequiredprivilege) + * 5.2.34.14 [RQ.SRS-006.RBAC.ShowCreateDictionary.RequiredPrivilege](#rqsrs-006rbacshowcreatedictionaryrequiredprivilege) + * 5.2.34.15 [RQ.SRS-006.RBAC.ExistsDictionary.RequiredPrivilege](#rqsrs-006rbacexistsdictionaryrequiredprivilege) + * 5.2.35 [Access Management](#access-management) + * 5.2.35.1 [RQ.SRS-006.RBAC.Privileges.CreateUser](#rqsrs-006rbacprivilegescreateuser) + * 5.2.35.2 [RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole](#rqsrs-006rbacprivilegescreateuserdefaultrole) + * 5.2.35.3 [RQ.SRS-006.RBAC.Privileges.AlterUser](#rqsrs-006rbacprivilegesalteruser) + * 5.2.35.4 [RQ.SRS-006.RBAC.Privileges.DropUser](#rqsrs-006rbacprivilegesdropuser) + * 5.2.35.5 [RQ.SRS-006.RBAC.Privileges.CreateRole](#rqsrs-006rbacprivilegescreaterole) + * 5.2.35.6 [RQ.SRS-006.RBAC.Privileges.AlterRole](#rqsrs-006rbacprivilegesalterrole) + * 5.2.35.7 [RQ.SRS-006.RBAC.Privileges.DropRole](#rqsrs-006rbacprivilegesdroprole) + * 5.2.35.8 [RQ.SRS-006.RBAC.Privileges.CreateRowPolicy](#rqsrs-006rbacprivilegescreaterowpolicy) + * 5.2.35.9 [RQ.SRS-006.RBAC.Privileges.AlterRowPolicy](#rqsrs-006rbacprivilegesalterrowpolicy) + * 5.2.35.10 [RQ.SRS-006.RBAC.Privileges.DropRowPolicy](#rqsrs-006rbacprivilegesdroprowpolicy) + * 5.2.35.11 [RQ.SRS-006.RBAC.Privileges.CreateQuota](#rqsrs-006rbacprivilegescreatequota) + * 5.2.35.12 [RQ.SRS-006.RBAC.Privileges.AlterQuota](#rqsrs-006rbacprivilegesalterquota) + * 5.2.35.13 [RQ.SRS-006.RBAC.Privileges.DropQuota](#rqsrs-006rbacprivilegesdropquota) + * 5.2.35.14 [RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile](#rqsrs-006rbacprivilegescreatesettingsprofile) + * 5.2.35.15 [RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile](#rqsrs-006rbacprivilegesaltersettingsprofile) + * 5.2.35.16 [RQ.SRS-006.RBAC.Privileges.DropSettingsProfile](#rqsrs-006rbacprivilegesdropsettingsprofile) + * 5.2.35.17 [RQ.SRS-006.RBAC.Privileges.RoleAdmin](#rqsrs-006rbacprivilegesroleadmin) + * 5.2.35.18 [Show Access](#show-access) + * 5.2.35.18.1 [RQ.SRS-006.RBAC.ShowUsers.Privilege](#rqsrs-006rbacshowusersprivilege) + * 5.2.35.18.2 [RQ.SRS-006.RBAC.ShowUsers.RequiredPrivilege](#rqsrs-006rbacshowusersrequiredprivilege) + * 5.2.35.18.3 [RQ.SRS-006.RBAC.ShowCreateUser.RequiredPrivilege](#rqsrs-006rbacshowcreateuserrequiredprivilege) + * 5.2.35.18.4 [RQ.SRS-006.RBAC.ShowRoles.Privilege](#rqsrs-006rbacshowrolesprivilege) + * 5.2.35.18.5 [RQ.SRS-006.RBAC.ShowRoles.RequiredPrivilege](#rqsrs-006rbacshowrolesrequiredprivilege) + * 5.2.35.18.6 [RQ.SRS-006.RBAC.ShowCreateRole.RequiredPrivilege](#rqsrs-006rbacshowcreaterolerequiredprivilege) + * 5.2.35.18.7 [RQ.SRS-006.RBAC.ShowRowPolicies.Privilege](#rqsrs-006rbacshowrowpoliciesprivilege) + * 5.2.35.18.8 [RQ.SRS-006.RBAC.ShowRowPolicies.RequiredPrivilege](#rqsrs-006rbacshowrowpoliciesrequiredprivilege) + * 5.2.35.18.9 [RQ.SRS-006.RBAC.ShowCreateRowPolicy.RequiredPrivilege](#rqsrs-006rbacshowcreaterowpolicyrequiredprivilege) + * 5.2.35.18.10 [RQ.SRS-006.RBAC.ShowQuotas.Privilege](#rqsrs-006rbacshowquotasprivilege) + * 5.2.35.18.11 [RQ.SRS-006.RBAC.ShowQuotas.RequiredPrivilege](#rqsrs-006rbacshowquotasrequiredprivilege) + * 5.2.35.18.12 [RQ.SRS-006.RBAC.ShowCreateQuota.RequiredPrivilege](#rqsrs-006rbacshowcreatequotarequiredprivilege) + * 5.2.35.18.13 [RQ.SRS-006.RBAC.ShowSettingsProfiles.Privilege](#rqsrs-006rbacshowsettingsprofilesprivilege) + * 5.2.35.18.14 [RQ.SRS-006.RBAC.ShowSettingsProfiles.RequiredPrivilege](#rqsrs-006rbacshowsettingsprofilesrequiredprivilege) + * 5.2.35.18.15 [RQ.SRS-006.RBAC.ShowCreateSettingsProfile.RequiredPrivilege](#rqsrs-006rbacshowcreatesettingsprofilerequiredprivilege) + * 5.2.36 [dictGet](#dictget) + * 5.2.36.1 [RQ.SRS-006.RBAC.dictGet.Privilege](#rqsrs-006rbacdictgetprivilege) + * 5.2.36.2 [RQ.SRS-006.RBAC.dictGet.RequiredPrivilege](#rqsrs-006rbacdictgetrequiredprivilege) + * 5.2.36.3 [RQ.SRS-006.RBAC.dictGet.Type.RequiredPrivilege](#rqsrs-006rbacdictgettyperequiredprivilege) + * 5.2.36.4 [RQ.SRS-006.RBAC.dictGet.OrDefault.RequiredPrivilege](#rqsrs-006rbacdictgetordefaultrequiredprivilege) + * 5.2.36.5 [RQ.SRS-006.RBAC.dictHas.RequiredPrivilege](#rqsrs-006rbacdicthasrequiredprivilege) + * 5.2.36.6 [RQ.SRS-006.RBAC.dictGetHierarchy.RequiredPrivilege](#rqsrs-006rbacdictgethierarchyrequiredprivilege) + * 5.2.36.7 [RQ.SRS-006.RBAC.dictIsIn.RequiredPrivilege](#rqsrs-006rbacdictisinrequiredprivilege) + * 5.2.37 [Introspection](#introspection) + * 5.2.37.1 [RQ.SRS-006.RBAC.Privileges.Introspection](#rqsrs-006rbacprivilegesintrospection) + * 5.2.37.2 [RQ.SRS-006.RBAC.Privileges.Introspection.addressToLine](#rqsrs-006rbacprivilegesintrospectionaddresstoline) + * 5.2.37.3 [RQ.SRS-006.RBAC.Privileges.Introspection.addressToSymbol](#rqsrs-006rbacprivilegesintrospectionaddresstosymbol) + * 5.2.37.4 [RQ.SRS-006.RBAC.Privileges.Introspection.demangle](#rqsrs-006rbacprivilegesintrospectiondemangle) + * 5.2.38 [System](#system) + * 5.2.38.1 [RQ.SRS-006.RBAC.Privileges.System.Shutdown](#rqsrs-006rbacprivilegessystemshutdown) + * 5.2.38.2 [RQ.SRS-006.RBAC.Privileges.System.DropCache](#rqsrs-006rbacprivilegessystemdropcache) + * 5.2.38.3 [RQ.SRS-006.RBAC.Privileges.System.DropCache.DNS](#rqsrs-006rbacprivilegessystemdropcachedns) + * 5.2.38.4 [RQ.SRS-006.RBAC.Privileges.System.DropCache.Mark](#rqsrs-006rbacprivilegessystemdropcachemark) + * 5.2.38.5 [RQ.SRS-006.RBAC.Privileges.System.DropCache.Uncompressed](#rqsrs-006rbacprivilegessystemdropcacheuncompressed) + * 5.2.38.6 [RQ.SRS-006.RBAC.Privileges.System.Reload](#rqsrs-006rbacprivilegessystemreload) + * 5.2.38.7 [RQ.SRS-006.RBAC.Privileges.System.Reload.Config](#rqsrs-006rbacprivilegessystemreloadconfig) + * 5.2.38.8 [RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionary](#rqsrs-006rbacprivilegessystemreloaddictionary) + * 5.2.38.9 [RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionaries](#rqsrs-006rbacprivilegessystemreloaddictionaries) + * 5.2.38.10 [RQ.SRS-006.RBAC.Privileges.System.Reload.EmbeddedDictionaries](#rqsrs-006rbacprivilegessystemreloadembeddeddictionaries) + * 5.2.38.11 [RQ.SRS-006.RBAC.Privileges.System.Merges](#rqsrs-006rbacprivilegessystemmerges) + * 5.2.38.12 [RQ.SRS-006.RBAC.Privileges.System.TTLMerges](#rqsrs-006rbacprivilegessystemttlmerges) + * 5.2.38.13 [RQ.SRS-006.RBAC.Privileges.System.Fetches](#rqsrs-006rbacprivilegessystemfetches) + * 5.2.38.14 [RQ.SRS-006.RBAC.Privileges.System.Moves](#rqsrs-006rbacprivilegessystemmoves) + * 5.2.38.15 [RQ.SRS-006.RBAC.Privileges.System.Sends](#rqsrs-006rbacprivilegessystemsends) + * 5.2.38.16 [RQ.SRS-006.RBAC.Privileges.System.Sends.Distributed](#rqsrs-006rbacprivilegessystemsendsdistributed) + * 5.2.38.17 [RQ.SRS-006.RBAC.Privileges.System.Sends.Replicated](#rqsrs-006rbacprivilegessystemsendsreplicated) + * 5.2.38.18 [RQ.SRS-006.RBAC.Privileges.System.ReplicationQueues](#rqsrs-006rbacprivilegessystemreplicationqueues) + * 5.2.38.19 [RQ.SRS-006.RBAC.Privileges.System.SyncReplica](#rqsrs-006rbacprivilegessystemsyncreplica) + * 5.2.38.20 [RQ.SRS-006.RBAC.Privileges.System.RestartReplica](#rqsrs-006rbacprivilegessystemrestartreplica) + * 5.2.38.21 [RQ.SRS-006.RBAC.Privileges.System.Flush](#rqsrs-006rbacprivilegessystemflush) + * 5.2.38.22 [RQ.SRS-006.RBAC.Privileges.System.Flush.Distributed](#rqsrs-006rbacprivilegessystemflushdistributed) + * 5.2.38.23 [RQ.SRS-006.RBAC.Privileges.System.Flush.Logs](#rqsrs-006rbacprivilegessystemflushlogs) + * 5.2.39 [Sources](#sources) + * 5.2.39.1 [RQ.SRS-006.RBAC.Privileges.Sources](#rqsrs-006rbacprivilegessources) + * 5.2.39.2 [RQ.SRS-006.RBAC.Privileges.Sources.File](#rqsrs-006rbacprivilegessourcesfile) + * 5.2.39.3 [RQ.SRS-006.RBAC.Privileges.Sources.URL](#rqsrs-006rbacprivilegessourcesurl) + * 5.2.39.4 [RQ.SRS-006.RBAC.Privileges.Sources.Remote](#rqsrs-006rbacprivilegessourcesremote) + * 5.2.39.5 [RQ.SRS-006.RBAC.Privileges.Sources.MySQL](#rqsrs-006rbacprivilegessourcesmysql) + * 5.2.39.6 [RQ.SRS-006.RBAC.Privileges.Sources.ODBC](#rqsrs-006rbacprivilegessourcesodbc) + * 5.2.39.7 [RQ.SRS-006.RBAC.Privileges.Sources.JDBC](#rqsrs-006rbacprivilegessourcesjdbc) + * 5.2.39.8 [RQ.SRS-006.RBAC.Privileges.Sources.HDFS](#rqsrs-006rbacprivilegessourceshdfs) + * 5.2.39.9 [RQ.SRS-006.RBAC.Privileges.Sources.S3](#rqsrs-006rbacprivilegessourcess3) + * 5.2.40 [RQ.SRS-006.RBAC.Privileges.GrantOption](#rqsrs-006rbacprivilegesgrantoption) + * 5.2.41 [RQ.SRS-006.RBAC.Privileges.All](#rqsrs-006rbacprivilegesall) + * 5.2.42 [RQ.SRS-006.RBAC.Privileges.AdminOption](#rqsrs-006rbacprivilegesadminoption) * 6 [References](#references) ## Revision History @@ -11160,60 +1545,12 @@ version: 1.0 [ClickHouse] SHALL support granting the **create** privilege to one or more users or roles using the `GRANT CREATE` statement. -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.Database -version: 1.0 - -[ClickHouse] SHALL support granting the **create database** privilege to one or more users or roles -for a database using the `GRANT CREATE DATABASE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.Dictionary -version: 1.0 - -[ClickHouse] SHALL support granting the **create dictionary** privilege to one or more users or roles -for a dictionary using the `GRANT CREATE DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.Table -version: 1.0 - -[ClickHouse] SHALL support granting the **create table** privilege to one or more users or roles -for a table using the `GRANT CREATE TABLE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Create.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support granting the **create temporary table** privilege to one or more users or roles -for a temporary table using the `GRANT CREATE TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Grant.Privilege.Drop version: 1.0 [ClickHouse] SHALL support granting the **drop** privilege to one or more users or roles using the `GRANT DROP` statement. -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.Database -version: 1.0 - -[ClickHouse] SHALL support granting the **drop database** privilege to one or more users or roles -for a database using the `GRANT DROP DATABASE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.Dictionary -version: 1.0 - -[ClickHouse] SHALL support granting the **drop dictionary** privilege to one or more users or roles -for a dictionary using the `GRANT DROP DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.Table -version: 1.0 - -[ClickHouse] SHALL support granting the **drop table** privilege to one or more users or roles -for a table using the `GRANT DROP TABLE` statement. - -##### RQ.SRS-006.RBAC.Grant.Privilege.Drop.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support granting the **drop temporary table** privilege to one or more users or roles -for a temporary table using the `GRANT DROP TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Grant.Privilege.Truncate version: 1.0 @@ -11365,60 +1702,12 @@ version: 1.0 [ClickHouse] SHALL support revoking the **create** privilege to one or more users or roles using the `REVOKE CREATE` statement. -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.Database -version: 1.0 - -[ClickHouse] SHALL support revoking the **create database** privilege to one or more users or roles -for a database using the `REVOKE CREATE DATABASE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.Dictionary -version: 1.0 - -[ClickHouse] SHALL support revoking the **create dictionary** privilege to one or more users or roles -for a dictionary using the `REVOKE CREATE DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.Table -version: 1.0 - -[ClickHouse] SHALL support revoking the **create table** privilege to one or more users or roles -for a table using the `REVOKE CREATE TABLE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Create.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support revoking the **create temporary table** privilege to one or more users or roles -for a temporary table using the `REVOKE CREATE TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop version: 1.0 [ClickHouse] SHALL support revoking the **drop** privilege to one or more users or roles using the `REVOKE DROP` statement. -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Database -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop database** privilege to one or more users or roles -for a database using the `REVOKE DROP DATABASE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Dictionary -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop dictionary** privilege to one or more users or roles -for a dictionary using the `REVOKE DROP DICTIONARY` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.Table -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop table** privilege to one or more users or roles -for a table using the `REVOKE DROP TABLE` statement. - -##### RQ.SRS-006.RBAC.Revoke.Privilege.Drop.TemporaryTable -version: 1.0 - -[ClickHouse] SHALL support revoking the **drop temporary table** privilege to one or more users or roles -for a temporary table using the `REVOKE DROP TEMPORARY TABLE` statement. - ##### RQ.SRS-006.RBAC.Revoke.Privilege.Truncate version: 1.0 @@ -12561,20 +2850,32 @@ version: 1.0 * system.contributors * system.functions -##### RQ.SRS-006.RBAC.Table.QueryLog +##### RQ.SRS-006.RBAC.Table.SensitiveTables version: 1.0 -[ClickHouse] SHALL return only queries executed by the user when the user is selecting from system.query_log. +[ClickHouse] SHALL not support a user with no privileges accessing the following `system` tables: -##### Distributed Tables +* processes +* query_log +* query_thread_log +* clusters +* events +* graphite_retentions +* stack_trace +* trace_log +* user_directories +* zookeeper +* macros -###### RQ.SRS-006.RBAC.Table.DistributedTable.Create +#### Distributed Tables + +##### RQ.SRS-006.RBAC.DistributedTable.Create version: 1.0 [ClickHouse] SHALL successfully `CREATE` a distributed table if and only if the user has **create table** privilege on the table and **remote** privilege on *.* -###### RQ.SRS-006.RBAC.Table.DistributedTable.Select +##### RQ.SRS-006.RBAC.DistributedTable.Select version: 1.0 [ClickHouse] SHALL successfully `SELECT` from a distributed table if and only if @@ -12582,7 +2883,7 @@ the user has **select** privilege on the table and on the remote table specified Does not require **select** privilege for the remote table if the remote table does not exist on the same server as the user. -###### RQ.SRS-006.RBAC.Table.DistributedTable.Insert +##### RQ.SRS-006.RBAC.DistributedTable.Insert version: 1.0 [ClickHouse] SHALL successfully `INSERT` into a distributed table if and only if @@ -12591,7 +2892,7 @@ the user has **insert** privilege on the table and on the remote table specified Does not require **insert** privilege for the remote table if the remote table does not exist on the same server as the user, insert executes into the remote table on a different server. -###### RQ.SRS-006.RBAC.Table.DistributedTable.SpecialTables +##### RQ.SRS-006.RBAC.DistributedTable.SpecialTables version: 1.0 [ClickHouse] SHALL successfully execute a query using a distributed table that uses one of the special tables if and only if @@ -12601,13 +2902,13 @@ Special tables include: * distributed table * source table of a materialized view -###### RQ.SRS-006.RBAC.Table.DistributedTable.LocalUser +##### RQ.SRS-006.RBAC.DistributedTable.LocalUser version: 1.0 [ClickHouse] SHALL successfully execute a query using a distributed table from a user present locally, but not remotely. -###### RQ.SRS-006.RBAC.Table.DistributedTable.SameUserDifferentNodesDifferentPrivileges +##### RQ.SRS-006.RBAC.DistributedTable.SameUserDifferentNodesDifferentPrivileges version: 1.0 [ClickHouse] SHALL successfully execute a query using a distributed table by a user that exists on multiple nodes @@ -12834,41 +3135,33 @@ version: 1.0 [ClickHouse] SHALL only successfully execute an `ALTER LIVE VIEW REFRESH` command if and only if the user has **refresh** privilege on that view either explicitly or through a role. -#### Privileges +#### Select -##### RQ.SRS-006.RBAC.Privileges.Usage +##### RQ.SRS-006.RBAC.Select version: 1.0 -[ClickHouse] SHALL support granting or revoking **usage** privilege -for a database or a specific table to one or more **users** or **roles**. - -##### Select - -###### RQ.SRS-006.RBAC.Privileges.Select -version: 1.0 - -[ClickHouse] SHALL not execute `SELECT INTO` if and only if the user +[ClickHouse] SHALL execute `SELECT` if and only if the user has the **select** privilege for the destination table either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Select.Column +##### RQ.SRS-006.RBAC.Select.Column version: 1.0 [ClickHouse] SHALL support granting or revoking **select** privilege for one or more specified columns in a table to one or more **users** or **roles**. -Any `SELECT INTO` statements SHALL not to be executed, unless the user +Any `SELECT` statements SHALL not to be executed, unless the user has the **select** privilege for the destination column either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Select.Cluster +##### RQ.SRS-006.RBAC.Select.Cluster version: 1.0 [ClickHouse] SHALL support granting or revoking **select** privilege on a specified cluster to one or more **users** or **roles**. -Any `SELECT INTO` statements SHALL succeed only on nodes where +Any `SELECT` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Select.TableEngines +##### RQ.SRS-006.RBAC.Select.TableEngines version: 1.0 [ClickHouse] SHALL support controlling access to the **select** privilege @@ -12889,16 +3182,16 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree -##### Insert +#### Insert -###### RQ.SRS-006.RBAC.Privileges.Insert +##### RQ.SRS-006.RBAC.Insert version: 1.0 [ClickHouse] SHALL execute `INSERT INTO` if and only if the user has the **insert** privilege for the destination table either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Insert.Column +##### RQ.SRS-006.RBAC.Insert.Column version: 1.0 [ClickHouse] SHALL support granting or revoking **insert** privilege @@ -12907,7 +3200,7 @@ Any `INSERT INTO` statements SHALL not to be executed, unless the user has the **insert** privilege for the destination column either because of the explicit grant or through one of the roles assigned to the user. -###### RQ.SRS-006.RBAC.Privileges.Insert.Cluster +##### RQ.SRS-006.RBAC.Insert.Cluster version: 1.0 [ClickHouse] SHALL support granting or revoking **insert** privilege @@ -12915,7 +3208,7 @@ on a specified cluster to one or more **users** or **roles**. Any `INSERT INTO` statements SHALL succeed only on nodes where the table exists and privilege was granted. -###### RQ.SRS-006.RBAC.Privileges.Insert.TableEngines +##### RQ.SRS-006.RBAC.Insert.TableEngines version: 1.0 [ClickHouse] SHALL support controlling access to the **insert** privilege @@ -12936,6 +3229,8 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree +#### Alter + ##### Alter Column ###### RQ.SRS-006.RBAC.Privileges.AlterColumn @@ -13400,7 +3695,7 @@ on tables created using the following engines * ReplicatedVersionedCollapsingMergeTree * ReplicatedGraphiteMergeTree -##### RQ.SRS-006.RBAC.Privileges.CreateTable +#### RQ.SRS-006.RBAC.Privileges.CreateTable version: 1.0 [ClickHouse] SHALL only successfully execute a `CREATE TABLE` command if and only if @@ -13418,108 +3713,110 @@ CREATE TABLE table AS SELECT column FROM table0 JOIN table1 USING column UNION A CREATE TABLE table0 AS SELECT column FROM table1 UNION ALL SELECT column FROM table2 ``` -##### RQ.SRS-006.RBAC.Privileges.CreateDatabase +#### RQ.SRS-006.RBAC.Privileges.CreateDatabase version: 1.0 [ClickHouse] SHALL successfully execute `CREATE DATABASE` statement if and only if the user has **create database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.CreateDictionary +#### RQ.SRS-006.RBAC.Privileges.CreateDictionary version: 1.0 [ClickHouse] SHALL successfully execute `CREATE DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable +#### RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable version: 1.0 [ClickHouse] SHALL successfully execute `CREATE TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachDatabase +#### RQ.SRS-006.RBAC.Privileges.AttachDatabase version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH DATABASE` statement if and only if the user has **create database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachDictionary +#### RQ.SRS-006.RBAC.Privileges.AttachDictionary version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable +#### RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.AttachTable +#### RQ.SRS-006.RBAC.Privileges.AttachTable version: 1.0 [ClickHouse] SHALL successfully execute `ATTACH TABLE` statement if and only if the user has **create table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DropTable +#### RQ.SRS-006.RBAC.Privileges.DropTable version: 1.0 [ClickHouse] SHALL successfully execute `DROP TABLE` statement if and only if the user has **drop table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DropDatabase +#### RQ.SRS-006.RBAC.Privileges.DropDatabase version: 1.0 [ClickHouse] SHALL successfully execute `DROP DATABASE` statement if and only if the user has **drop database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DropDictionary +#### RQ.SRS-006.RBAC.Privileges.DropDictionary version: 1.0 [ClickHouse] SHALL successfully execute `DROP DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachTable +#### RQ.SRS-006.RBAC.Privileges.DetachTable version: 1.0 [ClickHouse] SHALL successfully execute `DETACH TABLE` statement if and only if the user has **drop table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachView +#### RQ.SRS-006.RBAC.Privileges.DetachView version: 1.0 [ClickHouse] SHALL successfully execute `DETACH VIEW` statement if and only if the user has **drop view** privilege on the view, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachDatabase +#### RQ.SRS-006.RBAC.Privileges.DetachDatabase version: 1.0 [ClickHouse] SHALL successfully execute `DETACH DATABASE` statement if and only if the user has **drop database** privilege on the database, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DetachDictionary +#### RQ.SRS-006.RBAC.Privileges.DetachDictionary version: 1.0 [ClickHouse] SHALL successfully execute `DETACH DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.Truncate +#### RQ.SRS-006.RBAC.Privileges.Truncate version: 1.0 [ClickHouse] SHALL successfully execute `TRUNCATE TABLE` statement if and only if the user has **truncate table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.Optimize +#### RQ.SRS-006.RBAC.Privileges.Optimize version: 1.0 [ClickHouse] SHALL successfully execute `OPTIMIZE TABLE` statement if and only if the user has **optimize table** privilege on the table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.KillQuery +#### RQ.SRS-006.RBAC.Privileges.KillQuery version: 1.0 [ClickHouse] SHALL successfully execute `KILL QUERY` statement if and only if the user has **kill query** privilege, either directly or through a role. +#### Kill Mutation + ##### RQ.SRS-006.RBAC.Privileges.KillMutation version: 1.0 @@ -13545,95 +3842,99 @@ version: 1.0 [ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER DROP COLUMN` mutation if and only if the user has `ALTER DROP COLUMN` privilege on the table where the mutation was created, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowTables +#### Show + +##### RQ.SRS-006.RBAC.ShowTables.Privilege version: 1.0 [ClickHouse] SHALL grant **show tables** privilege on a table to a user if that user has recieved any grant, including `SHOW TABLES`, on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowTables.Query +##### RQ.SRS-006.RBAC.ShowTables.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW TABLES` statement if and only if the user has **show tables** privilege, or any privilege on the table either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ExistsTable +##### RQ.SRS-006.RBAC.ExistsTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `EXISTS table` statement if and only if the user has **show tables** privilege, or any privilege on the table either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.CheckTable +##### RQ.SRS-006.RBAC.CheckTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `CHECK table` statement if and only if the user has **show tables** privilege, or any privilege on the table either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDatabases +##### RQ.SRS-006.RBAC.ShowDatabases.Privilege version: 1.0 [ClickHouse] SHALL grant **show databases** privilege on a database to a user if that user has recieved any grant, including `SHOW DATABASES`, on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDatabases.Query +##### RQ.SRS-006.RBAC.ShowDatabases.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW DATABASES` statement if and only if the user has **show databases** privilege, or any privilege on the database either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateDatabase +##### RQ.SRS-006.RBAC.ShowCreateDatabase.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW CREATE DATABASE` statement if and only if the user has **show databases** privilege, or any privilege on the database either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.UseDatabase +##### RQ.SRS-006.RBAC.UseDatabase.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `USE database` statement if and only if the user has **show databases** privilege, or any privilege on the database either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowColumns +##### RQ.SRS-006.RBAC.ShowColumns.Privilege version: 1.0 -[ClickHouse] SHALL grant **show columns** privilege to a user if and only if it is granted directly or through a role. +[ClickHouse] SHALL support granting or revoking the `SHOW COLUMNS` privilege. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateTable +##### RQ.SRS-006.RBAC.ShowCreateTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW CREATE TABLE` statement if and only if the user has **show columns** privilege on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.DescribeTable +##### RQ.SRS-006.RBAC.DescribeTable.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `DESCRIBE table` statement if and only if the user has **show columns** privilege on that table, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDictionaries +##### RQ.SRS-006.RBAC.ShowDictionaries.Privilege version: 1.0 [ClickHouse] SHALL grant **show dictionaries** privilege on a dictionary to a user if that user has recieved any grant, including `SHOW DICTIONARIES`, on that dictionary, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowDictionaries.Query +##### RQ.SRS-006.RBAC.ShowDictionaries.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW DICTIONARIES` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateDictionary +##### RQ.SRS-006.RBAC.ShowCreateDictionary.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW CREATE DICTIONARY` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ExistsDictionary +##### RQ.SRS-006.RBAC.ExistsDictionary.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `EXISTS dictionary` statement if and only if the user has **show dictionaries** privilege, or any privilege on the dictionary either directly or through a role. +#### Access Management + ##### RQ.SRS-006.RBAC.Privileges.CreateUser version: 1.0 @@ -13735,215 +4036,404 @@ version: 1.0 [ClickHouse] SHALL successfully execute any role grant or revoke by a user with `ROLE ADMIN` privilege. -##### RQ.SRS-006.RBAC.Privileges.ShowUsers +##### Show Access + +###### RQ.SRS-006.RBAC.ShowUsers.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW USERS` privilege when the user is granted `SHOW USERS`, `SHOW CREATE USER`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowUsers.Query +###### RQ.SRS-006.RBAC.ShowUsers.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW USERS` statement if and only if the user has **show users** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateUser +###### RQ.SRS-006.RBAC.ShowCreateUser.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show create user** privilege, +[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show users** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowRoles +###### RQ.SRS-006.RBAC.ShowRoles.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW ROLES` privilege when the user is granted `SHOW ROLES`, `SHOW CREATE ROLE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowRoles.Query +###### RQ.SRS-006.RBAC.ShowRoles.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW ROLES` statement if and only if the user has **show roles** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateRole +###### RQ.SRS-006.RBAC.ShowCreateRole.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show create role** privilege, +[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show roles** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies +###### RQ.SRS-006.RBAC.ShowRowPolicies.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW ROW POLICIES` privilege when the user is granted `SHOW ROW POLICIES`, `SHOW POLICIES`, `SHOW CREATE ROW POLICY`, `SHOW CREATE POLICY`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowRowPolicies.Query +###### RQ.SRS-006.RBAC.ShowRowPolicies.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if the user has **show row policies** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if +the user has **show row policies** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateRowPolicy +###### RQ.SRS-006.RBAC.ShowCreateRowPolicy.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement if and only if the user has **show create row policy** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement +if and only if the user has **show row policies** privilege,either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowQuotas +###### RQ.SRS-006.RBAC.ShowQuotas.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW QUOTAS` privilege when the user is granted `SHOW QUOTAS`, `SHOW CREATE QUOTA`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowQuotas.Query +###### RQ.SRS-006.RBAC.ShowQuotas.RequiredPrivilege version: 1.0 [ClickHouse] SHALL successfully execute `SHOW QUOTAS` statement if and only if the user has **show quotas** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateQuota +###### RQ.SRS-006.RBAC.ShowCreateQuota.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if the user has **show create quota** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if +the user has **show quotas** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles +###### RQ.SRS-006.RBAC.ShowSettingsProfiles.Privilege version: 1.0 [ClickHouse] SHALL successfully grant `SHOW SETTINGS PROFILES` privilege when the user is granted `SHOW SETTINGS PROFILES`, `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`, `SHOW SETTINGS PROFILE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`. -##### RQ.SRS-006.RBAC.Privileges.ShowSettingsProfiles.Query +###### RQ.SRS-006.RBAC.ShowSettingsProfiles.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement if and only if the user has **show settings profiles** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement +if and only if the user has **show settings profiles** privilege, either directly or through a role. -##### RQ.SRS-006.RBAC.Privileges.ShowCreateSettingsProfile +###### RQ.SRS-006.RBAC.ShowCreateSettingsProfile.RequiredPrivilege version: 1.0 -[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement if and only if the user has **show create settings profile** privilege, -either directly or through a role. +[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement +if and only if the user has **show settings profiles** privilege, either directly or through a role. -##### Grant Option +#### dictGet -###### RQ.SRS-006.RBAC.Privileges.GrantOption +##### RQ.SRS-006.RBAC.dictGet.Privilege +version: 1.0 + +[ClickHouse] SHALL successfully grant `dictGet` privilege when +the user is granted `dictGet`, `dictHas`, `dictGetHierarchy`, or `dictIsIn`. + +##### RQ.SRS-006.RBAC.dictGet.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGet` statement +if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictGet.Type.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGet[TYPE]` statement +if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role. +Available types: + +* Int8 +* Int16 +* Int32 +* Int64 +* UInt8 +* UInt16 +* UInt32 +* UInt64 +* Float32 +* Float64 +* Date +* DateTime +* UUID +* String + +##### RQ.SRS-006.RBAC.dictGet.OrDefault.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGetOrDefault` statement +if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictHas.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictHas` statement +if and only if the user has **dictGet** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictGetHierarchy.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictGetHierarchy` statement +if and only if the user has **dictGet** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.dictIsIn.RequiredPrivilege +version: 1.0 + +[ClickHouse] SHALL successfully execute `dictIsIn` statement +if and only if the user has **dictGet** privilege, either directly or through a role. + +#### Introspection + +##### RQ.SRS-006.RBAC.Privileges.Introspection +version: 1.0 + +[ClickHouse] SHALL successfully grant `INTROSPECTION` privilege when +the user is granted `INTROSPECTION` or `INTROSPECTION FUNCTIONS`. + +##### RQ.SRS-006.RBAC.Privileges.Introspection.addressToLine +version: 1.0 + +[ClickHouse] SHALL successfully execute `addressToLine` statement if and only if +the user has **introspection** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Introspection.addressToSymbol +version: 1.0 + +[ClickHouse] SHALL successfully execute `addressToSymbol` statement if and only if +the user has **introspection** privilege, either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Introspection.demangle +version: 1.0 + +[ClickHouse] SHALL successfully execute `demangle` statement if and only if +the user has **introspection** privilege, either directly or through a role. + +#### System + +##### RQ.SRS-006.RBAC.Privileges.System.Shutdown +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM SHUTDOWN` privilege when +the user is granted `SYSTEM`, `SYSTEM SHUTDOWN`, `SHUTDOWN`,or `SYSTEM KILL`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, or `DROP CACHE`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache.DNS +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP DNS CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP DNS CACHE`, +`SYSTEM DROP DNS`, `DROP DNS CACHE`, or `DROP DNS`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache.Mark +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP MARK CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP MARK CACHE`, +`SYSTEM DROP MARK`, `DROP MARK CACHE`, or `DROP MARKS`. + +##### RQ.SRS-006.RBAC.Privileges.System.DropCache.Uncompressed +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DROP UNCOMPRESSED CACHE` privilege when +the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP UNCOMPRESSED CACHE`, +`SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, or `DROP UNCOMPRESSED`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD` privilege when +the user is granted `SYSTEM` or `SYSTEM RELOAD`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.Config +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD CONFIG` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD CONFIG`, or `RELOAD CONFIG`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionary +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD DICTIONARY` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARIES`, or `RELOAD DICTIONARY`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionaries +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD DICTIONARIES` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARIES`, or `RELOAD DICTIONARY`. + +##### RQ.SRS-006.RBAC.Privileges.System.Reload.EmbeddedDictionaries +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilege when +the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARY ON *.*`, or `SYSTEM RELOAD EMBEDDED DICTIONARIES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Merges +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM MERGES` privilege when +the user is granted `SYSTEM`, `SYSTEM MERGES`, `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, or `START MERGES`. + +##### RQ.SRS-006.RBAC.Privileges.System.TTLMerges +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM TTL MERGES` privilege when +the user is granted `SYSTEM`, `SYSTEM TTL MERGES`, `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, or `START TTL MERGES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Fetches +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FETCHES` privilege when +the user is granted `SYSTEM`, `SYSTEM FETCHES`, `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, or `START FETCHES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Moves +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM MOVES` privilege when +the user is granted `SYSTEM`, `SYSTEM MOVES`, `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, or `START MOVES`. + +##### RQ.SRS-006.RBAC.Privileges.System.Sends +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM SENDS` privilege when +the user is granted `SYSTEM`, `SYSTEM SENDS`, `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, or `START SENDS`. + +##### RQ.SRS-006.RBAC.Privileges.System.Sends.Distributed +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM DISTRIBUTED SENDS` privilege when +the user is granted `SYSTEM`, `SYSTEM DISTRIBUTED SENDS`, `SYSTEM STOP DISTRIBUTED SENDS`, +`SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, or `START DISTRIBUTED SENDS`. + +##### RQ.SRS-006.RBAC.Privileges.System.Sends.Replicated +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM REPLICATED SENDS` privilege when +the user is granted `SYSTEM`, `SYSTEM REPLICATED SENDS`, `SYSTEM STOP REPLICATED SENDS`, +`SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, or `START REPLICATED SENDS`. + +##### RQ.SRS-006.RBAC.Privileges.System.ReplicationQueues +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM REPLICATION QUEUES` privilege when +the user is granted `SYSTEM`, `SYSTEM REPLICATION QUEUES`, `SYSTEM STOP REPLICATION QUEUES`, +`SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, or `START REPLICATION QUEUES`. + +##### RQ.SRS-006.RBAC.Privileges.System.SyncReplica +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM SYNC REPLICA` privilege when +the user is granted `SYSTEM`, `SYSTEM SYNC REPLICA`, or `SYNC REPLICA`. + +##### RQ.SRS-006.RBAC.Privileges.System.RestartReplica +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM RESTART REPLICA` privilege when +the user is granted `SYSTEM`, `SYSTEM RESTART REPLICA`, or `RESTART REPLICA`. + +##### RQ.SRS-006.RBAC.Privileges.System.Flush +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FLUSH` privilege when +the user is granted `SYSTEM` or `SYSTEM FLUSH`. + +##### RQ.SRS-006.RBAC.Privileges.System.Flush.Distributed +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FLUSH DISTRIBUTED` privilege when +the user is granted `SYSTEM`, `SYSTEM FLUSH DISTRIBUTED`, or `FLUSH DISTRIBUTED`. + +##### RQ.SRS-006.RBAC.Privileges.System.Flush.Logs +version: 1.0 + +[ClickHouse] SHALL successfully grant `SYSTEM FLUSH LOGS` privilege when +the user is granted `SYSTEM`, `SYSTEM FLUSH LOGS`, or `FLUSH LOGS`. + +#### Sources + +##### RQ.SRS-006.RBAC.Privileges.Sources +version: 1.0 + +[ClickHouse] SHALL support granting or revoking `SOURCES` privilege from +the user, either directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.File +version: 1.0 + +[ClickHouse] SHALL support the use of `FILE` source by a user if and only if +the user has `FILE` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.URL +version: 1.0 + +[ClickHouse] SHALL support the use of `URL` source by a user if and only if +the user has `URL` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.Remote +version: 1.0 + +[ClickHouse] SHALL support the use of `REMOTE` source by a user if and only if +the user has `REMOTE` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.MySQL +version: 1.0 + +[ClickHouse] SHALL support the use of `MySQL` source by a user if and only if +the user has `MySQL` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.ODBC +version: 1.0 + +[ClickHouse] SHALL support the use of `ODBC` source by a user if and only if +the user has `ODBC` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.JDBC +version: 1.0 + +[ClickHouse] SHALL support the use of `JDBC` source by a user if and only if +the user has `JDBC` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.HDFS +version: 1.0 + +[ClickHouse] SHALL support the use of `HDFS` source by a user if and only if +the user has `HDFS` or `SOURCES` privileges granted to them directly or through a role. + +##### RQ.SRS-006.RBAC.Privileges.Sources.S3 +version: 1.0 + +[ClickHouse] SHALL support the use of `S3` source by a user if and only if +the user has `S3` or `SOURCES` privileges granted to them directly or through a role. + +#### RQ.SRS-006.RBAC.Privileges.GrantOption version: 1.0 [ClickHouse] SHALL successfully execute `GRANT` or `REVOKE` privilege statements by a user if and only if the user has that privilege with `GRANT OPTION`, either directly or through a role. -`GRANT OPTION` is supported by the following privileges - -* `ALTER MOVE PARTITION` -* `ALTER FETCH PARTITION` -* `ALTER FREEZE PARTITION` -* `ALTER DELETE` -* `ALTER UPDATE` -* `ALTER SETTINGS` -* `ALTER TTL` -* `ALTER CONSTRAINT` -* `ALTER COLUMN` -* `ALTER INDEX` -* `INSERT` -* `SELECT` -* `CREATE TABLE` -* `CREATE VIEW` -* `CREATE DATABASE` -* `CREATE DICTIONARY` -* `CREATE TEMPORARY TABLE` -* `DROP TABLE` -* `DROP VIEW` -* `DROP DATABASE` -* `DROP DICTIONARY` - -##### RQ.SRS-006.RBAC.Privileges.All +#### RQ.SRS-006.RBAC.Privileges.All version: 1.0 -[ClickHouse] SHALL include in the **all** privilege the same rights -as provided by **usage**, **select**, **select columns**, -**insert**, **delete**, **alter**, **create**, and **drop** privileges. +[ClickHouse] SHALL support granting or revoking `ALL` privilege. -##### RQ.SRS-006.RBAC.Privileges.All.GrantRevoke +#### RQ.SRS-006.RBAC.Privileges.AdminOption version: 1.0 -[ClickHouse] SHALL support granting or revoking **all** privileges -for a database or a specific table to one or more **users** or **roles**. - -##### RQ.SRS-006.RBAC.Privileges.AdminOption -version: 1.0 - -[ClickHouse] SHALL support granting or revoking **admin option** privilege -to one or more **users** or **roles**. - -#### Required Privileges - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Create -version: 1.0 - -[ClickHouse] SHALL not allow any `CREATE` statements -to be executed unless the user has the **create** privilege for the destination database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Alter -version: 1.0 - -[ClickHouse] SHALL not allow any `ALTER` statements -to be executed unless the user has the **alter** privilege for the destination table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop -version: 1.0 - -[ClickHouse] SHALL not allow any `DROP` statements -to be executed unless the user has the **drop** privilege for the destination database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Drop.Table -version: 1.0 - -[ClickHouse] SHALL not allow any `DROP TABLE` statements -to be executed unless the user has the **drop** privilege for the destination database or the table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.GrantRevoke -version: 1.0 - -[ClickHouse] SHALL not allow any `GRANT` or `REVOKE` statements -to be executed unless the user has the **grant option** privilege -for the privilege of the destination table -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Use -version: 1.0 - -[ClickHouse] SHALL not allow the `USE` statement to be executed -unless the user has at least one of the privileges for the database -or the table inside that database -either because of the explicit grant or through one of the roles assigned to the user. - -##### RQ.SRS-006.RBAC.RequiredPrivileges.Admin -version: 1.0 - -[ClickHouse] SHALL not allow any of the following statements - -* `SYSTEM` -* `SHOW` -* `ATTACH` -* `CHECK TABLE` -* `DESCRIBE TABLE` -* `DETACH` -* `EXISTS` -* `KILL QUERY` -* `KILL MUTATION` -* `OPTIMIZE` -* `RENAME` -* `TRUNCATE` - -to be executed unless the user has the **admin option** privilege -through one of the roles with **admin option** privilege assigned to the user. +[ClickHouse] SHALL support a user granting or revoking a role if and only if +the user has that role with `ADMIN OPTION` privilege. ## References @@ -13961,3 +4451,7845 @@ through one of the roles with **admin option** privilege assigned to the user. [MySQL]: https://dev.mysql.com/doc/refman/8.0/en/account-management-statements.html [PostgreSQL]: https://www.postgresql.org/docs/12/user-manag.html ''') + +RQ_SRS_006_RBAC = Requirement( + name='RQ.SRS-006.RBAC', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support role based access control.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Login = Requirement( + name='RQ.SRS-006.RBAC.Login', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only allow access to the server for a given\n' + 'user only when correct username and password are used during\n' + 'the connection to the server.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Login_DefaultUser = Requirement( + name='RQ.SRS-006.RBAC.Login.DefaultUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL use the **default user** when no username and password\n' + 'are specified during the connection to the server.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User = Requirement( + name='RQ.SRS-006.RBAC.User', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creation and manipulation of\n' + 'one or more **user** accounts to which roles, privileges,\n' + 'settings profile, quotas and row policies can be assigned.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Roles = Requirement( + name='RQ.SRS-006.RBAC.User.Roles', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more **roles**\n' + 'to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Privileges = Requirement( + name='RQ.SRS-006.RBAC.User.Privileges', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more privileges to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Variables = Requirement( + name='RQ.SRS-006.RBAC.User.Variables', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more variables to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Variables_Constraints = Requirement( + name='RQ.SRS-006.RBAC.User.Variables.Constraints', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning min, max and read-only constraints\n' + 'for the variables that can be set and read by the **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_SettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.User.SettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more **settings profiles**\n' + 'to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Quotas = Requirement( + name='RQ.SRS-006.RBAC.User.Quotas', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more **quotas** to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_RowPolicies = Requirement( + name='RQ.SRS-006.RBAC.User.RowPolicies', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more **row policies** to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_AccountLock = Requirement( + name='RQ.SRS-006.RBAC.User.AccountLock', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support locking and unlocking of **user** accounts.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_AccountLock_DenyAccess = Requirement( + name='RQ.SRS-006.RBAC.User.AccountLock.DenyAccess', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL deny access to the user whose account is locked.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_DefaultRole = Requirement( + name='RQ.SRS-006.RBAC.User.DefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning a default role to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_RoleSelection = Requirement( + name='RQ.SRS-006.RBAC.User.RoleSelection', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support selection of one or more **roles** from the available roles\n' + 'that are assigned to a **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_ShowCreate = Requirement( + name='RQ.SRS-006.RBAC.User.ShowCreate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the command of how **user** account was created.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_ShowPrivileges = Requirement( + name='RQ.SRS-006.RBAC.User.ShowPrivileges', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support listing the privileges of the **user**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role = Requirement( + name='RQ.SRS-006.RBAC.Role', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClikHouse] SHALL support creation and manipulation of **roles**\n' + 'to which privileges, settings profile, quotas and row policies can be\n' + 'assigned.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Privileges = Requirement( + name='RQ.SRS-006.RBAC.Role.Privileges', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more privileges to a **role**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Variables = Requirement( + name='RQ.SRS-006.RBAC.Role.Variables', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more variables to a **role**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_SettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Role.SettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more **settings profiles**\n' + 'to a **role**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Quotas = Requirement( + name='RQ.SRS-006.RBAC.Role.Quotas', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more **quotas** to a **role**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_RowPolicies = Requirement( + name='RQ.SRS-006.RBAC.Role.RowPolicies', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning one or more **row policies** to a **role**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_PartialRevokes = Requirement( + name='RQ.SRS-006.RBAC.PartialRevokes', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support partial revoking of privileges granted\n' + 'to a **user** or a **role**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creation and manipulation of **settings profiles**\n' + 'that can include value definition for one or more variables and can\n' + 'can be assigned to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Constraints = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Constraints', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning min, max and read-only constraints\n' + 'for the variables specified in the **settings profile**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_ShowCreate = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.ShowCreate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the command of how **setting profile** was created.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas = Requirement( + name='RQ.SRS-006.RBAC.Quotas', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creation and manipulation of **quotas**\n' + 'that can be used to limit resource usage by a **user** or a **role**\n' + 'over a period of time.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_Keyed = Requirement( + name='RQ.SRS-006.RBAC.Quotas.Keyed', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating **quotas** that are keyed\n' + 'so that a quota is tracked separately for each key value.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_Queries = Requirement( + name='RQ.SRS-006.RBAC.Quotas.Queries', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting **queries** quota to limit the total number of requests.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_Errors = Requirement( + name='RQ.SRS-006.RBAC.Quotas.Errors', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting **errors** quota to limit the number of queries that threw an exception.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_ResultRows = Requirement( + name='RQ.SRS-006.RBAC.Quotas.ResultRows', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting **result rows** quota to limit the\n' + 'the total number of rows given as the result.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_ReadRows = Requirement( + name='RQ.SRS-006.RBAC.Quotas.ReadRows', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting **read rows** quota to limit the total\n' + 'number of source rows read from tables for running the query on all remote servers.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_ResultBytes = Requirement( + name='RQ.SRS-006.RBAC.Quotas.ResultBytes', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting **result bytes** quota to limit the total number\n' + 'of bytes that can be returned as the result.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_ReadBytes = Requirement( + name='RQ.SRS-006.RBAC.Quotas.ReadBytes', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting **read bytes** quota to limit the total number\n' + 'of source bytes read from tables for running the query on all remote servers.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_ExecutionTime = Requirement( + name='RQ.SRS-006.RBAC.Quotas.ExecutionTime', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting **execution time** quota to limit the maximum\n' + 'query execution time.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quotas_ShowCreate = Requirement( + name='RQ.SRS-006.RBAC.Quotas.ShowCreate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the command of how **quota** was created.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creation and manipulation of table **row policies**\n' + 'that can be used to limit access to the table contents for a **user** or a **role**\n' + 'using a specified **condition**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Condition = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Condition', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support row policy **conditions** that can be any SQL\n' + 'expression that returns a boolean.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_ShowCreate = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.ShowCreate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the command of how **row policy** was created.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Use_DefaultRole = Requirement( + name='RQ.SRS-006.RBAC.User.Use.DefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL by default use default role or roles assigned\n' + 'to the user if specified.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Use_AllRolesWhenNoDefaultRole = Requirement( + name='RQ.SRS-006.RBAC.User.Use.AllRolesWhenNoDefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL by default use all the roles assigned to the user\n' + 'if no default role or roles are specified for the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create = Requirement( + name='RQ.SRS-006.RBAC.User.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating **user** accounts using `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_IfNotExists = Requirement( + name='RQ.SRS-006.RBAC.User.Create.IfNotExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE USER` statement\n' + 'to skip raising an exception if a user with the same **name** already exists.\n' + 'If the `IF NOT EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if a user with the same **name** already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Replace = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Replace', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE USER` statement\n' + 'to replace existing user account if already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_NoPassword = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.NoPassword', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying no password when creating\n' + 'user account using `IDENTIFIED WITH NO_PASSWORD` clause .\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_NoPassword_Login = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.NoPassword.Login', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL use no password for the user when connecting to the server\n' + 'when an account was created with `IDENTIFIED WITH NO_PASSWORD` clause.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_PlainText = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.PlainText', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying plaintext password when creating\n' + 'user account using `IDENTIFIED WITH PLAINTEXT_PASSWORD BY` clause.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_PlainText_Login = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.PlainText.Login', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL use the plaintext password passed by the user when connecting to the server\n' + 'when an account was created with `IDENTIFIED WITH PLAINTEXT_PASSWORD` clause\n' + 'and compare the password with the one used in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_Sha256Password = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Password', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the result of applying SHA256\n' + 'to some password when creating user account using `IDENTIFIED WITH SHA256_PASSWORD BY` or `IDENTIFIED BY`\n' + 'clause.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_Sha256Password_Login = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Password.Login', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL calculate `SHA256` of the password passed by the user when connecting to the server\n' + "when an account was created with `IDENTIFIED WITH SHA256_PASSWORD` or with 'IDENTIFIED BY' clause\n" + 'and compare the calculated hash to the one used in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_Sha256Hash = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the result of applying SHA256\n' + 'to some already calculated hash when creating user account using `IDENTIFIED WITH SHA256_HASH`\n' + 'clause.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_Sha256Hash_Login = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.Sha256Hash.Login', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL calculate `SHA256` of the already calculated hash passed by\n' + 'the user when connecting to the server\n' + 'when an account was created with `IDENTIFIED WITH SHA256_HASH` clause\n' + 'and compare the calculated hash to the one used in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Password = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the result of applying SHA1 two times\n' + 'to a password when creating user account using `IDENTIFIED WITH DOUBLE_SHA1_PASSWORD`\n' + 'clause.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Password_Login = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Password.Login', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL calculate `SHA1` two times over the password passed by\n' + 'the user when connecting to the server\n' + 'when an account was created with `IDENTIFIED WITH DOUBLE_SHA1_PASSWORD` clause\n' + 'and compare the calculated value to the one used in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Hash = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the result of applying SHA1 two times\n' + 'to a hash when creating user account using `IDENTIFIED WITH DOUBLE_SHA1_HASH`\n' + 'clause.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Password_DoubleSha1Hash_Login = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Password.DoubleSha1Hash.Login', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL calculate `SHA1` two times over the hash passed by\n' + 'the user when connecting to the server\n' + 'when an account was created with `IDENTIFIED WITH DOUBLE_SHA1_HASH` clause\n' + 'and compare the calculated value to the one used in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_Name = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.Name', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more hostnames from\n' + 'which user can access the server using the `HOST NAME` clause\n' + 'in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_Regexp = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.Regexp', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more regular expressions\n' + 'to match hostnames from which user can access the server\n' + 'using the `HOST REGEXP` clause in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_IP = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.IP', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more IP address or subnet from\n' + 'which user can access the server using the `HOST IP` clause in the\n' + '`CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_Any = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.Any', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying `HOST ANY` clause in the `CREATE USER` statement\n' + 'to indicate that user can access the server from any host.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_None = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support fobidding access from any host using `HOST NONE` clause in the\n' + '`CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_Local = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.Local', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting user access to local only using `HOST LOCAL` clause in the\n' + '`CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_Like = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.Like', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying host using `LIKE` command syntax using the\n' + '`HOST LIKE` clause in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Host_Default = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Host.Default', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support user access to server from any host\n' + 'if no `HOST` clause is specified in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_DefaultRole = Requirement( + name='RQ.SRS-006.RBAC.User.Create.DefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more default roles\n' + 'using `DEFAULT ROLE` clause in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_DefaultRole_None = Requirement( + name='RQ.SRS-006.RBAC.User.Create.DefaultRole.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying no default roles\n' + 'using `DEFAULT ROLE NONE` clause in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_DefaultRole_All = Requirement( + name='RQ.SRS-006.RBAC.User.Create.DefaultRole.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying all roles to be used as default\n' + 'using `DEFAULT ROLE ALL` clause in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Settings = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Settings', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying settings and profile\n' + 'using `SETTINGS` clause in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.User.Create.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying cluster on which the user\n' + 'will be created using `ON CLUSTER` clause in the `CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Create_Syntax = Requirement( + name='RQ.SRS-006.RBAC.User.Create.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for `CREATE USER` statement.\n' + '\n' + '```sql\n' + 'CREATE USER [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]\n' + " [IDENTIFIED [WITH {NO_PASSWORD|PLAINTEXT_PASSWORD|SHA256_PASSWORD|SHA256_HASH|DOUBLE_SHA1_PASSWORD|DOUBLE_SHA1_HASH}] BY {'password'|'hash'}]\n" + " [HOST {LOCAL | NAME 'name' | NAME REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]\n" + ' [DEFAULT ROLE role [,...]]\n' + " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter = Requirement( + name='RQ.SRS-006.RBAC.User.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering **user** accounts using `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_OrderOfEvaluation = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.OrderOfEvaluation', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support evaluating `ALTER USER` statement from left to right\n' + 'where things defined on the right override anything that was previously defined on\n' + 'the left.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_IfExists = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF EXISTS` clause in the `ALTER USER` statement\n' + 'to skip raising an exception (producing a warning instead) if a user with the specified **name** does not exist. If the `IF EXISTS` clause is not specified then an exception SHALL be raised if a user with the **name** does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Cluster = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the cluster the user is on\n' + 'when altering user account using `ON CLUSTER` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Rename = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Rename', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying a new name for the user when\n' + 'altering user account using `RENAME` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Password_PlainText = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Password.PlainText', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying plaintext password when altering\n' + 'user account using `IDENTIFIED WITH PLAINTEXT_PASSWORD BY` or\n' + 'using shorthand `IDENTIFIED BY` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Password_Sha256Password = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Password.Sha256Password', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the result of applying SHA256\n' + 'to some password as identification when altering user account using\n' + '`IDENTIFIED WITH SHA256_PASSWORD` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Password_DoubleSha1Password = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Password.DoubleSha1Password', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the result of applying Double SHA1\n' + 'to some password as identification when altering user account using\n' + '`IDENTIFIED WITH DOUBLE_SHA1_PASSWORD` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_AddDrop = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.AddDrop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering user by adding and dropping access to hosts with the `ADD HOST` or the `DROP HOST`in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_Local = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.Local', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting user access to local only using `HOST LOCAL` clause in the\n' + '`ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_Name = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.Name', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more hostnames from\n' + 'which user can access the server using the `HOST NAME` clause\n' + 'in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_Regexp = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.Regexp', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more regular expressions\n' + 'to match hostnames from which user can access the server\n' + 'using the `HOST REGEXP` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_IP = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.IP', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more IP address or subnet from\n' + 'which user can access the server using the `HOST IP` clause in the\n' + '`ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_Like = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.Like', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying sone or more similar hosts using `LIKE` command syntax using the `HOST LIKE` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_Any = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.Any', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying `HOST ANY` clause in the `ALTER USER` statement\n' + 'to indicate that user can access the server from any host.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Host_None = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Host.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support fobidding access from any host using `HOST NONE` clause in the\n' + '`ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_DefaultRole = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.DefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more default roles\n' + 'using `DEFAULT ROLE` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_DefaultRole_All = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.DefaultRole.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying all roles to be used as default\n' + 'using `DEFAULT ROLE ALL` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_DefaultRole_AllExcept = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.DefaultRole.AllExcept', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more roles which will not be used as default\n' + 'using `DEFAULT ROLE ALL EXCEPT` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Settings = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Settings', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying one or more variables\n' + 'using `SETTINGS` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Settings_Min = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Settings.Min', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying a minimum value for the variable specifed using `SETTINGS` with `MIN` clause in the `ALTER USER` statement.\n' + '\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Settings_Max = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Settings.Max', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying a maximum value for the variable specifed using `SETTINGS` with `MAX` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Settings_Profile = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Settings.Profile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying the name of a profile for the variable specifed using `SETTINGS` with `PROFILE` clause in the `ALTER USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Alter_Syntax = Requirement( + name='RQ.SRS-006.RBAC.User.Alter.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `ALTER USER` statement.\n' + '\n' + '```sql\n' + 'ALTER USER [IF EXISTS] name [ON CLUSTER cluster_name]\n' + ' [RENAME TO new_name]\n' + " [IDENTIFIED [WITH {PLAINTEXT_PASSWORD|SHA256_PASSWORD|DOUBLE_SHA1_PASSWORD}] BY {'password'|'hash'}]\n" + " [[ADD|DROP] HOST {LOCAL | NAME 'name' | REGEXP 'name_regexp' | IP 'address' | LIKE 'pattern'} [,...] | ANY | NONE]\n" + ' [DEFAULT ROLE role [,...] | ALL | ALL EXCEPT role [,...] ]\n' + " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetDefaultRole = Requirement( + name='RQ.SRS-006.RBAC.SetDefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting or changing granted roles to default for one or more\n' + 'users using `SET DEFAULT ROLE` statement which\n' + 'SHALL permanently change the default roles for the user or users if successful.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetDefaultRole_CurrentUser = Requirement( + name='RQ.SRS-006.RBAC.SetDefaultRole.CurrentUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting or changing granted roles to default for\n' + 'the current user using `CURRENT_USER` clause in the `SET DEFAULT ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetDefaultRole_All = Requirement( + name='RQ.SRS-006.RBAC.SetDefaultRole.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting or changing all granted roles to default\n' + 'for one or more users using `ALL` clause in the `SET DEFAULT ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetDefaultRole_AllExcept = Requirement( + name='RQ.SRS-006.RBAC.SetDefaultRole.AllExcept', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting or changing all granted roles except those specified\n' + 'to default for one or more users using `ALL EXCEPT` clause in the `SET DEFAULT ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetDefaultRole_None = Requirement( + name='RQ.SRS-006.RBAC.SetDefaultRole.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing all granted roles from default\n' + 'for one or more users using `NONE` clause in the `SET DEFAULT ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetDefaultRole_Syntax = Requirement( + name='RQ.SRS-006.RBAC.SetDefaultRole.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `SET DEFAULT ROLE` statement.\n' + '\n' + '```sql\n' + 'SET DEFAULT ROLE\n' + ' {NONE | role [,...] | ALL | ALL EXCEPT role [,...]}\n' + ' TO {user|CURRENT_USER} [,...]\n' + '\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetRole = Requirement( + name='RQ.SRS-006.RBAC.SetRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support activating role or roles for the current user\n' + 'using `SET ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetRole_Default = Requirement( + name='RQ.SRS-006.RBAC.SetRole.Default', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support activating default roles for the current user\n' + 'using `DEFAULT` clause in the `SET ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetRole_None = Requirement( + name='RQ.SRS-006.RBAC.SetRole.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support activating no roles for the current user\n' + 'using `NONE` clause in the `SET ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetRole_All = Requirement( + name='RQ.SRS-006.RBAC.SetRole.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support activating all roles for the current user\n' + 'using `ALL` clause in the `SET ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetRole_AllExcept = Requirement( + name='RQ.SRS-006.RBAC.SetRole.AllExcept', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support activating all roles except those specified\n' + 'for the current user using `ALL EXCEPT` clause in the `SET ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SetRole_Syntax = Requirement( + name='RQ.SRS-006.RBAC.SetRole.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '```sql\n' + 'SET ROLE {DEFAULT | NONE | role [,...] | ALL | ALL EXCEPT role [,...]}\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_ShowCreateUser = Requirement( + name='RQ.SRS-006.RBAC.User.ShowCreateUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the `CREATE USER` statement used to create the current user object\n' + 'using the `SHOW CREATE USER` statement with `CURRENT_USER` or no argument.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_ShowCreateUser_For = Requirement( + name='RQ.SRS-006.RBAC.User.ShowCreateUser.For', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the `CREATE USER` statement used to create the specified user object\n' + 'using the `FOR` clause in the `SHOW CREATE USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_ShowCreateUser_Syntax = Requirement( + name='RQ.SRS-006.RBAC.User.ShowCreateUser.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the following syntax for `SHOW CREATE USER` statement.\n' + '\n' + '```sql\n' + 'SHOW CREATE USER [name | CURRENT_USER]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Drop = Requirement( + name='RQ.SRS-006.RBAC.User.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing a user account using `DROP USER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Drop_IfExists = Requirement( + name='RQ.SRS-006.RBAC.User.Drop.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP USER` statement\n' + 'to skip raising an exception if the user account does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if a user does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Drop_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.User.Drop.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using `ON CLUSTER` clause in the `DROP USER` statement\n' + 'to specify the name of the cluster the user should be dropped from.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_User_Drop_Syntax = Requirement( + name='RQ.SRS-006.RBAC.User.Drop.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for `DROP USER` statement\n' + '\n' + '```sql\n' + 'DROP USER [IF EXISTS] name [,...] [ON CLUSTER cluster_name]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Create = Requirement( + name='RQ.SRS-006.RBAC.Role.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating a **role** using `CREATE ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Create_IfNotExists = Requirement( + name='RQ.SRS-006.RBAC.Role.Create.IfNotExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE ROLE` statement\n' + 'to raising an exception if a role with the same **name** already exists.\n' + 'If the `IF NOT EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if a role with the same **name** already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Create_Replace = Requirement( + name='RQ.SRS-006.RBAC.Role.Create.Replace', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE ROLE` statement\n' + 'to replace existing role if it already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Create_Settings = Requirement( + name='RQ.SRS-006.RBAC.Role.Create.Settings', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying settings and profile using `SETTINGS`\n' + 'clause in the `CREATE ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Create_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Role.Create.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `CREATE ROLE` statement\n' + '\n' + '``` sql\n' + 'CREATE ROLE [IF NOT EXISTS | OR REPLACE] name\n' + " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Alter = Requirement( + name='RQ.SRS-006.RBAC.Role.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Alter_IfExists = Requirement( + name='RQ.SRS-006.RBAC.Role.Alter.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE IF EXISTS` statement, where no exception\n' + 'will be thrown if the role does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Alter_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Role.Alter.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE role ON CLUSTER` statement to specify the\n' + 'cluster location of the specified role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Alter_Rename = Requirement( + name='RQ.SRS-006.RBAC.Role.Alter.Rename', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering one **role** using `ALTER ROLE role RENAME TO` statement which renames the\n' + 'role to a specified new name. If the new name already exists, that an exception SHALL be raised unless the\n' + '`IF EXISTS` clause is specified, by which no exception will be raised and nothing will change.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Alter_Settings = Requirement( + name='RQ.SRS-006.RBAC.Role.Alter.Settings', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the settings of one **role** using `ALTER ROLE role SETTINGS ...` statement.\n' + 'Altering variable values, creating max and min values, specifying readonly or writable, and specifying the\n' + 'profiles for which this alter change shall be applied to, are all supported, using the following syntax.\n' + '\n' + '```sql\n' + "[SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" + '```\n' + '\n' + 'One or more variables and profiles may be specified as shown above.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Alter_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Role.Alter.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '```sql\n' + 'ALTER ROLE [IF EXISTS] name [ON CLUSTER cluster_name]\n' + ' [RENAME TO new_name]\n' + " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | PROFILE 'profile_name'] [,...]\n" + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Drop = Requirement( + name='RQ.SRS-006.RBAC.Role.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing one or more roles using `DROP ROLE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Drop_IfExists = Requirement( + name='RQ.SRS-006.RBAC.Role.Drop.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP ROLE` statement\n' + 'to skip raising an exception if the role does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if a role does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Drop_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Role.Drop.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using `ON CLUSTER` clause in the `DROP ROLE` statement to specify the cluster from which to drop the specified role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_Drop_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Role.Drop.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `DROP ROLE` statement\n' + '\n' + '``` sql\n' + 'DROP ROLE [IF EXISTS] name [,...] [ON CLUSTER cluster_name]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_ShowCreate = Requirement( + name='RQ.SRS-006.RBAC.Role.ShowCreate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support viewing the settings for a role upon creation with the `SHOW CREATE ROLE`\n' + 'statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Role_ShowCreate_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Role.ShowCreate.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `SHOW CREATE ROLE` command.\n' + '\n' + '```sql\n' + 'SHOW CREATE ROLE name\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_To = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.To', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting privileges to one or more users or roles using `TO` clause\n' + 'in the `GRANT PRIVILEGE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_ToCurrentUser = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.ToCurrentUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting privileges to current user using `TO CURRENT_USER` clause\n' + 'in the `GRANT PRIVILEGE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Select = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **select** privilege to one or more users or roles\n' + 'for a database or a table using the `GRANT SELECT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Insert = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Insert', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **insert** privilege to one or more users or roles\n' + 'for a database or a table using the `GRANT INSERT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Alter = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **alter** privilege to one or more users or roles\n' + 'for a database or a table using the `GRANT ALTER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Create = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **create** privilege to one or more users or roles\n' + 'using the `GRANT CREATE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Drop = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **drop** privilege to one or more users or roles\n' + 'using the `GRANT DROP` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Truncate = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Truncate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **truncate** privilege to one or more users or roles\n' + 'for a database or a table using `GRANT TRUNCATE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Optimize = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Optimize', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **optimize** privilege to one or more users or roles\n' + 'for a database or a table using `GRANT OPTIMIZE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Show = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Show', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **show** privilege to one or more users or roles\n' + 'for a database or a table using `GRANT SHOW` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_KillQuery = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.KillQuery', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **kill query** privilege to one or more users or roles\n' + 'for a database or a table using `GRANT KILL QUERY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_AccessManagement = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.AccessManagement', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **access management** privileges to one or more users or roles\n' + 'for a database or a table using `GRANT ACCESS MANAGEMENT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_System = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.System', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **system** privileges to one or more users or roles\n' + 'for a database or a table using `GRANT SYSTEM` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Introspection = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Introspection', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **introspection** privileges to one or more users or roles\n' + 'for a database or a table using `GRANT INTROSPECTION` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Sources = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Sources', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **sources** privileges to one or more users or roles\n' + 'for a database or a table using `GRANT SOURCES` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_DictGet = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.DictGet', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **dictGet** privilege to one or more users or roles\n' + 'for a database or a table using `GRANT dictGet` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_None = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting no privileges to one or more users or roles\n' + 'for a database or a table using `GRANT NONE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_All = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **all** privileges to one or more users or roles\n' + 'for a database or a table using the `GRANT ALL` or `GRANT ALL PRIVILEGES` statements.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_GrantOption = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.GrantOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the **grant option** privilege to one or more users or roles\n' + 'for a database or a table using the `WITH GRANT OPTION` clause in the `GRANT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_On = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.On', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `ON` clause in the `GRANT` privilege statement\n' + 'which SHALL allow to specify one or more tables to which the privilege SHALL\n' + 'be granted using the following patterns\n' + '\n' + '* `*.*` any table in any database\n' + '* `database.*` any table in the specified database\n' + '* `database.table` specific table in the specified database\n' + '* `*` any table in the current database\n' + '* `table` specific table in the current database\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_PrivilegeColumns = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.PrivilegeColumns', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting the privilege **some_privilege** to one or more users or roles\n' + 'for a database or a table using the `GRANT some_privilege(column)` statement for one column.\n' + 'Multiple columns will be supported with `GRANT some_privilege(column1, column2...)` statement.\n' + 'The privileges will be granted for only the specified columns.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying cluster on which to grant privileges using the `ON CLUSTER`\n' + 'clause in the `GRANT PRIVILEGE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Privilege_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Grant.Privilege.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `GRANT` statement that\n' + 'grants explicit privileges to a user or a role.\n' + '\n' + '```sql\n' + 'GRANT [ON CLUSTER cluster_name] privilege[(column_name [,...])] [,...]\n' + ' ON {db.table|db.*|*.*|table|*}\n' + ' TO {user | role | CURRENT_USER} [,...]\n' + ' [WITH GRANT OPTION]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking privileges to one or more users or roles\n' + 'for a database or a table on some specific cluster using the `REVOKE ON CLUSTER cluster_name` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Any = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Any', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking ANY privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE some_privilege` statement.\n' + '**some_privilege** refers to any Clickhouse defined privilege, whose hierarchy includes\n' + 'SELECT, INSERT, ALTER, CREATE, DROP, TRUNCATE, OPTIMIZE, SHOW, KILL QUERY, ACCESS MANAGEMENT,\n' + 'SYSTEM, INTROSPECTION, SOURCES, dictGet and all of their sub-privileges.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Select = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **select** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE SELECT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Insert = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Insert', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **insert** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE INSERT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Alter = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **alter** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE ALTER` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Create = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **create** privilege to one or more users or roles\n' + 'using the `REVOKE CREATE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Drop = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **drop** privilege to one or more users or roles\n' + 'using the `REVOKE DROP` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Truncate = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Truncate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **truncate** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE TRUNCATE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Optimize = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Optimize', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **optimize** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE OPTIMIZE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Show = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Show', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **show** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE SHOW` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_KillQuery = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.KillQuery', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **kill query** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE KILL QUERY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_AccessManagement = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.AccessManagement', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **access management** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE ACCESS MANAGEMENT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_System = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.System', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **system** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE SYSTEM` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Introspection = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Introspection', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **introspection** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE INTROSPECTION` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Sources = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Sources', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **sources** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE SOURCES` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_DictGet = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.DictGet', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the **dictGet** privilege to one or more users or roles\n' + 'for a database or a table using the `REVOKE dictGet` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_PrivelegeColumns = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.PrivelegeColumns', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking the privilege **some_privilege** to one or more users or roles\n' + 'for a database or a table using the `REVOKE some_privilege(column)` statement for one column.\n' + 'Multiple columns will be supported with `REVOKE some_privilege(column1, column2...)` statement.\n' + 'The privileges will be revoked for only the specified columns.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Multiple = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Multiple', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking MULTIPLE **privileges** to one or more users or roles\n' + 'for a database or a table using the `REVOKE privilege1, privilege2...` statement.\n' + '**privileges** refers to any set of Clickhouse defined privilege, whose hierarchy includes\n' + 'SELECT, INSERT, ALTER, CREATE, DROP, TRUNCATE, OPTIMIZE, SHOW, KILL QUERY, ACCESS MANAGEMENT,\n' + 'SYSTEM, INTROSPECTION, SOURCES, dictGet and all of their sub-privileges.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_All = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **all** privileges to one or more users or roles\n' + 'for a database or a table using the `REVOKE ALL` or `REVOKE ALL PRIVILEGES` statements.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_None = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **no** privileges to one or more users or roles\n' + 'for a database or a table using the `REVOKE NONE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_On = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.On', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `ON` clause in the `REVOKE` privilege statement\n' + 'which SHALL allow to specify one or more tables to which the privilege SHALL\n' + 'be revoked using the following patterns\n' + '\n' + '* `db.table` specific table in the specified database\n' + '* `db.*` any table in the specified database\n' + '* `*.*` any table in any database\n' + '* `table` specific table in the current database\n' + '* `*` any table in the current database\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_From = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.From', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `FROM` clause in the `REVOKE` privilege statement\n' + 'which SHALL allow to specify one or more users to which the privilege SHALL\n' + 'be revoked using the following patterns\n' + '\n' + '* `{user | CURRENT_USER} [,...]` some combination of users by name, which may include the current user\n' + '* `ALL` all users\n' + '* `ALL EXCEPT {user | CURRENT_USER} [,...]` the logical reverse of the first pattern\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Privilege_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Privilege.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `REVOKE` statement that\n' + 'revokes explicit privileges of a user or a role.\n' + '\n' + '```sql\n' + 'REVOKE [ON CLUSTER cluster_name] privilege\n' + ' [(column_name [,...])] [,...]\n' + ' ON {db.table|db.*|*.*|table|*}\n' + ' FROM {user | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user | CURRENT_USER} [,...]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_PartialRevoke_Syntax = Requirement( + name='RQ.SRS-006.RBAC.PartialRevoke.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support partial revokes by using `partial_revokes` variable\n' + 'that can be set or unset using the following syntax.\n' + '\n' + 'To disable partial revokes the `partial_revokes` variable SHALL be set to `0`\n' + '\n' + '```sql\n' + 'SET partial_revokes = 0\n' + '```\n' + '\n' + 'To enable partial revokes the `partial revokes` variable SHALL be set to `1`\n' + '\n' + '```sql\n' + 'SET partial_revokes = 1\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Role = Requirement( + name='RQ.SRS-006.RBAC.Grant.Role', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting one or more roles to\n' + 'one or more users or roles using the `GRANT` role statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Role_CurrentUser = Requirement( + name='RQ.SRS-006.RBAC.Grant.Role.CurrentUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting one or more roles to current user using\n' + '`TO CURRENT_USER` clause in the `GRANT` role statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Role_AdminOption = Requirement( + name='RQ.SRS-006.RBAC.Grant.Role.AdminOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting `admin option` privilege\n' + 'to one or more users or roles using the `WITH ADMIN OPTION` clause\n' + 'in the `GRANT` role statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Role_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.Grant.Role.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying cluster on which the user is to be granted one or more roles\n' + 'using `ON CLUSTER` clause in the `GRANT` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Grant_Role_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Grant.Role.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for `GRANT` role statement\n' + '\n' + '``` sql\n' + 'GRANT\n' + ' ON CLUSTER cluster_name\n' + ' role [, role ...]\n' + ' TO {user | role | CURRENT_USER} [,...]\n' + ' [WITH ADMIN OPTION]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Role = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Role', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking one or more roles from\n' + 'one or more users or roles using the `REVOKE` role statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Role_Keywords = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Role.Keywords', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking one or more roles from\n' + 'special groupings of one or more users or roles with the `ALL`, `ALL EXCEPT`,\n' + 'and `CURRENT_USER` keywords.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Role_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Role.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking one or more roles from\n' + 'one or more users or roles from one or more clusters\n' + 'using the `REVOKE ON CLUSTER` role statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_AdminOption = Requirement( + name='RQ.SRS-006.RBAC.Revoke.AdminOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking `admin option` privilege\n' + 'in one or more users or roles using the `ADMIN OPTION FOR` clause\n' + 'in the `REVOKE` role statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Revoke_Role_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Revoke.Role.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `REVOKE` role statement\n' + '\n' + '```sql\n' + 'REVOKE [ON CLUSTER cluster_name] [ADMIN OPTION FOR]\n' + ' role [,...]\n' + ' FROM {user | role | CURRENT_USER} [,...] | ALL | ALL EXCEPT {user_name | role_name | CURRENT_USER} [,...]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Show_Grants = Requirement( + name='RQ.SRS-006.RBAC.Show.Grants', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support listing all the privileges granted to current user and role\n' + 'using the `SHOW GRANTS` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Show_Grants_For = Requirement( + name='RQ.SRS-006.RBAC.Show.Grants.For', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support listing all the privileges granted to a user or a role\n' + 'using the `FOR` clause in the `SHOW GRANTS` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Show_Grants_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Show.Grants.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[Clickhouse] SHALL use the following syntax for the `SHOW GRANTS` statement\n' + '\n' + '``` sql\n' + 'SHOW GRANTS [FOR user_or_role]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating settings profile using the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_IfNotExists = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.IfNotExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE SETTINGS PROFILE` statement\n' + 'to skip raising an exception if a settings profile with the same **name** already exists.\n' + 'If `IF NOT EXISTS` clause is not specified then an exception SHALL be raised if\n' + 'a settings profile with the same **name** already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Replace = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Replace', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE SETTINGS PROFILE` statement\n' + 'to replace existing settings profile if it already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Variables = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning values and constraints to one or more\n' + 'variables in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Value = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Value', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning variable value in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Variables_Constraints = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Variables.Constraints', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support setting `MIN`, `MAX`, `READONLY`, and `WRITABLE`\n' + 'constraints for the variables in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning settings profile to one or more users\n' + 'or roles in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_None = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning settings profile to no users or roles using\n' + '`TO NONE` clause in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_All = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning settings profile to all current users and roles\n' + 'using `TO ALL` clause in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Assignment_AllExcept = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Assignment.AllExcept', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support excluding assignment to one or more users or roles using\n' + 'the `ALL EXCEPT` clause in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Inherit = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Inherit', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support inheriting profile settings from indicated profile using\n' + 'the `INHERIT` clause in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying what cluster to create settings profile on\n' + 'using `ON CLUSTER` clause in the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Create_Syntax = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Create.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `CREATE SETTINGS PROFILE` statement.\n' + '\n' + '``` sql\n' + 'CREATE SETTINGS PROFILE [IF NOT EXISTS | OR REPLACE] name\n' + ' [ON CLUSTER cluster_name]\n' + " [SET varname [= value] [MIN min] [MAX max] [READONLY|WRITABLE] | [INHERIT 'profile_name'] [,...]]\n" + ' [TO {user_or_role [,...] | NONE | ALL | ALL EXCEPT user_or_role [,...]}]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering settings profile using the `ALTER STETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_IfExists = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF EXISTS` clause in the `ALTER SETTINGS PROFILE` statement\n' + 'to not raise exception if a settings profile does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if a settings profile does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Rename = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Rename', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support renaming settings profile using the `RANAME TO` clause\n' + 'in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering values and constraints of one or more\n' + 'variables in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Value = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Value', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering value of the variable in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Variables_Constraints = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Variables.Constraints', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering `MIN`, `MAX`, `READONLY`, and `WRITABLE`\n' + 'constraints for the variables in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning settings profile to one or more users\n' + 'or roles using the `TO` clause in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_None = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning settings profile to no users or roles using the\n' + '`TO NONE` clause in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_All = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning settings profile to all current users and roles\n' + 'using the `TO ALL` clause in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_AllExcept = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.AllExcept', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support excluding assignment to one or more users or roles using\n' + 'the `TO ALL EXCEPT` clause in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_Inherit = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.Inherit', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the settings profile by inheriting settings from\n' + 'specified profile using `INHERIT` clause in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Assignment_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Assignment.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the settings profile on a specified cluster using\n' + '`ON CLUSTER` clause in the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Alter_Syntax = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Alter.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `ALTER SETTINGS PROFILE` statement.\n' + '\n' + '``` sql\n' + 'ALTER SETTINGS PROFILE [IF EXISTS] name\n' + ' [ON CLUSTER cluster_name]\n' + ' [RENAME TO new_name]\n' + " [SETTINGS variable [= value] [MIN [=] min_value] [MAX [=] max_value] [READONLY|WRITABLE] | INHERIT 'profile_name'] [,...]\n" + ' [TO {user_or_role [,...] | NONE | ALL | ALL EXCEPT user_or_role [,...]]}\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Drop = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing one or more settings profiles using the `DROP SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Drop_IfExists = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Drop.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP SETTINGS PROFILE` statement\n' + 'to skip raising an exception if the settings profile does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if a settings profile does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Drop_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Drop.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support dropping one or more settings profiles on specified cluster using\n' + '`ON CLUSTER` clause in the `DROP SETTINGS PROFILE` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_Drop_Syntax = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.Drop.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `DROP SETTINGS PROFILE` statement\n' + '\n' + '``` sql\n' + 'DROP SETTINGS PROFILE [IF EXISTS] name [,name,...]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_SettingsProfile_ShowCreateSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.SettingsProfile.ShowCreateSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the `CREATE SETTINGS PROFILE` statement used to create the settings profile\n' + 'using the `SHOW CREATE SETTINGS PROFILE` statement with the following syntax\n' + '\n' + '``` sql\n' + 'SHOW CREATE SETTINGS PROFILE name\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating quotas using the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_IfNotExists = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.IfNotExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE QUOTA` statement\n' + 'to skip raising an exception if a quota with the same **name** already exists.\n' + 'If `IF NOT EXISTS` clause is not specified then an exception SHALL be raised if\n' + 'a quota with the same **name** already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Replace = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Replace', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE QUOTA` statement\n' + 'to replace existing quota if it already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating quotas on a specific cluster with the\n' + '`ON CLUSTER` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Interval = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Interval', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support defining the quota interval that specifies\n' + 'a period of time over for which the quota SHALL apply using the\n' + '`FOR INTERVAL` clause in the `CREATE QUOTA` statement.\n' + '\n' + 'This statement SHALL also support a number and a time period which will be one\n' + 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + '\n' + '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number\n' + 'to define the interval.\n' + '\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Interval_Randomized = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Interval.Randomized', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support defining the quota randomized interval that specifies\n' + 'a period of time over for which the quota SHALL apply using the\n' + '`FOR RANDOMIZED INTERVAL` clause in the `CREATE QUOTA` statement.\n' + '\n' + 'This statement SHALL also support a number and a time period which will be one\n' + 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + '\n' + '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some\n' + 'real number to define the interval.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Queries = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Queries', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting number of requests over a period of time\n' + 'using the `QUERIES` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Errors = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Errors', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting number of queries that threw an exception\n' + 'using the `ERRORS` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_ResultRows = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.ResultRows', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the total number of rows given as the result\n' + 'using the `RESULT ROWS` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_ReadRows = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.ReadRows', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the total number of source rows read from tables\n' + 'for running the query on all remote servers\n' + 'using the `READ ROWS` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_ResultBytes = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.ResultBytes', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the total number of bytes that can be returned as the result\n' + 'using the `RESULT BYTES` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_ReadBytes = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.ReadBytes', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the total number of source bytes read from tables\n' + 'for running the query on all remote servers\n' + 'using the `READ BYTES` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_ExecutionTime = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.ExecutionTime', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the maximum query execution time\n' + 'using the `EXECUTION TIME` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_NoLimits = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.NoLimits', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the maximum query execution time\n' + 'using the `NO LIMITS` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_TrackingOnly = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.TrackingOnly', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the maximum query execution time\n' + 'using the `TRACKING ONLY` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_KeyedBy = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.KeyedBy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support to track quota for some key\n' + 'following the `KEYED BY` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_KeyedByOptions = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.KeyedByOptions', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support to track quota separately for some parameter\n' + "using the `KEYED BY 'parameter'` clause in the `CREATE QUOTA` statement.\n" + '\n' + "'parameter' can be one of:\n" + "`{'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}`\n" + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Assignment = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Assignment', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning quota to one or more users\n' + 'or roles using the `TO` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Assignment_None = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Assignment.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning quota to no users or roles using\n' + '`TO NONE` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Assignment_All = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Assignment.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning quota to all current users and roles\n' + 'using `TO ALL` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Assignment_Except = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Assignment.Except', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support excluding assignment of quota to one or more users or roles using\n' + 'the `EXCEPT` clause in the `CREATE QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Create_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Quota.Create.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `CREATE QUOTA` statement\n' + '\n' + '```sql\n' + 'CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name [ON CLUSTER cluster_name]\n' + " [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}]\n" + ' [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}\n' + ' {MAX { {QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = number } [,...] |\n' + ' NO LIMITS | TRACKING ONLY} [,...]]\n' + ' [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering quotas using the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_IfExists = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF EXISTS` clause in the `ALTER QUOTA` statement\n' + 'to skip raising an exception if a quota does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be raised if\n' + 'a quota does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Rename = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Rename', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `RENAME TO` clause in the `ALTER QUOTA` statement\n' + 'to rename the quota to the specified name.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering quotas on a specific cluster with the\n' + '`ON CLUSTER` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Interval = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Interval', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support redefining the quota interval that specifies\n' + 'a period of time over for which the quota SHALL apply using the\n' + '`FOR INTERVAL` clause in the `ALTER QUOTA` statement.\n' + '\n' + 'This statement SHALL also support a number and a time period which will be one\n' + 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + '\n' + '`FOR INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some real number\n' + 'to define the interval.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Interval_Randomized = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Interval.Randomized', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support redefining the quota randomized interval that specifies\n' + 'a period of time over for which the quota SHALL apply using the\n' + '`FOR RANDOMIZED INTERVAL` clause in the `ALTER QUOTA` statement.\n' + '\n' + 'This statement SHALL also support a number and a time period which will be one\n' + 'of `{SECOND | MINUTE | HOUR | DAY | MONTH}`. Thus, the complete syntax SHALL be:\n' + '\n' + '`FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY}` where number is some\n' + 'real number to define the interval.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Queries = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Queries', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the limit of number of requests over a period of time\n' + 'using the `QUERIES` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Errors = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Errors', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the limit of number of queries that threw an exception\n' + 'using the `ERRORS` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_ResultRows = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.ResultRows', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the limit of the total number of rows given as the result\n' + 'using the `RESULT ROWS` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_ReadRows = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.ReadRows', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the limit of the total number of source rows read from tables\n' + 'for running the query on all remote servers\n' + 'using the `READ ROWS` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ALter_ResultBytes = Requirement( + name='RQ.SRS-006.RBAC.Quota.ALter.ResultBytes', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the limit of the total number of bytes that can be returned as the result\n' + 'using the `RESULT BYTES` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_ReadBytes = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.ReadBytes', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the limit of the total number of source bytes read from tables\n' + 'for running the query on all remote servers\n' + 'using the `READ BYTES` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_ExecutionTime = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.ExecutionTime', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering the limit of the maximum query execution time\n' + 'using the `EXECUTION TIME` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_NoLimits = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.NoLimits', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the maximum query execution time\n' + 'using the `NO LIMITS` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_TrackingOnly = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.TrackingOnly', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support limiting the maximum query execution time\n' + 'using the `TRACKING ONLY` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_KeyedBy = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.KeyedBy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering quota to track quota separately for some key\n' + 'following the `KEYED BY` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_KeyedByOptions = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.KeyedByOptions', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering quota to track quota separately for some parameter\n' + "using the `KEYED BY 'parameter'` clause in the `ALTER QUOTA` statement.\n" + '\n' + "'parameter' can be one of:\n" + "`{'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}`\n" + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Assignment = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Assignment', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning quota to one or more users\n' + 'or roles using the `TO` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Assignment_None = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning quota to no users or roles using\n' + '`TO NONE` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Assignment_All = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning quota to all current users and roles\n' + 'using `TO ALL` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Assignment_Except = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Assignment.Except', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support excluding assignment of quota to one or more users or roles using\n' + 'the `EXCEPT` clause in the `ALTER QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Alter_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Quota.Alter.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `ALTER QUOTA` statement\n' + '\n' + '``` sql\n' + 'ALTER QUOTA [IF EXIST] name\n' + ' {{{QUERIES | ERRORS | RESULT ROWS | READ ROWS | RESULT BYTES | READ BYTES | EXECUTION TIME} number} [, ...] FOR INTERVAL number time_unit} [, ...]\n' + ' [KEYED BY USERNAME | KEYED BY IP | NOT KEYED] [ALLOW CUSTOM KEY | DISALLOW CUSTOM KEY]\n' + ' [TO {user_or_role [,...] | NONE | ALL} [EXCEPT user_or_role [,...]]]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Drop = Requirement( + name='RQ.SRS-006.RBAC.Quota.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing one or more quotas using the `DROP QUOTA` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Drop_IfExists = Requirement( + name='RQ.SRS-006.RBAC.Quota.Drop.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using `IF EXISTS` clause in the `DROP QUOTA` statement\n' + 'to skip raising an exception when the quota does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if the quota does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Drop_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Quota.Drop.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using `ON CLUSTER` clause in the `DROP QUOTA` statement\n' + 'to indicate the cluster the quota to be dropped is located on.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_Drop_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Quota.Drop.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `DROP QUOTA` statement\n' + '\n' + '``` sql\n' + 'DROP QUOTA [IF EXISTS] name [,name...]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowQuotas = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowQuotas', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing all of the current quotas\n' + 'using the `SHOW QUOTAS` statement with the following syntax\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowQuotas_IntoOutfile = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowQuotas.IntoOutfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `INTO OUTFILE` clause in the `SHOW QUOTAS` statement to define an outfile by some given string literal.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowQuotas_Format = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Format', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `FORMAT` clause in the `SHOW QUOTAS` statement to define a format for the output quota list.\n' + '\n' + 'The types of valid formats are many, listed in output column:\n' + 'https://clickhouse.tech/docs/en/interfaces/formats/\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowQuotas_Settings = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Settings', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `SETTINGS` clause in the `SHOW QUOTAS` statement to define settings in the showing of all quotas.\n' + '\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowQuotas_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowQuotas.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using the `SHOW QUOTAS` statement\n' + 'with the following syntax\n' + '``` sql\n' + 'SHOW QUOTAS\n' + '```\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Name = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Name', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the `CREATE QUOTA` statement used to create the quota with some given name\n' + 'using the `SHOW CREATE QUOTA` statement with the following syntax\n' + '\n' + '``` sql\n' + 'SHOW CREATE QUOTA name\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Current = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Current', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the `CREATE QUOTA` statement used to create the CURRENT quota\n' + 'using the `SHOW CREATE QUOTA CURRENT` statement or the shorthand form\n' + '`SHOW CREATE QUOTA`\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Quota_ShowCreateQuota_Syntax = Requirement( + name='RQ.SRS-006.RBAC.Quota.ShowCreateQuota.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax when\n' + 'using the `SHOW CREATE QUOTA` statement.\n' + '\n' + '```sql\n' + 'SHOW CREATE QUOTA [name | CURRENT]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support creating row policy using the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_IfNotExists = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.IfNotExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `IF NOT EXISTS` clause in the `CREATE ROW POLICY` statement\n' + 'to skip raising an exception if a row policy with the same **name** already exists.\n' + 'If the `IF NOT EXISTS` clause is not specified then an exception SHALL be raised if\n' + 'a row policy with the same **name** already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Replace = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Replace', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support `OR REPLACE` clause in the `CREATE ROW POLICY` statement\n' + 'to replace existing row policy if it already exists.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying cluster on which to create the role policy\n' + 'using the `ON CLUSTER` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_On = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.On', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying table on which to create the role policy\n' + 'using the `ON` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Access = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Access', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support allowing or restricting access to rows using the\n' + '`AS` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Access_Permissive = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Access.Permissive', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support allowing access to rows using the\n' + '`AS PERMISSIVE` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Access_Restrictive = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Access.Restrictive', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support restricting access to rows using the\n' + '`AS RESTRICTIVE` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_ForSelect = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.ForSelect', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying which rows are affected\n' + 'using the `FOR SELECT` clause in the `CREATE ROW POLICY` statement.\n' + 'REQUIRES CONFIRMATION\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Condition = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Condition', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying a condition that\n' + 'that can be any SQL expression which returns a boolean using the `USING`\n' + 'clause in the `CREATE ROW POLOCY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Assignment = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning row policy to one or more users\n' + 'or roles using the `TO` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_None = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning row policy to no users or roles using\n' + 'the `TO NONE` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_All = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support assigning row policy to all current users and roles\n' + 'using `TO ALL` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Assignment_AllExcept = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Assignment.AllExcept', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support excluding assignment of row policy to one or more users or roles using\n' + 'the `ALL EXCEPT` clause in the `CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Create_Syntax = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Create.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `CRETE ROW POLICY` statement\n' + '\n' + '``` sql\n' + 'CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name [ON CLUSTER cluster_name] ON [db.]table\n' + ' [AS {PERMISSIVE | RESTRICTIVE}]\n' + ' [FOR SELECT]\n' + ' [USING condition]\n' + ' [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering row policy using the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_IfExists = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the `IF EXISTS` clause in the `ALTER ROW POLICY` statement\n' + 'to skip raising an exception if a row policy does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be raised if\n' + 'a row policy does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_ForSelect = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.ForSelect', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support modifying rows on which to apply the row policy\n' + 'using the `FOR SELECT` clause in the `ALTER ROW POLICY` statement.\n' + 'REQUIRES FUNCTION CONFIRMATION.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying cluster on which to alter the row policy\n' + 'using the `ON CLUSTER` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_On = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.On', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support specifying table on which to alter the row policy\n' + 'using the `ON` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Rename = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Rename', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support renaming the row policy using the `RENAME` clause\n' + 'in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Access = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support altering access to rows using the\n' + '`AS` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Access_Permissive = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Permissive', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support permitting access to rows using the\n' + '`AS PERMISSIVE` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Access_Restrictive = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Access.Restrictive', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support restricting access to rows using the\n' + '`AS RESTRICTIVE` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Condition = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Condition', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support re-specifying the row policy condition\n' + 'using the `USING` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Condition_None = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Condition.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing the row policy condition\n' + 'using the `USING NONE` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning row policy to one or more users\n' + 'or roles using the `TO` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_None = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.None', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning row policy to no users or roles using\n' + 'the `TO NONE` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_All = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support reassigning row policy to all current users and roles\n' + 'using the `TO ALL` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Assignment_AllExcept = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Assignment.AllExcept', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support excluding assignment of row policy to one or more users or roles using\n' + 'the `ALL EXCEPT` clause in the `ALTER ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Alter_Syntax = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Alter.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `ALTER ROW POLICY` statement\n' + '\n' + '``` sql\n' + 'ALTER [ROW] POLICY [IF EXISTS] name [ON CLUSTER cluster_name] ON [database.]table\n' + ' [RENAME TO new_name]\n' + ' [AS {PERMISSIVE | RESTRICTIVE}]\n' + ' [FOR SELECT]\n' + ' [USING {condition | NONE}][,...]\n' + ' [TO {role [,...] | ALL | ALL EXCEPT role [,...]}]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Drop = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing one or more row policies using the `DROP ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Drop_IfExists = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Drop.IfExists', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support using the `IF EXISTS` clause in the `DROP ROW POLICY` statement\n' + 'to skip raising an exception when the row policy does not exist.\n' + 'If the `IF EXISTS` clause is not specified then an exception SHALL be\n' + 'raised if the row policy does not exist.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Drop_On = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Drop.On', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing row policy from one or more specified tables\n' + 'using the `ON` clause in the `DROP ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Drop_OnCluster = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Drop.OnCluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support removing row policy from specified cluster\n' + 'using the `ON CLUSTER` clause in the `DROP ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_Drop_Syntax = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.Drop.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for the `DROP ROW POLICY` statement.\n' + '\n' + '``` sql\n' + 'DROP [ROW] POLICY [IF EXISTS] name [,...] ON [database.]table [,...] [ON CLUSTER cluster_name]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing the `CREATE ROW POLICY` statement used to create the row policy\n' + 'using the `SHOW CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy_On = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.On', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing statement used to create row policy on specific table\n' + 'using the `ON` in the `SHOW CREATE ROW POLICY` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_ShowCreateRowPolicy_Syntax = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.ShowCreateRowPolicy.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for `SHOW CREATE ROW POLICY`.\n' + '\n' + '``` sql\n' + 'SHOW CREATE [ROW] POLICY name ON [database.]table\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing row policies using the `SHOW ROW POLICIES` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies_On = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.On', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support showing row policies on a specific table\n' + 'using the `ON` clause in the `SHOW ROW POLICIES` statement.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_RowPolicy_ShowRowPolicies_Syntax = Requirement( + name='RQ.SRS-006.RBAC.RowPolicy.ShowRowPolicies.Syntax', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the following syntax for `SHOW ROW POLICIES`.\n' + '\n' + '```sql\n' + 'SHOW [ROW] POLICIES [ON [database.]table]\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Table_PublicTables = Requirement( + name='RQ.SRS-006.RBAC.Table.PublicTables', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support that a user without any privileges will be able to access the following tables\n' + '\n' + '* system.one\n' + '* system.numbers\n' + '* system.contributors\n' + '* system.functions\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Table_SensitiveTables = Requirement( + name='RQ.SRS-006.RBAC.Table.SensitiveTables', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL not support a user with no privileges accessing the following `system` tables:\n' + '\n' + '* processes\n' + '* query_log\n' + '* query_thread_log\n' + '* clusters\n' + '* events\n' + '* graphite_retentions\n' + '* stack_trace\n' + '* trace_log\n' + '* user_directories\n' + '* zookeeper\n' + '* macros\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_DistributedTable_Create = Requirement( + name='RQ.SRS-006.RBAC.DistributedTable.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully `CREATE` a distributed table if and only if\n' + 'the user has **create table** privilege on the table and **remote** privilege on *.*\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_DistributedTable_Select = Requirement( + name='RQ.SRS-006.RBAC.DistributedTable.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully `SELECT` from a distributed table if and only if\n' + 'the user has **select** privilege on the table and on the remote table specified in the `CREATE` query of the distributed table.\n' + '\n' + 'Does not require **select** privilege for the remote table if the remote table does not exist on the same server as the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_DistributedTable_Insert = Requirement( + name='RQ.SRS-006.RBAC.DistributedTable.Insert', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully `INSERT` into a distributed table if and only if\n' + 'the user has **insert** privilege on the table and on the remote table specified in the `CREATE` query of the distributed table.\n' + '\n' + 'Does not require **insert** privilege for the remote table if the remote table does not exist on the same server as the user,\n' + 'insert executes into the remote table on a different server.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_DistributedTable_SpecialTables = Requirement( + name='RQ.SRS-006.RBAC.DistributedTable.SpecialTables', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute a query using a distributed table that uses one of the special tables if and only if\n' + 'the user has the necessary privileges to interact with that special table, either granted directly or through a role.\n' + 'Special tables include:\n' + '* materialized view\n' + '* distributed table\n' + '* source table of a materialized view\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_DistributedTable_LocalUser = Requirement( + name='RQ.SRS-006.RBAC.DistributedTable.LocalUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute a query using a distributed table from\n' + 'a user present locally, but not remotely.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_DistributedTable_SameUserDifferentNodesDifferentPrivileges = Requirement( + name='RQ.SRS-006.RBAC.DistributedTable.SameUserDifferentNodesDifferentPrivileges', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute a query using a distributed table by a user that exists on multiple nodes\n' + 'if and only if the user has the required privileges on the node the query is being executed from.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_View = Requirement( + name='RQ.SRS-006.RBAC.View', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to **create**, **select** and **drop**\n' + 'privileges for a view for users or roles.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_View_Create = Requirement( + name='RQ.SRS-006.RBAC.View.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `CREATE VIEW` command if and only if\n' + 'the user has **create view** privilege either explicitly or through roles.\n' + '\n' + 'If the stored query includes one or more source tables, the user must have **select** privilege\n' + 'on all the source tables either explicitly or through a role.\n' + 'For example,\n' + '```sql\n' + 'CREATE VIEW view AS SELECT * FROM source_table\n' + 'CREATE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' + 'CREATE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' + 'CREATE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' + 'CREATE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' + 'CREATE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_View_Select = Requirement( + name='RQ.SRS-006.RBAC.View.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully `SELECT` from a view if and only if\n' + 'the user has **select** privilege for that view either explicitly or through a role.\n' + '\n' + 'If the stored query includes one or more source tables, the user must have **select** privilege\n' + 'on all the source tables either explicitly or through a role.\n' + 'For example,\n' + '```sql\n' + 'CREATE VIEW view AS SELECT * FROM source_table\n' + 'CREATE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' + 'CREATE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' + 'CREATE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' + 'CREATE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' + 'CREATE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' + '\n' + 'SELECT * FROM view\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_View_Drop = Requirement( + name='RQ.SRS-006.RBAC.View.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `DROP VIEW` command if and only if\n' + 'the user has **drop view** privilege on that view either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to **create**, **select**, **alter** and **drop**\n' + 'privileges for a materialized view for users or roles.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Create = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `CREATE MATERIALIZED VIEW` command if and only if\n' + 'the user has **create view** privilege either explicitly or through roles.\n' + '\n' + 'If `POPULATE` is specified, the user must have `INSERT` privilege on the view,\n' + 'either explicitly or through roles.\n' + 'For example,\n' + '```sql\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory POPULATE AS SELECT * FROM source_table\n' + '```\n' + '\n' + 'If the stored query includes one or more source tables, the user must have **select** privilege\n' + 'on all the source tables either explicitly or through a role.\n' + 'For example,\n' + '```sql\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM source_table\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 JOIN table1 USING column\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' + 'CREATE MATERIALIZED VIEW view0 ENGINE = Memory AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' + '```\n' + '\n' + 'If the materialized view has a target table explicitly declared in the `TO` clause, the user must have\n' + '**insert** and **select** privilege on the target table.\n' + 'For example,\n' + '```sql\n' + 'CREATE MATERIALIZED VIEW view TO target_table AS SELECT * FROM source_table\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Select = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully `SELECT` from a materialized view if and only if\n' + 'the user has **select** privilege for that view either explicitly or through a role.\n' + '\n' + 'If the stored query includes one or more source tables, the user must have **select** privilege\n' + 'on all the source tables either explicitly or through a role.\n' + 'For example,\n' + '```sql\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM source_table\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 JOIN table1 USING column\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' + 'CREATE MATERIALIZED VIEW view ENGINE = Memory AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' + 'CREATE MATERIALIZED VIEW view0 ENGINE = Memory AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' + '\n' + 'SELECT * FROM view\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Select_TargetTable = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Select.TargetTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully `SELECT` from the target table, implicit or explicit, of a materialized view if and only if\n' + 'the user has `SELECT` privilege for the table, either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Select_SourceTable = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Select.SourceTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully `SELECT` from the source table of a materialized view if and only if\n' + 'the user has `SELECT` privilege for the table, either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Drop = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `DROP VIEW` command if and only if\n' + 'the user has **drop view** privilege on that view either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_ModifyQuery = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.ModifyQuery', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `MODIFY QUERY` command if and only if\n' + 'the user has **modify query** privilege on that view either explicitly or through a role.\n' + '\n' + 'If the new query includes one or more source tables, the user must have **select** privilege\n' + 'on all the source tables either explicitly or through a role.\n' + 'For example,\n' + '```sql\n' + 'ALTER TABLE view MODIFY QUERY SELECT * FROM source_table\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Insert = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Insert', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only succesfully `INSERT` into a materialized view if and only if\n' + 'the user has `INSERT` privilege on the view, either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Insert_SourceTable = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Insert.SourceTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only succesfully `INSERT` into a source table of a materialized view if and only if\n' + 'the user has `INSERT` privilege on the source table, either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_MaterializedView_Insert_TargetTable = Requirement( + name='RQ.SRS-006.RBAC.MaterializedView.Insert.TargetTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only succesfully `INSERT` into a target table of a materialized view if and only if\n' + 'the user has `INSERT` privelege on the target table, either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_LiveView = Requirement( + name='RQ.SRS-006.RBAC.LiveView', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to **create**, **select**, **alter** and **drop**\n' + 'privileges for a live view for users or roles.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_LiveView_Create = Requirement( + name='RQ.SRS-006.RBAC.LiveView.Create', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `CREATE LIVE VIEW` command if and only if\n' + 'the user has **create view** privilege either explicitly or through roles.\n' + '\n' + 'If the stored query includes one or more source tables, the user must have **select** privilege\n' + 'on all the source tables either explicitly or through a role.\n' + 'For example,\n' + '```sql\n' + 'CREATE LIVE VIEW view AS SELECT * FROM source_table\n' + 'CREATE LIVE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' + 'CREATE LIVE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' + 'CREATE LIVE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' + 'CREATE LIVE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' + 'CREATE LIVE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_LiveView_Select = Requirement( + name='RQ.SRS-006.RBAC.LiveView.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully `SELECT` from a live view if and only if\n' + 'the user has **select** privilege for that view either explicitly or through a role.\n' + '\n' + 'If the stored query includes one or more source tables, the user must have **select** privilege\n' + 'on all the source tables either explicitly or through a role.\n' + 'For example,\n' + '```sql\n' + 'CREATE LIVE VIEW view AS SELECT * FROM source_table\n' + 'CREATE LIVE VIEW view AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' + 'CREATE LIVE VIEW view AS SELECT * FROM table0 JOIN table1 USING column\n' + 'CREATE LIVE VIEW view AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' + 'CREATE LIVE VIEW view AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' + 'CREATE LIVE VIEW view0 AS SELECT column FROM view1 UNION ALL SELECT column FROM view2\n' + '\n' + 'SELECT * FROM view\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_LiveView_Drop = Requirement( + name='RQ.SRS-006.RBAC.LiveView.Drop', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `DROP VIEW` command if and only if\n' + 'the user has **drop view** privilege on that view either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_LiveView_Refresh = Requirement( + name='RQ.SRS-006.RBAC.LiveView.Refresh', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute an `ALTER LIVE VIEW REFRESH` command if and only if\n' + 'the user has **refresh** privilege on that view either explicitly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Select = Requirement( + name='RQ.SRS-006.RBAC.Select', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL execute `SELECT` if and only if the user\n' + 'has the **select** privilege for the destination table\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Select_Column = Requirement( + name='RQ.SRS-006.RBAC.Select.Column', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **select** privilege\n' + 'for one or more specified columns in a table to one or more **users** or **roles**.\n' + 'Any `SELECT` statements SHALL not to be executed, unless the user\n' + 'has the **select** privilege for the destination column\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Select_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Select.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **select** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `SELECT` statements SHALL succeed only on nodes where\n' + 'the table exists and privilege was granted.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Select_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Select.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **select** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Insert = Requirement( + name='RQ.SRS-006.RBAC.Insert', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL execute `INSERT INTO` if and only if the user\n' + 'has the **insert** privilege for the destination table\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Insert_Column = Requirement( + name='RQ.SRS-006.RBAC.Insert.Column', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **insert** privilege\n' + 'for one or more specified columns in a table to one or more **users** or **roles**.\n' + 'Any `INSERT INTO` statements SHALL not to be executed, unless the user\n' + 'has the **insert** privilege for the destination column\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Insert_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Insert.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **insert** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `INSERT INTO` statements SHALL succeed only on nodes where\n' + 'the table exists and privilege was granted.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Insert_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Insert.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **insert** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterColumn = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterColumn', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter column** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL\n' + 'return an error, unless the user has the **alter column** privilege for\n' + 'the destination table either because of the explicit grant or through one of\n' + 'the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterColumn_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter column** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterColumn_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter column** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterColumn_Column = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Column', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter column** privilege\n' + 'for one or more specified columns in a table to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN` statements SHALL return an error,\n' + 'unless the user has the **alter column** privilege for the destination column\n' + 'either because of the explicit grant or through one of the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterColumn_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterColumn.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter column** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN`\n' + 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterColumn_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterColumn.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter column** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterIndex = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterIndex', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter index** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX` statements SHALL\n' + 'return an error, unless the user has the **alter index** privilege for\n' + 'the destination table either because of the explicit grant or through one of\n' + 'the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterIndex_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter index** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterIndex_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter index** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterIndex_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterIndex.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter index** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ORDER BY | ADD|DROP|MATERIALIZE|CLEAR INDEX`\n' + 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterIndex_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterIndex.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter index** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterConstraint = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterConstraint', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter constraint** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ADD|CREATE CONSTRAINT` statements SHALL\n' + 'return an error, unless the user has the **alter constraint** privilege for\n' + 'the destination table either because of the explicit grant or through one of\n' + 'the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterConstraint_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter constraint** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterConstraint_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter constraint** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterConstraint_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter constraint** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ADD|DROP CONSTRAINT`\n' + 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterConstraint_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterConstraint.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter constraint** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterTTL = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterTTL', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter ttl** or **alter materialize ttl** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL` statements SHALL\n' + 'return an error, unless the user has the **alter ttl** or **alter materialize ttl** privilege for\n' + 'the destination table either because of the explicit grant or through one of\n' + 'the roles assigned to the user.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterTTL_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter ttl** or **alter materialize ttl** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterTTL_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter ttl** or **alter materialize ttl** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterTTL_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterTTL.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter ttl** or **alter materialize ttl** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... ALTER TTL | ALTER MATERIALIZE TTL`\n' + 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterTTL_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterTTL.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter ttl** or **alter materialize ttl** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterSettings = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterSettings', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter settings** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... MODIFY SETTING setting` statements SHALL\n' + 'return an error, unless the user has the **alter settings** privilege for\n' + 'the destination table either because of the explicit grant or through one of\n' + 'the roles assigned to the user. The **alter settings** privilege allows\n' + 'modifying table engine settings. It doesn’t affect settings or server configuration parameters.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterSettings_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter settings** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterSettings_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter settings** privilege\n' + 'for a database or a specific table to one or more **users** or **roles**\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterSettings_Cluster = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterSettings.Cluster', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking **alter settings** privilege\n' + 'on a specified cluster to one or more **users** or **roles**.\n' + 'Any `ALTER TABLE ... MODIFY SETTING setting`\n' + 'statements SHALL succeed only on nodes where the table exists and privilege was granted.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterSettings_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterSettings.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter settings** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUpdate = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUpdate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER UPDATE` statement if and only if the user has **alter update** privilege for that column,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUpdate_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter update** privilege on a column level\n' + 'to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUpdate_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter update** privilege on a column level\n' + 'from one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUpdate_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUpdate.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter update** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterDelete = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterDelete', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER DELETE` statement if and only if the user has **alter delete** privilege for that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterDelete_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterDelete.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter delete** privilege on a column level\n' + 'to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterDelete_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterDelete.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter delete** privilege on a column level\n' + 'from one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterDelete_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterDelete.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter delete** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFreeze = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFreeze', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER FREEZE` statement if and only if the user has **alter freeze** privilege for that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFreeze_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter freeze** privilege on a column level\n' + 'to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFreeze_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter freeze** privilege on a column level\n' + 'from one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFreeze_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFreeze.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter freeze** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFetch = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFetch', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER FETCH` statement if and only if the user has **alter fetch** privilege for that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFetch_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFetch.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter fetch** privilege on a column level\n' + 'to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFetch_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFetch.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter fetch** privilege on a column level\n' + 'from one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterFetch_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterFetch.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter fetch** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterMove = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterMove', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER MOVE` statement if and only if the user has **alter move**, **select**, and **alter delete** privilege on the source table\n' + 'and **insert** privilege on the target table, either directly or through a role.\n' + 'For example,\n' + '```sql\n' + 'ALTER TABLE source_table MOVE PARTITION 1 TO target_table\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterMove_Grant = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterMove.Grant', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting **alter move** privilege on a column level\n' + 'to one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterMove_Revoke = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterMove.Revoke', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support revoking **alter move** privilege on a column level\n' + 'from one or more **users** or **roles**.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterMove_TableEngines = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterMove.TableEngines', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support controlling access to the **alter move** privilege\n' + 'on tables created using the following engines\n' + '\n' + '* MergeTree\n' + '* ReplacingMergeTree\n' + '* SummingMergeTree\n' + '* AggregatingMergeTree\n' + '* CollapsingMergeTree\n' + '* VersionedCollapsingMergeTree\n' + '* GraphiteMergeTree\n' + '* ReplicatedMergeTree\n' + '* ReplicatedSummingMergeTree\n' + '* ReplicatedReplacingMergeTree\n' + '* ReplicatedAggregatingMergeTree\n' + '* ReplicatedCollapsingMergeTree\n' + '* ReplicatedVersionedCollapsingMergeTree\n' + '* ReplicatedGraphiteMergeTree\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateTable = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL only successfully execute a `CREATE TABLE` command if and only if\n' + 'the user has **create table** privilege either explicitly or through roles.\n' + '\n' + 'If the stored query includes one or more source tables, the user must have **select** privilege\n' + "on all the source tables and **insert** for the table they're trying to create either explicitly or through a role.\n" + 'For example,\n' + '```sql\n' + 'CREATE TABLE table AS SELECT * FROM source_table\n' + 'CREATE TABLE table AS SELECT * FROM table0 WHERE column IN (SELECT column FROM table1 WHERE column IN (SELECT column FROM table2 WHERE expression))\n' + 'CREATE TABLE table AS SELECT * FROM table0 JOIN table1 USING column\n' + 'CREATE TABLE table AS SELECT * FROM table0 UNION ALL SELECT * FROM table1 UNION ALL SELECT * FROM table2\n' + 'CREATE TABLE table AS SELECT column FROM table0 JOIN table1 USING column UNION ALL SELECT column FROM table2 WHERE column IN (SELECT column FROM table3 WHERE column IN (SELECT column FROM table4 WHERE expression))\n' + 'CREATE TABLE table0 AS SELECT column FROM table1 UNION ALL SELECT column FROM table2\n' + '```\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateDatabase = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateDatabase', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE DATABASE` statement if and only if the user has **create database** privilege on the database,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateDictionary = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateDictionary', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateTemporaryTable = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateTemporaryTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AttachDatabase = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AttachDatabase', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ATTACH DATABASE` statement if and only if the user has **create database** privilege on the database,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AttachDictionary = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AttachDictionary', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ATTACH DICTIONARY` statement if and only if the user has **create dictionary** privilege on the dictionary,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AttachTemporaryTable = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AttachTemporaryTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ATTACH TEMPORARY TABLE` statement if and only if the user has **create temporary table** privilege on the table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AttachTable = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AttachTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ATTACH TABLE` statement if and only if the user has **create table** privilege on the table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropTable = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP TABLE` statement if and only if the user has **drop table** privilege on the table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropDatabase = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropDatabase', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP DATABASE` statement if and only if the user has **drop database** privilege on the database,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropDictionary = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropDictionary', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DetachTable = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DetachTable', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DETACH TABLE` statement if and only if the user has **drop table** privilege on the table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DetachView = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DetachView', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DETACH VIEW` statement if and only if the user has **drop view** privilege on the view,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DetachDatabase = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DetachDatabase', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DETACH DATABASE` statement if and only if the user has **drop database** privilege on the database,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DetachDictionary = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DetachDictionary', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DETACH DICTIONARY` statement if and only if the user has **drop dictionary** privilege on the dictionary,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Truncate = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Truncate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `TRUNCATE TABLE` statement if and only if the user has **truncate table** privilege on the table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Optimize = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Optimize', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `OPTIMIZE TABLE` statement if and only if the user has **optimize table** privilege on the table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_KillQuery = Requirement( + name='RQ.SRS-006.RBAC.Privileges.KillQuery', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `KILL QUERY` statement if and only if the user has **kill query** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_KillMutation = Requirement( + name='RQ.SRS-006.RBAC.Privileges.KillMutation', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `KILL MUTATION` statement if and only if\n' + 'the user has the privilege that created the mutation, either directly or through a role.\n' + 'For example, to `KILL MUTATION` after `ALTER UPDATE` query, the user needs `ALTER UPDATE` privilege.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_KillMutation_AlterUpdate = Requirement( + name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterUpdate', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER UPDATE` mutation if and only if\n' + 'the user has `ALTER UPDATE` privilege on the table where the mutation was created, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_KillMutation_AlterDelete = Requirement( + name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDelete', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER DELETE` mutation if and only if\n' + 'the user has `ALTER DELETE` privilege on the table where the mutation was created, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_KillMutation_AlterDropColumn = Requirement( + name='RQ.SRS-006.RBAC.Privileges.KillMutation.AlterDropColumn', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `KILL MUTATION` query on an `ALTER DROP COLUMN` mutation if and only if\n' + 'the user has `ALTER DROP COLUMN` privilege on the table where the mutation was created, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowTables_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowTables.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL grant **show tables** privilege on a table to a user if that user has recieved any grant,\n' + 'including `SHOW TABLES`, on that table, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowTables_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowTables.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW TABLES` statement if and only if the user has **show tables** privilege,\n' + 'or any privilege on the table either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ExistsTable_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ExistsTable.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `EXISTS table` statement if and only if the user has **show tables** privilege,\n' + 'or any privilege on the table either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_CheckTable_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.CheckTable.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CHECK table` statement if and only if the user has **show tables** privilege,\n' + 'or any privilege on the table either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowDatabases_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowDatabases.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL grant **show databases** privilege on a database to a user if that user has recieved any grant,\n' + 'including `SHOW DATABASES`, on that table, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowDatabases_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowDatabases.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW DATABASES` statement if and only if the user has **show databases** privilege,\n' + 'or any privilege on the database either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateDatabase_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateDatabase.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE DATABASE` statement if and only if the user has **show databases** privilege,\n' + 'or any privilege on the database either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_UseDatabase_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.UseDatabase.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `USE database` statement if and only if the user has **show databases** privilege,\n' + 'or any privilege on the database either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowColumns_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowColumns.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking the `SHOW COLUMNS` privilege.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateTable_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateTable.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE TABLE` statement if and only if the user has **show columns** privilege on that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_DescribeTable_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.DescribeTable.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DESCRIBE table` statement if and only if the user has **show columns** privilege on that table,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowDictionaries_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowDictionaries.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL grant **show dictionaries** privilege on a dictionary to a user if that user has recieved any grant,\n' + 'including `SHOW DICTIONARIES`, on that dictionary, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowDictionaries_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowDictionaries.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW DICTIONARIES` statement if and only if the user has **show dictionaries** privilege,\n' + 'or any privilege on the dictionary either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateDictionary_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateDictionary.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE DICTIONARY` statement if and only if the user has **show dictionaries** privilege,\n' + 'or any privilege on the dictionary either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ExistsDictionary_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ExistsDictionary.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `EXISTS dictionary` statement if and only if the user has **show dictionaries** privilege,\n' + 'or any privilege on the dictionary either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateUser = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE USER` statement if and only if the user has **create user** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateUser_DefaultRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateUser.DefaultRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE USER` statement with `DEFAULT ROLE ` clause if and only if\n' + 'the user has **create user** privilege and the role with **admin option**, or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterUser = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER USER` statement if and only if the user has **alter user** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropUser = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropUser', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP USER` statement if and only if the user has **drop user** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE ROLE` statement if and only if the user has **create role** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER ROLE` statement if and only if the user has **alter role** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropRole = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropRole', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP ROLE` statement if and only if the user has **drop role** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE ROW POLICY` statement if and only if the user has **create row policy** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER ROW POLICY` statement if and only if the user has **alter row policy** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropRowPolicy = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropRowPolicy', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP ROW POLICY` statement if and only if the user has **drop row policy** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateQuota = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateQuota', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE QUOTA` statement if and only if the user has **create quota** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterQuota = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterQuota', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER QUOTA` statement if and only if the user has **alter quota** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropQuota = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropQuota', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP QUOTA` statement if and only if the user has **drop quota** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_CreateSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Privileges.CreateSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `CREATE SETTINGS PROFILE` statement if and only if the user has **create settings profile** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AlterSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AlterSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `ALTER SETTINGS PROFILE` statement if and only if the user has **alter settings profile** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_DropSettingsProfile = Requirement( + name='RQ.SRS-006.RBAC.Privileges.DropSettingsProfile', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `DROP SETTINGS PROFILE` statement if and only if the user has **drop settings profile** privilege,\n' + 'or either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_RoleAdmin = Requirement( + name='RQ.SRS-006.RBAC.Privileges.RoleAdmin', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute any role grant or revoke by a user with `ROLE ADMIN` privilege.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowUsers_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowUsers.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW USERS` privilege when\n' + 'the user is granted `SHOW USERS`, `SHOW CREATE USER`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowUsers_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowUsers.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW USERS` statement if and only if the user has **show users** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateUser_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateUser.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE USER` statement if and only if the user has **show users** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowRoles_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowRoles.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW ROLES` privilege when\n' + 'the user is granted `SHOW ROLES`, `SHOW CREATE ROLE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowRoles_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowRoles.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW ROLES` statement if and only if the user has **show roles** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateRole_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateRole.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE ROLE` statement if and only if the user has **show roles** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowRowPolicies_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowRowPolicies.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW ROW POLICIES` privilege when\n' + 'the user is granted `SHOW ROW POLICIES`, `SHOW POLICIES`, `SHOW CREATE ROW POLICY`,\n' + '`SHOW CREATE POLICY`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowRowPolicies_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowRowPolicies.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW ROW POLICIES` or `SHOW POLICIES` statement if and only if\n' + 'the user has **show row policies** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateRowPolicy_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateRowPolicy.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE ROW POLICY` or `SHOW CREATE POLICY` statement\n' + 'if and only if the user has **show row policies** privilege,either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowQuotas_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowQuotas.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW QUOTAS` privilege when\n' + 'the user is granted `SHOW QUOTAS`, `SHOW CREATE QUOTA`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowQuotas_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowQuotas.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW QUOTAS` statement if and only if the user has **show quotas** privilege,\n' + 'either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateQuota_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateQuota.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE QUOTA` statement if and only if\n' + 'the user has **show quotas** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowSettingsProfiles_Privilege = Requirement( + name='RQ.SRS-006.RBAC.ShowSettingsProfiles.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SHOW SETTINGS PROFILES` privilege when\n' + 'the user is granted `SHOW SETTINGS PROFILES`, `SHOW PROFILES`, `SHOW CREATE SETTINGS PROFILE`,\n' + '`SHOW SETTINGS PROFILE`, `SHOW ACCESS`, or `ACCESS MANAGEMENT`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowSettingsProfiles_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowSettingsProfiles.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW SETTINGS PROFILES` or `SHOW PROFILES` statement\n' + 'if and only if the user has **show settings profiles** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_ShowCreateSettingsProfile_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.ShowCreateSettingsProfile.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `SHOW CREATE SETTINGS PROFILE` or `SHOW CREATE PROFILE` statement\n' + 'if and only if the user has **show settings profiles** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_dictGet_Privilege = Requirement( + name='RQ.SRS-006.RBAC.dictGet.Privilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `dictGet` privilege when\n' + 'the user is granted `dictGet`, `dictHas`, `dictGetHierarchy`, or `dictIsIn`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_dictGet_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.dictGet.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `dictGet` statement\n' + 'if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_dictGet_Type_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.dictGet.Type.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `dictGet[TYPE]` statement\n' + 'if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role.\n' + 'Available types:\n' + '\n' + '* Int8\n' + '* Int16\n' + '* Int32\n' + '* Int64\n' + '* UInt8\n' + '* UInt16\n' + '* UInt32\n' + '* UInt64\n' + '* Float32\n' + '* Float64\n' + '* Date\n' + '* DateTime\n' + '* UUID\n' + '* String\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_dictGet_OrDefault_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.dictGet.OrDefault.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `dictGetOrDefault` statement\n' + 'if and only if the user has **dictGet** privilege on that dictionary, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_dictHas_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.dictHas.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `dictHas` statement\n' + 'if and only if the user has **dictGet** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_dictGetHierarchy_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.dictGetHierarchy.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `dictGetHierarchy` statement\n' + 'if and only if the user has **dictGet** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_dictIsIn_RequiredPrivilege = Requirement( + name='RQ.SRS-006.RBAC.dictIsIn.RequiredPrivilege', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `dictIsIn` statement\n' + 'if and only if the user has **dictGet** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Introspection = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Introspection', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `INTROSPECTION` privilege when\n' + 'the user is granted `INTROSPECTION` or `INTROSPECTION FUNCTIONS`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Introspection_addressToLine = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Introspection.addressToLine', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `addressToLine` statement if and only if\n' + 'the user has **introspection** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Introspection_addressToSymbol = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Introspection.addressToSymbol', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `addressToSymbol` statement if and only if\n' + 'the user has **introspection** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Introspection_demangle = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Introspection.demangle', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `demangle` statement if and only if\n' + 'the user has **introspection** privilege, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Shutdown = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Shutdown', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM SHUTDOWN` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM SHUTDOWN`, `SHUTDOWN`,or `SYSTEM KILL`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_DropCache = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.DropCache', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM DROP CACHE` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, or `DROP CACHE`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_DropCache_DNS = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.DropCache.DNS', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM DROP DNS CACHE` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP DNS CACHE`,\n' + '`SYSTEM DROP DNS`, `DROP DNS CACHE`, or `DROP DNS`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_DropCache_Mark = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.DropCache.Mark', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM DROP MARK CACHE` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP MARK CACHE`,\n' + '`SYSTEM DROP MARK`, `DROP MARK CACHE`, or `DROP MARKS`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_DropCache_Uncompressed = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.DropCache.Uncompressed', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM DROP UNCOMPRESSED CACHE` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM DROP CACHE`, `DROP CACHE`, `SYSTEM DROP UNCOMPRESSED CACHE`,\n' + '`SYSTEM DROP UNCOMPRESSED`, `DROP UNCOMPRESSED CACHE`, or `DROP UNCOMPRESSED`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Reload = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Reload', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM RELOAD` privilege when\n' + 'the user is granted `SYSTEM` or `SYSTEM RELOAD`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Reload_Config = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Reload.Config', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM RELOAD CONFIG` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD CONFIG`, or `RELOAD CONFIG`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Reload_Dictionary = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionary', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM RELOAD DICTIONARY` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARIES`, or `RELOAD DICTIONARY`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Reload_Dictionaries = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Reload.Dictionaries', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM RELOAD DICTIONARIES` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARIES`, `RELOAD DICTIONARIES`, or `RELOAD DICTIONARY`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Reload_EmbeddedDictionaries = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Reload.EmbeddedDictionaries', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM RELOAD`, `SYSTEM RELOAD DICTIONARY ON *.*`, or `SYSTEM RELOAD EMBEDDED DICTIONARIES`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Merges = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Merges', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM MERGES` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM MERGES`, `SYSTEM STOP MERGES`, `SYSTEM START MERGES`, `STOP MERGES`, or `START MERGES`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_TTLMerges = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.TTLMerges', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM TTL MERGES` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM TTL MERGES`, `SYSTEM STOP TTL MERGES`, `SYSTEM START TTL MERGES`, `STOP TTL MERGES`, or `START TTL MERGES`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Fetches = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Fetches', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM FETCHES` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM FETCHES`, `SYSTEM STOP FETCHES`, `SYSTEM START FETCHES`, `STOP FETCHES`, or `START FETCHES`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Moves = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Moves', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM MOVES` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM MOVES`, `SYSTEM STOP MOVES`, `SYSTEM START MOVES`, `STOP MOVES`, or `START MOVES`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Sends = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Sends', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM SENDS` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM SENDS`, `SYSTEM STOP SENDS`, `SYSTEM START SENDS`, `STOP SENDS`, or `START SENDS`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Sends_Distributed = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Sends.Distributed', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM DISTRIBUTED SENDS` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM DISTRIBUTED SENDS`, `SYSTEM STOP DISTRIBUTED SENDS`,\n' + '`SYSTEM START DISTRIBUTED SENDS`, `STOP DISTRIBUTED SENDS`, or `START DISTRIBUTED SENDS`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Sends_Replicated = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Sends.Replicated', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM REPLICATED SENDS` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM REPLICATED SENDS`, `SYSTEM STOP REPLICATED SENDS`,\n' + '`SYSTEM START REPLICATED SENDS`, `STOP REPLICATED SENDS`, or `START REPLICATED SENDS`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_ReplicationQueues = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.ReplicationQueues', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM REPLICATION QUEUES` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM REPLICATION QUEUES`, `SYSTEM STOP REPLICATION QUEUES`,\n' + '`SYSTEM START REPLICATION QUEUES`, `STOP REPLICATION QUEUES`, or `START REPLICATION QUEUES`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_SyncReplica = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.SyncReplica', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM SYNC REPLICA` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM SYNC REPLICA`, or `SYNC REPLICA`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_RestartReplica = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.RestartReplica', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM RESTART REPLICA` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM RESTART REPLICA`, or `RESTART REPLICA`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Flush = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Flush', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM FLUSH` privilege when\n' + 'the user is granted `SYSTEM` or `SYSTEM FLUSH`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Flush_Distributed = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Flush.Distributed', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM FLUSH DISTRIBUTED` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM FLUSH DISTRIBUTED`, or `FLUSH DISTRIBUTED`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_System_Flush_Logs = Requirement( + name='RQ.SRS-006.RBAC.Privileges.System.Flush.Logs', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully grant `SYSTEM FLUSH LOGS` privilege when\n' + 'the user is granted `SYSTEM`, `SYSTEM FLUSH LOGS`, or `FLUSH LOGS`.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking `SOURCES` privilege from\n' + 'the user, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_File = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.File', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `FILE` source by a user if and only if\n' + 'the user has `FILE` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_URL = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.URL', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `URL` source by a user if and only if\n' + 'the user has `URL` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_Remote = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.Remote', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `REMOTE` source by a user if and only if\n' + 'the user has `REMOTE` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_MySQL = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.MySQL', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `MySQL` source by a user if and only if\n' + 'the user has `MySQL` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_ODBC = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.ODBC', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `ODBC` source by a user if and only if\n' + 'the user has `ODBC` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_JDBC = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.JDBC', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `JDBC` source by a user if and only if\n' + 'the user has `JDBC` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_HDFS = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.HDFS', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `HDFS` source by a user if and only if\n' + 'the user has `HDFS` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_Sources_S3 = Requirement( + name='RQ.SRS-006.RBAC.Privileges.Sources.S3', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support the use of `S3` source by a user if and only if\n' + 'the user has `S3` or `SOURCES` privileges granted to them directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_GrantOption = Requirement( + name='RQ.SRS-006.RBAC.Privileges.GrantOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL successfully execute `GRANT` or `REVOKE` privilege statements by a user if and only if\n' + 'the user has that privilege with `GRANT OPTION`, either directly or through a role.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_All = Requirement( + name='RQ.SRS-006.RBAC.Privileges.All', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support granting or revoking `ALL` privilege.\n' + '\n' + ), + link=None) + +RQ_SRS_006_RBAC_Privileges_AdminOption = Requirement( + name='RQ.SRS-006.RBAC.Privileges.AdminOption', + version='1.0', + priority=None, + group=None, + type=None, + uid=None, + description=( + '[ClickHouse] SHALL support a user granting or revoking a role if and only if\n' + 'the user has that role with `ADMIN OPTION` privilege.\n' + '\n' + ), + link=None) diff --git a/tests/testflows/rbac/tests/privileges/admin_option.py b/tests/testflows/rbac/tests/privileges/admin_option.py new file mode 100644 index 00000000000..88dadc8522c --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/admin_option.py @@ -0,0 +1,119 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to grant role with `ADMIN OPTION` privilege granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(test=grant_role, flags=TE)(grant_target_name=user_name, user_name=user_name) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to grant role with `ADMIN OPTION` privilege granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(test=grant_role, flags=TE)(grant_target_name=role_name, user_name=user_name) + +@TestSuite +def grant_role(self, grant_target_name, user_name, node=None): + """Check that user is able to execute to grant roles if and only if they have role with `ADMIN OPTION`. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("Grant role without privilege"): + grant_role_name = f"grant_role_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name), role(node, grant_role_name): + + with When("I check the user can't grant a role"): + node.query(f"GRANT {grant_role_name} TO {target_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("Grant role with privilege"): + grant_role_name = f"grant_role_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name), role(node, grant_role_name): + + with When(f"I grant ADMIN OPTION"): + node.query(f"GRANT {grant_role_name} TO {grant_target_name} WITH ADMIN OPTION") + + with Then("I check the user can grant a role"): + node.query(f"GRANT {grant_role_name} TO {target_user_name}", settings = [("user", f"{user_name}")]) + + with Scenario("Grant role on cluster"): + grant_role_name = f"grant_role_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + try: + with Given("I have a role on a cluster"): + node.query(f"CREATE ROLE {grant_role_name} ON CLUSTER sharded_cluster") + + with And("I have a user on a cluster"): + node.query(f"CREATE USER {target_user_name} ON CLUSTER sharded_cluster") + + with When("I grant ADMIN OPTION privilege"): + node.query(f"GRANT {grant_role_name} TO {grant_target_name} WITH ADMIN OPTION") + + with Then("I check the user can grant a role"): + node.query(f"GRANT {grant_role_name} TO {target_user_name} ON CLUSTER sharded_cluster", settings = [("user", f"{user_name}")]) + + finally: + with Finally("I drop the user"): + node.query(f"DROP ROLE IF EXISTS {grant_role_name} ON CLUSTER sharded_cluster") + + with Scenario("Grant role with revoked privilege"): + grant_role_name = f"grant_role_{getuid()}" + target_user_name = f"target_user_{getuid()}" + + with user(node, target_user_name), role(node, grant_role_name): + + with When(f"I grant ADMIN OPTION"): + node.query(f"GRANT {grant_role_name} TO {grant_target_name} WITH ADMIN OPTION") + + with And(f"I revoke ADMIN OPTION"): + node.query(f"REVOKE {grant_role_name} FROM {grant_target_name}") + + with Then("I check the user cannot grant a role"): + node.query(f"GRANT {grant_role_name} TO {target_user_name}", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("admin option") +@Requirements( + RQ_SRS_006_RBAC_Privileges_AdminOption("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of ADMIN OPTION. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_column.py b/tests/testflows/rbac/tests/privileges/alter/alter_column.py index f32ca8ce9e2..430872029b5 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_column.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_column.py @@ -24,7 +24,7 @@ aliases = { "RENAME COLUMN" : ["ALTER RENAME COLUMN", "RENAME COLUMN"], "COMMENT COLUMN": ["ALTER COMMENT COLUMN", "COMMENT COLUMN"], "DROP COLUMN": ["ALTER DROP COLUMN", "DROP COLUMN"], - "ALTER COLUMN" : ["ALTER COLUMN"], #super-privilege + "ALTER COLUMN" : ["ALTER COLUMN", "ALL"], #super-privilege } # extra permutation is for 'ALTER COLUMN' super-privilege @@ -675,7 +675,8 @@ def scenario_parallelization(self, table_type, permutation): @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterColumn("1.0"), - RQ_SRS_006_RBAC_Privileges_AlterColumn_TableEngines("1.0") + RQ_SRS_006_RBAC_Privileges_AlterColumn_TableEngines("1.0"), + RQ_SRS_006_RBAC_Privileges_All("1.0") ) @Examples("table_type", [ (key,) for key in table_types.keys() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py b/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py index ea040c74a97..59ff1828222 100755 --- a/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_constraint.py @@ -17,7 +17,7 @@ subprivileges = { aliases = { "ADD CONSTRAINT" : ["ALTER ADD CONSTRAINT", "ADD CONSTRAINT"], "DROP CONSTRAINT": ["ALTER DROP CONSTRAINT", "DROP CONSTRAINT"], - "ALTER CONSTRAINT": ["ALTER CONSTRAINT", "CONSTRAINT"] # super-privilege + "ALTER CONSTRAINT": ["ALTER CONSTRAINT", "CONSTRAINT", "ALL"] # super-privilege } # Extra permutation is for 'ALTER CONSTRAINT' super-privilege @@ -274,7 +274,8 @@ def user_with_privileges_on_cluster(self, table_type, node=None): @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterConstraint("1.0"), - RQ_SRS_006_RBAC_Privileges_AlterConstraint_TableEngines("1.0") + RQ_SRS_006_RBAC_Privileges_AlterConstraint_TableEngines("1.0"), + RQ_SRS_006_RBAC_Privileges_All("1.0") ) @Examples("table_type", [ (key,) for key in table_types.keys() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_delete.py b/tests/testflows/rbac/tests/privileges/alter/alter_delete.py index a3b5c39271c..9f46e6cdb3d 100644 --- a/tests/testflows/rbac/tests/privileges/alter/alter_delete.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_delete.py @@ -7,7 +7,7 @@ from rbac.requirements import * from rbac.helper.common import * import rbac.helper.errors as errors -aliases = {"ALTER DELETE", "DELETE"} +aliases = {"ALTER DELETE", "DELETE", "ALL"} @TestSuite def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): @@ -65,6 +65,7 @@ def privilege_check(grant_target_name, user_name, table_type, privilege, node=No @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterDelete("1.0"), + RQ_SRS_006_RBAC_Privileges_All("1.0") ) @Examples("table_type", [ (key,) for key in table_types.keys() diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_fetch.py b/tests/testflows/rbac/tests/privileges/alter/alter_fetch.py index 54d1dcf1f51..56f2d48e7d2 100644 --- a/tests/testflows/rbac/tests/privileges/alter/alter_fetch.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_fetch.py @@ -7,7 +7,7 @@ from rbac.requirements import * from rbac.helper.common import * import rbac.helper.errors as errors -aliases = {"ALTER FETCH PARTITION", "FETCH PARTITION"} +aliases = {"ALTER FETCH PARTITION", "FETCH PARTITION", "ALL"} @TestSuite def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): @@ -44,7 +44,7 @@ def privilege_check(grant_target_name, user_name, table_type, privilege, node=No with table(node, table_name, table_type): with When("I attempt to fetch a partition without privilege"): - node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/tables/{{shard}}/{table_name}'", settings = [("user", user_name)], + node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/'", settings = [("user", user_name)], exitcode=exitcode, message=message) with Scenario("user with privilege", setup=instrument_clickhouse_server_log): @@ -55,7 +55,7 @@ def privilege_check(grant_target_name, user_name, table_type, privilege, node=No node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") with Then("I attempt to fetch a partition"): - node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/tables/{{shard}}/{table_name}'", settings = [("user", user_name)], + node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/'", settings = [("user", user_name)], exitcode=231, message="DB::Exception: No node") with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): @@ -69,12 +69,13 @@ def privilege_check(grant_target_name, user_name, table_type, privilege, node=No node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_target_name}") with Then("I attempt to fetch a partition"): - node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/tables/{{shard}}/{table_name}'", settings = [("user", user_name)], + node.query(f"ALTER TABLE {table_name} FETCH PARTITION 1 FROM '/clickhouse/'", settings = [("user", user_name)], exitcode=exitcode, message=message) @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterFetch("1.0"), + RQ_SRS_006_RBAC_Privileges_All("1.0") ) @Examples("table_type",[ ("ReplicatedMergeTree-sharded_cluster",), diff --git a/tests/testflows/rbac/tests/privileges/alter/alter_freeze.py b/tests/testflows/rbac/tests/privileges/alter/alter_freeze.py index b3d068b34a5..32bd4602044 100644 --- a/tests/testflows/rbac/tests/privileges/alter/alter_freeze.py +++ b/tests/testflows/rbac/tests/privileges/alter/alter_freeze.py @@ -7,7 +7,7 @@ from rbac.requirements import * from rbac.helper.common import * import rbac.helper.errors as errors -aliases = {"ALTER FREEZE PARTITION", "FREEZE PARTITION"} +aliases = {"ALTER FREEZE PARTITION", "FREEZE PARTITION", "ALL"} @TestSuite def privilege_granted_directly_or_via_role(self, table_type, privilege, node=None): @@ -39,6 +39,7 @@ def privilege_check(grant_target_name, user_name, table_type, privilege, node=No with Scenario("user without privilege", setup=instrument_clickhouse_server_log): table_name = f"merge_tree_{getuid()}" with table(node, table_name, table_type): + with When("I attempt to freeze partitions without privilege"): node.query(f"ALTER TABLE {table_name} FREEZE", settings = [("user", user_name)], exitcode=exitcode, message=message) @@ -46,18 +47,22 @@ def privilege_check(grant_target_name, user_name, table_type, privilege, node=No with Scenario("user with privilege", setup=instrument_clickhouse_server_log): table_name = f"merge_tree_{getuid()}" with table(node, table_name, table_type): + with When("I grant the freeze privilege"): node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") + with Then("I attempt to freeze partitions"): node.query(f"ALTER TABLE {table_name} FREEZE", settings = [("user", user_name)]) with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): table_name = f"merge_tree_{getuid()}" with table(node, table_name, table_type): + with When("I grant the freeze privilege"): node.query(f"GRANT {privilege} ON {table_name} TO {grant_target_name}") with And("I revoke the freeze privilege"): node.query(f"REVOKE {privilege} ON {table_name} FROM {grant_target_name}") + with Then("I attempt to freeze partitions"): node.query(f"ALTER TABLE {table_name} FREEZE", settings = [("user", user_name)], exitcode=exitcode, message=message) @@ -65,6 +70,7 @@ def privilege_check(grant_target_name, user_name, table_type, privilege, node=No @TestFeature @Requirements( RQ_SRS_006_RBAC_Privileges_AlterFreeze("1.0"), + RQ_SRS_006_RBAC_Privileges_All("1.0") ) @Examples("table_type", [ (key,) for key in table_types.keys() diff --git a/tests/testflows/rbac/tests/privileges/attach/attach_table.py b/tests/testflows/rbac/tests/privileges/attach/attach_table.py index 604e5a119e8..55c9efd369c 100644 --- a/tests/testflows/rbac/tests/privileges/attach/attach_table.py +++ b/tests/testflows/rbac/tests/privileges/attach/attach_table.py @@ -36,8 +36,9 @@ def privilege_check(grant_target_name, user_name, node=None): table_name = f"table_{getuid()}" try: + with When("I attempt to attach a table without privilege"): - node.query(f"ATTACH TABLE {table_name} (x Int8) ENGINE = Memory", settings = [("user", user_name)], + node.query(f"ATTACH TABLE {table_name}", settings = [("user", user_name)], exitcode=exitcode, message=message) finally: @@ -48,12 +49,12 @@ def privilege_check(grant_target_name, user_name, node=None): table_name = f"table_{getuid()}" try: - with When("I grant create table privilege"): + with When("I grant create table privilege"): node.query(f"GRANT CREATE TABLE ON *.* TO {grant_target_name}") with Then("I attempt to attach a table"): - node.query(f"ATTACH TABLE {table_name} (x Int8) ENGINE = Memory", settings = [("user", user_name)], - exitcode=80, message="DB::Exception: Incorrect ATTACH TABLE query") + node.query(f"ATTACH TABLE {table_name}", settings = [("user", user_name)], + exitcode=134, message=f"DB::Exception: Table `{table_name}` doesn't exist.") finally: with Finally("I drop the table"): @@ -70,7 +71,7 @@ def privilege_check(grant_target_name, user_name, node=None): node.query(f"REVOKE CREATE TABLE ON *.* FROM {grant_target_name}") with Then("I attempt to attach a table"): - node.query(f"ATTACH TABLE {table_name} (x Int8) ENGINE = Memory", settings = [("user", user_name)], + node.query(f"ATTACH TABLE {table_name}", settings = [("user", user_name)], exitcode=exitcode, message=message) finally: diff --git a/tests/testflows/rbac/tests/privileges/create/create_table.py b/tests/testflows/rbac/tests/privileges/create/create_table.py index c39b0c9a5e4..919e683f0f1 100644 --- a/tests/testflows/rbac/tests/privileges/create/create_table.py +++ b/tests/testflows/rbac/tests/privileges/create/create_table.py @@ -706,7 +706,7 @@ def create_as_merge(self, node=None): with When("I grant CREATE TABLE privilege to a user"): node.query(f"GRANT CREATE TABLE ON {table_name} TO {user_name}") - with And("I grant SELECT privilege to a user to allow executing the table function merge()"): + with And("I grant SELECT privilege on the source table"): node.query(f"GRANT SELECT ON {source_table_name} TO {user_name}") with Then("I try to create a table as another table"): @@ -722,6 +722,8 @@ def create_as_merge(self, node=None): ) @Name("create table") def feature(self, stress=None, parallel=None, node="clickhouse1"): + """Check the RBAC functionality of CREATE TABLE. + """ self.context.node = self.context.cluster.node(node) if stress is not None: diff --git a/tests/testflows/rbac/tests/privileges/detach/detach_view.py b/tests/testflows/rbac/tests/privileges/detach/detach_view.py index 899d8126c42..e6e8adad065 100644 --- a/tests/testflows/rbac/tests/privileges/detach/detach_view.py +++ b/tests/testflows/rbac/tests/privileges/detach/detach_view.py @@ -45,9 +45,9 @@ def privilege_check(grant_target_name, user_name, node=None): finally: with Finally("I reattach the view as a table", flags=TE): - node.query(f"ATTACH TABLE IF NOT EXISTS {view_name}") + node.query(f"ATTACH VIEW IF NOT EXISTS {view_name} AS SELECT 1") with And("I drop the view", flags=TE): - node.query(f"DROP TABLE IF EXISTS {view_name}") + node.query(f"DROP VIEW IF EXISTS {view_name}") with Scenario("user with privilege", setup=instrument_clickhouse_server_log): view_name = f"view_{getuid()}" @@ -64,9 +64,9 @@ def privilege_check(grant_target_name, user_name, node=None): finally: with Finally("I reattach the view as a table", flags=TE): - node.query(f"ATTACH TABLE IF NOT EXISTS {view_name}") + node.query(f"ATTACH VIEW IF NOT EXISTS {view_name} AS SELECT 1") with And("I drop the table", flags=TE): - node.query(f"DROP TABLE IF EXISTS {view_name}") + node.query(f"DROP VIEW IF EXISTS {view_name}") with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): view_name = f"view_{getuid()}" @@ -87,9 +87,9 @@ def privilege_check(grant_target_name, user_name, node=None): finally: with Finally("I reattach the view as a table", flags=TE): - node.query(f"ATTACH TABLE IF NOT EXISTS {view_name}") + node.query(f"ATTACH VIEW IF NOT EXISTS {view_name} AS SELECT 1") with And("I drop the view", flags=TE): - node.query(f"DROP TABLE IF EXISTS {view_name}") + node.query(f"DROP VIEW IF EXISTS {view_name}") @TestFeature @Requirements( diff --git a/tests/testflows/rbac/tests/privileges/dictGet.py b/tests/testflows/rbac/tests/privileges/dictGet.py new file mode 100644 index 00000000000..532fa798eb2 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/dictGet.py @@ -0,0 +1,646 @@ +import os + +from contextlib import contextmanager + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def dict_setup(node, table_name, dict_name, type="UInt64"): + """Setup and teardown of table and dictionary needed for the tests. + """ + + try: + with Given("I have a table"): + node.query(f"CREATE TABLE {table_name} (x UInt64, y UInt64, z {type}) ENGINE = Memory") + + with And("I have a dictionary"): + node.query(f"CREATE DICTIONARY {dict_name} (x UInt64 HIERARCHICAL IS_OBJECT_ID, y UInt64 HIERARCHICAL, z {type}) PRIMARY KEY x LAYOUT(FLAT()) SOURCE(CLICKHOUSE(host 'localhost' port 9000 user 'default' password '' db 'default' table '{table_name}')) LIFETIME(0)") + + yield + + finally: + with Finally("I drop the table", flags=TE): + node.query(f"DROP TABLE IF EXISTS {table_name}") + + with And("I drop the dictionary", flags=TE): + node.query(f"DROP DICTIONARY IF EXISTS {dict_name}") + +@TestSuite +def dictGet_granted_directly(self, node=None): + """Run dictGet checks with privileges granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=dictGet_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dictGet_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dictGet_granted_via_role(self, node=None): + """Run dictGet checks with privileges granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictGet_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dictGet_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("dictGet",), + ("dictHas",), + ("dictGetHierarchy",), + ("dictIsIn",), +]) +@Requirements( + RQ_SRS_006_RBAC_dictGet_RequiredPrivilege("1.0") +) +def dictGet_check(self, privilege, grant_target_name, user_name, node=None): + """Check that user is able to execute `dictGet` if and only if they have the necessary privileges. + """ + if node is None: + node = self.context.node + + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I attempt to dictGet without privilege"): + node.query(f"SELECT dictGet ({dict_name},'y',toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When(f"I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with Then("I attempt to dictGet with privilege"): + node.query(f"SELECT dictGet ({dict_name},'y',toUInt64(1))", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with And("I revoke privilege"): + node.query(f"REVOKE {privilege} ON {dict_name} FROM {grant_target_name}") + + with When("I attempt to dictGet without privilege"): + node.query(f"SELECT dictGet ({dict_name},'y',toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + +@TestSuite +def dictGetOrDefault_granted_directly(self, node=None): + """Run dictGetOrDefault checks with privileges granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=dictGetOrDefault_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dictGetOrDefault_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dictGetOrDefault_granted_via_role(self, node=None): + """Run dictGetOrDefault checks with privileges granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictGetOrDefault_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dictGetOrDefault_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("dictGet",), + ("dictHas",), + ("dictGetHierarchy",), + ("dictIsIn",), +]) +@Requirements( + RQ_SRS_006_RBAC_dictGet_OrDefault_RequiredPrivilege("1.0") +) +def dictGetOrDefault_check(self, privilege, grant_target_name, user_name, node=None): + """Check that user is able to execute `dictGetOrDefault` if and only if they have the necessary privileges. + """ + if node is None: + node = self.context.node + + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I attempt to dictGetOrDefault without privilege"): + node.query(f"SELECT dictGetOrDefault ({dict_name},'y',toUInt64(1),toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When(f"I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with Then("I attempt to dictGetOrDefault with privilege"): + node.query(f"SELECT dictGetOrDefault ({dict_name},'y',toUInt64(1),toUInt64(1))", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with And("I revoke privilege"): + node.query(f"REVOKE {privilege} ON {dict_name} FROM {grant_target_name}") + + with When("I attempt to dictGetOrDefault without privilege"): + node.query(f"SELECT dictGetOrDefault ({dict_name},'y',toUInt64(1),toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + +@TestSuite +def dictHas_granted_directly(self, node=None): + """Run dictHas checks with privileges granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=dictHas_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dictHas_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dictHas_granted_via_role(self, node=None): + """Run checks with privileges granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictHas_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dictHas_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("dictGet",), + ("dictHas",), + ("dictGetHierarchy",), + ("dictIsIn",), +]) +@Requirements( + RQ_SRS_006_RBAC_dictHas_RequiredPrivilege("1.0") +) +def dictHas_check(self, privilege, grant_target_name, user_name, node=None): + """Check that user is able to execute `dictHas` if and only if they have the necessary privileges. + """ + if node is None: + node = self.context.node + + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I attempt to dictHas without privilege"): + node.query(f"SELECT dictHas({dict_name},toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with Then("I attempt to dictHas with privilege"): + node.query(f"SELECT dictHas({dict_name},toUInt64(1))", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with And("I revoke privilege"): + node.query(f"REVOKE {privilege} ON {dict_name} FROM {grant_target_name}") + + with When("I attempt to dictHas without privilege"): + node.query(f"SELECT dictHas({dict_name},toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + +@TestSuite +def dictGetHierarchy_granted_directly(self, node=None): + """Run dictGetHierarchy checks with privileges granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + Suite(run=dictGetHierarchy_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dictGetHierarchy_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dictGetHierarchy_granted_via_role(self, node=None): + """Run checks with privileges granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictGetHierarchy_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dictGetHierarchy_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("dictGet",), + ("dictHas",), + ("dictGetHierarchy",), + ("dictIsIn",), +]) +@Requirements( + RQ_SRS_006_RBAC_dictGetHierarchy_RequiredPrivilege("1.0") +) +def dictGetHierarchy_check(self, privilege, grant_target_name, user_name, node=None): + """Check that user is able to execute `dictGetHierarchy` if and only if they have the necessary privileges. + """ + if node is None: + node = self.context.node + + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I attempt to dictGetHierarchy without privilege"): + node.query(f"SELECT dictGetHierarchy({dict_name},toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with Then("I attempt to dictGetHierarchy with privilege"): + node.query(f"SELECT dictGetHierarchy({dict_name},toUInt64(1))", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with And("I revoke privilege"): + node.query(f"REVOKE {privilege} ON {dict_name} FROM {grant_target_name}") + + with When("I attempt to dictGetHierarchy without privilege"): + node.query(f"SELECT dictGetHierarchy({dict_name},toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + +@TestSuite +def dictIsIn_granted_directly(self, node=None): + """Run dictIsIn checks with privileges granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + Suite(run=dictIsIn_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dictIsIn_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dictIsIn_granted_via_role(self, node=None): + """Run checks with privileges granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictIsIn_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dictIsIn_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("dictGet",), + ("dictHas",), + ("dictGetHierarchy",), + ("dictIsIn",), +]) +@Requirements( + RQ_SRS_006_RBAC_dictIsIn_RequiredPrivilege("1.0") +) +def dictIsIn_check(self, privilege, grant_target_name, user_name, node=None): + """Check that user is able to execute `dictIsIn` if and only if they have the necessary privileges. + """ + if node is None: + node = self.context.node + + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I attempt to dictIsIn without privilege"): + node.query(f"SELECT dictIsIn({dict_name},toUInt64(1),toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with Then("I attempt to dictIsIn with privilege"): + node.query(f"SELECT dictIsIn({dict_name},toUInt64(1),toUInt64(1))", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with And("I revoke privilege"): + node.query(f"REVOKE {privilege} ON {dict_name} FROM {grant_target_name}") + + with When("I attempt to dictIsIn without privilege"): + node.query(f"SELECT dictIsIn({dict_name},toUInt64(1),toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + +@TestSuite +@Examples("type",[ + ("Int8",), + ("Int16",), + ("Int32",), + ("Int64",), + ("UInt8",), + ("UInt16",), + ("UInt32",), + ("UInt64",), + ("Float32",), + ("Float64",), + ("Date",), + ("DateTime",), + ("UUID",), + ("String",), +]) +def dictGetType_granted_directly(self, type, node=None): + """Run checks on dictGet with a type specified with privileges granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + Suite(run=dictGetType_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name type", [ + tuple(list(row)+[user_name,user_name,type]) for row in dictGetType_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +@Examples("type",[ + ("Int8",), + ("Int16",), + ("Int32",), + ("Int64",), + ("UInt8",), + ("UInt16",), + ("UInt32",), + ("UInt64",), + ("Float32",), + ("Float64",), + ("Date",), + ("DateTime",), + ("UUID",), + ("String",), +]) +def dictGetType_granted_via_role(self, type, node=None): + """Run checks on dictGet with a type specified with privileges granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictGetType_check, setup=instrument_clickhouse_server_log, + examples=Examples("privilege grant_target_name user_name type", [ + tuple(list(row)+[role_name,user_name,type]) for row in dictGetType_check.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("dictGet",), + ("dictHas",), + ("dictGetHierarchy",), + ("dictIsIn",), +]) +@Requirements( + RQ_SRS_006_RBAC_dictGet_Type_RequiredPrivilege("1.0") +) +def dictGetType_check(self, privilege, grant_target_name, user_name, type, node=None): + """Check that user is able to execute `dictGet` if and only if they have the necessary privileges. + """ + if node is None: + node = self.context.node + + exitcode, message = errors.not_enough_privileges(name=f"{user_name}") + + with Scenario("user without privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name, type): + + with When("I attempt to dictGet without privilege"): + node.query(f"SELECT dictGet{type}({dict_name},'z',toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + + with Scenario("user with privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name, type): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with Then("I attempt to dictGet with privilege"): + node.query(f"SELECT dictGet{type}({dict_name},'z',toUInt64(1))", settings = [("user", user_name)]) + + with Scenario("user with revoked privilege", setup=instrument_clickhouse_server_log): + dict_name = f"dict_{getuid()}" + + with dict_setup(node, table_name, dict_name, type): + + with When("I grant privilege"): + node.query(f"GRANT {privilege} ON {dict_name} TO {grant_target_name}") + + with And("I revoke privilege"): + node.query(f"REVOKE {privilege} ON {dict_name} FROM {grant_target_name}") + + with When("I attempt to dictGet without privilege"): + node.query(f"SELECT dictGet{type}({dict_name},'z',toUInt64(1))", settings = [("user", user_name)], exitcode=exitcode, message=message) + +@TestFeature +@Requirements( + RQ_SRS_006_RBAC_dictGet_Privilege("1.0") +) +@Name("dictGet") +def feature(self, node="clickhouse1", stress=None, parallel=None): + """Check the RBAC functionality of dictGet. + """ + self.context.node = self.context.cluster.node(node) + + if parallel is not None: + self.context.parallel = parallel + if stress is not None: + self.context.stress = stress + + pool = Pool(20) + try: + tasks = [] + try: + + run_scenario(pool, tasks, Suite(test=dictGet_granted_directly)) + run_scenario(pool, tasks, Suite(test=dictGet_granted_via_role)) + run_scenario(pool, tasks, Suite(test=dictGetOrDefault_granted_directly)) + run_scenario(pool, tasks, Suite(test=dictGetOrDefault_granted_via_role)) + run_scenario(pool, tasks, Suite(test=dictHas_granted_directly)) + run_scenario(pool, tasks, Suite(test=dictHas_granted_via_role)) + run_scenario(pool, tasks, Suite(test=dictGetHierarchy_granted_directly)) + run_scenario(pool, tasks, Suite(test=dictGetHierarchy_granted_via_role)) + run_scenario(pool, tasks, Suite(test=dictIsIn_granted_directly)) + run_scenario(pool, tasks, Suite(test=dictIsIn_granted_via_role)) + + for example in dictGetType_granted_directly.examples: + type, = example + + with Example(example): + run_scenario(pool, tasks, Suite(test=dictGetType_granted_directly),{"type" : type}) + run_scenario(pool, tasks, Suite(test=dictGetType_granted_via_role),{"type" : type}) + + finally: + join(tasks) + finally: + pool.close() diff --git a/tests/testflows/rbac/tests/privileges/distributed_table.py b/tests/testflows/rbac/tests/privileges/distributed_table.py index 39c95c73f37..5b62448a446 100755 --- a/tests/testflows/rbac/tests/privileges/distributed_table.py +++ b/tests/testflows/rbac/tests/privileges/distributed_table.py @@ -61,7 +61,7 @@ def table(self, name, cluster=None, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Table_DistributedTable_Create("1.0"), + RQ_SRS_006_RBAC_DistributedTable_Create("1.0"), ) def create(self): """Check the RBAC functionality of distributed table with CREATE. @@ -161,7 +161,7 @@ def create_with_privilege(self, user_name, grant_target_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Table_DistributedTable_Select("1.0"), + RQ_SRS_006_RBAC_DistributedTable_Select("1.0"), ) def select(self): """Check the RBAC functionality of distributed table with SELECT. @@ -275,7 +275,7 @@ def select_with_privilege(self, user_name, grant_target_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Table_DistributedTable_Insert("1.0"), + RQ_SRS_006_RBAC_DistributedTable_Insert("1.0"), ) def insert(self): """Check the RBAC functionality of distributed table with INSERT. @@ -389,7 +389,7 @@ def insert_with_privilege(self, user_name, grant_target_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Table_DistributedTable_SpecialTables("1.0"), + RQ_SRS_006_RBAC_DistributedTable_SpecialTables("1.0"), ) def special_cases(self): """Check that the user is able to successfully execute queries on distributed tables using special tables, @@ -875,7 +875,7 @@ def insert_with_table_on_distributed_table(self, user_name, grant_target_name, n " where one replica is on clickhouse1 and another on clickhouse2 accessed from clickhouse1")), ]) @Requirements( - RQ_SRS_006_RBAC_Table_DistributedTable_LocalUser("1.0") + RQ_SRS_006_RBAC_DistributedTable_LocalUser("1.0") ) def local_user(self, cluster, node=None): """Check that a user that exists locally and not present on the remote nodes @@ -912,7 +912,7 @@ def local_user(self, cluster, node=None): @TestScenario @Requirements( - RQ_SRS_006_RBAC_Table_DistributedTable_SameUserDifferentNodesDifferentPrivileges("1.0") + RQ_SRS_006_RBAC_DistributedTable_SameUserDifferentNodesDifferentPrivileges("1.0") ) def multiple_node_user(self, node=None): """Check that a user that exists on multiple nodes with different privileges on each is able to execute queries diff --git a/tests/testflows/rbac/tests/privileges/feature.py b/tests/testflows/rbac/tests/privileges/feature.py index ed9c0ae32e8..ee141ef12dc 100755 --- a/tests/testflows/rbac/tests/privileges/feature.py +++ b/tests/testflows/rbac/tests/privileges/feature.py @@ -7,7 +7,7 @@ from rbac.helper.common import * def feature(self): tasks = [] - pool = Pool(16) + pool = Pool(10) try: try: @@ -21,6 +21,10 @@ def feature(self): run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.kill_query", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.kill_mutation", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.role_admin", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.dictGet", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.introspection", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.sources", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.admin_option", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_tables", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.show.show_dictionaries", "feature"), flags=TE), {}) @@ -76,6 +80,20 @@ def feature(self): run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.detach.detach_dictionary", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.detach.detach_table", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.detach.detach_view", "feature"), flags=TE), {}) + + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.drop_cache", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.reload", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.flush", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.merges", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.moves", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.replication_queues", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.ttl_merges", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.restart_replica", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.sends", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.sync_replica", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.fetches", "feature"), flags=TE), {}) + run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.shutdown", "feature"), flags=TE), {}) + finally: join(tasks) finally: diff --git a/tests/testflows/rbac/tests/privileges/insert.py b/tests/testflows/rbac/tests/privileges/insert.py index 372aa5c43a5..78478967d37 100755 --- a/tests/testflows/rbac/tests/privileges/insert.py +++ b/tests/testflows/rbac/tests/privileges/insert.py @@ -96,7 +96,7 @@ def user_with_privilege_on_columns(self, table_type): @TestOutline @Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_Column("1.0"), + RQ_SRS_006_RBAC_Insert_Column("1.0"), ) @Examples("grant_columns revoke_columns insert_columns_fail insert_columns_pass data_fail data_pass", [ ("d", "d", "x", "d", '\'woo\'', '\'2020-01-01\''), @@ -219,7 +219,7 @@ def role_with_privilege_on_columns(self, table_type): @TestOutline @Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_Column("1.0"), + RQ_SRS_006_RBAC_Insert_Column("1.0"), ) @Examples("grant_columns revoke_columns insert_columns_fail insert_columns_pass data_fail data_pass", [ ("d", "d", "x", "d", '\'woo\'', '\'2020-01-01\''), @@ -264,7 +264,7 @@ def role_column_privileges(self, grant_columns, insert_columns_pass, data_fail, @TestScenario @Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_Cluster("1.0"), + RQ_SRS_006_RBAC_Insert_Cluster("1.0"), ) def user_with_privilege_on_cluster(self, table_type, node=None): """Check that user is able or unable to insert into a table @@ -305,7 +305,7 @@ def user_with_privilege_on_cluster(self, table_type, node=None): @TestScenario @Requirements( - RQ_SRS_006_RBAC_Privileges_Insert_Cluster("1.0"), + RQ_SRS_006_RBAC_Insert_Cluster("1.0"), ) def role_with_privilege_on_cluster(self, table_type, node=None): """Check that user with role is able to insert into a table @@ -351,8 +351,8 @@ def role_with_privilege_on_cluster(self, table_type, node=None): @TestOutline(Feature) @Requirements( - RQ_SRS_006_RBAC_Privileges_Insert("1.0"), - RQ_SRS_006_RBAC_Privileges_Insert_TableEngines("1.0") + RQ_SRS_006_RBAC_Insert("1.0"), + RQ_SRS_006_RBAC_Insert_TableEngines("1.0") ) @Examples("table_type", [ (key,) for key in table_types.keys() @@ -360,6 +360,8 @@ def role_with_privilege_on_cluster(self, table_type, node=None): @Flags(TE) @Name("insert") def feature(self, table_type, parallel=None, stress=None, node="clickhouse1"): + """Check the RBAC functionality of INSERT. + """ self.context.node = self.context.cluster.node(node) self.context.node1 = self.context.cluster.node("clickhouse1") diff --git a/tests/testflows/rbac/tests/privileges/introspection.py b/tests/testflows/rbac/tests/privileges/introspection.py new file mode 100644 index 00000000000..f8d774902ab --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/introspection.py @@ -0,0 +1,285 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def allow_introspection_functions(node): + setting = ("allow_introspection_functions", 1) + default_query_settings = None + + try: + with Given("I add allow_introspection_functions to the default query settings"): + default_query_settings = getsattr(current().context, "default_query_settings", []) + default_query_settings.append(setting) + yield + finally: + with Finally("I remove allow_introspection_functions from the default query settings"): + if default_query_settings: + try: + default_query_settings.pop(default_query_settings.index(setting)) + except ValueError: + pass + +@TestSuite +def addressToLine_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `addressToLine` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=addressToLine, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in addressToLine.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def addressToLine_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `addressToLine` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=addressToLine, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in addressToLine.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("INTROSPECTION",), + ("INTROSPECTION FUNCTIONS",), + ("addressToLine",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Introspection_addressToLine("1.0"), +) +def addressToLine(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `addressToLine` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("addressToLine without privilege"): + + with When("I check the user can't use addressToLine"): + node.query(f"WITH addressToLine(toUInt64(dummy)) AS addr SELECT 1 WHERE addr = ''", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("addressToLine with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use addressToLine"): + node.query(f"WITH addressToLine(toUInt64(dummy)) AS addr SELECT 1 WHERE addr = ''", settings = [("user", f"{user_name}")]) + + with Scenario("addressToLine with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use addressToLine"): + node.query(f"WITH addressToLine(toUInt64(dummy)) AS addr SELECT 1 WHERE addr = ''", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def addressToSymbol_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `addressToSymbol` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=addressToSymbol, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in addressToSymbol.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def addressToSymbol_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `addressToSymbol` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=addressToSymbol, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in addressToSymbol.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("INTROSPECTION",), + ("INTROSPECTION FUNCTIONS",), + ("addressToSymbol",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Introspection_addressToSymbol("1.0"), +) +def addressToSymbol(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `addressToSymbol` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("addressToSymbol without privilege"): + + with When("I check the user can't use addressToSymbol"): + node.query(f"WITH addressToSymbol(toUInt64(dummy)) AS addr SELECT 1 WHERE addr = ''", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("addressToSymbol with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use addressToSymbol"): + node.query(f"WITH addressToSymbol(toUInt64(dummy)) AS addr SELECT 1 WHERE addr = ''", settings = [("user", f"{user_name}")]) + + with Scenario("addressToSymbol with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use addressToSymbol"): + node.query(f"WITH addressToSymbol(toUInt64(dummy)) AS addr SELECT 1 WHERE addr = ''", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def demangle_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `demangle` with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=demangle, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in demangle.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def demangle_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `demangle` with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=demangle, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in demangle.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("INTROSPECTION",), + ("INTROSPECTION FUNCTIONS",), + ("demangle",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Introspection_demangle("1.0"), +) +def demangle(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `demangle` when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("demangle without privilege"): + + with When("I check the user can't use demangle"): + node.query(f"WITH demangle(toString(dummy)) AS addr SELECT 1 WHERE addr = ''", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("demangle with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use demangle"): + node.query(f"WITH demangle(toString(dummy)) AS addr SELECT 1 WHERE addr = ''", settings = [("user", f"{user_name}")]) + + with Scenario("demangle with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use demangle"): + node.query(f"WITH demangle(toString(dummy)) AS addr SELECT 1 WHERE addr = ''", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("introspection") +@Requirements( + RQ_SRS_006_RBAC_Privileges_Introspection("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of INTROSPECTION. + """ + self.context.node = self.context.cluster.node(node) + + with allow_introspection_functions(self.context.node): + Suite(run=addressToLine_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=addressToLine_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=addressToSymbol_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=addressToSymbol_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=demangle_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=demangle_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/public_tables.py b/tests/testflows/rbac/tests/privileges/public_tables.py index f4e457d71c4..98077438946 100755 --- a/tests/testflows/rbac/tests/privileges/public_tables.py +++ b/tests/testflows/rbac/tests/privileges/public_tables.py @@ -18,7 +18,8 @@ def public_tables(self, node=None): node = self.context.node with user(node, f"{user_name}"): - with Then("I check the user is able to select on system.one"): + + with When("I check the user is able to select on system.one"): node.query("SELECT count(*) FROM system.one", settings = [("user",user_name)]) with And("I check the user is able to select on system.numbers"): @@ -32,10 +33,10 @@ def public_tables(self, node=None): @TestScenario @Requirements( - RQ_SRS_006_RBAC_Table_QueryLog("1.0"), + RQ_SRS_006_RBAC_Table_SensitiveTables("1.0"), ) -def query_log(self, node=None): - """Check that a user with no privilege is only able to see their own queries. +def sensitive_tables(self, node=None): + """Check that a user with no privilege is not able to see from these tables. """ user_name = f"user_{getuid()}" if node is None: @@ -45,8 +46,48 @@ def query_log(self, node=None): with Given("I create a query"): node.query("SELECT 1") - with Then("The user reads system.query_log"): - output = node.query("SELECT count() FROM system.query_log", settings = [("user",user_name)]).output + with When("I select from processes"): + output = node.query("SELECT count(*) FROM system.processes", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from query_log"): + output = node.query("SELECT count(*) FROM system.query_log", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from query_thread_log"): + output = node.query("SELECT count(*) FROM system.query_thread_log", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from clusters"): + output = node.query("SELECT count(*) FROM system.clusters", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from events"): + output = node.query("SELECT count(*) FROM system.events", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from graphite_retentions"): + output = node.query("SELECT count(*) FROM system.graphite_retentions", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from stack_trace"): + output = node.query("SELECT count(*) FROM system.stack_trace", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from trace_log"): + output = node.query("SELECT count(*) FROM system.trace_log", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from user_directories"): + output = node.query("SELECT count(*) FROM system.user_directories", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from zookeeper"): + output = node.query("SELECT count(*) FROM system.zookeeper WHERE path = '/clickhouse' ", settings = [("user",user_name)]).output + assert output == 0, error() + + with And("I select from macros"): + output = node.query("SELECT count(*) FROM system.macros", settings = [("user",user_name)]).output assert output == 0, error() @TestFeature @@ -55,4 +96,4 @@ def feature(self, node="clickhouse1"): self.context.node = self.context.cluster.node(node) Scenario(run=public_tables, setup=instrument_clickhouse_server_log, flags=TE) - Scenario(run=query_log, setup=instrument_clickhouse_server_log, flags=TE) \ No newline at end of file + Scenario(run=sensitive_tables, setup=instrument_clickhouse_server_log, flags=TE) \ No newline at end of file diff --git a/tests/testflows/rbac/tests/privileges/role_admin.py b/tests/testflows/rbac/tests/privileges/role_admin.py index 91d663d0de8..955b0fcd258 100644 --- a/tests/testflows/rbac/tests/privileges/role_admin.py +++ b/tests/testflows/rbac/tests/privileges/role_admin.py @@ -95,10 +95,10 @@ def role_admin(self, grant_target_name, user_name, node=None): with user(node, target_user_name), role(node, role_admin_name): - with When(f"I grant ROLE ADMIN on the database"): + with When(f"I grant ROLE ADMIN"): node.query(f"GRANT ROLE ADMIN ON *.* TO {grant_target_name}") - with And(f"I revoke ROLE ADMIN on the database"): + with And(f"I revoke ROLE ADMIN"): node.query(f"REVOKE ROLE ADMIN ON *.* FROM {grant_target_name}") with Then("I check the user cannot grant a role"): diff --git a/tests/testflows/rbac/tests/privileges/select.py b/tests/testflows/rbac/tests/privileges/select.py index 581f7c902f1..e58dd8184d2 100755 --- a/tests/testflows/rbac/tests/privileges/select.py +++ b/tests/testflows/rbac/tests/privileges/select.py @@ -76,7 +76,7 @@ def user_with_privilege_on_columns(self, table_type): @TestOutline @Requirements( - RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"), + RQ_SRS_006_RBAC_Select_Column("1.0"), ) @Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass", [ ("d", "d", "x", "d", '\'2020-01-01\''), @@ -197,7 +197,7 @@ def role_with_privilege_on_columns(self, table_type): @TestOutline @Requirements( - RQ_SRS_006_RBAC_Privileges_Select_Column("1.0"), + RQ_SRS_006_RBAC_Select_Column("1.0"), ) @Examples("grant_columns revoke_columns select_columns_fail select_columns_pass data_pass", [ ("d", "d", "x", "d", '\'2020-01-01\''), @@ -241,7 +241,7 @@ def role_column_privileges(self, grant_columns, select_columns_pass, data_pass, @TestScenario @Requirements( - RQ_SRS_006_RBAC_Privileges_Select_Cluster("1.0"), + RQ_SRS_006_RBAC_Select_Cluster("1.0"), ) def user_with_privilege_on_cluster(self, table_type, node=None): """Check that user is able to select from a table with @@ -270,14 +270,16 @@ def user_with_privilege_on_cluster(self, table_type, node=None): @TestOutline(Feature) @Requirements( - RQ_SRS_006_RBAC_Privileges_Select("1.0"), - RQ_SRS_006_RBAC_Privileges_Select_TableEngines("1.0") + RQ_SRS_006_RBAC_Select("1.0"), + RQ_SRS_006_RBAC_Select_TableEngines("1.0") ) @Examples("table_type", [ (key,) for key in table_types.keys() ]) @Name("select") def feature(self, table_type, parallel=None, stress=None, node="clickhouse1"): + """Check the RBAC functionality of SELECT. + """ self.context.node = self.context.cluster.node(node) if stress is not None: diff --git a/tests/testflows/rbac/tests/privileges/show/show_columns.py b/tests/testflows/rbac/tests/privileges/show/show_columns.py index d57d0e0146c..996663cdcbc 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_columns.py +++ b/tests/testflows/rbac/tests/privileges/show/show_columns.py @@ -41,7 +41,7 @@ def describe_with_privilege_granted_via_role(self, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_DescribeTable("1.0"), + RQ_SRS_006_RBAC_DescribeTable_RequiredPrivilege("1.0"), ) def describe(self, grant_target_name, user_name, table_name, node=None): """Check that user is able to execute DESCRIBE only when they have SHOW COLUMNS privilege. @@ -112,7 +112,7 @@ def show_create_with_privilege_granted_via_role(self, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateTable("1.0"), + RQ_SRS_006_RBAC_ShowCreateTable_RequiredPrivilege("1.0"), ) def show_create(self, grant_target_name, user_name, table_name, node=None): """Check that user is able to execute SHOW CREATE on a table only when they have SHOW COLUMNS privilege. @@ -150,7 +150,7 @@ def show_create(self, grant_target_name, user_name, table_name, node=None): @TestFeature @Name("show columns") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowColumns("1.0") + RQ_SRS_006_RBAC_ShowColumns_Privilege("1.0") ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW COLUMNS. diff --git a/tests/testflows/rbac/tests/privileges/show/show_databases.py b/tests/testflows/rbac/tests/privileges/show/show_databases.py index 7b13dfe3ad2..27d10ef2b73 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_databases.py +++ b/tests/testflows/rbac/tests/privileges/show/show_databases.py @@ -71,7 +71,7 @@ def check_privilege(self, privilege, on, grant_target_name, user_name, db_name, @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowDatabases_Query("1.0"), + RQ_SRS_006_RBAC_ShowDatabases_RequiredPrivilege("1.0"), ) def show_db(self, privilege, on, grant_target_name, user_name, db_name, node=None): """Check that user is only able to see a database in SHOW DATABASES when they have a privilege on that database. @@ -114,7 +114,7 @@ def show_db(self, privilege, on, grant_target_name, user_name, db_name, node=Non @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_UseDatabase("1.0"), + RQ_SRS_006_RBAC_UseDatabase_RequiredPrivilege("1.0"), ) def use(self, privilege, on, grant_target_name, user_name, db_name, node=None): """Check that user is able to execute EXISTS on a database if and only if the user has SHOW DATABASE privilege @@ -158,7 +158,7 @@ def use(self, privilege, on, grant_target_name, user_name, db_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateDatabase("1.0"), + RQ_SRS_006_RBAC_ShowCreateDatabase_RequiredPrivilege("1.0"), ) def show_create(self, privilege, on, grant_target_name, user_name, db_name, node=None): """Check that user is able to execute EXISTS on a database if and only if the user has SHOW DATABASE privilege @@ -203,7 +203,7 @@ def show_create(self, privilege, on, grant_target_name, user_name, db_name, node @TestFeature @Name("show databases") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowDatabases("1.0") + RQ_SRS_006_RBAC_ShowDatabases_Privilege("1.0") ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW DATABASES. diff --git a/tests/testflows/rbac/tests/privileges/show/show_dictionaries.py b/tests/testflows/rbac/tests/privileges/show/show_dictionaries.py index de076dd9bd2..ec5617af904 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_dictionaries.py +++ b/tests/testflows/rbac/tests/privileges/show/show_dictionaries.py @@ -71,7 +71,7 @@ def check_privilege(self, privilege, on, grant_target_name, user_name, dict_name @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowDictionaries_Query("1.0"), + RQ_SRS_006_RBAC_ShowDictionaries_RequiredPrivilege("1.0"), ) def show_dict(self, privilege, on, grant_target_name, user_name, dict_name, node=None): """Check that user is only able to see a dictionary in SHOW DICTIONARIES @@ -115,7 +115,7 @@ def show_dict(self, privilege, on, grant_target_name, user_name, dict_name, node @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ExistsDictionary("1.0"), + RQ_SRS_006_RBAC_ExistsDictionary_RequiredPrivilege("1.0"), ) def exists(self, privilege, on, grant_target_name, user_name, dict_name, node=None): """Check that user is able to execute EXISTS on a dictionary if and only if the user has SHOW DICTIONARY privilege @@ -159,7 +159,7 @@ def exists(self, privilege, on, grant_target_name, user_name, dict_name, node=No @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateDictionary("1.0"), + RQ_SRS_006_RBAC_ShowCreateDictionary_RequiredPrivilege("1.0"), ) def show_create(self, privilege, on, grant_target_name, user_name, dict_name, node=None): """Check that user is able to execute SHOW CREATE on a dictionary if and only if the user has SHOW DICTIONARY privilege @@ -204,7 +204,7 @@ def show_create(self, privilege, on, grant_target_name, user_name, dict_name, no @TestFeature @Name("show dictionaries") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowDictionaries("1.0"), + RQ_SRS_006_RBAC_ShowDictionaries_Privilege("1.0"), ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW DICTIONARIES. diff --git a/tests/testflows/rbac/tests/privileges/show/show_quotas.py b/tests/testflows/rbac/tests/privileges/show/show_quotas.py index 74c77db9f98..d84b5192677 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_quotas.py +++ b/tests/testflows/rbac/tests/privileges/show/show_quotas.py @@ -74,7 +74,7 @@ def check_privilege(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowQuotas_Query("1.0"), + RQ_SRS_006_RBAC_ShowQuotas_RequiredPrivilege("1.0"), ) def show_quotas(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW QUOTAS` when they have the necessary privilege. @@ -112,7 +112,7 @@ def show_quotas(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateQuota("1.0"), + RQ_SRS_006_RBAC_ShowCreateQuota_RequiredPrivilege("1.0"), ) def show_create(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW CREATE QUOTA` when they have the necessary privilege. @@ -160,7 +160,7 @@ def show_create(self, privilege, grant_target_name, user_name, node=None): @TestFeature @Name("show quotas") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowQuotas("1.0"), + RQ_SRS_006_RBAC_ShowQuotas_Privilege("1.0"), ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW QUOTAS. diff --git a/tests/testflows/rbac/tests/privileges/show/show_roles.py b/tests/testflows/rbac/tests/privileges/show/show_roles.py index 64eeb5c599d..3106e1c5df3 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_roles.py +++ b/tests/testflows/rbac/tests/privileges/show/show_roles.py @@ -62,7 +62,7 @@ def check_privilege(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowRoles_Query("1.0"), + RQ_SRS_006_RBAC_ShowRoles_RequiredPrivilege("1.0"), ) def show_roles(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW ROLES` when they have the necessary privilege. @@ -100,7 +100,7 @@ def show_roles(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateRole("1.0"), + RQ_SRS_006_RBAC_ShowCreateRole_RequiredPrivilege("1.0"), ) def show_create(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW CREATE ROLE` when they have the necessary privilege. @@ -148,7 +148,7 @@ def show_create(self, privilege, grant_target_name, user_name, node=None): @TestFeature @Name("show roles") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowRoles("1.0"), + RQ_SRS_006_RBAC_ShowRoles_Privilege("1.0"), ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW ROLES. diff --git a/tests/testflows/rbac/tests/privileges/show/show_row_policies.py b/tests/testflows/rbac/tests/privileges/show/show_row_policies.py index a375545e6b8..cfa25284cee 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_row_policies.py +++ b/tests/testflows/rbac/tests/privileges/show/show_row_policies.py @@ -76,7 +76,7 @@ def check_privilege(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowRowPolicies_Query("1.0"), + RQ_SRS_006_RBAC_ShowRowPolicies_RequiredPrivilege("1.0"), ) def show_row_policies(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW ROW POLICIES` when they have the necessary privilege. @@ -114,7 +114,7 @@ def show_row_policies(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateRowPolicy("1.0"), + RQ_SRS_006_RBAC_ShowCreateRowPolicy_RequiredPrivilege("1.0"), ) def show_create(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW CREATE ROW POLICY` when they have the necessary privilege. @@ -165,7 +165,7 @@ def show_create(self, privilege, grant_target_name, user_name, node=None): @TestFeature @Name("show row policies") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowRowPolicies("1.0"), + RQ_SRS_006_RBAC_ShowRowPolicies_Privilege("1.0"), ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW ROW POLICYS. diff --git a/tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py b/tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py index 1b1e660fffd..8c29a7f462e 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py +++ b/tests/testflows/rbac/tests/privileges/show/show_settings_profiles.py @@ -76,7 +76,7 @@ def check_privilege(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles_Query("1.0"), + RQ_SRS_006_RBAC_ShowSettingsProfiles_RequiredPrivilege("1.0"), ) def show_settings_profiles(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW SETTINGS PROFILES` when they have the necessary privilege. @@ -114,7 +114,7 @@ def show_settings_profiles(self, privilege, grant_target_name, user_name, node=N @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateSettingsProfile("1.0"), + RQ_SRS_006_RBAC_ShowCreateSettingsProfile_RequiredPrivilege("1.0"), ) def show_create(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW CREATE SETTINGS PROFILE` when they have the necessary privilege. @@ -162,7 +162,7 @@ def show_create(self, privilege, grant_target_name, user_name, node=None): @TestFeature @Name("show settings profiles") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowSettingsProfiles("1.0"), + RQ_SRS_006_RBAC_ShowSettingsProfiles_Privilege("1.0"), ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW SETTINGS PROFILES. diff --git a/tests/testflows/rbac/tests/privileges/show/show_tables.py b/tests/testflows/rbac/tests/privileges/show/show_tables.py index 6cda5abb3ca..913b64cef69 100755 --- a/tests/testflows/rbac/tests/privileges/show/show_tables.py +++ b/tests/testflows/rbac/tests/privileges/show/show_tables.py @@ -73,7 +73,7 @@ def check_privilege(self, privilege, on, grant_target_name, user_name, table_nam @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowTables_Query("1.0"), + RQ_SRS_006_RBAC_ShowTables_RequiredPrivilege("1.0"), ) def show_tables(self, privilege, on, grant_target_name, user_name, table_name, node=None): """Check that user is only able to see a table in SHOW TABLES when they have a privilege on that table. @@ -112,7 +112,7 @@ def show_tables(self, privilege, on, grant_target_name, user_name, table_name, n @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ExistsTable("1.0"), + RQ_SRS_006_RBAC_ExistsTable_RequiredPrivilege("1.0"), ) def exists(self, privilege, on, grant_target_name, user_name, table_name, node=None): """Check that user is able to execute EXISTS on a table if and only if the user has SHOW TABLE privilege @@ -152,7 +152,7 @@ def exists(self, privilege, on, grant_target_name, user_name, table_name, node=N @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_CheckTable("1.0"), + RQ_SRS_006_RBAC_CheckTable_RequiredPrivilege("1.0"), ) def check(self, privilege, on, grant_target_name, user_name, table_name, node=None): """Check that user is able to execute CHECK on a table if and only if the user has SHOW TABLE privilege @@ -193,7 +193,7 @@ def check(self, privilege, on, grant_target_name, user_name, table_name, node=No @TestFeature @Name("show tables") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowTables("1.0"), + RQ_SRS_006_RBAC_ShowTables_Privilege("1.0"), ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW TABLES. diff --git a/tests/testflows/rbac/tests/privileges/show/show_users.py b/tests/testflows/rbac/tests/privileges/show/show_users.py index 08c5ca65051..48e6ba51f48 100644 --- a/tests/testflows/rbac/tests/privileges/show/show_users.py +++ b/tests/testflows/rbac/tests/privileges/show/show_users.py @@ -62,7 +62,7 @@ def check_privilege(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowUsers_Query("1.0"), + RQ_SRS_006_RBAC_ShowUsers_RequiredPrivilege("1.0"), ) def show_users(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW USERS` when they have the necessary privilege. @@ -100,7 +100,7 @@ def show_users(self, privilege, grant_target_name, user_name, node=None): @TestSuite @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowCreateUser("1.0"), + RQ_SRS_006_RBAC_ShowCreateUser_RequiredPrivilege("1.0"), ) def show_create(self, privilege, grant_target_name, user_name, node=None): """Check that user is only able to execute `SHOW CREATE USER` when they have the necessary privilege. @@ -148,7 +148,7 @@ def show_create(self, privilege, grant_target_name, user_name, node=None): @TestFeature @Name("show users") @Requirements( - RQ_SRS_006_RBAC_Privileges_ShowUsers("1.0"), + RQ_SRS_006_RBAC_ShowUsers_Privilege("1.0"), ) def feature(self, node="clickhouse1"): """Check the RBAC functionality of SHOW USERS. diff --git a/tests/testflows/rbac/tests/privileges/sources.py b/tests/testflows/rbac/tests/privileges/sources.py new file mode 100644 index 00000000000..8c1b61ee401 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/sources.py @@ -0,0 +1,713 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def file_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a `File` source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=file, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in file.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def file_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a `File` source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=file, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in file.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("FILE",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_File("1.0"), +) +def file(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a `File` source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("File source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the File source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=File()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("File source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the File source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=File()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("File source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the File source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=File()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def url_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a `URL` source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=url, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in url.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def url_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a `URL` source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=url, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in url.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("URL",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_URL("1.0"), +) +def url(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a `URL` source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("URL source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the URL source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=URL()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("URL source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the URL source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=URL()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("URL source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the URL source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=URL()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def remote_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a Remote source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=remote, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in remote.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def remote_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a Remote source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=remote, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in remote.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("REMOTE",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_Remote("1.0"), +) +def remote(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a remote source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("Remote source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the Remote source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE = Distributed()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("Remote source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the Remote source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE = Distributed()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("Remote source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the Remote source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE = Distributed()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def MySQL_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a `MySQL` source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=MySQL, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in MySQL.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def MySQL_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a `MySQL` source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=MySQL, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in MySQL.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("MYSQL",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_MySQL("1.0"), +) +def MySQL(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a `MySQL` source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("MySQL source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the MySQL source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=MySQL()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("MySQL source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the MySQL source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=MySQL()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("MySQL source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the MySQL source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=MySQL()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def ODBC_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a `ODBC` source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=ODBC, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in ODBC.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def ODBC_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a `ODBC` source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=ODBC, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in ODBC.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("ODBC",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_ODBC("1.0"), +) +def ODBC(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a `ODBC` source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("ODBC source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the ODBC source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=ODBC()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("ODBC source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the ODBC source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=ODBC()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("ODBC source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the ODBC source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=ODBC()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def JDBC_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a `JDBC` source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=JDBC, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in JDBC.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def JDBC_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a `JDBC` source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=JDBC, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in JDBC.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("JDBC",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_JDBC("1.0"), +) +def JDBC(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a `JDBC` source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("JDBC source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the JDBC source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=JDBC()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("JDBC source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the JDBC source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=JDBC()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("JDBC source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the JDBC source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=JDBC()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def HDFS_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a `HDFS` source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=HDFS, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in HDFS.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def HDFS_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a `HDFS` source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=HDFS, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in HDFS.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("HDFS",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_HDFS("1.0"), +) +def HDFS(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a `HDFS` source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("HDFS source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the HDFS source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=HDFS()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("HDFS source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the HDFS source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=HDFS()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("HDFS source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the HDFS source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=HDFS()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def S3_privileges_granted_directly(self, node=None): + """Check that a user is able to create a table from a `S3` source with privileges are granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=S3, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in S3.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def S3_privileges_granted_via_role(self, node=None): + """Check that a user is able to create a table from a `S3` source with privileges are granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=S3, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in S3.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SOURCES",), + ("S3",), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources_S3("1.0"), +) +def S3(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to to create a table from a `S3` source when they have the necessary privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("S3 source without privilege"): + table_name = f'table_{getuid()}' + + with Given("The user has table privilege"): + node.query(f"GRANT CREATE TABLE ON {table_name} TO {grant_target_name}") + + with When("I check the user can't use the S3 source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=S3()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("S3 source with privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use the S3 source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=S3()", settings = [("user", f"{user_name}")], + exitcode=42, message='Exception: Storage') + + with Scenario("S3 source with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use the S3 source"): + node.query(f"CREATE TABLE {table_name} (x String) ENGINE=S3()", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("sources") +@Requirements( + RQ_SRS_006_RBAC_Privileges_Sources("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SOURCES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=file_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=file_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=url_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=url_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=remote_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=remote_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=MySQL_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=MySQL_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=ODBC_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=ODBC_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=JDBC_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=JDBC_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=HDFS_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=HDFS_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=S3_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=S3_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/__init__.py b/tests/testflows/rbac/tests/privileges/system/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/testflows/rbac/tests/privileges/system/drop_cache.py b/tests/testflows/rbac/tests/privileges/system/drop_cache.py new file mode 100644 index 00000000000..6439beb248d --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/drop_cache.py @@ -0,0 +1,269 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def dns_cache_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM DROP DNS CACHE` if and only if + they have `SYSTEM DROP DNS CACHE` privilege granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=dns_cache, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dns_cache.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dns_cache_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM DROP DNS CACHE` if and only if + they have `SYSTEM DROP DNS CACHE` privilege granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dns_cache, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dns_cache.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_DropCache_DNS("1.0"), +) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM DROP CACHE",), + ("SYSTEM DROP DNS CACHE",), + ("DROP CACHE",), + ("DROP DNS CACHE",), + ("SYSTEM DROP DNS",), + ("DROP DNS",), +]) +def dns_cache(self, privilege, grant_target_name, user_name, node=None): + """Run checks for `SYSTEM DROP DNS CACHE` privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM DROP DNS CACHE without privilege"): + with When("I check the user is unable to execute SYSTEM DROP DNS CACHE"): + node.query("SYSTEM DROP DNS CACHE", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM DROP DNS CACHE with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user is bale to execute SYSTEM DROP DNS CACHE"): + node.query("SYSTEM DROP DNS CACHE", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM DROP DNS CACHE with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user is unable to execute SYSTEM DROP DNS CACHE"): + node.query("SYSTEM DROP DNS CACHE", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def mark_cache_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM DROP MARK CACHE` if and only if + they have `SYSTEM DROP MARK CACHE` privilege granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=mark_cache, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in mark_cache.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def mark_cache_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM DROP MARK CACHE` if and only if + they have `SYSTEM DROP MARK CACHE` privilege granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=mark_cache, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in mark_cache.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_DropCache_Mark("1.0"), +) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM DROP CACHE",), + ("SYSTEM DROP MARK CACHE",), + ("DROP CACHE",), + ("DROP MARK CACHE",), + ("SYSTEM DROP MARK",), + ("DROP MARKS",), +]) +def mark_cache(self, privilege, grant_target_name, user_name, node=None): + """Run checks for `SYSTEM DROP MARK CACHE` privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM DROP MARK CACHE without privilege"): + with When("I check the user is unable to execute SYSTEM DROP MARK CACHE"): + node.query("SYSTEM DROP MARK CACHE", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM DROP MARK CACHE with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user is bale to execute SYSTEM DROP MARK CACHE"): + node.query("SYSTEM DROP MARK CACHE", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM DROP MARK CACHE with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user is unable to execute SYSTEM DROP MARK CACHE"): + node.query("SYSTEM DROP MARK CACHE", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def uncompressed_cache_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM DROP UNCOMPRESSED CACHE` if and only if + they have `SYSTEM DROP UNCOMPRESSED CACHE` privilege granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=uncompressed_cache, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in uncompressed_cache.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def uncompressed_cache_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM DROP UNCOMPRESSED CACHE` if and only if + they have `SYSTEM DROP UNCOMPRESSED CACHE` privilege granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=uncompressed_cache, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in uncompressed_cache.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_DropCache_Uncompressed("1.0"), +) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM DROP CACHE",), + ("SYSTEM DROP UNCOMPRESSED CACHE",), + ("DROP CACHE",), + ("DROP UNCOMPRESSED CACHE",), + ("SYSTEM DROP UNCOMPRESSED",), + ("DROP UNCOMPRESSED",), +]) +def uncompressed_cache(self, privilege, grant_target_name, user_name, node=None): + """Run checks for `SYSTEM DROP UNCOMPRESSED CACHE` privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM DROP UNCOMPRESSED CACHE without privilege"): + with When("I check the user is unable to execute SYSTEM DROP UNCOMPRESSED CACHE"): + node.query("SYSTEM DROP UNCOMPRESSED CACHE", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM DROP UNCOMPRESSED CACHE with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user is bale to execute SYSTEM DROP UNCOMPRESSED CACHE"): + node.query("SYSTEM DROP UNCOMPRESSED CACHE", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM DROP UNCOMPRESSED CACHE with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user is unable to execute SYSTEM DROP UNCOMPRESSED CACHE"): + node.query("SYSTEM DROP UNCOMPRESSED CACHE", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system drop cache") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_DropCache("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM DROP CACHE. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=dns_cache_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=dns_cache_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=mark_cache_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=mark_cache_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=uncompressed_cache_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=uncompressed_cache_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/fetches.py b/tests/testflows/rbac/tests/privileges/system/fetches.py new file mode 100644 index 00000000000..14c046f4fbe --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/fetches.py @@ -0,0 +1,150 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def replicated_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM FETCHES` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_replicated_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_replicated_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def replicated_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM FETCHES` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_replicated_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_replicated_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM FETCHES", "table"), + ("SYSTEM STOP FETCHES", "table"), + ("SYSTEM START FETCHES", "table"), + ("START FETCHES", "table"), + ("STOP FETCHES", "table"), +]) +def check_replicated_privilege(self, privilege, on, grant_target_name, user_name, node=None): + """Run checks for commands that require SYSTEM FETCHES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=start_replication_queues, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=stop_replication_queues, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +def start_replication_queues(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM START FETCHES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM START FETCHES without privilege"): + with When("I check the user can't start fetches"): + node.query(f"SYSTEM START FETCHES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM START FETCHES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start fetches"): + node.query(f"SYSTEM START FETCHES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM START FETCHES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start fetches"): + node.query(f"SYSTEM START FETCHES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def stop_replication_queues(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM STOP FETCHES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM STOP FETCHES without privilege"): + with When("I check the user can't stop fetches"): + node.query(f"SYSTEM STOP FETCHES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM STOP FETCHES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start fetches"): + node.query(f"SYSTEM STOP FETCHES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM STOP FETCHES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start fetches"): + node.query(f"SYSTEM STOP FETCHES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system fetches") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Fetches("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM FETCHES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=replicated_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=replicated_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/flush.py b/tests/testflows/rbac/tests/privileges/system/flush.py new file mode 100644 index 00000000000..8835b51db9e --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/flush.py @@ -0,0 +1,194 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM FLUSH LOGS` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=flush_logs, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in flush_logs.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM FLUSH LOGS` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=flush_logs, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in flush_logs.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM FLUSH", "*.*"), + ("SYSTEM FLUSH LOGS", "*.*"), + ("FLUSH LOGS", "*.*"), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Flush_Logs("1.0"), +) +def flush_logs(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM START REPLICATED FLUSH` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM FLUSH LOGS without privilege"): + with When("I check the user can't flush logs"): + node.query(f"SYSTEM FLUSH LOGS", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM FLUSH LOGS with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can flush logs"): + node.query(f"SYSTEM FLUSH LOGS", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM FLUSH LOGS with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't flush logs"): + node.query(f"SYSTEM FLUSH LOGS", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def distributed_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM FLUSH DISTRIBUTED` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + table_name = f"table_name_{getuid()}" + + Suite(run=flush_distributed, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[user_name,user_name,table_name]) for row in flush_distributed.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def distributed_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM FLUSH DISTRIBUTED` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + table_name = f"table_name_{getuid()}" + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=flush_distributed, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[role_name,user_name,table_name]) for row in flush_distributed.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM FLUSH", "*.*"), + ("SYSTEM FLUSH DISTRIBUTED", "table"), + ("FLUSH DISTRIBUTED", "table"), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Flush_Distributed("1.0"), +) +def flush_distributed(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM FLUSH DISTRIBUTED` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table0_name = f"table0_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table0_name): + try: + with Given("I have a distributed table"): + node.query(f"CREATE TABLE {table_name} (a UInt64) ENGINE = Distributed(sharded_cluster, default, {table0_name}, rand())") + + with Scenario("SYSTEM FLUSH DISTRIBUTED without privilege"): + with When("I check the user can't flush distributed"): + node.query(f"SYSTEM FLUSH DISTRIBUTED {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM FLUSH DISTRIBUTED with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can flush distributed"): + node.query(f"SYSTEM FLUSH DISTRIBUTED {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM FLUSH DISTRIBUTED with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't flush distributed"): + node.query(f"SYSTEM FLUSH DISTRIBUTED {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the distributed table"): + node.query(f"DROP TABLE IF EXISTS {table_name}") + + +@TestFeature +@Name("system flush") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Flush("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM FLUSH. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=distributed_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=distributed_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/merges.py b/tests/testflows/rbac/tests/privileges/system/merges.py new file mode 100644 index 00000000000..0f347299c44 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/merges.py @@ -0,0 +1,150 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM MERGES` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + table_name = f"table_name_{getuid()}" + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[user_name,user_name,table_name]) for row in check_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM MERGES` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + table_name = f"table_name_{getuid()}" + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[role_name,user_name,table_name]) for row in check_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM MERGES", "table"), + ("SYSTEM STOP MERGES", "table"), + ("SYSTEM START MERGES", "table"), + ("START MERGES", "table"), + ("STOP MERGES", "table"), +]) +def check_privilege(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Run checks for commands that require SYSTEM MERGES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=start_merges, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + Suite(test=stop_merges, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + +@TestSuite +def start_merges(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM START MERGES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name): + + with Scenario("SYSTEM START MERGES without privilege"): + with When("I check the user can't start merges"): + node.query(f"SYSTEM START MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM START MERGES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start merges"): + node.query(f"SYSTEM START MERGES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM START MERGES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start merges"): + node.query(f"SYSTEM START MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def stop_merges(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM STOP MERGES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name): + + with Scenario("SYSTEM STOP MERGES without privilege"): + with When("I check the user can't stop merges"): + node.query(f"SYSTEM STOP MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM STOP MERGES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can stop merges"): + node.query(f"SYSTEM STOP MERGES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM STOP MERGES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't stop merges"): + node.query(f"SYSTEM STOP MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system merges") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Merges("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM MERGES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/moves.py b/tests/testflows/rbac/tests/privileges/system/moves.py new file mode 100644 index 00000000000..2081e6dfe22 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/moves.py @@ -0,0 +1,150 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM MOVES` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + table_name = f"table_name_{getuid()}" + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[user_name,user_name,table_name]) for row in check_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM MOVES` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + table_name = f"table_name_{getuid()}" + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[role_name,user_name,table_name]) for row in check_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM MOVES", "table"), + ("SYSTEM STOP MOVES", "table"), + ("SYSTEM START MOVES", "table"), + ("START MOVES", "table"), + ("STOP MOVES", "table"), +]) +def check_privilege(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Run checks for commands that require SYSTEM MOVES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=start_moves, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + Suite(test=stop_moves, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + +@TestSuite +def start_moves(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM START MOVES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name): + + with Scenario("SYSTEM START MOVES without privilege"): + with When("I check the user can't start moves"): + node.query(f"SYSTEM START MOVES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM START MOVES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start moves"): + node.query(f"SYSTEM START MOVES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM START MOVES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start moves"): + node.query(f"SYSTEM START MOVES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def stop_moves(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM STOP MOVES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name): + + with Scenario("SYSTEM STOP MOVES without privilege"): + with When("I check the user can't stop moves"): + node.query(f"SYSTEM STOP MOVES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM STOP MOVES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can stop moves"): + node.query(f"SYSTEM STOP MOVES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM STOP MOVES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't stop moves"): + node.query(f"SYSTEM STOP MOVES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system moves") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Moves("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM MOVES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/reload.py b/tests/testflows/rbac/tests/privileges/system/reload.py new file mode 100644 index 00000000000..cfc752fb253 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/reload.py @@ -0,0 +1,379 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@contextmanager +def dict_setup(node, table_name, dict_name): + """Setup and teardown of table and dictionary needed for the tests. + """ + + try: + with Given("I have a table"): + node.query(f"CREATE TABLE {table_name} (key UInt64, val UInt64) Engine=Memory()") + + with And("I have a dictionary"): + node.query(f"CREATE DICTIONARY {dict_name} (key UInt64 DEFAULT 0, val UInt64 DEFAULT 10) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE '{table_name}' PASSWORD '' DB 'default')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT())") + + yield + + finally: + with Finally("I drop the table", flags=TE): + node.query(f"DROP TABLE IF EXISTS {table_name}") + + with And("I drop the dictionary", flags=TE): + node.query(f"DROP DICTIONARY IF EXISTS default.{dict_name}") + +@TestSuite +def config_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD CONFIG` if and only if + they have `SYSTEM RELOAD CONFIG` privilege granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=config, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in config.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def config_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD CONFIG` if and only if + they have `SYSTEM RELOAD CONFIG` privilege granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=config, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in config.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Reload_Config("1.0"), +) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM RELOAD",), + ("SYSTEM RELOAD CONFIG",), + ("RELOAD CONFIG",), +]) +def config(self, privilege, grant_target_name, user_name, node=None): + """Run checks for `SYSTEM RELOAD CONFIG` privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM RELOAD CONFIG without privilege"): + with When("I check the user is unable to execute SYSTEM RELOAD CONFIG"): + node.query("SYSTEM RELOAD CONFIG", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM RELOAD CONFIG with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user is bale to execute SYSTEM RELOAD CONFIG"): + node.query("SYSTEM RELOAD CONFIG", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM RELOAD CONFIG with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user is unable to execute SYSTEM RELOAD CONFIG"): + node.query("SYSTEM RELOAD CONFIG", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def dictionary_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD DICTIONARY` if and only if + they have `SYSTEM RELOAD DICTIONARY` privilege granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=dictionary, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dictionary.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dictionary_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD DICTIONARY` if and only if + they have `SYSTEM RELOAD DICTIONARY` privilege granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictionary, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dictionary.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Reload_Dictionary("1.0"), +) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM RELOAD",), + ("SYSTEM RELOAD DICTIONARIES",), + ("RELOAD DICTIONARIES",), + ("RELOAD DICTIONARY",), +]) +def dictionary(self, privilege, grant_target_name, user_name, node=None): + """Run checks for `SYSTEM RELOAD DICTIONARY` privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM RELOAD DICTIONARY without privilege"): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When("I check the user is unable to execute SYSTEM RELOAD DICTIONARY"): + node.query(f"SYSTEM RELOAD DICTIONARY default.{dict_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM RELOAD DICTIONARY with privilege"): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user is bale to execute SYSTEM RELOAD DICTIONARY"): + node.query(f"SYSTEM RELOAD DICTIONARY default.{dict_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM RELOAD DICTIONARY with revoked privilege"): + dict_name = f"dict_{getuid()}" + table_name = f"table_{getuid()}" + + with dict_setup(node, table_name, dict_name): + + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user is unable to execute SYSTEM RELOAD DICTIONARY"): + node.query(f"SYSTEM RELOAD DICTIONARY default.{dict_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def dictionaries_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD DICTIONARIES` if and only if + they have `SYSTEM RELOAD DICTIONARIES` privilege granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=dictionaries, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in dictionaries.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def dictionaries_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD DICTIONARIES` if and only if + they have `SYSTEM RELOAD DICTIONARIES` privilege granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=dictionaries, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in dictionaries.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Reload_Dictionaries("1.0"), +) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM RELOAD",), + ("SYSTEM RELOAD DICTIONARIES",), + ("RELOAD DICTIONARIES",), + ("RELOAD DICTIONARY",), +]) +def dictionaries(self, privilege, grant_target_name, user_name, node=None): + """Run checks for `SYSTEM RELOAD DICTIONARIES` privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM RELOAD DICTIONARIES without privilege"): + with When("I check the user is unable to execute SYSTEM RELOAD DICTIONARIES"): + node.query("SYSTEM RELOAD DICTIONARIES", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM RELOAD DICTIONARIES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user is bale to execute SYSTEM RELOAD DICTIONARIES"): + node.query("SYSTEM RELOAD DICTIONARIES", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM RELOAD DICTIONARIES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user is unable to execute SYSTEM RELOAD DICTIONARIES"): + node.query("SYSTEM RELOAD DICTIONARIES", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def embedded_dictionaries_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD EMBEDDED DICTIONARIES` if and only if + they have `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilege granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=embedded_dictionaries, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in embedded_dictionaries.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def embedded_dictionaries_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM RELOAD EMBEDDED DICTIONARIES` if and only if + they have `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilege granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=embedded_dictionaries, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in embedded_dictionaries.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Reload_EmbeddedDictionaries("1.0"), +) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM RELOAD",), + ("SYSTEM RELOAD EMBEDDED DICTIONARIES",), + ("SYSTEM RELOAD DICTIONARY",), +]) +def embedded_dictionaries(self, privilege, grant_target_name, user_name, node=None): + """Run checks for `SYSTEM RELOAD EMBEDDED DICTIONARIES` privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM RELOAD EMBEDDED DICTIONARIES without privilege"): + with When("I check the user is unable to execute SYSTEM RELOAD EMBEDDED DICTIONARIES"): + node.query("SYSTEM RELOAD EMBEDDED DICTIONARIES", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM RELOAD EMBEDDED DICTIONARIES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user is bale to execute SYSTEM RELOAD EMBEDDED DICTIONARIES"): + node.query("SYSTEM RELOAD EMBEDDED DICTIONARIES", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM RELOAD EMBEDDED DICTIONARIES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user is unable to execute SYSTEM RELOAD EMBEDDED DICTIONARIES"): + node.query("SYSTEM RELOAD EMBEDDED DICTIONARIES", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system reload") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Reload("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM RELOAD. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=config_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=config_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=dictionary_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=dictionary_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=dictionaries_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=dictionaries_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=embedded_dictionaries_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=embedded_dictionaries_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/replication_queues.py b/tests/testflows/rbac/tests/privileges/system/replication_queues.py new file mode 100644 index 00000000000..3ac2e09418a --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/replication_queues.py @@ -0,0 +1,150 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def replicated_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM REPLICATION QUEUES` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_replicated_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_replicated_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def replicated_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM REPLICATION QUEUES` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_replicated_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_replicated_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM REPLICATION QUEUES", "table"), + ("SYSTEM STOP REPLICATION QUEUES", "table"), + ("SYSTEM START REPLICATION QUEUES", "table"), + ("START REPLICATION QUEUES", "table"), + ("STOP REPLICATION QUEUES", "table"), +]) +def check_replicated_privilege(self, privilege, on, grant_target_name, user_name, node=None): + """Run checks for commands that require SYSTEM REPLICATION QUEUES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=start_replication_queues, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=stop_replication_queues, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +def start_replication_queues(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM START REPLICATION QUEUES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM START REPLICATION QUEUES without privilege"): + with When("I check the user can't start sends"): + node.query(f"SYSTEM START REPLICATION QUEUES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM START REPLICATION QUEUES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start sends"): + node.query(f"SYSTEM START REPLICATION QUEUES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM START REPLICATION QUEUES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start sends"): + node.query(f"SYSTEM START REPLICATION QUEUES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def stop_replication_queues(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM STOP REPLICATION QUEUES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM STOP REPLICATION QUEUES without privilege"): + with When("I check the user can't stop sends"): + node.query(f"SYSTEM STOP REPLICATION QUEUES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM STOP REPLICATION QUEUES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start sends"): + node.query(f"SYSTEM STOP REPLICATION QUEUES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM STOP REPLICATION QUEUES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start sends"): + node.query(f"SYSTEM STOP REPLICATION QUEUES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system replication queues") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_ReplicationQueues("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM REPLICATION QUEUES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=replicated_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=replicated_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/restart_replica.py b/tests/testflows/rbac/tests/privileges/system/restart_replica.py new file mode 100644 index 00000000000..0e3e61d04bb --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/restart_replica.py @@ -0,0 +1,99 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM RESTART REPLICA` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=restart_replica, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in restart_replica.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM RESTART REPLICA` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=restart_replica, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in restart_replica.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM RESTART REPLICA", "table"), + ("RESTART REPLICA", "table"), +]) +def restart_replica(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM RESTARTE REPLICA` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM RESTART REPLICA without privilege"): + with When("I check the user can't restart replica"): + node.query(f"SYSTEM RESTART REPLICA {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM RESTART REPLICA with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can restart replica"): + node.query(f"SYSTEM RESTART REPLICA {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM RESTART REPLICA with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't restart replica"): + node.query(f"SYSTEM RESTART REPLICA {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system restart replica") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_RestartReplica("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM RESTART REPLICA. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/sends.py b/tests/testflows/rbac/tests/privileges/system/sends.py new file mode 100644 index 00000000000..24865088703 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/sends.py @@ -0,0 +1,314 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def replicated_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM REPLICATED SENDS` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_replicated_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_replicated_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def replicated_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM REPLICATED SENDS` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_replicated_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_replicated_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM SENDS", "*.*"), + ("SYSTEM START SENDS", "*.*"), + ("SYSTEM STOP SENDS", "*.*"), + ("START SENDS", "*.*"), + ("STOP SENDS", "*.*"), + ("SYSTEM REPLICATED SENDS", "table"), + ("SYSTEM STOP REPLICATED SENDS", "table"), + ("SYSTEM START REPLICATED SENDS", "table"), + ("START REPLICATED SENDS", "table"), + ("STOP REPLICATED SENDS", "table"), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Sends_Replicated("1.0"), +) +def check_replicated_privilege(self, privilege, on, grant_target_name, user_name, node=None): + """Run checks for commands that require SYSTEM REPLICATED SENDS privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=start_replicated_sends, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=stop_replicated_sends, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +def start_replicated_sends(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM START REPLICATED SENDS` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM START REPLICATED SENDS without privilege"): + with When("I check the user can't start sends"): + node.query(f"SYSTEM START REPLICATED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM START REPLICATED SENDS with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start sends"): + node.query(f"SYSTEM START REPLICATED SENDS {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM START REPLICATED SENDS with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start sends"): + node.query(f"SYSTEM START REPLICATED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def stop_replicated_sends(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM STOP REPLICATED SENDS` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM STOP REPLICATED SENDS without privilege"): + with When("I check the user can't stop sends"): + node.query(f"SYSTEM STOP REPLICATED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM STOP REPLICATED SENDS with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can stop sends"): + node.query(f"SYSTEM STOP REPLICATED SENDS {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM STOP REPLICATED SENDS with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't stop sends"): + node.query(f"SYSTEM STOP REPLICATED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def distributed_privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM DISTRIBUTED SENDS` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + table_name = f"table_name_{getuid()}" + + Suite(run=check_distributed_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[user_name,user_name,table_name]) for row in check_distributed_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def distributed_privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM DISTRIBUTED SENDS` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + table_name = f"table_name_{getuid()}" + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_distributed_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[role_name,user_name,table_name]) for row in check_distributed_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM SENDS", "*.*"), + ("SYSTEM START SENDS", "*.*"), + ("SYSTEM STOP SENDS", "*.*"), + ("START SENDS", "*.*"), + ("STOP SENDS", "*.*"), + ("SYSTEM DISTRIBUTED SENDS", "table"), + ("SYSTEM STOP DISTRIBUTED SENDS", "table"), + ("SYSTEM START DISTRIBUTED SENDS", "table"), + ("START DISTRIBUTED SENDS", "table"), + ("STOP DISTRIBUTED SENDS", "table"), +]) +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Sends_Distributed("1.0"), +) +def check_distributed_privilege(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Run checks for commands that require SYSTEM DISTRIBUTED SENDS privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=start_distributed_moves, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + Suite(test=stop_distributed_moves, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + +@TestSuite +def start_distributed_moves(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM START DISTRIBUTED SENDS` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table0_name = f"table0_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table0_name): + try: + with Given("I have a distributed table"): + node.query(f"CREATE TABLE {table_name} (a UInt64) ENGINE = Distributed(sharded_cluster, default, {table0_name}, rand())") + + with Scenario("SYSTEM START DISTRIBUTED SENDS without privilege"): + with When("I check the user can't start merges"): + node.query(f"SYSTEM START DISTRIBUTED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM START DISTRIBUTED SENDS with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start merges"): + node.query(f"SYSTEM START DISTRIBUTED SENDS {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM START DISTRIBUTED SENDS with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start merges"): + node.query(f"SYSTEM START DISTRIBUTED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + finally: + with Finally("I drop the distributed table"): + node.query(f"DROP TABLE IF EXISTS {table_name}") + +@TestSuite +def stop_distributed_moves(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM STOP DISTRIBUTED SENDS` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table0_name = f"table0_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table0_name): + try: + with Given("I have a distributed table"): + node.query(f"CREATE TABLE {table_name} (a UInt64) ENGINE = Distributed(sharded_cluster, default, {table0_name}, rand())") + + with Scenario("SYSTEM STOP DISTRIBUTED SENDS without privilege"): + with When("I check the user can't stop merges"): + node.query(f"SYSTEM STOP DISTRIBUTED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM STOP DISTRIBUTED SENDS with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can stop merges"): + node.query(f"SYSTEM STOP DISTRIBUTED SENDS {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM STOP DISTRIBUTED SENDS with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't stop merges"): + node.query(f"SYSTEM STOP DISTRIBUTED SENDS {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + finally: + with Finally("I drop the distributed table"): + node.query(f"DROP TABLE IF EXISTS {table_name}") + +@TestFeature +@Name("system sends") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Sends("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM SENDS. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=replicated_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=replicated_privileges_granted_via_role, setup=instrument_clickhouse_server_log) + Suite(run=distributed_privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=distributed_privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/shutdown.py b/tests/testflows/rbac/tests/privileges/system/shutdown.py new file mode 100644 index 00000000000..290f6d8e5d1 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/shutdown.py @@ -0,0 +1,200 @@ +import time + +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Run checks with privileges granted directly. + """ + + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Run checks with privileges granted through a role. + """ + + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in check_privilege.examples + ], args=Args(name="privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege",[ + ("SYSTEM",), + ("SYSTEM SHUTDOWN",), + ("SHUTDOWN",), + ("SYSTEM KILL",), +]) +def check_privilege(self, privilege, grant_target_name, user_name, node=None): + """Run checks for commands that require SYSTEM SHUTDOWN privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=shutdown, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + Suite(test=kill, setup=instrument_clickhouse_server_log)(privilege=privilege, grant_target_name=grant_target_name, user_name=user_name) + +@TestSuite +def shutdown(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM SHUTDOWN` when they have the necessary privilege. + """ + cluster = self.context.cluster + + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM SHUTDOWN without privilege"): + + with When("I check the user can't use SYSTEM SHUTDOWN"): + node.query(f"SYSTEM SHUTDOWN", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM SHUTDOWN with privilege"): + timeout = 60 + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SYSTEM SHUTDOWN"): + node.query(f"SYSTEM SHUTDOWN", settings = [("user", f"{user_name}")]) + + with And("I close all connections to the node"): + node.close_bashes() + + with And("I check that system is down"): + command = f"echo -e \"SELECT 1\" | {cluster.docker_compose} exec -T {node.name} clickhouse client -n" + + start_time = time.time() + + while True: + r = cluster.bash(None)(command) + if r.exitcode != 0: + break + if time.time() - start_time > timeout: + break + time.sleep(1) + + assert r.exitcode != 0, error(r.output) + + finally: + with Finally("I relaunch the server"): + node.restart(safe=False) + + with Scenario("SYSTEM SHUTDOWN with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SYSTEM SHUTDOWN"): + node.query(f"SYSTEM SHUTDOWN", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestSuite +def kill(self, privilege, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM KILL` when they have the necessary privilege. + """ + cluster = self.context.cluster + + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + with Scenario("SYSTEM KILL without privilege"): + + with When("I check the user can't use SYSTEM KILL"): + node.query(f"SYSTEM KILL", settings=[("user",user_name)], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM KILL with privilege"): + timeout = 60 + + try: + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with Then("I check the user can use SYSTEM KILL"): + command = f"echo -e \"SYSTEM KILL\" | clickhouse client -n" + with By("executing command", description=command): + self.context.cluster.bash(node.name).send(command) + + with And("I close all connections to the node"): + node.close_bashes() + + with And("I check that system is down"): + command = f"echo -e \"SELECT 1\" | {cluster.docker_compose} exec -T {node.name} clickhouse client -n" + + start_time = time.time() + + while True: + r = cluster.bash(None)(command) + if r.exitcode != 0: + break + if time.time() - start_time > timeout: + break + time.sleep(1) + + assert r.exitcode != 0, error(r.output) + + finally: + with Finally("I relaunch the server"): + node.restart(safe=False) + + with Scenario("SYSTEM KILL with revoked privilege"): + + with When(f"I grant {privilege}"): + node.query(f"GRANT {privilege} ON *.* TO {grant_target_name}") + + with And(f"I revoke {privilege}"): + node.query(f"REVOKE {privilege} ON *.* FROM {grant_target_name}") + + with Then("I check the user cannot use SYSTEM KILL"): + node.query(f"SYSTEM KILL", settings=[("user",user_name)], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system shutdown") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_Shutdown("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM SHUTDOWN. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/sync_replica.py b/tests/testflows/rbac/tests/privileges/system/sync_replica.py new file mode 100644 index 00000000000..7df697fd6e5 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/sync_replica.py @@ -0,0 +1,99 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM SYNC REPLICA` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + + Suite(run=sync_replica, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[user_name,user_name]) for row in sync_replica.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM SYNC REPLICA` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=sync_replica, flags=TE, + examples=Examples("privilege on grant_target_name user_name", [ + tuple(list(row)+[role_name,user_name]) for row in sync_replica.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM SYNC REPLICA", "table"), + ("SYNC REPLICA", "table"), +]) +def sync_replica(self, privilege, on, grant_target_name, user_name, node=None): + """Check that user is only able to execute `SYSTEM SYNCE REPLICA` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + table_name = f"table_name_{getuid()}" + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name, "ReplicatedMergeTree-sharded_cluster"): + + with Scenario("SYSTEM SYNC REPLICA without privilege"): + with When("I check the user can't sync replica"): + node.query(f"SYSTEM SYNC REPLICA {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM SYNC REPLICA with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can sync replica"): + node.query(f"SYSTEM SYNC REPLICA {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM SYNC REPLICA with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't sync replica"): + node.query(f"SYSTEM SYNC REPLICA {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system sync replica") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_SyncReplica("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM SYNC REPLICA. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/rbac/tests/privileges/system/ttl_merges.py b/tests/testflows/rbac/tests/privileges/system/ttl_merges.py new file mode 100644 index 00000000000..74f99026fe4 --- /dev/null +++ b/tests/testflows/rbac/tests/privileges/system/ttl_merges.py @@ -0,0 +1,150 @@ +from testflows.core import * +from testflows.asserts import error + +from rbac.requirements import * +from rbac.helper.common import * +import rbac.helper.errors as errors + +@TestSuite +def privileges_granted_directly(self, node=None): + """Check that a user is able to execute `SYSTEM TTL MERGES` commands if and only if + the privilege has been granted directly. + """ + user_name = f"user_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"): + table_name = f"table_name_{getuid()}" + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[user_name,user_name,table_name]) for row in check_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestSuite +def privileges_granted_via_role(self, node=None): + """Check that a user is able to execute `SYSTEM TTL MERGES` commands if and only if + the privilege has been granted via role. + """ + user_name = f"user_{getuid()}" + role_name = f"role_{getuid()}" + + if node is None: + node = self.context.node + + with user(node, f"{user_name}"), role(node, f"{role_name}"): + table_name = f"table_name_{getuid()}" + + with When("I grant the role to the user"): + node.query(f"GRANT {role_name} TO {user_name}") + + Suite(run=check_privilege, flags=TE, + examples=Examples("privilege on grant_target_name user_name table_name", [ + tuple(list(row)+[role_name,user_name,table_name]) for row in check_privilege.examples + ], args=Args(name="check privilege={privilege}", format_name=True))) + +@TestOutline(Suite) +@Examples("privilege on",[ + ("SYSTEM", "*.*"), + ("SYSTEM TTL MERGES", "table"), + ("SYSTEM STOP TTL MERGES", "table"), + ("SYSTEM START TTL MERGES", "table"), + ("START TTL MERGES", "table"), + ("STOP TTL MERGES", "table"), +]) +def check_privilege(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Run checks for commands that require SYSTEM TTL MERGES privilege. + """ + + if node is None: + node = self.context.node + + Suite(test=start_ttl_merges, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + Suite(test=stop_ttl_merges, setup=instrument_clickhouse_server_log)(privilege=privilege, on=on, grant_target_name=grant_target_name, user_name=user_name, table_name=table_name) + +@TestSuite +def start_ttl_merges(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM START TTL MERGES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name): + + with Scenario("SYSTEM START TTL MERGES without privilege"): + with When("I check the user can't start merges"): + node.query(f"SYSTEM START TTL MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM START TTL MERGES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can start merges"): + node.query(f"SYSTEM START TTL MERGES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM START TTL MERGES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't start merges"): + node.query(f"SYSTEM START TTL MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestSuite +def stop_ttl_merges(self, privilege, on, grant_target_name, user_name, table_name, node=None): + """Check that user is only able to execute `SYSTEM STOP TTL MERGES` when they have privilege. + """ + exitcode, message = errors.not_enough_privileges(name=user_name) + + if node is None: + node = self.context.node + + on = on.replace("table", f"{table_name}") + + with table(node, table_name): + + with Scenario("SYSTEM STOP TTL MERGES without privilege"): + with When("I check the user can't stop merges"): + node.query(f"SYSTEM STOP TTL MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + + with Scenario("SYSTEM STOP TTL MERGES with privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with Then("I check the user can stop merges"): + node.query(f"SYSTEM STOP TTL MERGES {table_name}", settings = [("user", f"{user_name}")]) + + with Scenario("SYSTEM STOP TTL MERGES with revoked privilege"): + with When(f"I grant {privilege} on the table"): + node.query(f"GRANT {privilege} ON {on} TO {grant_target_name}") + + with And(f"I revoke {privilege} on the table"): + node.query(f"REVOKE {privilege} ON {on} FROM {grant_target_name}") + + with Then("I check the user can't stop merges"): + node.query(f"SYSTEM STOP TTL MERGES {table_name}", settings = [("user", f"{user_name}")], + exitcode=exitcode, message=message) + +@TestFeature +@Name("system ttl merges") +@Requirements( + RQ_SRS_006_RBAC_Privileges_System_TTLMerges("1.0"), +) +def feature(self, node="clickhouse1"): + """Check the RBAC functionality of SYSTEM TTL MERGES. + """ + self.context.node = self.context.cluster.node(node) + + Suite(run=privileges_granted_directly, setup=instrument_clickhouse_server_log) + Suite(run=privileges_granted_via_role, setup=instrument_clickhouse_server_log) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index ef2008aa173..dcc93abda60 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -16,7 +16,10 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): # Feature(test=load("example.regression", "regression"))(**args) # Feature(test=load("ldap.regression", "regression"))(**args) -# Feature(test=load("rbac.regression", "regression"))(**args) + for i in range(100): + '''Run the checks 100 times to spot any fails. + ''' + Feature(test=load("rbac.regression", "regression"))(**args) # Feature(test=load("aes_encryption.regression", "regression"))(**args) if main(): From 47000e7b555b20973b9822c69ad0f461c73c0267 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 13 Jan 2021 19:36:32 +0300 Subject: [PATCH 128/264] Update 01184_insert_values_huge_strings.sh --- .../0_stateless/01184_insert_values_huge_strings.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01184_insert_values_huge_strings.sh b/tests/queries/0_stateless/01184_insert_values_huge_strings.sh index 63287582cbd..9b63f401a59 100755 --- a/tests/queries/0_stateless/01184_insert_values_huge_strings.sh +++ b/tests/queries/0_stateless/01184_insert_values_huge_strings.sh @@ -4,12 +4,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists default.huge_strings" -$CLICKHOUSE_CLIENT -q "create table default.huge_strings (n UInt64, l UInt64, s String, h UInt64) engine=MergeTree order by n" +$CLICKHOUSE_CLIENT -q "drop table if exists huge_strings" +$CLICKHOUSE_CLIENT -q "create table huge_strings (n UInt64, l UInt64, s String, h UInt64) engine=MergeTree order by n" for _ in {1..10}; do - $CLICKHOUSE_CLIENT -q "select number, (rand() % 100*1000*1000) as l, repeat(randomString(l/1000/1000), 1000*1000) as s, cityHash64(s) from numbers(10) format Values" | $CLICKHOUSE_CLIENT -q "insert into default.huge_strings values" & - $CLICKHOUSE_CLIENT -q "select number % 10, (rand() % 100) as l, randomString(l) as s, cityHash64(s) from numbers(100000)" | $CLICKHOUSE_CLIENT -q "insert into default.huge_strings format TSV" & + $CLICKHOUSE_CLIENT -q "select number, (rand() % 100*1000*1000) as l, repeat(randomString(l/1000/1000), 1000*1000) as s, cityHash64(s) from numbers(10) format Values" | $CLICKHOUSE_CLIENT -q "insert into huge_strings values" & + $CLICKHOUSE_CLIENT -q "select number % 10, (rand() % 100) as l, randomString(l) as s, cityHash64(s) from numbers(100000)" | $CLICKHOUSE_CLIENT -q "insert into huge_strings format TSV" & done; wait From 944d53d69129401f95917711ed5bfc5ae94ee4bc Mon Sep 17 00:00:00 2001 From: MyroTk Date: Wed, 13 Jan 2021 18:20:29 +0100 Subject: [PATCH 129/264] Running RBAC just once --- tests/testflows/regression.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index dcc93abda60..ac17ec4e535 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -16,10 +16,7 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): # Feature(test=load("example.regression", "regression"))(**args) # Feature(test=load("ldap.regression", "regression"))(**args) - for i in range(100): - '''Run the checks 100 times to spot any fails. - ''' - Feature(test=load("rbac.regression", "regression"))(**args) + Feature(test=load("rbac.regression", "regression"))(**args) # Feature(test=load("aes_encryption.regression", "regression"))(**args) if main(): From 5f0f8ae3cddef3c8384b1694b09469ecba476251 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 13 Jan 2021 20:59:20 +0300 Subject: [PATCH 130/264] fix duplicate UUIDs of LiveView on server startup --- src/Interpreters/Context.cpp | 3 -- src/Interpreters/DatabaseCatalog.cpp | 8 ++++ .../LiveView/TemporaryLiveViewCleaner.cpp | 42 +++++++++++++++---- .../LiveView/TemporaryLiveViewCleaner.h | 4 ++ 4 files changed, 45 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9fb74a4f800..2a8fdce869b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include @@ -429,7 +428,6 @@ struct ContextShared if (system_logs) system_logs->shutdown(); - TemporaryLiveViewCleaner::shutdown(); DatabaseCatalog::shutdown(); /// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference). @@ -493,7 +491,6 @@ Context Context::createGlobal(ContextShared * shared) void Context::initGlobal() { DatabaseCatalog::init(*this); - TemporaryLiveViewCleaner::init(*this); } SharedContextHolder Context::createShared() diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index e9caaab4ef9..18cf69675ba 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -148,10 +149,16 @@ void DatabaseCatalog::loadDatabases() std::lock_guard lock{tables_marked_dropped_mutex}; if (!tables_marked_dropped.empty()) (*drop_task)->schedule(); + + /// Another background thread which drops temporary LiveViews. + /// We should start it after loadMarkedAsDroppedTables() to avoid race condition. + TemporaryLiveViewCleaner::instance().startupIfNecessary(); } void DatabaseCatalog::shutdownImpl() { + TemporaryLiveViewCleaner::shutdown(); + if (drop_task) (*drop_task)->deactivate(); @@ -524,6 +531,7 @@ std::unique_ptr DatabaseCatalog::database_catalog; DatabaseCatalog::DatabaseCatalog(Context & global_context_) : global_context(global_context_), log(&Poco::Logger::get("DatabaseCatalog")) { + TemporaryLiveViewCleaner::init(global_context); } DatabaseCatalog & DatabaseCatalog::init(Context & global_context_) diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp index 052b22fd666..4f6eb1bb3e1 100644 --- a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp @@ -48,6 +48,37 @@ void TemporaryLiveViewCleaner::init(Context & global_context_) the_instance.reset(new TemporaryLiveViewCleaner(global_context_)); } +void TemporaryLiveViewCleaner::startupIfNecessary() +{ + std::lock_guard lock{mutex}; + if (background_thread_should_exit) + return; + if (!views.empty()) + startupIfNecessaryImpl(lock); +} + +void TemporaryLiveViewCleaner::startupIfNecessaryImpl(const std::lock_guard &) +{ + /// If views.empty() the background thread isn't running or it's going to stop right now. + /// If can_start_background_thread is false, then the thread has not been started previously. + bool background_thread_is_running; + if (can_start_background_thread) + { + background_thread_is_running = !views.empty(); + } + else + { + can_start_background_thread = true; + background_thread_is_running = false; + } + + if (!background_thread_is_running) + { + if (background_thread.joinable()) + background_thread.join(); + background_thread = ThreadFromGlobalPool{&TemporaryLiveViewCleaner::backgroundThreadFunc, this}; + } +} void TemporaryLiveViewCleaner::shutdown() { @@ -79,20 +110,13 @@ void TemporaryLiveViewCleaner::addView(const std::shared_ptr & if (background_thread_should_exit) return; - /// If views.empty() the background thread isn't running or it's going to stop right now. - bool background_thread_is_running = !views.empty(); + if (can_start_background_thread) + startupIfNecessaryImpl(lock); /// Keep the vector `views` sorted by time of next check. StorageAndTimeOfCheck storage_and_time_of_check{view, time_of_next_check}; views.insert(std::upper_bound(views.begin(), views.end(), storage_and_time_of_check), storage_and_time_of_check); - if (!background_thread_is_running) - { - if (background_thread.joinable()) - background_thread.join(); - background_thread = ThreadFromGlobalPool{&TemporaryLiveViewCleaner::backgroundThreadFunc, this}; - } - background_thread_wake_up.notify_one(); } diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.h b/src/Storages/LiveView/TemporaryLiveViewCleaner.h index 57c12bd1c07..8d57aa9fbfa 100644 --- a/src/Storages/LiveView/TemporaryLiveViewCleaner.h +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.h @@ -23,6 +23,9 @@ public: static void init(Context & global_context_); static void shutdown(); + void startupIfNecessary(); + void startupIfNecessaryImpl(const std::lock_guard &); + private: friend std::unique_ptr::deleter_type; @@ -44,6 +47,7 @@ private: std::mutex mutex; std::vector views; ThreadFromGlobalPool background_thread; + bool can_start_background_thread = false; std::atomic background_thread_should_exit = false; std::condition_variable background_thread_wake_up; }; From 871ac3eeb783e8e6e6123076768a83937d11e871 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jan 2021 21:37:54 +0300 Subject: [PATCH 131/264] Update building sets. --- src/Interpreters/ExpressionAnalyzer.cpp | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index ae1aa1a0bb1..42311abd9bf 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -53,6 +53,8 @@ #include #include +#include + namespace DB { @@ -318,13 +320,14 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr } auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, {}, query_options); - auto stream = interpreter_subquery->execute().getInputStream(); + auto io = interpreter_subquery->execute(); + PullingPipelineExecutor executor(io.pipeline); SetPtr set = std::make_shared(settings.size_limits_for_set, true, context.getSettingsRef().transform_null_in); - set->setHeader(stream->getHeader()); + set->setHeader(executor.getHeader()); - stream->readPrefix(); - while (Block block = stream->read()) + Block block; + while (executor.pull(block)) { /// If the limits have been exceeded, give up and let the default subquery processing actions take place. if (!set->insertFromBlock(block)) @@ -332,7 +335,6 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr } set->finishInsert(); - stream->readSuffix(); prepared_sets[set_key] = std::move(set); } From b03f28f28ce3bc7a46469dd77d9a282706761489 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jan 2021 21:58:00 +0300 Subject: [PATCH 132/264] Update building sets. --- src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index f7a1fc83182..b3e8af90f28 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -21,6 +21,8 @@ #include +#include + namespace DB { @@ -116,13 +118,12 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr } else { - auto stream = interpreter.execute().getInputStream(); + auto io = interpreter.execute(); try { - block = stream->read(); - - if (!block) + PullingPipelineExecutor executor(io.pipeline); + if (!executor.pull(block)) { /// Interpret subquery with empty result as Null literal auto ast_new = std::make_unique(Null()); @@ -131,7 +132,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr return; } - if (block.rows() != 1 || stream->read()) + if (block.rows() != 1 || executor.pull(block)) throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); } catch (const Exception & e) From 7df99ff4ff4514946dc02c34a483c528676df83d Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Wed, 13 Jan 2021 23:56:24 +0300 Subject: [PATCH 133/264] Update docs/en/sql-reference/functions/string-search-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 4036974dd37..76627d78464 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -560,7 +560,7 @@ countSubstrings(haystack, needle[, start_pos]) - Number of occurrences. -Type: [UInt64](../../sql-reference/data-types/int-uint.md). +Type: Positive integer. **Examples** From d8a38c7e57ca47bc824092b0790d2e75ea963553 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Wed, 13 Jan 2021 23:56:40 +0300 Subject: [PATCH 134/264] Update docs/en/sql-reference/functions/string-search-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 76627d78464..fb70dbf9028 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -626,7 +626,7 @@ countSubstringsCaseInsensitive(haystack, needle[, start_pos]) - Number of occurrences. -Type: [UInt64](../../sql-reference/data-types/int-uint.md). +Type: Positive integer. **Examples** From e363a240eee247c2ab7833618bbe3b3c070b6723 Mon Sep 17 00:00:00 2001 From: George Date: Thu, 14 Jan 2021 00:13:36 +0300 Subject: [PATCH 135/264] Undo changes --- docs/en/sql-reference/functions/string-search-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index fb70dbf9028..4036974dd37 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -560,7 +560,7 @@ countSubstrings(haystack, needle[, start_pos]) - Number of occurrences. -Type: Positive integer. +Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** @@ -626,7 +626,7 @@ countSubstringsCaseInsensitive(haystack, needle[, start_pos]) - Number of occurrences. -Type: Positive integer. +Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** From fd67eb125150590bd35037c989a702058c64071d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 13 Jan 2021 18:53:28 -0400 Subject: [PATCH 136/264] Update partition.md tuple req. in part expressions --- docs/ru/sql-reference/statements/alter/partition.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index b43340467fc..4b9ae116ec2 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -288,7 +288,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать: - Имя партиции. Посмотреть имя партиции можно в столбце `partition` системной таблицы [system.parts](../../../operations/system-tables/parts.md#system_tables-parts). Например, `ALTER TABLE visits DETACH PARTITION 201901`. -- Произвольное выражение из столбцов исходной таблицы. Также поддерживаются константы и константные выражения. Например, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. +- Произвольное выражение из столбцов исходной таблицы. Также поддерживаются константы и константные выражения. Выражение необходимо обернуть в функцию tuple(). Например, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Строковый идентификатор партиции. Идентификатор партиции используется для именования кусков партиции на файловой системе и в ZooKeeper. В запросах `ALTER` идентификатор партиции нужно указывать в секции `PARTITION ID`, в одинарных кавычках. Например, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - Для запросов [ATTACH PART](#alter_attach-partition) и [DROP DETACHED PART](#alter_drop-detached): чтобы задать имя куска партиции, используйте строковой литерал со значением из столбца `name` системной таблицы [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts). Например, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. @@ -306,4 +306,4 @@ OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL; Примеры запросов `ALTER ... PARTITION` можно посмотреть в тестах: [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) и [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/alter/partition/) \ No newline at end of file +[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/alter/partition/) From ae017c10d35e41d774cf03c2e118570549ef9826 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 13 Jan 2021 18:55:24 -0400 Subject: [PATCH 137/264] Update partition.md tuple req. in part expressions --- docs/en/sql-reference/statements/alter/partition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 2d46ee609f1..2a65bc83824 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -286,7 +286,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; You can specify the partition expression in `ALTER ... PARTITION` queries in different ways: - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. -- As the expression from the table column. Constants and constant expressions are supported. For example, `ALTER TABLE visits DETACH PARTITION toYYYYMM(toDate('2019-01-25'))`. +- As the expression from the table column. Constants and constant expressions are supported. The expression must be wrapped in a `tuple()` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. From 2b40ac29e05883539539db713793af4e2ba599fc Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 13 Jan 2021 18:56:24 -0400 Subject: [PATCH 138/264] Update partition.md --- docs/ru/sql-reference/statements/alter/partition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index 4b9ae116ec2..5e6f323200f 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -288,7 +288,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать: - Имя партиции. Посмотреть имя партиции можно в столбце `partition` системной таблицы [system.parts](../../../operations/system-tables/parts.md#system_tables-parts). Например, `ALTER TABLE visits DETACH PARTITION 201901`. -- Произвольное выражение из столбцов исходной таблицы. Также поддерживаются константы и константные выражения. Выражение необходимо обернуть в функцию tuple(). Например, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. +- Произвольное выражение из столбцов исходной таблицы. Также поддерживаются константы и константные выражения. Выражение необходимо обернуть в функцию `tuple()`. Например, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Строковый идентификатор партиции. Идентификатор партиции используется для именования кусков партиции на файловой системе и в ZooKeeper. В запросах `ALTER` идентификатор партиции нужно указывать в секции `PARTITION ID`, в одинарных кавычках. Например, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - Для запросов [ATTACH PART](#alter_attach-partition) и [DROP DETACHED PART](#alter_drop-detached): чтобы задать имя куска партиции, используйте строковой литерал со значением из столбца `name` системной таблицы [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts). Например, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. From 8550fb813b6bb9034b4294d3329dec748f5b468f Mon Sep 17 00:00:00 2001 From: centos7 Date: Thu, 14 Jan 2021 10:57:52 +0800 Subject: [PATCH 139/264] fix cloneresize impl --- .../01552_impl_aggfunc_cloneresize.reference | 1 + .../01552_impl_aggfunc_cloneresize.sql | 57 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.reference create mode 100644 tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.sql diff --git a/tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.reference b/tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.reference new file mode 100644 index 00000000000..9972842f982 --- /dev/null +++ b/tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.reference @@ -0,0 +1 @@ +1 1 diff --git a/tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.sql b/tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.sql new file mode 100644 index 00000000000..849a4378e45 --- /dev/null +++ b/tests/queries/0_stateless/01552_impl_aggfunc_cloneresize.sql @@ -0,0 +1,57 @@ +drop table if EXISTS test_bm; + +drop table if EXISTS test_bm_join; + +create table test_bm( + dim UInt64, + id UInt64 ) +ENGINE = MergeTree() +ORDER BY( dim, id ) +SETTINGS index_granularity = 8192; + +create table test_bm_join( + dim UInt64, + id UInt64 ) +ENGINE = MergeTree() +ORDER BY(dim,id) +SETTINGS index_granularity = 8192; + +insert into test_bm VALUES (1,1),(2,2),(3,3),(4,4); + +select + dim , + sum(idnum) +from + test_bm_join +right join( + select + dim, + bitmapOrCardinality(ids,ids2) as idnum + from + ( + select + dim, + groupBitmapState(toUInt64(id)) as ids + FROM + test_bm + where + dim >2 + group by + dim ) A all + right join ( + select + dim, + groupBitmapState(toUInt64(id)) as ids2 + FROM + test_bm + where + dim < 2 + group by + dim ) B + using(dim) ) C +using(dim) +group by dim; + +drop table test_bm; + +drop table test_bm_join; From bfc27254b2f31ec02fcdccd5c34d566b751d13ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Jan 2021 11:07:13 +0300 Subject: [PATCH 140/264] Avoid redundant exception while dropping part --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e4a1c04f414..afa25c728a2 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6175,6 +6175,11 @@ bool StorageReplicatedMergeTree::dropPart( LOG_TRACE(log, "A new log entry appeared while trying to commit DROP RANGE. Retry."); continue; } + else if (rc == Coordination::Error::ZNONODE) + { + LOG_TRACE(log, "Other replica already removing same part {} or part deduplication node was removed by background thread. Retry.", part_name); + continue; + } else zkutil::KeeperMultiException::check(rc, ops, responses); From cefd941457dc7df330aa826dfd59864d78a8b58e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 11:38:24 +0300 Subject: [PATCH 141/264] Update test --- .../0_stateless/01651_bugs_from_15889.sql | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index fa4b7ca2268..621e573ffd0 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -13,6 +13,28 @@ SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } DROP TABLE IF EXISTS xp; DROP TABLE IF EXISTS xp_d; +DROP TABLE IF EXISTS trace_log; +CREATE TABLE trace_log +( + `event_date` Date, + `event_time` DateTime, + `event_time_microseconds` DateTime64(6), + `timestamp_ns` UInt64, + `revision` UInt32, + `trace_type` Enum8('Real' = 0, 'CPU' = 1, 'Memory' = 2, 'MemorySample' = 3), + `thread_id` UInt64, + `query_id` String, + `trace` Array(UInt64), + `size` Int64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192; + +INSERT INTO trace_log values ('2020-10-06','2020-10-06 13:43:39','2020-10-06 13:43:39.208819',1601981019208819975,54441,'Real',20412,'2e8ddf40-48da-4641-8ccc-573dd487753f',[140316350688023,130685338,226362737,224904385,227758790,227742969,227761037,224450136,219847931,219844987,219854151,223212098,223208665,228194329,228227607,257889111,257890159,258775545,258767526,140316350645979,140316343425599],0); + + set allow_introspection_functions = 1; WITH concat(addressToLine(arrayJoin(trace) AS addr), '#') AS symbol @@ -44,6 +66,8 @@ WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterd LIMIT 1 )); -- { serverError 42 } +DROP TABLE IF EXISTS trace_log; + WITH ( ( SELECT query_start_time_microseconds From 3194933788f9081adb43acf5921a282abbd7ebae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 11:39:27 +0300 Subject: [PATCH 142/264] Update test --- tests/queries/0_stateless/01651_bugs_from_15889.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 621e573ffd0..6eeb0ca583b 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -39,7 +39,7 @@ set allow_introspection_functions = 1; WITH concat(addressToLine(arrayJoin(trace) AS addr), '#') AS symbol SELECT count() > 7 -FROM system.trace_log AS t +FROM trace_log AS t WHERE (query_id = ( SELECT @@ -54,7 +54,7 @@ WHERE (query_id = WITH addressToSymbol(arrayJoin(trace)) AS symbol SELECT count() > 0 -FROM system.trace_log AS t +FROM trace_log AS t WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterday()) AND (trace_type = 'Memory') AND (query_id = ( SELECT From 64975c08e7f67607d461ed7d290f07f411fddb87 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Jan 2021 13:33:39 +0300 Subject: [PATCH 143/264] fix --- src/Storages/LiveView/TemporaryLiveViewCleaner.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp index 4f6eb1bb3e1..143e7460cc3 100644 --- a/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp +++ b/src/Storages/LiveView/TemporaryLiveViewCleaner.cpp @@ -55,6 +55,8 @@ void TemporaryLiveViewCleaner::startupIfNecessary() return; if (!views.empty()) startupIfNecessaryImpl(lock); + else + can_start_background_thread = true; } void TemporaryLiveViewCleaner::startupIfNecessaryImpl(const std::lock_guard &) From 29685aac3cbfe5ad89939e4a0dd1f30316534eff Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Jan 2021 13:42:51 +0300 Subject: [PATCH 144/264] More debug for stateless tests writer --- docker/test/stateless/run.sh | 2 ++ src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 309328bc8e2..371bd3539d5 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -66,3 +66,5 @@ function run_tests() export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: + +tar -cvf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index ef64ec28e79..506d7a0bcaa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -165,6 +165,11 @@ void MergeTreeDataPartWriterWide::shiftCurrentMark(const Granules & granules_wri void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Permutation * permutation) { + /// TODO: remove me + const auto storage_settings = storage.getSettings(); + if (settings.blocks_are_granules_size && block.rows() > storage_settings->index_granularity) + throw Exception(ErrorCodes::LOGICAL_ERROR, "When blocks are granules size block size ({} rows) cannot be bigger than fixed granularity ({} rows)", + block.rows(), storage_settings->index_granularity); /// Fill index granularity for this block /// if it's unknown (in case of insert data or horizontal merge, /// but not in case of vertical part of vertical merge) From f9ee3f8178609498f4d0ce6349b9abded52d13ab Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 14 Jan 2021 14:09:13 +0300 Subject: [PATCH 145/264] update test --- src/IO/tests/gtest_peekable_read_buffer.cpp | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/IO/tests/gtest_peekable_read_buffer.cpp b/src/IO/tests/gtest_peekable_read_buffer.cpp index f28f5588006..07d6404bbef 100644 --- a/src/IO/tests/gtest_peekable_read_buffer.cpp +++ b/src/IO/tests/gtest_peekable_read_buffer.cpp @@ -48,6 +48,8 @@ try DB::PeekableReadBufferCheckpoint checkpoint{peekable}; readAndAssert(peekable, "01234"); } + +#ifndef ABORT_ON_LOGICAL_ERROR bool exception = false; try { @@ -60,6 +62,7 @@ try exception = true; } ASSERT_TRUE(exception); +#endif assertAvailable(peekable, "56789"); readAndAssert(peekable, "56"); @@ -70,19 +73,10 @@ try peekable.dropCheckpoint(); assertAvailable(peekable, "789"); - exception = false; - try { DB::PeekableReadBufferCheckpoint checkpoint{peekable, true}; - peekable.ignore(30); + peekable.ignore(20); } - catch (DB::Exception & e) - { - if (e.code() != DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED) - throw; - exception = true; - } - ASSERT_TRUE(exception); assertAvailable(peekable, "789qwertyuiop"); readAndAssert(peekable, "789qwertyu"); From 7be81c3a7167925761e51481cf62b08329ddd16a Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 14 Jan 2021 12:45:56 +0100 Subject: [PATCH 146/264] Fix images dependencies --- docker/images.json | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docker/images.json b/docker/images.json index e0f2608d848..6ab2d287b74 100644 --- a/docker/images.json +++ b/docker/images.json @@ -45,7 +45,8 @@ "name": "yandex/clickhouse-stateless-test", "dependent": [ "docker/test/stateful", - "docker/test/coverage" + "docker/test/coverage", + "docker/test/unit" ] }, "docker/test/stateless_pytest": { @@ -134,7 +135,9 @@ "name": "yandex/clickhouse-test-base", "dependent": [ "docker/test/stateless", - "docker/test/stateless_pytest" + "docker/test/stateless_unbundled", + "docker/test/stateless_pytest", + "docker/test/integration/base" ] }, "docker/packager/unbundled": { From 6bec839ab3acb2faea96f85f8a27a0f52bf2fa47 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 14 Jan 2021 12:46:24 +0100 Subject: [PATCH 147/264] Touch dockerfiles to trigger rebuild --- docker/test/integration/base/Dockerfile | 1 + docker/test/stateless_unbundled/Dockerfile | 1 + docker/test/unit/Dockerfile | 1 + 3 files changed, 3 insertions(+) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 99095de60fb..4963ff0094d 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -30,3 +30,4 @@ RUN curl 'https://cdn.mysql.com//Downloads/Connector-ODBC/8.0/mysql-connector-od ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone + diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index d212290d553..9efe08dbf23 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -86,3 +86,4 @@ RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone COPY run.sh / CMD ["/bin/bash", "/run.sh"] + diff --git a/docker/test/unit/Dockerfile b/docker/test/unit/Dockerfile index 0f65649fb76..f01ed613918 100644 --- a/docker/test/unit/Dockerfile +++ b/docker/test/unit/Dockerfile @@ -7,3 +7,4 @@ RUN apt-get install gdb CMD service zookeeper start && sleep 7 && /usr/share/zookeeper/bin/zkCli.sh -server localhost:2181 -create create /clickhouse_test ''; \ gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt + From ac426c3da641644144e76f730fdbb5d044d01e04 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Jan 2021 15:32:20 +0300 Subject: [PATCH 148/264] restrict MODIFY TTL for tables created in old syntax --- src/Storages/MergeTree/MergeTreeData.cpp | 6 ++++++ tests/queries/0_stateless/01652_ttl_old_syntax.reference | 0 tests/queries/0_stateless/01652_ttl_old_syntax.sql | 6 ++++++ 3 files changed, 12 insertions(+) create mode 100644 tests/queries/0_stateless/01652_ttl_old_syntax.reference create mode 100644 tests/queries/0_stateless/01652_ttl_old_syntax.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 897aea4363a..f84de847acd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1506,6 +1506,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S "ALTER MODIFY ORDER BY is not supported for default-partitioned tables created with the old syntax", ErrorCodes::BAD_ARGUMENTS); } + if (command.type == AlterCommand::MODIFY_TTL && !is_custom_partitioned) + { + throw Exception( + "ALTER MODIFY TTL is not supported for default-partitioned tables created with the old syntax", + ErrorCodes::BAD_ARGUMENTS); + } if (command.type == AlterCommand::MODIFY_SAMPLE_BY) { if (!is_custom_partitioned) diff --git a/tests/queries/0_stateless/01652_ttl_old_syntax.reference b/tests/queries/0_stateless/01652_ttl_old_syntax.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01652_ttl_old_syntax.sql b/tests/queries/0_stateless/01652_ttl_old_syntax.sql new file mode 100644 index 00000000000..05c391b85e5 --- /dev/null +++ b/tests/queries/0_stateless/01652_ttl_old_syntax.sql @@ -0,0 +1,6 @@ +DROP TABLE IF EXISTS ttl_old_syntax; + +CREATE TABLE ttl_old_syntax (d Date, i Int) ENGINE = MergeTree(d, i, 8291); +ALTER TABLE ttl_old_syntax MODIFY TTL toDate('2020-01-01'); -- { serverError 36 } + +DROP TABLE ttl_old_syntax; From 9e08890f956f4a47dca7a209347c72353bd7617b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 16:53:42 +0300 Subject: [PATCH 149/264] Update 01651_bugs_from_15889.sql --- tests/queries/0_stateless/01651_bugs_from_15889.sql | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 6eeb0ca583b..1275f258225 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -1,6 +1,8 @@ DROP TABLE IF EXISTS xp; DROP TABLE IF EXISTS xp_d; +SET log_queries = 1; + CREATE TABLE xp (`A` Date, `B` Int64, `S` String) ENGINE = MergeTree PARTITION BY toYYYYMM(A) ORDER BY B; INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); @@ -68,6 +70,8 @@ WHERE greaterOrEquals(event_date, ignore(ignore(ignore(NULL, '')), 256), yesterd DROP TABLE IF EXISTS trace_log; +SYSTEM FLUSH LOGS; + WITH ( ( SELECT query_start_time_microseconds From c65f79d69b9f67e117370c569199504b29a9faec Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 17:29:51 +0300 Subject: [PATCH 150/264] Simplify tupleHammingDistance --- src/Functions/tupleHammingDistance.cpp | 266 +++++++++---------------- 1 file changed, 91 insertions(+), 175 deletions(-) diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index 67d5f73065b..698b51d7cb4 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -15,201 +15,117 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; } -template -struct TupleHammingDistanceImpl -{ - using ResultType = UInt8; - - static void NO_INLINE vectorVector( - const PaddedPODArray & a1, - const PaddedPODArray & b1, - const PaddedPODArray & a2, - const PaddedPODArray & b2, - PaddedPODArray & c) - { - size_t size = a1.size(); - for (size_t i = 0; i < size; ++i) - c[i] = apply(a1[i], a2[i]) + apply(b1[i], b2[i]); - } - - static void NO_INLINE - vectorConstant(const PaddedPODArray & a1, const PaddedPODArray & b1, UInt64 a2, UInt64 b2, PaddedPODArray & c) - { - size_t size = a1.size(); - for (size_t i = 0; i < size; ++i) - c[i] = apply(a1[i], a2) + apply(b1[i], b2); - } - - static void NO_INLINE - constantVector(UInt64 a1, UInt64 b1, const PaddedPODArray & a2, const PaddedPODArray & b2, PaddedPODArray & c) - { - size_t size = a2.size(); - for (size_t i = 0; i < size; ++i) - c[i] = apply(a1, a2[i]) + apply(b1, b2[i]); - } - - static ResultType constantConstant(UInt64 a1, UInt64 b1, UInt64 a2, UInt64 b2) { return apply(a1, a2) + apply(b1, b2); } - -private: - static inline UInt8 apply(UInt64 a, UInt64 b) { return a != b; } -}; - -template -bool castType(const IDataType * type, F && f) -{ - return castTypeToEither< - DataTypeInt8, - DataTypeInt16, - DataTypeInt32, - DataTypeInt64, - DataTypeUInt8, - DataTypeUInt16, - DataTypeUInt32, - DataTypeUInt64>(type, std::forward(f)); -} - -template -static bool castBothTypes(const IDataType * left, const IDataType * right, F && f) -{ - return castType(left, [&](const auto & left_) { return castType(right, [&](const auto & right_) { return f(left_, right_); }); }); -} - -// tupleHammingDistance function: (Tuple(Integer, Integer), Tuple(Integer, Integer))->0/1/2 -// in order to avoid code bloating, for non-constant tuple, we make sure that the elements -// in the tuple should have same data type, and for constant tuple, elements can be any integer -// data type, we cast all of them into UInt64 +/// tupleHammingDistance function: (Tuple(...), Tuple(...))-> N +/// Return the number of non-equal tuple elements class FunctionTupleHammingDistance : public IFunction { +private: + const Context & context; + public: static constexpr auto name = "tupleHammingDistance"; using ResultType = UInt8; - static FunctionPtr create(const Context &) { return std::make_shared(); } + + explicit FunctionTupleHammingDistance(const Context & context_) : context(context_) {} + static FunctionPtr create(const Context & context) { return std::make_shared(context); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isTuple(arguments[0])) - throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (!isTuple(arguments[1])) - throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(); + const auto * left_tuple = checkAndGetDataType(arguments[0].type.get()); + const auto * right_tuple = checkAndGetDataType(arguments[1].type.get()); + const auto * left_col = checkAndGetColumn(arguments[0].column.get()); + const auto * right_col = checkAndGetColumn(arguments[1].column.get()); + + if (!left_tuple) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 0 of function {} should be tuples, got {}", + getName(), arguments[0].type->getName()); + + if (!right_tuple) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 1 of function {} should be tuples, got {}", + getName(), arguments[1].type->getName()); + + const auto & left_elements = left_tuple->getElements(); + const auto & right_elements = right_tuple->getElements(); + + if (left_elements.size() != right_elements.size()) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Expected tuples of the same size as arguments of function {}. Got {} and {}", + getName(), arguments[0].type->getName(), arguments[1].type->getName()); + + size_t tuple_size = left_elements.size(); + if (tuple_size == 0) + return std::make_shared(); + + auto compare = FunctionFactory::instance().get("equals", context); + auto plus = FunctionFactory::instance().get("plus", context); + DataTypes types(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) + { + try + { + ColumnWithTypeAndName left{left_col ? left_col->getColumnPtr(i) : nullptr, left_elements[i], {}}; + ColumnWithTypeAndName right{right_col ? right_col->getColumnPtr(i) : nullptr, right_elements[i], {}}; + auto elem_compare = compare->build(ColumnsWithTypeAndName{left, right}); + types[i] = elem_compare->getResultType(); + } + catch (DB::Exception & e) + { + e.addMessage("While executing function {} for tuple element {}", getName(), i); + throw; + } + } + + auto res_type = types[0]; + for (size_t i = 1; i < tuple_size; ++i) + { + ColumnWithTypeAndName left{res_type, {}}; + ColumnWithTypeAndName right{types[i], {}}; + auto plus_elem = plus->build({left, right}); + res_type = plus_elem->getResultType(); + } + + return res_type; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - const ColumnWithTypeAndName & arg1 = arguments[0]; - const ColumnWithTypeAndName & arg2 = arguments[1]; - const DataTypeTuple & type1 = static_cast(*arg1.type); - const DataTypeTuple & type2 = static_cast(*arg2.type); - const auto & left_elems = type1.getElements(); - const auto & right_elems = type2.getElements(); - if (left_elems.size() != 2 || right_elems.size() != 2) - throw Exception( - "Illegal column of arguments of function " + getName() + ", tuple should have exactly two elements.", - ErrorCodes::ILLEGAL_COLUMN); + const auto * left_tuple = checkAndGetDataType(arguments[0].type.get()); + const auto * right_tuple = checkAndGetDataType(arguments[1].type.get()); + const auto * left_col = checkAndGetColumn(arguments[0].column.get()); + const auto * right_col = checkAndGetColumn(arguments[1].column.get()); + const auto & left_elements = left_tuple->getElements(); + const auto & right_elements = right_tuple->getElements(); - ColumnPtr result_column; + size_t tuple_size = left_elements.size(); + if (tuple_size == 0) + return DataTypeUInt8().createColumnConstWithDefaultValue(input_rows_count); - bool valid = castBothTypes(left_elems[0].get(), right_elems[0].get(), [&](const auto & left, const auto & right) + auto compare = FunctionFactory::instance().get("equals", context); + auto plus = FunctionFactory::instance().get("plus", context); + ColumnsWithTypeAndName columns(tuple_size); + for (size_t i = 0; i < tuple_size; ++i) { - using LeftDataType = std::decay_t; - using RightDataType = std::decay_t; - using T0 = typename LeftDataType::FieldType; - using T1 = typename RightDataType::FieldType; - using ColVecT0 = ColumnVector; - using ColVecT1 = ColumnVector; - using ColVecResult = ColumnVector; + ColumnWithTypeAndName left{left_col->getColumnPtr(i), left_elements[i], {}}; + ColumnWithTypeAndName right{right_col->getColumnPtr(i), right_elements[i], {}}; + auto elem_compare = compare->build(ColumnsWithTypeAndName{left, right}); + columns[i].type = elem_compare->getResultType(); + columns[i].column = elem_compare->execute({left, right}, columns[i].type, input_rows_count); + } - using OpImpl = TupleHammingDistanceImpl; + auto res = columns[0]; + for (size_t i = 1; i < tuple_size; ++i) + { + auto plus_elem = plus->build({res, columns[i]}); + auto res_type = plus_elem->getResultType(); + res.column = plus_elem->execute({res, columns[i]}, res_type, input_rows_count); + res.type = res_type; + } - // we can not useDefaultImplementationForConstants, - // because with that, tupleHammingDistance((10, 300), (10, 20)) does not work, - // since 10 has data type UInt8, and 300 has data type UInt16 - if (const ColumnConst * const_col_left = checkAndGetColumnConst(arg1.column.get())) - { - if (const ColumnConst * const_col_right = checkAndGetColumnConst(arg2.column.get())) - { - auto cols1 = convertConstTupleToConstantElements(*const_col_left); - auto cols2 = convertConstTupleToConstantElements(*const_col_right); - Field a1, b1, a2, b2; - cols1[0]->get(0, a1); - cols1[1]->get(0, b1); - cols2[0]->get(0, a2); - cols2[1]->get(0, b2); - auto res = OpImpl::constantConstant(a1.get(), b1.get(), a2.get(), b2.get()); - result_column = DataTypeUInt8().createColumnConst(const_col_left->size(), toField(res)); - return true; - } - } - - typename ColVecResult::MutablePtr col_res = nullptr; - col_res = ColVecResult::create(); - auto & vec_res = col_res->getData(); - vec_res.resize(input_rows_count); - // constant tuple - non-constant tuple - if (const ColumnConst * const_col_left = checkAndGetColumnConst(arg1.column.get())) - { - if (const ColumnTuple * col_right = typeid_cast(arg2.column.get())) - { - auto const_cols = convertConstTupleToConstantElements(*const_col_left); - Field a1, b1; - const_cols[0]->get(0, a1); - const_cols[1]->get(0, b1); - auto col_r1 = checkAndGetColumn(&col_right->getColumn(0)); - auto col_r2 = checkAndGetColumn(&col_right->getColumn(1)); - if (col_r1 && col_r2) - OpImpl::constantVector(a1.get(), b1.get(), col_r1->getData(), col_r2->getData(), vec_res); - else - return false; - } - else - return false; - } - else if (const ColumnTuple * col_left = typeid_cast(arg1.column.get())) - { - auto col_l1 = checkAndGetColumn(&col_left->getColumn(0)); - auto col_l2 = checkAndGetColumn(&col_left->getColumn(1)); - if (col_l1 && col_l2) - { - // non-constant tuple - constant tuple - if (const ColumnConst * const_col_right = checkAndGetColumnConst(arg2.column.get())) - { - auto const_cols = convertConstTupleToConstantElements(*const_col_right); - Field a2, b2; - const_cols[0]->get(0, a2); - const_cols[1]->get(0, b2); - OpImpl::vectorConstant(col_l1->getData(), col_l2->getData(), a2.get(), a2.get(), vec_res); - } - // non-constant tuple - non-constant tuple - else if (const ColumnTuple * col_right = typeid_cast(arg2.column.get())) - { - auto col_r1 = checkAndGetColumn(&col_right->getColumn(0)); - auto col_r2 = checkAndGetColumn(&col_right->getColumn(1)); - if (col_r1 && col_r2) - OpImpl::vectorVector(col_l1->getData(), col_l2->getData(), col_r1->getData(), col_r2->getData(), vec_res); - else - return false; - } - else - return false; - } - else - return false; - } - else - return false; - result_column = std::move(col_res); - return true; - }); - if (!valid) - throw Exception(getName() + "'s arguments do not match the expected data types", ErrorCodes::ILLEGAL_COLUMN); - - return result_column; + return res.column; } }; From 3ee4e9eda821b192e7f56df7949e59f43b5faf4f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 18:29:42 +0300 Subject: [PATCH 151/264] Simplify tupleHammingDistance --- src/Functions/tupleHammingDistance.cpp | 35 ++++++++++++++++++++------ 1 file changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index 698b51d7cb4..6cb3883cd08 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -33,12 +33,29 @@ public: size_t getNumberOfArguments() const override { return 2; } + bool useDefaultImplementationForConstants() const override { return true; } + + Columns getTupleElements(const IColumn & column) const + { + if (const auto * const_column = typeid_cast(&column)) + return convertConstTupleToConstantElements(*const_column); + + if (const auto * column_tuple = typeid_cast(&column)) + { + Columns columns(column_tuple->tupleSize()); + for (size_t i = 0; i < columns.size(); ++i) + columns[i] = column_tuple->getColumnPtr(i); + return columns; + } + + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument of function {} should be tuples, got {}", + getName(), column.getName()) + } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { const auto * left_tuple = checkAndGetDataType(arguments[0].type.get()); const auto * right_tuple = checkAndGetDataType(arguments[1].type.get()); - const auto * left_col = checkAndGetColumn(arguments[0].column.get()); - const auto * right_col = checkAndGetColumn(arguments[1].column.get()); if (!left_tuple) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 0 of function {} should be tuples, got {}", @@ -48,8 +65,12 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 1 of function {} should be tuples, got {}", getName(), arguments[1].type->getName()); - const auto & left_elements = left_tuple->getElements(); - const auto & right_elements = right_tuple->getElements(); + Columns left_elements; + Columns right_elements; + if (arguments[0].column) + left_elements = getTupleElements(*arguments[0].column); + if (arguments[1].column) + right_elements = getTupleElements(*arguments[1].column); if (left_elements.size() != right_elements.size()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, @@ -95,10 +116,8 @@ public: { const auto * left_tuple = checkAndGetDataType(arguments[0].type.get()); const auto * right_tuple = checkAndGetDataType(arguments[1].type.get()); - const auto * left_col = checkAndGetColumn(arguments[0].column.get()); - const auto * right_col = checkAndGetColumn(arguments[1].column.get()); - const auto & left_elements = left_tuple->getElements(); - const auto & right_elements = right_tuple->getElements(); + auto left_elements = getTupleElements(*arguments[0].column); + auto right_elements = getTupleElements(*arguments[1].column); size_t tuple_size = left_elements.size(); if (tuple_size == 0) From fc57ca5f500ba57678439b47323a479571855aea Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 18:34:44 +0300 Subject: [PATCH 152/264] Simplify tupleHammingDistance --- src/Functions/tupleHammingDistance.cpp | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index 6cb3883cd08..bd7e3db5c0c 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -49,7 +49,7 @@ public: } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument of function {} should be tuples, got {}", - getName(), column.getName()) + getName(), column.getName()); } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override @@ -65,6 +65,9 @@ public: throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument 1 of function {} should be tuples, got {}", getName(), arguments[1].type->getName()); + const auto & left_types = left_tuple->getElements(); + const auto & right_types = right_tuple->getElements(); + Columns left_elements; Columns right_elements; if (arguments[0].column) @@ -88,8 +91,8 @@ public: { try { - ColumnWithTypeAndName left{left_col ? left_col->getColumnPtr(i) : nullptr, left_elements[i], {}}; - ColumnWithTypeAndName right{right_col ? right_col->getColumnPtr(i) : nullptr, right_elements[i], {}}; + ColumnWithTypeAndName left{left_elements.empty() ? nullptr : left_elements[i], left_types[i], {}}; + ColumnWithTypeAndName right{right_elements.empty() ? nullptr : right_elements[i], right_types[i], {}}; auto elem_compare = compare->build(ColumnsWithTypeAndName{left, right}); types[i] = elem_compare->getResultType(); } @@ -116,6 +119,8 @@ public: { const auto * left_tuple = checkAndGetDataType(arguments[0].type.get()); const auto * right_tuple = checkAndGetDataType(arguments[1].type.get()); + const auto & left_types = left_tuple->getElements(); + const auto & right_types = right_tuple->getElements(); auto left_elements = getTupleElements(*arguments[0].column); auto right_elements = getTupleElements(*arguments[1].column); @@ -128,8 +133,8 @@ public: ColumnsWithTypeAndName columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) { - ColumnWithTypeAndName left{left_col->getColumnPtr(i), left_elements[i], {}}; - ColumnWithTypeAndName right{right_col->getColumnPtr(i), right_elements[i], {}}; + ColumnWithTypeAndName left{left_elements[i], left_types[i], {}}; + ColumnWithTypeAndName right{right_elements[i], right_types[i], {}}; auto elem_compare = compare->build(ColumnsWithTypeAndName{left, right}); columns[i].type = elem_compare->getResultType(); columns[i].column = elem_compare->execute({left, right}, columns[i].type, input_rows_count); From 4bb0bd16e1b61d2dd79fd06d6f64343499e6d2c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 19:02:17 +0300 Subject: [PATCH 153/264] Simplify tupleHammingDistance --- src/Functions/tupleHammingDistance.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index bd7e3db5c0c..9785c118e34 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -75,16 +75,16 @@ public: if (arguments[1].column) right_elements = getTupleElements(*arguments[1].column); - if (left_elements.size() != right_elements.size()) + if (left_types.size() != right_types.size()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Expected tuples of the same size as arguments of function {}. Got {} and {}", getName(), arguments[0].type->getName(), arguments[1].type->getName()); - size_t tuple_size = left_elements.size(); + size_t tuple_size = left_types.size(); if (tuple_size == 0) return std::make_shared(); - auto compare = FunctionFactory::instance().get("equals", context); + auto compare = FunctionFactory::instance().get("notEquals", context); auto plus = FunctionFactory::instance().get("plus", context); DataTypes types(tuple_size); for (size_t i = 0; i < tuple_size; ++i) @@ -128,7 +128,7 @@ public: if (tuple_size == 0) return DataTypeUInt8().createColumnConstWithDefaultValue(input_rows_count); - auto compare = FunctionFactory::instance().get("equals", context); + auto compare = FunctionFactory::instance().get("notEquals", context); auto plus = FunctionFactory::instance().get("plus", context); ColumnsWithTypeAndName columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) From 39de699790439bff3469a46a1255764961eee234 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 19:13:14 +0300 Subject: [PATCH 154/264] Added test. --- .../01653_tuple_hamming_distance_2.reference | 18 +++++++++++++++ .../01653_tuple_hamming_distance_2.sql | 23 +++++++++++++++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/01653_tuple_hamming_distance_2.reference create mode 100644 tests/queries/0_stateless/01653_tuple_hamming_distance_2.sql diff --git a/tests/queries/0_stateless/01653_tuple_hamming_distance_2.reference b/tests/queries/0_stateless/01653_tuple_hamming_distance_2.reference new file mode 100644 index 00000000000..613ab096c8c --- /dev/null +++ b/tests/queries/0_stateless/01653_tuple_hamming_distance_2.reference @@ -0,0 +1,18 @@ +0 +1 +\N +\N +2 +1 +1 +1 +0 +1 +2 +\N +\N +\N +1 +0 +1 +2 diff --git a/tests/queries/0_stateless/01653_tuple_hamming_distance_2.sql b/tests/queries/0_stateless/01653_tuple_hamming_distance_2.sql new file mode 100644 index 00000000000..81afb1e1201 --- /dev/null +++ b/tests/queries/0_stateless/01653_tuple_hamming_distance_2.sql @@ -0,0 +1,23 @@ +SELECT tupleHammingDistance(tuple(1), tuple(1)); +SELECT tupleHammingDistance(tuple(1), tuple(2)); +SELECT tupleHammingDistance(tuple(1), tuple(Null)); +SELECT tupleHammingDistance(tuple(Null), tuple(Null)); +SELECT tupleHammingDistance((1, 2), (3, 4)); +SELECT tupleHammingDistance((1, 2), (1, 4)); +SELECT tupleHammingDistance(materialize((1, 2)), (1, 4)); +SELECT tupleHammingDistance(materialize((1, 2)),materialize ((1, 4))); +SELECT tupleHammingDistance((1, 2), (1, 2)); +SELECT tupleHammingDistance((1, 2), (1, 257)); +SELECT tupleHammingDistance((1, 2, 3), (1, 257, 65537)); +SELECT tupleHammingDistance((1, 2), (1, Null)); +SELECT tupleHammingDistance((1, Null), (1, Null)); +SELECT tupleHammingDistance((Null, Null), (Null, Null)); +SELECT tupleHammingDistance(('abc', 2), ('abc', 257)); +SELECT tupleHammingDistance(('abc', (1, 2)), ('abc', (1, 2))); +SELECT tupleHammingDistance(('abc', (1, 2)), ('def', (1, 2))); +SELECT tupleHammingDistance(('abc', (1, 2)), ('def', (1, 3))); + + +SELECT tupleHammingDistance(tuple(1), tuple(1, 1)); --{serverError 43} +SELECT tupleHammingDistance(tuple(1), tuple('a')); --{serverError 386} +SELECT tupleHammingDistance((1, 3), (3, 'a')); --{serverError 386} From 490326e6b09765dad67b162d1cabc93e6d00a106 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 14 Jan 2021 19:24:06 +0300 Subject: [PATCH 155/264] Fix symlinks --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 371bd3539d5..3d42e4295fe 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -67,4 +67,4 @@ export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: -tar -cvf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log +tar -chvf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log From 8276a1c8d23190426cd34c4d5cf81c6fff56b098 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Jan 2021 19:24:13 +0300 Subject: [PATCH 156/264] Faster parts removal, more safe and efficient interface of IDisk --- src/Common/ErrorCodes.cpp | 1 + src/Disks/DiskCacheWrapper.cpp | 18 +++++++++-- src/Disks/DiskCacheWrapper.h | 4 ++- src/Disks/DiskDecorator.cpp | 14 ++++++-- src/Disks/DiskDecorator.h | 4 ++- src/Disks/DiskLocal.cpp | 19 +++++++++-- src/Disks/DiskLocal.h | 5 +-- src/Disks/DiskMemory.cpp | 32 +++++++++++++++++-- src/Disks/DiskMemory.h | 5 +-- src/Disks/IDisk.h | 17 +++++----- src/Disks/S3/DiskS3.cpp | 31 ++++++++++++------ src/Disks/S3/DiskS3.h | 5 +-- src/Disks/S3/registerDiskS3.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 14 ++++---- .../MergeTree/IMergedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 14 ++++---- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeMutationEntry.cpp | 2 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 4 +-- 20 files changed, 141 insertions(+), 56 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 1ceb9df011d..a2cd65137c0 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -532,6 +532,7 @@ M(563, CANNOT_READ_MAP_FROM_TEXT) \ M(564, INTERSERVER_SCHEME_DOESNT_MATCH) \ M(565, TOO_MANY_PARTITIONS) \ + M(566, CANNOT_RMDIR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index 652ea9935ed..26e6b7609f3 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -252,11 +252,23 @@ void DiskCacheWrapper::copyFile(const String & from_path, const String & to_path DiskDecorator::copyFile(from_path, to_path); } -void DiskCacheWrapper::remove(const String & path) +void DiskCacheWrapper::removeFile(const String & path) +{ + cache_disk->removeFileIfExists(path); + DiskDecorator::removeFile(path); +} + +void DiskCacheWrapper::removeFileIfExists(const String & path) +{ + cache_disk->removeFileIfExists(path); + DiskDecorator::removeFileIfExists(path); +} + +void DiskCacheWrapper::removeDirectory(const String & path) { if (cache_disk->exists(path)) - cache_disk->remove(path); - DiskDecorator::remove(path); + cache_disk->removeDirectory(path); + DiskDecorator::removeDirectory(path); } void DiskCacheWrapper::removeRecursive(const String & path) diff --git a/src/Disks/DiskCacheWrapper.h b/src/Disks/DiskCacheWrapper.h index 9e2b3c3b592..10d24bf92e8 100644 --- a/src/Disks/DiskCacheWrapper.h +++ b/src/Disks/DiskCacheWrapper.h @@ -37,7 +37,9 @@ public: readFile(const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold) const override; std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; - void remove(const String & path) override; + void removeFile(const String & path) override; + void removeFileIfExists(const String & path) override; + void removeDirectory(const String & path) override; void removeRecursive(const String & path) override; void createHardLink(const String & src_path, const String & dst_path) override; ReservationPtr reserve(UInt64 bytes) override; diff --git a/src/Disks/DiskDecorator.cpp b/src/Disks/DiskDecorator.cpp index e62f2737ec5..a475f85f9d2 100644 --- a/src/Disks/DiskDecorator.cpp +++ b/src/Disks/DiskDecorator.cpp @@ -130,9 +130,19 @@ DiskDecorator::writeFile(const String & path, size_t buf_size, WriteMode mode) return delegate->writeFile(path, buf_size, mode); } -void DiskDecorator::remove(const String & path) +void DiskDecorator::removeFile(const String & path) { - delegate->remove(path); + delegate->removeFile(path); +} + +void DiskDecorator::removeFileIfExists(const String & path) +{ + delegate->removeFileIfExists(path); +} + +void DiskDecorator::removeDirectory(const String & path) +{ + delegate->removeDirectory(path); } void DiskDecorator::removeRecursive(const String & path) diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 61b7ee9ecee..87668e82a3d 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -39,7 +39,9 @@ public: readFile(const String & path, size_t buf_size, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold) const override; std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; - void remove(const String & path) override; + void removeFile(const String & path) override; + void removeFileIfExists(const String & path) override; + void removeDirectory(const String & path) override; void removeRecursive(const String & path) override; void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; Poco::Timestamp getLastModified(const String & path) override; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index e2d6a5410a3..2b778b234fa 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -25,6 +25,8 @@ namespace ErrorCodes extern const int CANNOT_FSYNC; extern const int CANNOT_CLOSE_FILE; extern const int CANNOT_TRUNCATE_FILE; + extern const int CANNOT_UNLINK; + extern const int CANNOT_RMDIR; } std::mutex DiskLocal::reservation_mutex; @@ -237,9 +239,22 @@ DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) return std::make_unique(disk_path + path, buf_size, flags); } -void DiskLocal::remove(const String & path) +void DiskLocal::removeFile(const String & path) { - Poco::File(disk_path + path).remove(false); + if (0 != unlink(path.c_str())) + throwFromErrnoWithPath("Cannot unlink file " + path, path, ErrorCodes::CANNOT_UNLINK); +} + +void DiskLocal::removeFileIfExists(const String & path) +{ + if (0 != unlink(path.c_str()) && errno != ENOENT) + throwFromErrnoWithPath("Cannot unlink file " + path, path, ErrorCodes::CANNOT_UNLINK); +} + +void DiskLocal::removeDirectory(const String & path) +{ + if (0 != rmdir(path.c_str())) + throwFromErrnoWithPath("Cannot rmdir " + path, path, ErrorCodes::CANNOT_RMDIR); } void DiskLocal::removeRecursive(const String & path) diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index d862f72d4f8..e82bac57825 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -85,8 +85,9 @@ public: size_t buf_size, WriteMode mode) override; - void remove(const String & path) override; - + void removeFile(const String & path) override; + void removeFileIfExists(const String & path) override; + void removeDirectory(const String & path) override; void removeRecursive(const String & path) override; void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index aecdbf02f50..a3e57f32dcc 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -348,7 +348,21 @@ std::unique_ptr DiskMemory::writeFile(const String & pa return std::make_unique(this, path, mode, buf_size); } -void DiskMemory::remove(const String & path) +void DiskMemory::removeFile(const String & path) +{ + std::lock_guard lock(mutex); + + auto file_it = files.find(path); + if (file_it == files.end()) + throw Exception("File '" + path + "' doesn't exist", ErrorCodes::FILE_DOESNT_EXIST); + + if (file_it->second.type == FileType::Directory) + throw Exception("Path '" + path + "' is a directory", ErrorCodes::CANNOT_DELETE_DIRECTORY); + else + files.erase(file_it); +} + +void DiskMemory::removeDirectory(const String & path) { std::lock_guard lock(mutex); @@ -364,10 +378,24 @@ void DiskMemory::remove(const String & path) } else { - files.erase(file_it); + throw Exception("Path '" + path + "' is not a directory", ErrorCodes::CANNOT_DELETE_DIRECTORY); } } +void DiskMemory::removeFileIfExists(const String & path) +{ + std::lock_guard lock(mutex); + + auto file_it = files.find(path); + if (file_it == files.end()) + return; + + if (file_it->second.type == FileType::Directory) + throw Exception("Path '" + path + "' is a directory", ErrorCodes::CANNOT_DELETE_DIRECTORY); + else + files.erase(file_it); +} + void DiskMemory::removeRecursive(const String & path) { std::lock_guard lock(mutex); diff --git a/src/Disks/DiskMemory.h b/src/Disks/DiskMemory.h index e24f3c9c184..f8c2815b269 100644 --- a/src/Disks/DiskMemory.h +++ b/src/Disks/DiskMemory.h @@ -76,8 +76,9 @@ public: size_t buf_size, WriteMode mode) override; - void remove(const String & path) override; - + void removeFile(const String & path) override; + void removeFileIfExists(const String & path) override; + void removeDirectory(const String & path) override; void removeRecursive(const String & path) override; void setLastModified(const String &, const Poco::Timestamp &) override {} diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 437718924db..36f07c3b110 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -150,19 +150,18 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite) = 0; - /// Remove file or directory. Throws exception if file doesn't exists or if directory is not empty. - virtual void remove(const String & path) = 0; + /// Remove file. Throws exception if file doesn't exists or it's a directory. + virtual void removeFile(const String & path) = 0; + + /// Remove file if it exists. + virtual void removeFileIfExists(const String & path) = 0; + + /// Remove directory. Throws exception if it's not a directory or if directory is not empty. + virtual void removeDirectory(const String & path) = 0; /// Remove file or directory with all children. Use with extra caution. Throws exception if file doesn't exists. virtual void removeRecursive(const String & path) = 0; - /// Remove file or directory if it exists. - void removeIfExists(const String & path) - { - if (exists(path)) - remove(path); - } - /// Set last modified time to file or directory at `path`. virtual void setLastModified(const String & path, const Poco::Timestamp & timestamp) = 0; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index d9b2eaefa81..9a60879085e 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -38,6 +38,7 @@ namespace ErrorCodes extern const int INCORRECT_DISK_INDEX; extern const int NOT_IMPLEMENTED; extern const int PATH_ACCESS_DENIED; + extern const int CANNOT_DELETE_DIRECTORY; } @@ -606,7 +607,7 @@ void DiskS3::clearDirectory(const String & path) { for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) if (isFile(it->path())) - remove(it->path()); + removeFile(it->path()); } void DiskS3::moveFile(const String & from_path, const String & to_path) @@ -625,7 +626,7 @@ void DiskS3::replaceFile(const String & from_path, const String & to_path) Poco::File tmp_file(metadata_path + to_path + ".old"); to_file.renameTo(tmp_file.path()); from_file.renameTo(metadata_path + to_path); - remove(to_path + ".old"); + removeFile(to_path + ".old"); } else from_file.renameTo(to_file.path()); @@ -634,7 +635,7 @@ void DiskS3::replaceFile(const String & from_path, const String & to_path) void DiskS3::copyFile(const String & from_path, const String & to_path) { if (exists(to_path)) - remove(to_path); + removeFile(to_path); auto from = readMeta(from_path); auto to = createMeta(to_path); @@ -678,7 +679,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, { /// If metadata file exists - remove and create new. if (exist) - remove(path); + removeFile(path); auto metadata = createMeta(path); /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. @@ -708,10 +709,7 @@ void DiskS3::removeMeta(const String & path, AwsS3KeyKeeper & keys) Poco::File file(metadata_path + path); if (!file.isFile()) - { - file.remove(); - return; - } + throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path); try { @@ -785,13 +783,28 @@ void DiskS3::removeAws(const AwsS3KeyKeeper & keys) } } -void DiskS3::remove(const String & path) +void DiskS3::removeFile(const String & path) { AwsS3KeyKeeper keys; removeMeta(path, keys); removeAws(keys); } +void DiskS3::removeFileIfExists(const String & path) +{ + AwsS3KeyKeeper keys; + if (Poco::File(metadata_path + path).exists()) + { + removeMeta(path, keys); + removeAws(keys); + } +} + +void DiskS3::removeDirectory(const String & path) +{ + Poco::File(metadata_path + path).remove(); +} + void DiskS3::removeRecursive(const String & path) { AwsS3KeyKeeper keys; diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 900378f46cd..abdc049a58e 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -90,8 +90,9 @@ public: size_t buf_size, WriteMode mode) override; - void remove(const String & path) override; - + void removeFile(const String & path) override; + void removeFileIfExists(const String & path) override; + void removeDirectory(const String & path) override; void removeRecursive(const String & path) override; void createHardLink(const String & src_path, const String & dst_path) override; diff --git a/src/Disks/S3/registerDiskS3.cpp b/src/Disks/S3/registerDiskS3.cpp index fd658d95327..a1872a9ad6c 100644 --- a/src/Disks/S3/registerDiskS3.cpp +++ b/src/Disks/S3/registerDiskS3.cpp @@ -37,7 +37,7 @@ namespace throw Exception("No read access to S3 bucket in disk " + disk_name, ErrorCodes::PATH_ACCESS_DENIED); } - void checkRemoveAccess(IDisk & disk) { disk.remove("test_acl"); } + void checkRemoveAccess(IDisk & disk) { disk.removeFile("test_acl"); } std::shared_ptr getProxyResolverConfiguration( const String & prefix, const Poco::Util::AbstractConfiguration & proxy_resolver_config) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5d0f79f4679..0b2b9b9ff62 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -931,18 +931,18 @@ void IMergeTreeDataPart::remove() const # pragma GCC diagnostic ignored "-Wunused-variable" #endif for (const auto & [file, _] : checksums.files) - volume->getDisk()->remove(to + "/" + file); + volume->getDisk()->removeFile(to + "/" + file); #if !__clang__ # pragma GCC diagnostic pop #endif for (const auto & file : {"checksums.txt", "columns.txt"}) - volume->getDisk()->remove(to + "/" + file); + volume->getDisk()->removeFile(to + "/" + file); - volume->getDisk()->removeIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME); - volume->getDisk()->removeIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); + volume->getDisk()->removeFileIfExists(to + "/" + DEFAULT_COMPRESSION_CODEC_FILE_NAME); + volume->getDisk()->removeFileIfExists(to + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); - volume->getDisk()->remove(to); + volume->getDisk()->removeDirectory(to); } catch (...) { @@ -995,7 +995,7 @@ void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const Storag /// Backup is not recursive (max_level is 0), so do not copy inner directories localBackup(volume->getDisk(), getFullRelativePath(), destination_path, 0); - volume->getDisk()->removeIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); + volume->getDisk()->removeFileIfExists(destination_path + "/" + DELETE_ON_DESTROY_MARKER_FILE_NAME); } void IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const @@ -1017,7 +1017,7 @@ void IMergeTreeDataPart::makeCloneOnDisk(const DiskPtr & disk, const String & di disk->createDirectories(path_to_clone); volume->getDisk()->copy(getFullRelativePath(), disk, path_to_clone); - volume->getDisk()->removeIfExists(path_to_clone + '/' + DELETE_ON_DESTROY_MARKER_FILE_NAME); + volume->getDisk()->removeFileIfExists(path_to_clone + '/' + DELETE_ON_DESTROY_MARKER_FILE_NAME); } void IMergeTreeDataPart::checkConsistencyBase() const diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index dd293bf2502..1ea74d9fb27 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -63,7 +63,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( { if (checksums.files.count(removed_file)) { - data_part->volume->getDisk()->remove(data_part->getFullRelativePath() + removed_file); + data_part->volume->getDisk()->removeFile(data_part->getFullRelativePath() + removed_file); checksums.files.erase(removed_file); } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 897aea4363a..5617b9b4b0f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1215,8 +1215,8 @@ void MergeTreeData::clearOldWriteAheadLogs() 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()); + LOG_DEBUG(log, "Removing from filesystem the outdated WAL file " + it->name()); + disk_ptr->removeFile(relative_data_path + it->name()); } } } @@ -1316,9 +1316,9 @@ void MergeTreeData::dropIfEmpty() for (const auto & [path, disk] : getRelativeDataPathsWithDisks()) { /// Non recursive, exception is thrown if there are more files. - disk->remove(path + "format_version.txt"); - disk->remove(path + "detached"); - disk->remove(path); + disk->removeFile(path + "format_version.txt"); + disk->removeDirectory(path + "detached"); + disk->removeDirectory(path); } } @@ -3545,7 +3545,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk( LOG_DEBUG(log, "Cloning part {} to {}", fullPath(disk, src_part_path), fullPath(disk, dst_part_path)); localBackup(disk, src_part_path, dst_part_path); - disk->removeIfExists(dst_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + disk->removeFileIfExists(dst_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); auto single_disk_volume = std::make_shared(disk->getName(), disk, 0); auto dst_data_part = createPart(dst_part_name, dst_part_info, single_disk_volume, tmp_dst_part_name); @@ -3636,7 +3636,7 @@ PartitionCommandsResultInfo MergeTreeData::freezePartitionsByMatcher(MatcherFn m else localBackup(part->volume->getDisk(), part->getFullRelativePath(), backup_part_path); - part->volume->getDisk()->removeIfExists(backup_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); + part->volume->getDisk()->removeFileIfExists(backup_part_path + "/" + IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME); part->is_frozen.store(true, std::memory_order_relaxed); result.push_back(PartitionCommandResultInfo{ diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a735b939cd5..0704ad48435 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1065,7 +1065,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merge_entry->progress.store(progress_before + column_sizes->columnWeight(column_name), std::memory_order_relaxed); } - tmp_disk->remove(rows_sources_file_path); + tmp_disk->removeFile(rows_sources_file_path); } for (const auto & part : parts) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index feb7dcb7685..44c4b3c4d10 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -54,7 +54,7 @@ void MergeTreeMutationEntry::removeFile() if (!disk->exists(path_prefix + file_name)) return; - disk->remove(path_prefix + file_name); + disk->removeFile(path_prefix + file_name); file_name.clear(); } } diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index f36275bfd35..e726f0ffd51 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -179,7 +179,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor /// 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) - disk->remove(path); + disk->removeFile(path); else if (name == DEFAULT_WAL_FILE_NAME) rotate(lock); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8d1bee3e889..070e6eb0483 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -625,7 +625,7 @@ void StorageMergeTree::loadMutations() } else if (startsWith(it->name(), "tmp_mutation_")) { - disk->remove(it->path()); + disk->removeFile(it->path()); } } } @@ -1437,7 +1437,7 @@ CheckResults StorageMergeTree::checkData(const ASTPtr & query, const Context & c catch (const Exception & ex) { if (disk->exists(tmp_checksums_path)) - disk->remove(tmp_checksums_path); + disk->removeFile(tmp_checksums_path); results.emplace_back(part->name, false, "Check of part finished with error: '" + ex.message() + "'"); From 0df681a58bf2918e4222123c97ae5f26129f3a73 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Thu, 14 Jan 2021 17:57:44 +0100 Subject: [PATCH 157/264] Moving SYSTEM SHUTDOWN out of parallel --- tests/testflows/rbac/tests/privileges/feature.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/testflows/rbac/tests/privileges/feature.py b/tests/testflows/rbac/tests/privileges/feature.py index ee141ef12dc..e7a0cf2d368 100755 --- a/tests/testflows/rbac/tests/privileges/feature.py +++ b/tests/testflows/rbac/tests/privileges/feature.py @@ -92,9 +92,10 @@ def feature(self): run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.sends", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.sync_replica", "feature"), flags=TE), {}) run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.fetches", "feature"), flags=TE), {}) - run_scenario(pool, tasks, Feature(test=load("rbac.tests.privileges.system.shutdown", "feature"), flags=TE), {}) finally: join(tasks) finally: pool.close() + + Feature(test=load("rbac.tests.privileges.system.shutdown", "feature"), flags=TE) From ee13dd4fefe59ca380c5e6cf5487b8925c9ee553 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Jan 2021 19:58:44 +0300 Subject: [PATCH 158/264] Fix error --- src/Disks/DiskLocal.cpp | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index 2b778b234fa..2193d2a2a73 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -241,20 +241,23 @@ DiskLocal::writeFile(const String & path, size_t buf_size, WriteMode mode) void DiskLocal::removeFile(const String & path) { - if (0 != unlink(path.c_str())) - throwFromErrnoWithPath("Cannot unlink file " + path, path, ErrorCodes::CANNOT_UNLINK); + auto fs_path = disk_path + path; + if (0 != unlink(fs_path.c_str())) + throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeFileIfExists(const String & path) { - if (0 != unlink(path.c_str()) && errno != ENOENT) - throwFromErrnoWithPath("Cannot unlink file " + path, path, ErrorCodes::CANNOT_UNLINK); + auto fs_path = disk_path + path; + if (0 != unlink(fs_path.c_str()) && errno != ENOENT) + throwFromErrnoWithPath("Cannot unlink file " + fs_path, fs_path, ErrorCodes::CANNOT_UNLINK); } void DiskLocal::removeDirectory(const String & path) { - if (0 != rmdir(path.c_str())) - throwFromErrnoWithPath("Cannot rmdir " + path, path, ErrorCodes::CANNOT_RMDIR); + auto fs_path = disk_path + path; + if (0 != rmdir(fs_path.c_str())) + throwFromErrnoWithPath("Cannot rmdir " + fs_path, fs_path, ErrorCodes::CANNOT_RMDIR); } void DiskLocal::removeRecursive(const String & path) From d120801df3f0d6abf608c8cad5749561524c1b3f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jan 2021 20:16:31 +0300 Subject: [PATCH 159/264] Fix style --- src/Functions/tupleHammingDistance.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Functions/tupleHammingDistance.cpp b/src/Functions/tupleHammingDistance.cpp index 9785c118e34..0fd21e0ea91 100644 --- a/src/Functions/tupleHammingDistance.cpp +++ b/src/Functions/tupleHammingDistance.cpp @@ -11,7 +11,6 @@ namespace DB { namespace ErrorCodes { - extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } From 5143378bb416fa3dd2e863fe06280df6e81902b3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 14 Jan 2021 13:54:17 -0400 Subject: [PATCH 160/264] Update partition.md --- docs/ru/sql-reference/statements/alter/partition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index 5e6f323200f..9fbbb02c8a8 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -288,7 +288,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать: - Имя партиции. Посмотреть имя партиции можно в столбце `partition` системной таблицы [system.parts](../../../operations/system-tables/parts.md#system_tables-parts). Например, `ALTER TABLE visits DETACH PARTITION 201901`. -- Произвольное выражение из столбцов исходной таблицы. Также поддерживаются константы и константные выражения. Выражение необходимо обернуть в функцию `tuple()`. Например, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. +- Кортеж из выражений, совпадающий (в типах) с кортежем партиционирования. В случае ключа партиционирования из одного элемента, следует обернуть его в функцию `tuple(...)`. Например, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Строковый идентификатор партиции. Идентификатор партиции используется для именования кусков партиции на файловой системе и в ZooKeeper. В запросах `ALTER` идентификатор партиции нужно указывать в секции `PARTITION ID`, в одинарных кавычках. Например, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - Для запросов [ATTACH PART](#alter_attach-partition) и [DROP DETACHED PART](#alter_drop-detached): чтобы задать имя куска партиции, используйте строковой литерал со значением из столбца `name` системной таблицы [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts). Например, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. From 2c41360270f64a832d762b72f928e69378a993e3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 14 Jan 2021 13:57:02 -0400 Subject: [PATCH 161/264] Update partition.md --- docs/en/sql-reference/statements/alter/partition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 2a65bc83824..922952f0e8b 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -286,7 +286,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; You can specify the partition expression in `ALTER ... PARTITION` queries in different ways: - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. -- As the expression from the table column. Constants and constant expressions are supported. The expression must be wrapped in a `tuple()` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. +- As the expression tuple that matches (in types) a table partitioning keys tuple. In the case of a single element partitioning key, you should wrap it in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. From b2aa4d97097b6104580dc32cd70b369cdde50775 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 14 Jan 2021 14:22:19 -0400 Subject: [PATCH 162/264] Update partition.md --- docs/ru/sql-reference/statements/alter/partition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/alter/partition.md b/docs/ru/sql-reference/statements/alter/partition.md index 9fbbb02c8a8..8776c70c89e 100644 --- a/docs/ru/sql-reference/statements/alter/partition.md +++ b/docs/ru/sql-reference/statements/alter/partition.md @@ -288,7 +288,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; Чтобы задать нужную партицию в запросах `ALTER ... PARTITION`, можно использовать: - Имя партиции. Посмотреть имя партиции можно в столбце `partition` системной таблицы [system.parts](../../../operations/system-tables/parts.md#system_tables-parts). Например, `ALTER TABLE visits DETACH PARTITION 201901`. -- Кортеж из выражений, совпадающий (в типах) с кортежем партиционирования. В случае ключа партиционирования из одного элемента, следует обернуть его в функцию `tuple(...)`. Например, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. +- Кортеж из выражений или констант, совпадающий (в типах) с кортежем партиционирования. В случае ключа партиционирования из одного элемента, выражение следует обернуть в функцию `tuple(...)`. Например, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Строковый идентификатор партиции. Идентификатор партиции используется для именования кусков партиции на файловой системе и в ZooKeeper. В запросах `ALTER` идентификатор партиции нужно указывать в секции `PARTITION ID`, в одинарных кавычках. Например, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - Для запросов [ATTACH PART](#alter_attach-partition) и [DROP DETACHED PART](#alter_drop-detached): чтобы задать имя куска партиции, используйте строковой литерал со значением из столбца `name` системной таблицы [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts). Например, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. From 55986d038396bad05470d45711e7f939b5b5554c Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 14 Jan 2021 14:24:49 -0400 Subject: [PATCH 163/264] Update partition.md --- docs/en/sql-reference/statements/alter/partition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 922952f0e8b..bd8916711c3 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -286,7 +286,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; You can specify the partition expression in `ALTER ... PARTITION` queries in different ways: - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. -- As the expression tuple that matches (in types) a table partitioning keys tuple. In the case of a single element partitioning key, you should wrap it in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. +- As the tuple expressions or constants that matches (in types) a table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. From 7c8ec40555f951121a01ac909b88d42357c77d48 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 14 Jan 2021 14:31:26 -0400 Subject: [PATCH 164/264] Update partition.md --- docs/en/sql-reference/statements/alter/partition.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index bd8916711c3..42396223b86 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -286,7 +286,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; You can specify the partition expression in `ALTER ... PARTITION` queries in different ways: - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. -- As the tuple expressions or constants that matches (in types) a table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. +- As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. - In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. From 27600064159ac63e05800ef37afa41b317771352 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 14 Jan 2021 22:29:34 +0300 Subject: [PATCH 165/264] Update gtest_peekable_read_buffer.cpp --- src/IO/tests/gtest_peekable_read_buffer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/tests/gtest_peekable_read_buffer.cpp b/src/IO/tests/gtest_peekable_read_buffer.cpp index 07d6404bbef..8c491338bd3 100644 --- a/src/IO/tests/gtest_peekable_read_buffer.cpp +++ b/src/IO/tests/gtest_peekable_read_buffer.cpp @@ -9,7 +9,6 @@ namespace DB::ErrorCodes { extern const int LOGICAL_ERROR; - extern const int MEMORY_LIMIT_EXCEEDED; } static void readAndAssert(DB::ReadBuffer & buf, const char * str) From ffca270512f511ecd23b84915ce9a7fc833c7fc6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Jan 2021 22:34:13 +0300 Subject: [PATCH 166/264] Fix Arcadia --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index a4507555373..116d0beb80b 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -183,3 +183,4 @@ 01636_nullable_fuzz2 01639_distributed_sync_insert_zero_rows 01644_distributed_async_insert_fsync_smoke +01552_impl_aggfunc_cloneresize From ecae6c1c609fbabaf940ec41c40c7f3b2401b412 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jan 2021 22:36:30 +0300 Subject: [PATCH 167/264] Avoid reading the distributed batch just to read the block header Before this patch batched mode of the DirectoryMonitor is 2x slower then non-batched, after it should be more or less the same as non-batched. --- src/Storages/Distributed/DirectoryMonitor.cpp | 42 +++++++------------ .../DistributedBlockOutputStream.cpp | 1 + 2 files changed, 15 insertions(+), 28 deletions(-) diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index 62fa2b3f23a..2a29f2559b6 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -88,6 +88,8 @@ namespace /// .bin file cannot have zero rows/bytes. size_t rows = 0; size_t bytes = 0; + + std::string header; }; DistributedHeader readDistributedHeader(ReadBuffer & in, Poco::Logger * log) @@ -128,6 +130,7 @@ namespace { readVarUInt(header.rows, header_buf); readVarUInt(header.bytes, header_buf); + readStringBinary(header.header, header_buf); } /// Add handling new data here, for example: @@ -437,20 +440,20 @@ struct StorageDistributedDirectoryMonitor::BatchHeader Settings settings; String query; ClientInfo client_info; - Block sample_block; + String sample_block_structure; - BatchHeader(Settings settings_, String query_, ClientInfo client_info_, Block sample_block_) + BatchHeader(Settings settings_, String query_, ClientInfo client_info_, String sample_block_structure_) : settings(std::move(settings_)) , query(std::move(query_)) , client_info(std::move(client_info_)) - , sample_block(std::move(sample_block_)) + , sample_block_structure(std::move(sample_block_structure_)) { } bool operator==(const BatchHeader & other) const { - return settings == other.settings && query == other.query && client_info.query_kind == other.client_info.query_kind - && blocksHaveEqualStructure(sample_block, other.sample_block); + return std::tie(settings, query, client_info.query_kind, sample_block_structure) == + std::tie(other.settings, other.query, other.client_info.query_kind, other.sample_block_structure); } struct Hash @@ -459,14 +462,7 @@ struct StorageDistributedDirectoryMonitor::BatchHeader { SipHash hash_state; hash_state.update(batch_header.query.data(), batch_header.query.size()); - - size_t num_columns = batch_header.sample_block.columns(); - for (size_t i = 0; i < num_columns; ++i) - { - const String & type_name = batch_header.sample_block.getByPosition(i).type->getName(); - hash_state.update(type_name.data(), type_name.size()); - } - + hash_state.update(batch_header.sample_block_structure.data(), batch_header.sample_block_structure.size()); return hash_state.get64(); } }; @@ -730,7 +726,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map size_t total_rows = 0; size_t total_bytes = 0; - Block sample_block; + std::string sample_block_structure; DistributedHeader header; try { @@ -742,17 +738,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map { total_rows += header.rows; total_bytes += header.bytes; - - CompressedReadBuffer decompressing_in(in); - NativeBlockInputStream block_in(decompressing_in, DBMS_TCP_PROTOCOL_VERSION); - block_in.readPrefix(); - - /// We still need to read one block for the header. - while (Block block = block_in.read()) - { - sample_block = block.cloneEmpty(); - break; - } + sample_block_structure = header.header; } else { @@ -765,8 +751,8 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map total_rows += block.rows(); total_bytes += block.bytes(); - if (!sample_block) - sample_block = block.cloneEmpty(); + if (sample_block_structure.empty()) + sample_block_structure = block.cloneEmpty().dumpStructure(); } block_in.readSuffix(); } @@ -782,7 +768,7 @@ void StorageDistributedDirectoryMonitor::processFilesWithBatching(const std::map throw; } - BatchHeader batch_header(std::move(header.insert_settings), std::move(header.insert_query), std::move(header.client_info), std::move(sample_block)); + BatchHeader batch_header(std::move(header.insert_settings), std::move(header.insert_query), std::move(header.client_info), std::move(sample_block_structure)); Batch & batch = header_to_batch.try_emplace(batch_header, *this, files).first->second; batch.file_indices.push_back(file_idx); diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 977d8d3e81c..577808d72fd 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -655,6 +655,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: context.getClientInfo().write(header_buf, DBMS_TCP_PROTOCOL_VERSION); writeVarUInt(block.rows(), header_buf); writeVarUInt(block.bytes(), header_buf); + writeStringBinary(block.cloneEmpty().dumpStructure(), header_buf); /// Add new fields here, for example: /// writeVarUInt(my_new_data, header_buf); From 522ccddbeca32692b51c5ca03bfb878ced0493c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 14 Jan 2021 23:05:00 +0300 Subject: [PATCH 168/264] Better code --- src/Columns/ColumnAggregateFunction.cpp | 35 +++++++++++-------------- 1 file changed, 16 insertions(+), 19 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 0cffdde7c1c..d0a5e120a07 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -669,36 +669,33 @@ ColumnAggregateFunction::ColumnAggregateFunction(const ColumnAggregateFunction & func(src_.func), src(src_.getPtr()), data(src_.data.begin(), src_.data.end()) { } -//override method cloneResized + MutableColumnPtr ColumnAggregateFunction::cloneResized(size_t size) const { - //create a new col to return - MutableColumnPtr cloned_col = cloneEmpty(); - auto * res = typeid_cast(cloned_col.get()); if (size == 0) - return cloned_col; + return cloneEmpty(); size_t from_size = data.size(); - auto & res_data = res->data; - //copy data to cloned column + if (size <= from_size) { - res_data.resize(size); - res->insertRangeFrom(*this, 0, size); + auto res = createView(); + auto & res_data = res->data; + res_data.assign(data.begin(), data.begin() + size); + return res; } else { - res_data.resize(from_size); - if (from_size > 0) - { - res->insertRangeFrom(*this, 0, from_size); - } - res->ensureOwnership(); - for (size_t i = 0; i < size - from_size; ++i) - { + /// Create a new column to return. + MutableColumnPtr cloned_col = cloneEmpty(); + auto * res = typeid_cast(cloned_col.get()); + + res->insertRangeFrom(*this, 0, from_size); + for (size_t i = from_size; i < size; ++i) res->insertDefault(); - } + + return cloned_col; } - return cloned_col; } + } From 489d9d186ab9f0844cc55421739070f46796732c Mon Sep 17 00:00:00 2001 From: MyroTk Date: Thu, 14 Jan 2021 21:25:37 +0100 Subject: [PATCH 169/264] Looping RBAC several times --- tests/testflows/regression.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index ac17ec4e535..20ba26318ba 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -16,7 +16,8 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): # Feature(test=load("example.regression", "regression"))(**args) # Feature(test=load("ldap.regression", "regression"))(**args) - Feature(test=load("rbac.regression", "regression"))(**args) + for i in range(10): + Feature(test=load("rbac.regression", "regression"))(**args) # Feature(test=load("aes_encryption.regression", "regression"))(**args) if main(): From 58f10799dd359eb51117047bf32884f29528c15c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 00:14:10 +0300 Subject: [PATCH 170/264] Remove changes --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 506d7a0bcaa..ef64ec28e79 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -165,11 +165,6 @@ void MergeTreeDataPartWriterWide::shiftCurrentMark(const Granules & granules_wri void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Permutation * permutation) { - /// TODO: remove me - const auto storage_settings = storage.getSettings(); - if (settings.blocks_are_granules_size && block.rows() > storage_settings->index_granularity) - throw Exception(ErrorCodes::LOGICAL_ERROR, "When blocks are granules size block size ({} rows) cannot be bigger than fixed granularity ({} rows)", - block.rows(), storage_settings->index_granularity); /// Fill index granularity for this block /// if it's unknown (in case of insert data or horizontal merge, /// but not in case of vertical part of vertical merge) From 8da81a47916d5c5e45e6118fac6f8195fbcffbd6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 00:15:48 +0300 Subject: [PATCH 171/264] No errors --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 3d42e4295fe..21dede4db10 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -67,4 +67,4 @@ export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: -tar -chvf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log +tar -chvf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log ||: From b6170c1326cfb47f25464c16eda2428ebb23e774 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 00:18:50 +0300 Subject: [PATCH 172/264] No logs --- docker/test/stateless/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 21dede4db10..515b7e0a816 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -67,4 +67,4 @@ export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: -tar -chvf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log ||: +tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log ||: From 9db1299f4135831c6cbc5cf9dddd88596bb9d91b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 00:25:19 +0300 Subject: [PATCH 173/264] Improve code --- src/Functions/decodeXMLComponent.cpp | 307 ++++++++++----------------- 1 file changed, 113 insertions(+), 194 deletions(-) diff --git a/src/Functions/decodeXMLComponent.cpp b/src/Functions/decodeXMLComponent.cpp index ffc10d3a7bb..9b9cf5a081d 100644 --- a/src/Functions/decodeXMLComponent.cpp +++ b/src/Functions/decodeXMLComponent.cpp @@ -5,6 +5,7 @@ #include #include + namespace DB { namespace ErrorCodes @@ -28,7 +29,11 @@ namespace ColumnString::Chars & res_data, ColumnString::Offsets & res_offsets) { + /// The size of result is always not more than the size of source. + /// Because entities decodes to the shorter byte sequence. + /// Example: &#xx... &#xx... will decode to UTF-8 byte sequence not longer than 4 bytes. res_data.resize(data.size()); + size_t size = offsets.size(); res_offsets.resize(size); @@ -56,255 +61,169 @@ namespace private: static const int max_legal_unicode_value = 0x10FFFF; - static const int max_legal_unicode_bits = 7; + static const int max_decimal_length_of_unicode_point = 7; /// 1114111 + static size_t execute(const char * src, size_t src_size, char * dst) { - const char * src_prev_pos = src; - const char * src_curr_pos = src; - const char * src_next_pos = src; + const char * src_pos = src; const char * src_end = src + src_size; char * dst_pos = dst; while (true) { - src_curr_pos = find_first_symbols<'&'>(src_curr_pos, src_end); + const char * entity_pos = find_first_symbols<'&'>(src_pos, src_end); - if (src_curr_pos == src_end) - { + if (entity_pos + strlen("lt;") >= src_end) break; - } - else if (*src_curr_pos == '&') + + /// Copy text between entities. + size_t bytes_to_copy = entity_pos - src_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_pos, bytes_to_copy); + dst_pos += bytes_to_copy; + src_pos = entity_pos; + + ++entity_pos; + + const char * entity_end = find_first_symbols<';'>(entity_pos, src_end); + if (entity_end == src_end) + break; + + bool parsed = false; + + /// &#NNNN; or &#xNNNN; + uint32_t code_point = 0; + if (isValidNumericEntity(entity_pos, entity_end, code_point)) { - src_next_pos = find_first_symbols<';'>(src_curr_pos, src_end); - if (src_next_pos == src_end) + codePointToUTF8(code_point, dst_pos); + parsed = true; + } + else if (entity_end - entity_pos == 2) + { + if (memcmp(entity_pos, "lt", 2) == 0) { - src_curr_pos = src_end; - break; + *dst_pos = '<'; + ++dst_pos; + parsed = true; } - else if (isValidNumeric(src_curr_pos, src_next_pos)) + else if (memcmp(entity_pos, "gt", 2) == 0) { - int numeric_entity; - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - if (*(src_curr_pos + 2) == 'x' || *(src_curr_pos + 2) == 'X') - { - numeric_entity = hexOrDecStrToInt(src_curr_pos + 3, src_next_pos, 0x10); - } - else - { - numeric_entity = hexOrDecStrToInt(src_curr_pos + 2, src_next_pos, 10); - } - if (numeric_entity > max_legal_unicode_value) - { - bytes_to_copy = src_next_pos - src_curr_pos + 1; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_curr_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - } - else - { - decodeNumericPart(numeric_entity, dst_pos); - } - src_prev_pos = src_next_pos + 1; - src_curr_pos = src_next_pos + 1; + *dst_pos = '>'; + ++dst_pos; + parsed = true; } - else if (src_next_pos - src_curr_pos == 3) + } + else if (entity_end - entity_pos == 3) + { + if (memcmp(entity_pos, "amp", 3) == 0) { - if (strncmp(src_curr_pos, "<", 3) == 0) - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - *dst_pos = '<'; - ++dst_pos; - src_prev_pos = src_curr_pos + 4; - } - else if (strncmp(src_curr_pos, ">", 3) == 0) - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - *dst_pos = '>'; - ++dst_pos; - src_prev_pos = src_curr_pos + 4; - } - else - { - ++src_curr_pos; - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - src_prev_pos = src_curr_pos; - continue; - } - src_curr_pos += 4; + *dst_pos = '&'; + ++dst_pos; + parsed = true; } - else if (src_next_pos - src_curr_pos == 4) + } + else if (entity_end - entity_pos == 4) + { + if (memcmp(entity_pos, "quot", 4) == 0) { - if (strncmp(src_curr_pos, "&", 4) == 0) - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - *dst_pos = '&'; - ++dst_pos; - src_prev_pos = src_curr_pos + 5; - } - else - { - ++src_curr_pos; - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - src_prev_pos = src_curr_pos; - continue; - } - src_curr_pos += 5; + *dst_pos = '"'; + ++dst_pos; + parsed = true; } - else if (src_next_pos - src_curr_pos == 5) + else if (memcmp(entity_pos, "apos", 4) == 0) { - if (strncmp(src_curr_pos, """, 5) == 0) - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - *dst_pos = '"'; - ++dst_pos; - src_prev_pos = src_curr_pos + 6; - } - else if (strncmp(src_curr_pos, "&apos", 5) == 0) - { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - *dst_pos = '\''; - ++dst_pos; - src_prev_pos = src_curr_pos + 6; - } - else - { - ++src_curr_pos; - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - src_prev_pos = src_curr_pos; - continue; - } - src_curr_pos += 6; - } - else - { - ++src_curr_pos; - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); - dst_pos += bytes_to_copy; - src_prev_pos = src_curr_pos; + *dst_pos = '\''; + ++dst_pos; + parsed = true; } } + + if (parsed) + { + /// Skip the parsed entity. + src_pos = entity_end + 1; + } + else + { + /// Copy one byte as is and skip it. + *dst_pos = *src_pos; + ++dst_pos; + ++src_pos; + } } - if (src_prev_pos < src_curr_pos) + /// Copy the rest of the string. + if (src_pos < src_end) { - size_t bytes_to_copy = src_curr_pos - src_prev_pos; - memcpySmallAllowReadWriteOverflow15(dst_pos, src_prev_pos, bytes_to_copy); + size_t bytes_to_copy = src_end - src_pos; + memcpySmallAllowReadWriteOverflow15(dst_pos, src_pos, bytes_to_copy); dst_pos += bytes_to_copy; } return dst_pos - dst; } - static void decodeNumericPart(int numeric_entity, char *& dst_pos) + static void codePointToUTF8(uint32_t code_point, char *& dst_pos) { - const auto num_bits = numBitsCount(numeric_entity); - if (num_bits <= 7) + if (code_point < (1 << 7)) { - *(dst_pos++) = '\0' + (numeric_entity & 0x7F); + dst_pos[0] = (code_point & 0x7F); + ++dst_pos; } - else if (num_bits <= 11) + else if (code_point < (1 << 11)) { - *(dst_pos++) = '\0' + ((numeric_entity >> 6) & 0x1F) + 0xC0; - *(dst_pos++) = '\0' + (numeric_entity & 0x3F) + 0x80; + dst_pos[0] = ((code_point >> 6) & 0x1F) + 0xC0; + dst_pos[1] = (code_point & 0x3F) + 0x80; + dst_pos += 2; } - else if (num_bits <= 16) + else if (code_point < (1 << 16)) { - *(dst_pos++) = '\0' + ((numeric_entity >> 12) & 0x0F) + 0xE0; - *(dst_pos++) = '\0' + ((numeric_entity >> 6) & 0x3F) + 0x80; - *(dst_pos++) = '\0' + (numeric_entity & 0x3F) + 0x80; + dst_pos[0] = ((code_point >> 12) & 0x0F) + 0xE0; + dst_pos[1] = ((code_point >> 6) & 0x3F) + 0x80; + dst_pos[2] = (code_point & 0x3F) + 0x80; + dst_pos += 3; } else { - *(dst_pos++) = '\0' + ((numeric_entity >> 18) & 0x07) + 0xF0; - *(dst_pos++) = '\0' + ((numeric_entity >> 12) & 0x3F) + 0x80; - *(dst_pos++) = '\0' + ((numeric_entity >> 6) & 0x3F) + 0x80; - *(dst_pos++) = '\0' + (numeric_entity & 0x3F) + 0x80; + dst_pos[0] = ((code_point >> 18) & 0x07) + 0xF0; + dst_pos[1] = ((code_point >> 12) & 0x3F) + 0x80; + dst_pos[2] = ((code_point >> 6) & 0x3F) + 0x80; + dst_pos[3] = (code_point & 0x3F) + 0x80; + dst_pos += 4; } } - static int hexOrDecStrToInt(const char * src, const char * end, int base) + static bool isValidNumericEntity(const char * src, const char * end, uint32_t & code_point) { - int numeric_ans = 0; - int pos = 0; - if (base == 0x10) - { - while (src + pos != end) - { - numeric_ans = numeric_ans * 0x10 + static_cast(unhex(*(src + pos))); - ++pos; - } - } - else - { - while (src + pos != end) - { - numeric_ans = numeric_ans * base + (*(src + pos) - '0'); - ++pos; - } - } - return numeric_ans; - } - static int numBitsCount(int integer) - { - size_t num_bits = 0; - while (integer > 0) - { - ++num_bits; - integer >>= 1; - } - return num_bits; - } - static bool isValidNumeric(const char * src, const char * end) - { - int pos; - if (*src != '&' || *(src + 1) != '#' || (end - (src + 2) > max_legal_unicode_bits)) - { + if (src + strlen("#") >= end) return false; - } - if (*(src + 2) == 'x' || *(src + 2) == 'X') + + if (src[0] != '#' || (end - src > 1 + max_decimal_length_of_unicode_point)) + return false; + + if (src + 2 < end && (src[1] == 'x' || src[1] == 'X')) { - pos = 3; - while (src + pos != end) + src += 2; + for (; src < end; ++src) { - if (!isHexDigit(*(src + pos))) - { + if (!isHexDigit(*src)) return false; - } - ++pos; + code_point *= 16; + code_point += unhex(*src); } - return true; } else { - pos = 2; - while (src + pos != end) + src += 1; + for (; src < end; ++src) { - if (!isNumericASCII(*(src + pos))) - { + if (!isNumericASCII(*src)) return false; - } - ++pos; + code_point *= 10; + code_point += *src - '0'; } - return true; } + + return code_point <= max_legal_unicode_value; } }; From 8fe85b5fcbcc8a7427a7cc438845d3697ef00fa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 00:33:54 +0300 Subject: [PATCH 174/264] Add stateful test --- tests/queries/1_stateful/00160_decode_xml_component.reference | 1 + tests/queries/1_stateful/00160_decode_xml_component.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/1_stateful/00160_decode_xml_component.reference create mode 100644 tests/queries/1_stateful/00160_decode_xml_component.sql diff --git a/tests/queries/1_stateful/00160_decode_xml_component.reference b/tests/queries/1_stateful/00160_decode_xml_component.reference new file mode 100644 index 00000000000..96234a2262e --- /dev/null +++ b/tests/queries/1_stateful/00160_decode_xml_component.reference @@ -0,0 +1 @@ +10601114492838968014 diff --git a/tests/queries/1_stateful/00160_decode_xml_component.sql b/tests/queries/1_stateful/00160_decode_xml_component.sql new file mode 100644 index 00000000000..0194eb330a9 --- /dev/null +++ b/tests/queries/1_stateful/00160_decode_xml_component.sql @@ -0,0 +1 @@ +SELECT sum(DISTINCT sipHash64(decodeXMLComponent(Title) AS decoded)) FROM test.hits WHERE Title != decoded; From f6f7ef65a2e07a3f0ff507f8bdb3eb18abb7bec3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 00:34:53 +0300 Subject: [PATCH 175/264] Add perf test --- tests/performance/url_hits.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/performance/url_hits.xml b/tests/performance/url_hits.xml index f0ad6a786e0..a699ef6ba97 100644 --- a/tests/performance/url_hits.xml +++ b/tests/performance/url_hits.xml @@ -32,6 +32,7 @@ extractURLParameters extractURLParameterNames decodeURLComponent + decodeXMLComponent cutWWW cutQueryString cutQueryStringAndFragment From 0d23f3f16abdf489be10359c6059ce6ee87584ec Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 14 Jan 2021 23:37:27 +0100 Subject: [PATCH 176/264] Docker: fix uid/gid of the clickhouse user Explicitly set uid / gid of clickhouse user & group to the fixed values 101. It is especially important for rootless containers: in that case entrypoint can't do chown and owners of mounted volumes should be configured externally. We do that in advance at the begining of Dockerfile before any packages will be installed to prevent picking those uid / gid by some unrelated software. The same uid / gid (101) is used both for alpine and ubuntu. Number 101 is used by default in openshift, and was used by all clickhouse-server docker images before 20.10. In 20.11 it was changed (by accident) to 999. --- docker/server/Dockerfile | 5 ++++- docker/server/Dockerfile.alpine | 6 +++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 890aa35fe92..0759d92b325 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -4,7 +4,10 @@ ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" ARG version=21.1.0 ARG gosu_ver=1.10 -RUN apt-get update \ +# user/group precreated explicitly with fixed uid/gid on purpose (see commit) +RUN groupadd -r clickhouse --gid=101 \ + && useradd -r -g clickhouse --uid=101 --home-dir=/var/lib/clickhouse --shell=/bin/bash clickhouse \ + && apt-get update \ && apt-get install --yes --no-install-recommends \ apt-transport-https \ ca-certificates \ diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 2de834e7b9c..0d28e59d602 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -8,9 +8,9 @@ ENV LANG=en_US.UTF-8 \ COPY alpine-root/ / -# from https://github.com/ClickHouse/ClickHouse/blob/master/debian/clickhouse-server.postinst -RUN addgroup clickhouse \ - && adduser -S -H -h /nonexistent -s /bin/false -G clickhouse -g "ClickHouse server" clickhouse \ +# user/group precreated explicitly with fixed uid/gid on purpose (see commit) +RUN addgroup -S -g 101 clickhouse \ + && adduser -S -h /var/lib/clickhouse -s /bin/bash -G clickhouse -g "ClickHouse server" -u 101 clickhouse \ && chown clickhouse:clickhouse /var/lib/clickhouse \ && chmod 700 /var/lib/clickhouse \ && chown root:clickhouse /var/log/clickhouse-server \ From 82d3c4c9fedf21be334e1e52151818ac784fdbfc Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Fri, 15 Jan 2021 12:00:56 +0800 Subject: [PATCH 177/264] update chinese docs of insert-into and select-all to latest --- .../sql-reference/statements/insert-into.md | 51 +++++++++++++++++-- .../zh/sql-reference/statements/select/all.md | 17 +++++++ 2 files changed, 65 insertions(+), 3 deletions(-) create mode 100644 docs/zh/sql-reference/statements/select/all.md diff --git a/docs/zh/sql-reference/statements/insert-into.md b/docs/zh/sql-reference/statements/insert-into.md index b71e84ef0f6..92b24c74493 100644 --- a/docs/zh/sql-reference/statements/insert-into.md +++ b/docs/zh/sql-reference/statements/insert-into.md @@ -1,6 +1,6 @@ -## INSERT {#insert} +## INSERT INTO 语句 {#insert} -INSERT查询主要用于向系统中添加数据. +INSERT INTO 语句主要用于向系统中添加数据. 查询的基本格式: @@ -8,7 +8,52 @@ INSERT查询主要用于向系统中添加数据. INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` -您可以在查询中指定插入的列的列表,如:`[(c1, c2, c3)]`。对于存在于表结构中但不存在于插入列表中的列,它们将会按照如下方式填充数据: +您可以在查询中指定要插入的列的列表,如:`[(c1, c2, c3)]`。您还可以使用列[匹配器](../../sql-reference/statements/select/index.md#asterisk)的表达式,例如`*`和/或[修饰符](../../sql-reference/statements/select/index.md#select-modifiers),例如 [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#apply-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier)。 + +例如,考虑该表: + +``` sql +SHOW CREATE insert_select_testtable; +``` + +```text +CREATE TABLE insert_select_testtable +( + `a` Int8, + `b` String, + `c` Int8 +) +ENGINE = MergeTree() +ORDER BY a +SETTINGS index_granularity = 8192 +``` + +``` sql +INSERT INTO insert_select_testtable (*) VALUES (1, 'a', 1) ; +``` + +如果要在除了'b'列以外的所有列中插入数据,您需要传递和括号中选择的列数一样多的值: + +``` sql +INSERT INTO insert_select_testtable (* EXCEPT(b)) Values (2, 2); +``` + +``` sql +SELECT * FROM insert_select_testtable; +``` + +``` +┌─a─┬─b─┬─c─┐ +│ 2 │ │ 2 │ +└───┴───┴───┘ +┌─a─┬─b─┬─c─┐ +│ 1 │ a │ 1 │ +└───┴───┴───┘ +``` + +在这个示例中,我们看到插入的第二行的`a`和`c`列的值由传递的值填充,而`b`列由默认值填充。 + +对于存在于表结构中但不存在于插入列表中的列,它们将会按照如下方式填充数据: - 如果存在`DEFAULT`表达式,根据`DEFAULT`表达式计算被填充的值。 - 如果没有定义`DEFAULT`表达式,则填充零或空字符串。 diff --git a/docs/zh/sql-reference/statements/select/all.md b/docs/zh/sql-reference/statements/select/all.md new file mode 100644 index 00000000000..b23a3973502 --- /dev/null +++ b/docs/zh/sql-reference/statements/select/all.md @@ -0,0 +1,17 @@ +# ALL 子句 {#select-all} + +`SELECT ALL` 和 `SELECT` 不带 `DISTINCT` 是一样的。 + +- 如果指定了 `ALL` ,则忽略它。 +- 如果同时指定了 `ALL` 和 `DISTINCT` ,则会抛出异常。 + +`ALL` 也可以在聚合函数中指定,具有相同的效果(空操作)。例如: + +```sql +SELECT sum(ALL number) FROM numbers(10); +``` +等于 + +```sql +SELECT sum(number) FROM numbers(10); +``` From dfc1e8ef1d62c7166aaf2a15ce2bca054c5061ad Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 10:53:37 +0300 Subject: [PATCH 178/264] Also upload query log --- docker/test/stateless/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 515b7e0a816..d9a03f84726 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -68,3 +68,4 @@ export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log ||: +tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_log ||: From 0662d6bd7dab61ab48046ef67d0ea12355cdb366 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 12:04:23 +0300 Subject: [PATCH 179/264] Fix compression codec read for empty files --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 20 ++++++-- .../test_compression_codec_read/__init__.py | 1 + .../test_compression_codec_read/test.py | 46 +++++++++++++++++++ 3 files changed, 63 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_compression_codec_read/__init__.py create mode 100644 tests/integration/test_compression_codec_read/test.py diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5d0f79f4679..73fd5b7cab5 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -549,14 +549,26 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const auto column_size = getColumnSize(part_column.name, *part_column.type); if (column_size.data_compressed != 0 && !storage_columns.hasCompressionCodec(part_column.name)) { - String path_to_data_file = getFullRelativePath() + getFileNameForColumn(part_column) + ".bin"; - if (!volume->getDisk()->exists(path_to_data_file)) + String path_to_data_file; + part_column.type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path, const IDataType & /* substream_type */) { - LOG_WARNING(storage.log, "Part's {} column {} has non zero data compressed size, but data file {} doesn't exists", name, backQuoteIfNeed(part_column.name), path_to_data_file); + if (path_to_data_file.empty()) + { + String candidate_path = getFullRelativePath() + IDataType::getFileNameForStream(part_column.name, substream_path) + ".bin"; + + /// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns. + if (volume->getDisk()->exists(candidate_path) && volume->getDisk()->getFileSize(candidate_path) != 0) + path_to_data_file = candidate_path; + } + }); + + if (path_to_data_file.empty()) + { + LOG_WARNING(storage.log, "Part's {} column {} has non zero data compressed size, but all data files don't exist or empty", name, backQuoteIfNeed(part_column.name)); continue; } - result = getCompressionCodecForFile(volume->getDisk(), getFullRelativePath() + getFileNameForColumn(part_column) + ".bin"); + result = getCompressionCodecForFile(volume->getDisk(), path_to_data_file); break; } } diff --git a/tests/integration/test_compression_codec_read/__init__.py b/tests/integration/test_compression_codec_read/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_compression_codec_read/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_compression_codec_read/test.py b/tests/integration/test_compression_codec_read/test.py new file mode 100644 index 00000000000..0eb1f5aa867 --- /dev/null +++ b/tests/integration/test_compression_codec_read/test.py @@ -0,0 +1,46 @@ +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', image='yandex/clickhouse-server', tag='20.8.11.17', with_installed_binary=True, stay_alive=True) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def test_default_codec_read(start_cluster): + node1.query(""" + CREATE TABLE test_18340 + ( + `lns` LowCardinality(Nullable(String)), + `ns` Nullable(String), + `s` String, + `ni64` Nullable(Int64), + `ui64` UInt64, + `alns` Array(LowCardinality(Nullable(String))), + `ans` Array(Nullable(String)), + `dt` DateTime, + `i32` Int32 + ) + ENGINE = MergeTree() + PARTITION BY i32 + ORDER BY (s, farmHash64(s)) + SAMPLE BY farmHash64(s) + """) + + node1.query("insert into test_18340 values ('test', 'test', 'test', 0, 0, ['a'], ['a'], now(), 0)") + + + assert node1.query("SELECT COUNT() FROM test_18340") == "1\n" + + node1.restart_with_latest_version() + + assert node1.query("SELECT COUNT() FROM test_18340") == "1\n" From e106df2ad08a68f59c96174180910a5846945633 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 12:10:03 +0300 Subject: [PATCH 180/264] Fix comment --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 73fd5b7cab5..3bd06c7f758 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -556,7 +556,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { String candidate_path = getFullRelativePath() + IDataType::getFileNameForStream(part_column.name, substream_path) + ".bin"; - /// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns. + /// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns and column_name.dict.null.bin file. if (volume->getDisk()->exists(candidate_path) && volume->getDisk()->getFileSize(candidate_path) != 0) path_to_data_file = candidate_path; } From 33d045b72e336ac1737222edc5437159456b6182 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 13:17:09 +0300 Subject: [PATCH 181/264] Fix issue in function "bar" --- src/Common/UnicodeBar.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/UnicodeBar.cpp b/src/Common/UnicodeBar.cpp index 29a9838cd62..0e1f792c9fe 100644 --- a/src/Common/UnicodeBar.cpp +++ b/src/Common/UnicodeBar.cpp @@ -14,7 +14,7 @@ namespace UnicodeBar { double getWidth(double x, double min, double max, double max_width) { - if (isNaN(x)) + if (isNaN(x) || isNaN(min) || isNaN(max)) return 0; if (x <= min) From 55265db5834f6ee8421ec2429337ef40c13fb683 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 13:17:55 +0300 Subject: [PATCH 182/264] Add a test --- tests/queries/0_stateless/01654_bar_nan.reference | 1 + tests/queries/0_stateless/01654_bar_nan.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01654_bar_nan.reference create mode 100644 tests/queries/0_stateless/01654_bar_nan.sql diff --git a/tests/queries/0_stateless/01654_bar_nan.reference b/tests/queries/0_stateless/01654_bar_nan.reference new file mode 100644 index 00000000000..8b137891791 --- /dev/null +++ b/tests/queries/0_stateless/01654_bar_nan.reference @@ -0,0 +1 @@ + diff --git a/tests/queries/0_stateless/01654_bar_nan.sql b/tests/queries/0_stateless/01654_bar_nan.sql new file mode 100644 index 00000000000..f9a4f0ca325 --- /dev/null +++ b/tests/queries/0_stateless/01654_bar_nan.sql @@ -0,0 +1 @@ +SELECT bar(-1, -9223372036854775808, nan); From aa51463c933e4af08227df68192ddfe237be09b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 13:22:51 +0300 Subject: [PATCH 183/264] Adjust perf test --- tests/performance/if_to_multiif.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/if_to_multiif.xml b/tests/performance/if_to_multiif.xml index 65bda1b23c6..085a604649b 100644 --- a/tests/performance/if_to_multiif.xml +++ b/tests/performance/if_to_multiif.xml @@ -2,7 +2,7 @@ - + From 49059e5cce6607836aa73ed872a82f9aedb04e3c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 15 Jan 2021 14:31:41 +0300 Subject: [PATCH 184/264] Update 01615_two_args_function_index_fix.sql --- tests/queries/0_stateless/01615_two_args_function_index_fix.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01615_two_args_function_index_fix.sql b/tests/queries/0_stateless/01615_two_args_function_index_fix.sql index 258a07d25f2..31c6a2b9656 100644 --- a/tests/queries/0_stateless/01615_two_args_function_index_fix.sql +++ b/tests/queries/0_stateless/01615_two_args_function_index_fix.sql @@ -1,6 +1,6 @@ drop table if exists bad_date_time; -create table bad_date_time (time Datetime, count UInt16) Engine = MergeTree() ORDER BY (time); +create table bad_date_time (time Datetime('Europe/Moscow'), count UInt16) Engine = MergeTree() ORDER BY (time); insert into bad_date_time values('2020-12-20 20:59:52', 1), ('2020-12-20 21:59:52', 1), ('2020-12-20 01:59:52', 1); From 3c1c30d15b0df7ec60887f418f97558316794a19 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 15 Jan 2021 14:32:42 +0300 Subject: [PATCH 185/264] Update 01615_two_args_function_index_fix.sql --- tests/queries/0_stateless/01615_two_args_function_index_fix.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01615_two_args_function_index_fix.sql b/tests/queries/0_stateless/01615_two_args_function_index_fix.sql index 31c6a2b9656..dd2bde2eafc 100644 --- a/tests/queries/0_stateless/01615_two_args_function_index_fix.sql +++ b/tests/queries/0_stateless/01615_two_args_function_index_fix.sql @@ -4,6 +4,7 @@ create table bad_date_time (time Datetime('Europe/Moscow'), count UInt16) Engine insert into bad_date_time values('2020-12-20 20:59:52', 1), ('2020-12-20 21:59:52', 1), ('2020-12-20 01:59:52', 1); +-- primary key analysis was wrong in previous versions and did not take the timezone argument into account, so empty result was given. select toDate(time, 'UTC') dt, min(toDateTime(time, 'UTC')), max(toDateTime(time, 'UTC')), sum(count) from bad_date_time where toDate(time, 'UTC') = '2020-12-19' group by dt; drop table if exists bad_date_time; From 4a71971b43e1549f3e9b8ac3fc0c011f5d0c6cad Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 15 Jan 2021 14:36:07 +0300 Subject: [PATCH 186/264] Update KeyCondition.cpp --- src/Storages/MergeTree/KeyCondition.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 60f896844ac..53d3b29aa95 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -873,6 +873,11 @@ bool KeyCondition::tryPrepareSetIndex( return true; } + +/** Allow to use two argument function with constant argument to be analyzed as a single argument function. + * In other words, it performs "currying" (binding of arguments). + * This is needed, for example, to support correct analysis of `toDate(time, 'UTC')`. + */ class FunctionWithOptionalConstArg : public IFunctionBase { public: From 6a2a5e53ed7334247d1ee9e2ba3d6e789ac296ed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 15:15:13 +0300 Subject: [PATCH 187/264] Slightly better code of IMergeTreeDataPart #18955 --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 12 ++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 5 ++- src/Storages/MergeTree/MergeTreeData.cpp | 38 +++++++++---------- src/Storages/MergeTree/MergeTreeData.h | 4 +- 4 files changed, 36 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5d0f79f4679..912196528e2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -192,6 +192,18 @@ std::optional IMergeTreeDataPart::getColumnPosition(const String & colum return it->second; } + +void IMergeTreeDataPart::setState(IMergeTreeDataPart::State new_state) const +{ + state = new_state; +} + +IMergeTreeDataPart::State IMergeTreeDataPart::getState() const +{ + return state; +} + + DayNum IMergeTreeDataPart::getMinDate() const { if (storage.minmax_idx_date_column_pos != -1 && minmax_idx.initialized) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 1c617402f67..9c8247ba9bd 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -221,7 +221,8 @@ public: TTLInfos ttl_infos; /// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex - mutable State state{State::Temporary}; + void setState(State new_state) const; + State getState() const; /// Returns name of state static String stateToString(State state); @@ -423,6 +424,8 @@ private: /// Found column without specific compression and return codec /// for this column with default parameters. CompressionCodecPtr detectDefaultCompressionCodec() const; + + mutable State state{State::Temporary}; }; using MergeTreeDataPartState = IMergeTreeDataPart::State; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f84de847acd..7ee67f9f016 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -874,7 +874,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) part->modification_time = part_disk_ptr->getLastModified(relative_data_path + part_name).epochTime(); /// Assume that all parts are Committed, covered parts will be detected and marked as Outdated later - part->state = DataPartState::Committed; + part->setState(DataPartState::Committed); std::lock_guard loading_lock(mutex); if (!data_parts_indexes.insert(part).second) @@ -893,7 +893,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) 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; + part->setState(DataPartState::Committed); if (!data_parts_indexes.insert(part).second) throw Exception("Part " + part->name + " already exists", ErrorCodes::DUPLICATE_DATA_PART); @@ -935,7 +935,7 @@ void MergeTreeData::loadDataParts(bool skip_sanity_checks) (*prev_jt)->assertState({DataPartState::Committed}); - while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->state == DataPartState::Committed) + while (curr_jt != data_parts_by_state_and_info.end() && (*curr_jt)->getState() == DataPartState::Committed) { /// Don't consider data parts belonging to different partitions. if ((*curr_jt)->info.partition_id != (*prev_jt)->info.partition_id) @@ -1999,7 +1999,7 @@ bool MergeTreeData::renameTempPartAndReplace( part->name = part_name; part->info = part_info; part->is_temp = false; - part->state = DataPartState::PreCommitted; + part->setState(DataPartState::PreCommitted); part->renameTo(part_name, true); auto part_it = data_parts_indexes.insert(part).first; @@ -2068,16 +2068,16 @@ void MergeTreeData::removePartsFromWorkingSet(const MergeTreeData::DataPartsVect for (const DataPartPtr & part : remove) { - if (part->state == IMergeTreeDataPart::State::Committed) + if (part->getState() == IMergeTreeDataPart::State::Committed) { removePartContributionToColumnSizes(part); removePartContributionToDataVolume(part); } - if (part->state == IMergeTreeDataPart::State::Committed || clear_without_timeout) + if (part->getState() == IMergeTreeDataPart::State::Committed || clear_without_timeout) part->remove_time.store(remove_time, std::memory_order_relaxed); - if (part->state != IMergeTreeDataPart::State::Outdated) + if (part->getState() != IMergeTreeDataPart::State::Outdated) modifyPartState(part, IMergeTreeDataPart::State::Outdated); if (isInMemoryPart(part) && getSettings()->in_memory_parts_enable_wal) @@ -2161,7 +2161,7 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSet(c continue; } - if (part->state != DataPartState::Deleting) + if (part->getState() != DataPartState::Deleting) parts_to_remove.emplace_back(part); } @@ -2184,7 +2184,7 @@ restore_covered) /// What if part_to_detach is a reference to *it_part? Make a new owner just in case. DataPartPtr part = *it_part; - if (part->state == DataPartState::Committed) + if (part->getState() == DataPartState::Committed) { removePartContributionToDataVolume(part); removePartContributionToColumnSizes(part); @@ -2227,13 +2227,13 @@ restore_covered) { auto it = std::prev(it_middle); - if (part->contains(**it) && is_appropriate_state((*it)->state)) + if (part->contains(**it) && is_appropriate_state((*it)->getState())) { /// Maybe, we must consider part level somehow if ((*it)->info.min_block != part->info.min_block) update_error(it); - if ((*it)->state != DataPartState::Committed) + if ((*it)->getState() != DataPartState::Committed) { addPartContributionToColumnSizes(*it); addPartContributionToDataVolume(*it); @@ -2255,7 +2255,7 @@ restore_covered) if ((*it)->info.min_block < pos) continue; - if (!is_appropriate_state((*it)->state)) + if (!is_appropriate_state((*it)->getState())) { update_error(it); continue; @@ -2264,7 +2264,7 @@ restore_covered) if ((*it)->info.min_block > pos) update_error(it); - if ((*it)->state != DataPartState::Committed) + if ((*it)->getState() != DataPartState::Committed) { addPartContributionToColumnSizes(*it); addPartContributionToDataVolume(*it); @@ -2305,7 +2305,7 @@ void MergeTreeData::tryRemovePartImmediately(DataPartPtr && part) part.reset(); - if (!((*it)->state == DataPartState::Outdated && it->unique())) + if (!((*it)->getState() == DataPartState::Outdated && it->unique())) return; modifyPartState(it, DataPartState::Deleting); @@ -2522,10 +2522,8 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const MergeTreePartInf return nullptr; for (auto state : valid_states) - { - if ((*it)->state == state) + if ((*it)->getState() == state) return *it; - } return nullptr; } @@ -2961,7 +2959,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector(const DataPartS { out_states->resize(res.size()); for (size_t i = 0; i < res.size(); ++i) - (*out_states)[i] = res[i]->state; + (*out_states)[i] = res[i]->getState(); } } @@ -2979,7 +2977,7 @@ MergeTreeData::DataPartsVector MergeTreeData::getAllDataPartsVector(MergeTreeDat { out_states->resize(res.size()); for (size_t i = 0; i < res.size(); ++i) - (*out_states)[i] = res[i]->state; + (*out_states)[i] = res[i]->getState(); } } @@ -3317,7 +3315,7 @@ MergeTreeData::DataPartPtr MergeTreeData::getAnyPartInPartition( { auto it = data_parts_by_state_and_info.lower_bound(DataPartStateAndPartitionID{DataPartState::Committed, partition_id}); - if (it != data_parts_by_state_and_info.end() && (*it)->state == DataPartState::Committed && (*it)->info.partition_id == partition_id) + if (it != data_parts_by_state_and_info.end() && (*it)->getState() == DataPartState::Committed && (*it)->info.partition_id == partition_id) return *it; return nullptr; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 52c0b61b977..6098cd22c98 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -765,7 +765,7 @@ protected: static DataPartStateAndInfo dataPartPtrToStateAndInfo(const DataPartPtr & part) { - return {part->state, part->info}; + return {part->getState(), part->info}; } using DataPartsIndexes = boost::multi_index_containerstate = state; }; + return [state] (const DataPartPtr & part) { part->setState(state); }; } void modifyPartState(DataPartIteratorByStateAndInfo it, DataPartState state) From e238fd64acabf3c1adaaf05340a563d837887040 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 15:28:53 +0300 Subject: [PATCH 188/264] Add part metrics --- src/Common/CurrentMetrics.cpp | 8 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 78 ++++++++++++++++++- 2 files changed, 83 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index d3a4a41046e..d8d836503f4 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -56,7 +56,13 @@ M(LocalThreadActive, "Number of threads in local thread pools running a task.") \ M(DistributedFilesToInsert, "Number of pending files to process for asynchronous insertion into Distributed tables. Number of files for every shard is summed.") \ M(TablesToDropQueueSize, "Number of dropped tables, that are waiting for background data removal.") \ - M(MaxDDLEntryID, "Max processed DDL entry of DDLWorker.") \ + M(MaxDDLEntryID, "Max processed DDL entry of DDLWorker.") \ + M(PartsTemporary, "The part is generating now, it is not in data_parts list.") \ + M(PartsPreCommitted, "The part is in data_parts, but not used for SELECTs.") \ + M(PartsCommitted, "Active data part, used by current and upcoming SELECTs.") \ + M(PartsOutdated, "Not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes.") \ + M(PartsDeleting, "Not active data part with identity refcounter, it is deleting right now by a cleaner.") \ + M(PartsDeleteOnDestroy, "Part was moved to another disk and should be deleted in own destructor.") \ namespace CurrentMetrics { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 912196528e2..a6375731bb0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -12,11 +12,23 @@ #include #include #include +#include #include #include #include #include + +namespace CurrentMetrics +{ + extern const Metric PartsTemporary; + extern const Metric PartsPreCommitted; + extern const Metric PartsCommitted; + extern const Metric PartsOutdated; + extern const Metric PartsDeleting; + extern const Metric PartsDeleteOnDestroy; +} + namespace DB { namespace ErrorCodes @@ -137,6 +149,61 @@ void IMergeTreeDataPart::MinMaxIndex::merge(const MinMaxIndex & other) } +static void incrementMetric(IMergeTreeDataPart::State state) +{ + switch (state) + { + case IMergeTreeDataPart::State::Temporary: + CurrentMetrics::add(CurrentMetrics::PartsTemporary); + return; + case IMergeTreeDataPart::State::PreCommitted: + CurrentMetrics::add(CurrentMetrics::PartsPreCommitted); + return; + case IMergeTreeDataPart::State::Committed: + CurrentMetrics::add(CurrentMetrics::PartsCommitted); + return; + case IMergeTreeDataPart::State::Outdated: + CurrentMetrics::add(CurrentMetrics::PartsOutdated); + return; + case IMergeTreeDataPart::State::Deleting: + CurrentMetrics::add(CurrentMetrics::PartsDeleting); + return; + case IMergeTreeDataPart::State::DeleteOnDestroy: + CurrentMetrics::add(CurrentMetrics::PartsDeleteOnDestroy); + return; + } + + __builtin_unreachable(); +} + +static void decrementMetric(IMergeTreeDataPart::State state) +{ + switch (state) + { + case IMergeTreeDataPart::State::Temporary: + CurrentMetrics::sub(CurrentMetrics::PartsTemporary); + return; + case IMergeTreeDataPart::State::PreCommitted: + CurrentMetrics::sub(CurrentMetrics::PartsPreCommitted); + return; + case IMergeTreeDataPart::State::Committed: + CurrentMetrics::sub(CurrentMetrics::PartsCommitted); + return; + case IMergeTreeDataPart::State::Outdated: + CurrentMetrics::sub(CurrentMetrics::PartsOutdated); + return; + case IMergeTreeDataPart::State::Deleting: + CurrentMetrics::sub(CurrentMetrics::PartsDeleting); + return; + case IMergeTreeDataPart::State::DeleteOnDestroy: + CurrentMetrics::sub(CurrentMetrics::PartsDeleteOnDestroy); + return; + } + + __builtin_unreachable(); +} + + IMergeTreeDataPart::IMergeTreeDataPart( MergeTreeData & storage_, const String & name_, const VolumePtr & volume_, const std::optional & relative_path_, Type part_type_) : storage(storage_) @@ -147,6 +214,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) { + incrementMetric(state); } IMergeTreeDataPart::IMergeTreeDataPart( @@ -164,6 +232,12 @@ IMergeTreeDataPart::IMergeTreeDataPart( , index_granularity_info(storage_, part_type_) , part_type(part_type_) { + incrementMetric(state); +} + +IMergeTreeDataPart::~IMergeTreeDataPart() +{ + decrementMetric(state); } @@ -195,7 +269,9 @@ std::optional IMergeTreeDataPart::getColumnPosition(const String & colum void IMergeTreeDataPart::setState(IMergeTreeDataPart::State new_state) const { + decrementMetric(state); state = new_state; + incrementMetric(state); } IMergeTreeDataPart::State IMergeTreeDataPart::getState() const @@ -248,8 +324,6 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) column_name_to_position.emplace(column.name, pos++); } -IMergeTreeDataPart::~IMergeTreeDataPart() = default; - void IMergeTreeDataPart::removeIfNeeded() { if (state == State::DeleteOnDestroy || is_temp) From 16c3526864f2d4d851939b88a07b5f9f2372e6c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 15:34:37 +0300 Subject: [PATCH 189/264] Add a test --- .../01600_count_of_parts_metrics.reference | 4 ++ .../01600_count_of_parts_metrics.sh | 38 +++++++++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 tests/queries/0_stateless/01600_count_of_parts_metrics.reference create mode 100755 tests/queries/0_stateless/01600_count_of_parts_metrics.sh diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.reference b/tests/queries/0_stateless/01600_count_of_parts_metrics.reference new file mode 100644 index 00000000000..98fb6a68656 --- /dev/null +++ b/tests/queries/0_stateless/01600_count_of_parts_metrics.reference @@ -0,0 +1,4 @@ +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/01600_count_of_parts_metrics.sh b/tests/queries/0_stateless/01600_count_of_parts_metrics.sh new file mode 100755 index 00000000000..45ec6701fdf --- /dev/null +++ b/tests/queries/0_stateless/01600_count_of_parts_metrics.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +verify_sql="SELECT + (SELECT sumIf(value, metric = 'PartsCommitted'), sumIf(value, metric = 'PartsOutdated') FROM system.metrics) + = (SELECT sum(active), sum(NOT active) FROM system.parts)" + +# The query is not atomic - it can compare states between system.parts and system.metrics from different points in time. +# So, there is inherent race condition. But it should get expected result eventually. +# In case of test failure, this code will do infinite loop and timeout. +verify() +{ + while true + do + result=$( $CLICKHOUSE_CLIENT -m --query="$verify_sql" ) + [ "$result" = "1" ] && break + sleep 0.1 + done + echo 1 +} + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS test_table" +$CLICKHOUSE_CLIENT --query="CREATE TABLE test_table(data Date) ENGINE = MergeTree PARTITION BY toYear(data) ORDER BY data;" + +$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-01')" +verify + +$CLICKHOUSE_CLIENT --query="INSERT INTO test_table VALUES ('1992-01-02')" +verify + +$CLICKHOUSE_CLIENT --query="OPTIMIZE TABLE test_table FINAL" +verify + +$CLICKHOUSE_CLIENT --query="DROP TABLE test_table" +verify From 8ccaa6ede9423c397638232b70899718b3b6a4f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 15:40:37 +0300 Subject: [PATCH 190/264] Additional check for huge granules in MergeTreeDataWriter --- src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index ef64ec28e79..56341390581 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -26,7 +26,7 @@ Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, Granules result; size_t current_row = 0; - /// When our last mark is not finished yet and we have to write in rows into it + /// When our last mark is not finished yet and we have to write rows into it if (rows_written_in_last_mark > 0) { size_t rows_left_in_last_mark = index_granularity.getMarkRows(current_mark) - rows_written_in_last_mark; @@ -440,6 +440,13 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, "Still have {} rows in bin stream, last mark #{} index granularity size {}, last rows {}", column->size(), mark_num, index_granularity.getMarksCount(), index_granularity_rows); } + if (index_granularity_rows > data_part->index_granularity_info.fixed_index_granularity) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Mark #{} has {} rows, but max fixed granularity is {}, index granularity size {}", + mark_num, index_granularity_rows, data_part->index_granularity_info.fixed_index_granularity, index_granularity.getMarksCount()); + } + if (index_granularity_rows != index_granularity.getMarkRows(mark_num)) throw Exception( ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{} (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", From ef29f0744a089ad80dc0b84e0b808e29214f32ea Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 15 Jan 2021 12:49:43 +0000 Subject: [PATCH 191/264] Translate commercial section to Chinese and add Tencent Cloud to provider list update --- docs/en/commercial/cloud.md | 10 +++++++++ docs/zh/commercial/cloud.md | 40 +++++++++++++++++++++++++---------- docs/zh/commercial/index.md | 16 ++++++++++---- docs/zh/commercial/support.md | 12 +++++------ 4 files changed, 56 insertions(+), 22 deletions(-) diff --git a/docs/en/commercial/cloud.md b/docs/en/commercial/cloud.md index 9dd08124f64..0490881c622 100644 --- a/docs/en/commercial/cloud.md +++ b/docs/en/commercial/cloud.md @@ -29,4 +29,14 @@ toc_title: Cloud - Cross-AZ scaling for performance and high availability - Built-in monitoring and SQL query editor +## Tencent Cloud {#tencent-cloud} + +[Tencent Managed Service for ClickHouse](https://cloud.tencent.com/product/cdwch) provides the following key features: + +- Easy to deploy and manage on Tencent Cloud +- Highly scalable and available +- Integrated monitor and alert service +- High security with isolated per cluster VPCs +- On-demand pricing with no upfront costs or long-term commitments + {## [Original article](https://clickhouse.tech/docs/en/commercial/cloud/) ##} diff --git a/docs/zh/commercial/cloud.md b/docs/zh/commercial/cloud.md index beeec5057db..9cca8776d14 100644 --- a/docs/zh/commercial/cloud.md +++ b/docs/zh/commercial/cloud.md @@ -1,23 +1,41 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd toc_priority: 1 -toc_title: "\u4E91" +toc_title: 云 --- -# ツ环板Providersョツ嘉ッ {#clickhouse-cloud-service-providers} +# ClickHouse 云服务提供商 {#clickhouse-cloud-service-providers} -!!! info "信息" - 如果您已经启动了带有托管ClickHouse服务的公共云,请随时 [打开拉取请求](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/cloud.md) 将其添加到以下列表。 +!!! info "注意" + 如果您已经推出具有托管 ClickHouse 服务的公共云,请随时[提交一个 pull request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/cloud.md) 将其添加到以下列表。 -## Yandex云 {#yandex-cloud} +## Yandex 云 {#yandex-cloud} -[Yandex的ClickHouse托管服务](https://cloud.yandex.com/services/managed-clickhouse?utm_source=referrals&utm_medium=clickhouseofficialsite&utm_campaign=link3) 提供以下主要功能: +[Yandex的 ClickHouse 托管服务](https://cloud.yandex.com/services/managed-clickhouse?utm_source=referrals&utm_medium=clickhouseofficialsite&utm_campaign=link3) 提供以下主要功能: -- 全面管理的动物园管理员服务 [ClickHouse复制](../engines/table-engines/mergetree-family/replication.md) +- 用于 ClickHouse 复制的完全托管的 ZooKeeper 服务 [ClickHouse复制](../engines/table-engines/mergetree-family/replication.md) - 多种存储类型选择 -- 不同可用区中的副本 -- 加密和隔离 +- 不同可用区副本 +- 加密与隔离 - 自动化维护 +## Altinity.Cloud {#altinity.cloud} + +[Altinity.Cloud](https://altinity.com/cloud-database/) 是针对 Amazon 公共云的完全托管的 ClickHouse-as-a-Service + +- 在 Amazon 资源上快速部署 ClickHouse 集群 +- 轻松进行横向扩展/纵向扩展以及节点的垂直扩展 +- 具有公共端点或VPC对等的租户隔离 +- 可配置存储类型以及卷配置 +- 跨可用区扩展以实现性能和高可用性 +- 内置监控和SQL查询编辑器 + +## 腾讯云 {#tencent-cloud} + +[腾讯云的 ClickHouse 托管服务](https://cloud.tencent.com/product/cdwch)提供以下主要功能: + +- 易于部署和管理, 集成监控与警报服务 +- 高可用高扩展 +- 通过集群级别的 VPC 保证安全可靠 +- 按需定价,无需前期成本或长期承诺 + {## [原始文章](https://clickhouse.tech/docs/en/commercial/cloud/) ##} diff --git a/docs/zh/commercial/index.md b/docs/zh/commercial/index.md index ef285e28108..2bfd0767d1b 100644 --- a/docs/zh/commercial/index.md +++ b/docs/zh/commercial/index.md @@ -1,9 +1,17 @@ --- -machine_translated: true -machine_translated_rev: 72537a2d527c63c07aa5d2361a8829f3895cf2bd -toc_folder_title: "\u5546\u4E1A" +toc_folder_title: 商业支持 toc_priority: 70 -toc_title: "\u5546\u4E1A" +toc_title: 简介 --- +# ClickHouse 商业服务 {#clickhouse-commercial-services} +本节是专门从事 ClickHouse 的服务提供商的目录,它们是一些独立的公司,不一定与 Yandex 有关系。 + +服务类别: + +- [云](../commercial/cloud.md) +- [支持](../commercial/support.md) + +!!! note "对于服务提供商" +如果您碰巧是其中之一,可以随时提交一个 pull request,将您的公司添加到对应的章节(如果服务不属于现有的任何目录,也可以添加新的章节)。提交关于文档的 pull request 最简单的方式是点击右上角的“铅笔”编辑按钮。如果您的服务在某些本地市场上有售,请确保在本地化的文档界面中也提及它(或至少在 pull request 请求描述中指出)。 diff --git a/docs/zh/commercial/support.md b/docs/zh/commercial/support.md index f543338d4e6..44b6d1eab8d 100644 --- a/docs/zh/commercial/support.md +++ b/docs/zh/commercial/support.md @@ -1,18 +1,16 @@ --- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 toc_priority: 3 -toc_title: "\u5546\u4e1a\u652f\u6301" +toc_title: 支持 --- -# ClickHouse商业支持服务提供商 {#clickhouse-commercial-support-service-providers} +# ClickHouse 商业支持服务提供商 {#clickhouse-commercial-support-service-providers} -!!! info "信息" - 如果您已经推出ClickHouse商业支持服务,请随时 [提交一个 pull-request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) 将其添加到以下列表。 +!!! info "注意" + 如果您已经推出 ClickHouse 商业支持服务,请随时[提交一个 pull request](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/commercial/support.md) 将其添加到以下列表。 ## Altinity {#altinity} - Altinity 自从 2017 年开始已经为企业提供 ClickHouse 支持服务。Altinity 的客户范围包含百强企业到初创企业。访问 [www.altinity.com](https://www.altinity.com/) 了解更多信息。 + Altinity 自从 2017 年开始为企业提供 ClickHouse 支持服务。Altinity 的客户范围包含百强企业到初创企业等。访问 [www.altinity.com](https://www.altinity.com/) 了解更多信息。 ## Mafiree {#mafiree} From af44ff2390990241983abe8d0fc3bab95dedd749 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Jan 2021 16:22:53 +0300 Subject: [PATCH 192/264] Add test to arcadia skiplist --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 116d0beb80b..107e811cbc3 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -184,3 +184,4 @@ 01639_distributed_sync_insert_zero_rows 01644_distributed_async_insert_fsync_smoke 01552_impl_aggfunc_cloneresize +01651_bugs_from_15889 From 1b37d7716f76137e5a98010ac39e81317c7678a6 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Fri, 15 Jan 2021 18:06:51 +0300 Subject: [PATCH 193/264] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 0488cdce155..889d7308ed8 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -72,7 +72,7 @@ function watchdog function fuzz { - ./clickhouse-server --config-file db/config.xml -- --path db 2>&1 | tail -10000 > server.log & + ./clickhouse-server --config-file db/config.xml -- --path db 2>&1 | tail -100000 > server.log & server_pid=$! kill -0 $server_pid while ! ./clickhouse-client --query "select 1" && kill -0 $server_pid ; do echo . ; sleep 1 ; done @@ -85,7 +85,7 @@ function fuzz # SC2046: Quote this to prevent word splitting. Actually I need word splitting. # shellcheck disable=SC2012,SC2046 ./clickhouse-client --query-fuzzer-runs=1000 --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \ - > >(tail -n 10000 > fuzzer.log) \ + > >(tail -n 100000 > fuzzer.log) \ 2>&1 \ || fuzzer_exit_code=$? From 76495124cdce01bf4e07398b0c0c375cc30be1b4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Jan 2021 18:45:29 +0300 Subject: [PATCH 194/264] Fix readign from fd for ORCBlockInputFormat. --- src/Processors/Formats/Impl/ORCBlockInputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 8bbf0fc089b..7776a904f1c 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -28,7 +28,7 @@ Chunk ORCBlockInputFormat::generate() Chunk res; const Block & header = getPort().getHeader(); - if (in.eof()) + if (file_reader) return res; arrow::Status open_status = arrow::adapters::orc::ORCFileReader::Open(asArrowFile(in), arrow::default_memory_pool(), &file_reader); From 9829c09720cb0748254087e30ac150c08803f675 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 15 Jan 2021 15:54:35 +0000 Subject: [PATCH 195/264] fix --- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index fcb2349d14d..ab5fe4eae9f 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -56,7 +56,7 @@ namespace ErrorCodes InterpreterInsertQuery::InterpreterInsertQuery( const ASTPtr & query_ptr_, const Context & context_, bool allow_materialized_, bool no_squash_, bool no_destination_) - : query_ptr(query_ptr_->clone()) + : query_ptr(query_ptr_) , context(context_) , allow_materialized(allow_materialized_) , no_squash(no_squash_) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 577808d72fd..e85cb227310 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -352,7 +352,9 @@ DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobRep /// Forward user settings job.local_context = std::make_unique(context); - InterpreterInsertQuery interp(query_ast, *job.local_context); + auto copy_query_ast = query_ast->clone(); + + InterpreterInsertQuery interp(copy_query_ast, *job.local_context); auto block_io = interp.execute(); job.stream = block_io.out; From 8b079674a921bf2f3e8c69a4ee81dc15fdeb07b5 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 16 Jan 2021 00:37:41 +0800 Subject: [PATCH 196/264] Remove useless code --- src/IO/ReadHelpers.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 97a8d937d39..9cd8747da64 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -253,9 +253,6 @@ void readStringUntilEOFInto(Vector & s, ReadBuffer & buf) { appendToStringOrVector(s, buf, buf.buffer().end()); buf.position() = buf.buffer().end(); - - if (buf.hasPendingData()) - return; } } From 066242c64d3fc2b440c683777f2512fe25939977 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 15 Jan 2021 19:57:44 +0300 Subject: [PATCH 197/264] Print errors through LOG_ERROR --- src/Parsers/New/LexerErrorListener.cpp | 3 ++- src/Parsers/New/ParserErrorListener.cpp | 8 +++++--- src/Parsers/New/README.md | 12 ++++++++++++ 3 files changed, 19 insertions(+), 4 deletions(-) create mode 100644 src/Parsers/New/README.md diff --git a/src/Parsers/New/LexerErrorListener.cpp b/src/Parsers/New/LexerErrorListener.cpp index d40d470486e..ed6dc358c52 100644 --- a/src/Parsers/New/LexerErrorListener.cpp +++ b/src/Parsers/New/LexerErrorListener.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -17,7 +18,7 @@ extern int SYNTAX_ERROR; void LexerErrorListener::syntaxError(Recognizer *, Token *, size_t, size_t, const std::string & message, std::exception_ptr) { - std::cerr << "Lexer error: " << message << std::endl; + LOG_ERROR(&Poco::Logger::get("ClickHouseLexer"), "Lexer error: {}", message); throw DB::Exception("Can't recognize input: " + message, ErrorCodes::SYNTAX_ERROR); } diff --git a/src/Parsers/New/ParserErrorListener.cpp b/src/Parsers/New/ParserErrorListener.cpp index 0f637b1fa12..5c1b9238d15 100644 --- a/src/Parsers/New/ParserErrorListener.cpp +++ b/src/Parsers/New/ParserErrorListener.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -24,9 +25,10 @@ void ParserErrorListener::syntaxError( { auto * parser = dynamic_cast(recognizer); - std::cerr << "Last element parsed so far:" << std::endl - << parser->getRuleContext()->toStringTree(parser, true) << std::endl - << "Parser error: (pos " << token->getStartIndex() << ") " << message << std::endl; + LOG_ERROR(&Poco::Logger::get("ClickHouseParser"), + "Last element parsed so far:\n" + "{}\n" + "Parser error: (pos {}) {}", parser->getRuleContext()->toStringTree(parser, true), token->getStartIndex(), message); throw DB::Exception("Can't parse input: " + message, ErrorCodes::SYNTAX_ERROR); } diff --git a/src/Parsers/New/README.md b/src/Parsers/New/README.md new file mode 100644 index 00000000000..50bf34ab432 --- /dev/null +++ b/src/Parsers/New/README.md @@ -0,0 +1,12 @@ +## How to generate source code files from grammar + +Grammar is located inside `ClickHouseLexer.g4` and `ClickHouseParser.g4` files. + +To generate source code you need to install locally the `antlr4` binary: +``` +cd src/Parsers/New +antlr4 -no-listener -visitor -package DB -Dlanguage=Cpp ClickHouseLexer.g4 # if you have changes in a lexer part of grammar +antlr4 -no-listener -visitor -package DB -Dlanguage=Cpp ClickHouseParser.g4 +``` + +Commit only git-tracked generated files - not all of the generated content is required. From 83c5539df606d766a1dfffa0b39e8333d7667977 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 15 Jan 2021 20:39:32 +0300 Subject: [PATCH 198/264] Update 01651_bugs_from_15889.sql --- tests/queries/0_stateless/01651_bugs_from_15889.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 1275f258225..7b69061eb04 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -39,6 +39,9 @@ INSERT INTO trace_log values ('2020-10-06','2020-10-06 13:43:39','2020-10-06 13: set allow_introspection_functions = 1; +-- make sure query_log exists +SYSTEM FLUSH LOGS; + WITH concat(addressToLine(arrayJoin(trace) AS addr), '#') AS symbol SELECT count() > 7 FROM trace_log AS t From 142bcd4142c74cef40d6a996ed4b0c1515c882d6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 15 Jan 2021 20:44:16 +0300 Subject: [PATCH 199/264] Update test --- tests/queries/0_stateless/00900_orc_load.reference | 2 ++ tests/queries/0_stateless/00900_orc_load.sh | 1 + 2 files changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/00900_orc_load.reference b/tests/queries/0_stateless/00900_orc_load.reference index fe79e37ee18..04c72eccb17 100644 --- a/tests/queries/0_stateless/00900_orc_load.reference +++ b/tests/queries/0_stateless/00900_orc_load.reference @@ -1,2 +1,4 @@ 0 0 0 0 0 2019-01-01 test1 2147483647 -1 9223372036854775806 123.345345 345345.3453451212 2019-01-01 test2 +0 0 0 0 0 2019-01-01 test1 +2147483647 -1 9223372036854775806 123.345345 345345.3453451212 2019-01-01 test2 diff --git a/tests/queries/0_stateless/00900_orc_load.sh b/tests/queries/0_stateless/00900_orc_load.sh index a0bacff43e5..aae752be3e9 100755 --- a/tests/queries/0_stateless/00900_orc_load.sh +++ b/tests/queries/0_stateless/00900_orc_load.sh @@ -9,6 +9,7 @@ DATA_FILE=$CUR_DIR/data_orc/test.orc ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (int Int32, smallint Int8, bigint Int64, float Float32, double Float64, date Date, y String) ENGINE = Memory" cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE ${CLICKHOUSE_CLIENT} --query="select * from orc_load" ${CLICKHOUSE_CLIENT} --query="drop table orc_load" From c846f76ffacb9e587bb0c5acf3bbc42a9df8d106 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 15 Jan 2021 20:58:51 +0300 Subject: [PATCH 200/264] Update Dockerfile --- docker/server/Dockerfile | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 0759d92b325..fa9c0ae5f3a 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -4,7 +4,14 @@ ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" ARG version=21.1.0 ARG gosu_ver=1.10 -# user/group precreated explicitly with fixed uid/gid on purpose (see commit) +# user/group precreated explicitly with fixed uid/gid on purpose. +# It is especially important for rootless containers: in that case entrypoint +# can't do chown and owners of mounted volumes should be configured externally. +# We do that in advance at the begining of Dockerfile before any packages will be +# installed to prevent picking those uid / gid by some unrelated software. +# The same uid / gid (101) is used both for alpine and ubuntu. +# Number 101 is used by default in openshift + RUN groupadd -r clickhouse --gid=101 \ && useradd -r -g clickhouse --uid=101 --home-dir=/var/lib/clickhouse --shell=/bin/bash clickhouse \ && apt-get update \ From e0c90cd20e5637aa9cba9e4c88f7481d55f3998f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 15 Jan 2021 20:59:12 +0300 Subject: [PATCH 201/264] Update Dockerfile.alpine --- docker/server/Dockerfile.alpine | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index 0d28e59d602..090e20edab8 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -8,7 +8,14 @@ ENV LANG=en_US.UTF-8 \ COPY alpine-root/ / -# user/group precreated explicitly with fixed uid/gid on purpose (see commit) +# user/group precreated explicitly with fixed uid/gid on purpose. +# It is especially important for rootless containers: in that case entrypoint +# can't do chown and owners of mounted volumes should be configured externally. +# We do that in advance at the begining of Dockerfile before any packages will be +# installed to prevent picking those uid / gid by some unrelated software. +# The same uid / gid (101) is used both for alpine and ubuntu. +# Number 101 is used by default in openshift + RUN addgroup -S -g 101 clickhouse \ && adduser -S -h /var/lib/clickhouse -s /bin/bash -G clickhouse -g "ClickHouse server" -u 101 clickhouse \ && chown clickhouse:clickhouse /var/lib/clickhouse \ From 184dbedb06b68380d6f012a082768dc1f57ded3b Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jan 2021 21:50:30 +0300 Subject: [PATCH 202/264] Fix stupid error --- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 9 ++- .../01654_test_writer_block_sequence.python | 58 +++++++++++++++++++ ...01654_test_writer_block_sequence.reference | 1 + .../01654_test_writer_block_sequence.sh | 7 +++ 4 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01654_test_writer_block_sequence.python create mode 100644 tests/queries/0_stateless/01654_test_writer_block_sequence.reference create mode 100755 tests/queries/0_stateless/01654_test_writer_block_sequence.sh diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 56341390581..982345f8240 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -183,8 +183,8 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm /// adjust last mark rows and flush to disk. if (rows_written_in_last_mark >= index_granularity_for_block) adjustLastMarkIfNeedAndFlushToDisk(rows_written_in_last_mark); - else /// We still can write some rows from new block into previous granule. - adjustLastMarkIfNeedAndFlushToDisk(index_granularity_for_block - rows_written_in_last_mark); + else /// We still can write some rows from new block into previous granule. So the granule size will be block granularity size. + adjustLastMarkIfNeedAndFlushToDisk(index_granularity_for_block); } } @@ -614,6 +614,11 @@ void MergeTreeDataPartWriterWide::fillIndexGranularity(size_t index_granularity_ void MergeTreeDataPartWriterWide::adjustLastMarkIfNeedAndFlushToDisk(size_t new_rows_in_last_mark) { + /// We don't want to split already written granules to smaller + if (rows_written_in_last_mark > new_rows_in_last_mark) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Tryin to make mark #{} smaller ({} rows) then it already has {}", + getCurrentMark(), new_rows_in_last_mark, rows_written_in_last_mark); + /// We can adjust marks only if we computed granularity for blocks. /// Otherwise we cannot change granularity because it will differ from /// other columns diff --git a/tests/queries/0_stateless/01654_test_writer_block_sequence.python b/tests/queries/0_stateless/01654_test_writer_block_sequence.python new file mode 100644 index 00000000000..e80cc273076 --- /dev/null +++ b/tests/queries/0_stateless/01654_test_writer_block_sequence.python @@ -0,0 +1,58 @@ +#!/usr/bin/env python3 +import os +import sys +import random +import string + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from pure_http_client import ClickHouseClient + +def get_random_string(length): + return ''.join(random.choice(string.ascii_uppercase + string.digits) for _ in range(length)) + +client = ClickHouseClient() + +def insert_block(table_name, block_granularity_rows, block_rows): + global client + block_data = [] + index_granularity_bytes = 10 * 1024 * 1024 + row_bytes = index_granularity_bytes // block_granularity_rows + for _ in range(block_rows): + block_data.append(get_random_string(row_bytes - 1)) + + values_row = ", ".join("(1, '" + row + "')" for row in block_data) + client.query("INSERT INTO {} VALUES {}".format(table_name, values_row)) + +try: + client.query("DROP TABLE IF EXISTS t") + client.query("CREATE TABLE t (v UInt8, data String) ENGINE = MergeTree() ORDER BY tuple() SETTINGS min_bytes_for_wide_part = 0") + + client.query("SYSTEM STOP MERGES t") + + # These blocks size taken from the real table which reproduces the error + # when we get granule with more rows then fixed granularity after horizontal merge. + # About 10k rows when max is 8912. + # + # Why these blocks are special? + # 1) The first one contains 1811 rows, but its granule should have 6853. + # So we write 1811 and get unfinished granule with 6853 - 1811 = 5042 rows to write from the next blocks. + # + # 2) The second block has fewer rows than rows left in the unfinished granule (3094 < 5042). + # It can be written entirely in this unfinished granule and we will still have some rows left. But it's granularity + # should be smaller than rows left in granule (3094 < 5042), so clickhouse will adjust (make smaller) this last unfinished granule. + # This adjust logic contained a bug: we adjust not to the new block's granularity (3094), but to the difference of the new block granularity and + # already written rows (3094 - 1811 = 1283). This lead to several unsigned integer overflows in code and huge granules as result. + # + # 3) Last block just triggers the check that each granule has fewer rows than fixed granularity rows. If the bug from 2) exists then it will fail. + insert_block("t", block_granularity_rows=6853, block_rows=1811) + insert_block("t", block_granularity_rows=3094, block_rows=3094) + insert_block("t", block_granularity_rows=6092, block_rows=6092) + + client.query("SYSTEM START MERGES t") + client.query("OPTIMIZE TABLE t FINAL") + + print(client.query_return_df("SELECT COUNT() as C FROM t FORMAT TabSeparatedWithNames")['C'][0]) +finally: + client.query("DROP TABLE IF EXISTS t") diff --git a/tests/queries/0_stateless/01654_test_writer_block_sequence.reference b/tests/queries/0_stateless/01654_test_writer_block_sequence.reference new file mode 100644 index 00000000000..5aebdef792b --- /dev/null +++ b/tests/queries/0_stateless/01654_test_writer_block_sequence.reference @@ -0,0 +1 @@ +10997 diff --git a/tests/queries/0_stateless/01654_test_writer_block_sequence.sh b/tests/queries/0_stateless/01654_test_writer_block_sequence.sh new file mode 100755 index 00000000000..3330148d91a --- /dev/null +++ b/tests/queries/0_stateless/01654_test_writer_block_sequence.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +python3 "$CURDIR"/01654_test_writer_block_sequence.python From 61b2d0ce423a9325e6da0be4c32f7aa6e3f40f10 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 15 Jan 2021 22:39:10 +0300 Subject: [PATCH 203/264] MemoryTracker: Do not ignore server memory limits during blocking by default --- src/Common/MemoryTracker.cpp | 10 +++++----- src/Common/MemoryTracker.h | 4 ++-- src/Interpreters/executeQuery.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/StorageBuffer.cpp | 4 ++-- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index e62f15d4fd1..e5682cf37d0 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -181,7 +181,7 @@ void MemoryTracker::alloc(Int64 size) if (unlikely(fault_probability && fault(thread_local_rng)) && memoryTrackerCanThrow(level, true)) { /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc - BlockerInThread untrack_lock; + BlockerInThread untrack_lock(VariableContext::Global); ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); const auto * description = description_ptr.load(std::memory_order_relaxed); @@ -195,7 +195,7 @@ void MemoryTracker::alloc(Int64 size) if (unlikely(current_profiler_limit && will_be > current_profiler_limit)) { - BlockerInThread untrack_lock; + BlockerInThread untrack_lock(VariableContext::Global); DB::TraceCollector::collect(DB::TraceType::Memory, StackTrace(), size); setOrRaiseProfilerLimit((will_be + profiler_step - 1) / profiler_step * profiler_step); } @@ -203,14 +203,14 @@ void MemoryTracker::alloc(Int64 size) std::bernoulli_distribution sample(sample_probability); if (unlikely(sample_probability && sample(thread_local_rng))) { - BlockerInThread untrack_lock; + BlockerInThread untrack_lock(VariableContext::Global); DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), size); } if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false)) { /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc - BlockerInThread untrack_lock; + BlockerInThread untrack_lock(VariableContext::Global); ProfileEvents::increment(ProfileEvents::QueryMemoryLimitExceeded); const auto * description = description_ptr.load(std::memory_order_relaxed); @@ -256,7 +256,7 @@ void MemoryTracker::free(Int64 size) std::bernoulli_distribution sample(sample_probability); if (unlikely(sample_probability && sample(thread_local_rng))) { - BlockerInThread untrack_lock; + BlockerInThread untrack_lock(VariableContext::Global); DB::TraceCollector::collect(DB::TraceType::MemorySample, StackTrace(), -size); } diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 9a2c3a399ea..961c8722afa 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -165,7 +165,7 @@ public: VariableContext previous_level; public: /// level_ - block in level and above - BlockerInThread(VariableContext level_ = VariableContext::Global); + BlockerInThread(VariableContext level_ = VariableContext::User); ~BlockerInThread(); static bool isBlocked(VariableContext current_level) @@ -201,7 +201,7 @@ public: public: /// level_ - block in level and above /// block_fault_injections_ - block in fault injection too - LockExceptionInThread(VariableContext level_ = VariableContext::Global, bool block_fault_injections_ = true); + LockExceptionInThread(VariableContext level_ = VariableContext::User, bool block_fault_injections_ = true); ~LockExceptionInThread(); static bool isBlocked(VariableContext current_level, bool fault_injection) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 5928da156f3..e6aec4149fd 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -180,7 +180,7 @@ static void setExceptionStackTrace(QueryLogElement & elem) { /// Disable memory tracker for stack trace. /// Because if exception is "Memory limit (for query) exceed", then we probably can't allocate another one string. - MemoryTracker::BlockerInThread temporarily_disable_memory_tracker; + MemoryTracker::BlockerInThread temporarily_disable_memory_tracker(VariableContext::Global); try { diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index be87a81772f..d1abc7d6c80 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -408,7 +408,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks /// Memory should not be limited during ATTACH TABLE query. /// This is already true at the server startup but must be also ensured for manual table ATTACH. /// Motivation: memory for index is shared between queries - not belong to the query itself. - MemoryTracker::BlockerInThread temporarily_disable_memory_tracker; + MemoryTracker::BlockerInThread temporarily_disable_memory_tracker(VariableContext::Global); loadUUID(); loadColumns(require_columns_checksums); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 13bab7a00d9..3bc89053c74 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -387,7 +387,7 @@ static void appendBlock(const Block & from, Block & to) MutableColumnPtr last_col; try { - MemoryTracker::BlockerInThread temporarily_disable_memory_tracker(VariableContext::User); + MemoryTracker::BlockerInThread temporarily_disable_memory_tracker; for (size_t column_no = 0, columns = to.columns(); column_no < columns; ++column_no) { @@ -779,7 +779,7 @@ void StorageBuffer::writeBlockToDestination(const Block & block, StoragePtr tabl } auto destination_metadata_snapshot = table->getInMemoryMetadataPtr(); - MemoryTracker::BlockerInThread temporarily_disable_memory_tracker(VariableContext::User); + MemoryTracker::BlockerInThread temporarily_disable_memory_tracker; auto insert = std::make_shared(); insert->table_id = destination_id; From ba1921f38012219c972fa93d85b97f1e95323134 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 22:59:49 +0300 Subject: [PATCH 204/264] Minor changes --- src/Databases/DatabaseFactory.cpp | 1 + .../PostgreSQL/DatabasePostgreSQL.cpp | 2 + src/Databases/PostgreSQL/DatabasePostgreSQL.h | 4 +- .../PostgreSQLDictionarySource.cpp | 1 + .../PostgreSQL/PostgreSQLConnection.cpp | 26 ++++++++++ .../PostgreSQL/PostgreSQLConnection.h | 48 +++++++++++++++++++ src/Storages/StoragePostgreSQL.cpp | 1 + src/Storages/StoragePostgreSQL.h | 31 +----------- .../TableFunctionPostgreSQL.cpp | 1 + src/TableFunctions/TableFunctionPostgreSQL.h | 5 +- 10 files changed, 87 insertions(+), 33 deletions(-) create mode 100644 src/Storages/PostgreSQL/PostgreSQLConnection.cpp create mode 100644 src/Storages/PostgreSQL/PostgreSQLConnection.h diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 7bfb977e7cb..c184f4c9dbe 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -33,6 +33,7 @@ #if USE_LIBPQXX #include // Y_IGNORE +#include #endif namespace DB diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index d3ee726b84e..a41b87759f3 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -4,6 +4,8 @@ #include #include +#include +#include #include #include #include diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index 950481cb292..56ea6645f15 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -9,13 +9,15 @@ #include #include #include -#include namespace DB { class Context; +class PostgreSQLConnection; +using PostgreSQLConnectionPtr = std::shared_ptr; + /** Real-time access to table list and table structure from remote PostgreSQL. * All tables are created after pull-out structure from remote PostgreSQL. diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 67e959c34b0..c36402e450d 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include "readInvalidateQuery.h" #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.cpp b/src/Storages/PostgreSQL/PostgreSQLConnection.cpp new file mode 100644 index 00000000000..a95f99ee1e1 --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLConnection.cpp @@ -0,0 +1,26 @@ +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX +#include + + +namespace DB +{ + +PostgreSQLConnection::ConnectionPtr conn() +{ + checkUpdateConnection(); + return connection; +} + +void PostgreSQLConnection::checkUpdateConnection() +{ + if (!connection || !connection->is_open()) + connection = std::make_unique(connection_str); +} + +} + +#endif diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.h b/src/Storages/PostgreSQL/PostgreSQLConnection.h new file mode 100644 index 00000000000..3fa67bf7a1b --- /dev/null +++ b/src/Storages/PostgreSQL/PostgreSQLConnection.h @@ -0,0 +1,48 @@ +#pragma once + +#if !defined(ARCADIA_BUILD) +#include "config_core.h" +#endif + +#if USE_LIBPQXX +#include + + +namespace DB +{ + +/// Tiny connection class to make it more convenient to use. +/// Connection is not made until actually used. +class PostgreSQLConnection +{ + using ConnectionPtr = std::shared_ptr; + +public: + PostgreSQLConnection(const std::string & connection_str_) : connection_str(connection_str_) {} + PostgreSQLConnection(const PostgreSQLConnection &) = delete; + PostgreSQLConnection operator =(const PostgreSQLConnection &) = delete; + + ConnectionPtr conn() + { + checkUpdateConnection(); + return connection; + } + + std::string & conn_str() { return connection_str; } + +private: + ConnectionPtr connection; + std::string connection_str; + + void checkUpdateConnection() + { + if (!connection || !connection->is_open()) + connection = std::make_unique(connection_str); + } +}; + +using PostgreSQLConnectionPtr = std::shared_ptr; + +} + +#endif diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index c45336d1515..6fc128aa976 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 684c9cf2b17..8fc7a93b579 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -9,7 +9,7 @@ #include #include #include -#include "pqxx/pqxx" +#include namespace DB @@ -52,35 +52,6 @@ private: PostgreSQLConnectionPtr connection; }; - -/// Tiny connection class to make it more convenient to use. -/// Connection is not made until actually used. -class PostgreSQLConnection -{ -public: - PostgreSQLConnection(const std::string & connection_str_) : connection_str(connection_str_) {} - PostgreSQLConnection(const PostgreSQLConnection &) = delete; - PostgreSQLConnection operator =(const PostgreSQLConnection &) = delete; - - ConnectionPtr conn() - { - checkUpdateConnection(); - return connection; - } - - std::string & conn_str() { return connection_str; } - -private: - ConnectionPtr connection; - std::string connection_str; - - void checkUpdateConnection() - { - if (!connection || !connection->is_open()) - connection = std::make_unique(connection_str); - } -}; - } #endif diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index bbbc9742015..b6b6c40786c 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -10,6 +10,7 @@ #include #include "registerTableFunctions.h" #include +#include namespace DB diff --git a/src/TableFunctions/TableFunctionPostgreSQL.h b/src/TableFunctions/TableFunctionPostgreSQL.h index 7af01ecb053..e625cbd9bf6 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.h +++ b/src/TableFunctions/TableFunctionPostgreSQL.h @@ -5,13 +5,14 @@ #if USE_LIBPQXX #include -#include -#include "pqxx/pqxx" namespace DB { +class PostgreSQLConnection; +using PostgreSQLConnectionPtr = std::shared_ptr; + class TableFunctionPostgreSQL : public ITableFunction { public: From 52cfc1d110e1686545cf53152d7656716080f87e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 23:09:31 +0300 Subject: [PATCH 205/264] Minor changes --- src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp | 3 ++- src/Databases/PostgreSQL/FetchFromPostgreSQL.h | 4 +++- src/Storages/StoragePostgreSQL.cpp | 11 +++++------ src/Storages/StoragePostgreSQL.h | 1 - 4 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index b07c71d33a7..a12778159b3 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -24,7 +24,8 @@ namespace ErrorCodes } -std::shared_ptr fetchPostgreSQLTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls) +std::shared_ptr fetchPostgreSQLTableStructure( + std::shared_ptr connection, const String & postgres_table_name, bool use_nulls) { auto columns = NamesAndTypesList(); diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h index b0b103251a0..ada8a5f8920 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.h @@ -11,7 +11,9 @@ namespace DB { -std::shared_ptr fetchPostgreSQLTableStructure(ConnectionPtr connection, const String & postgres_table_name, bool use_nulls); +std::shared_ptr fetchPostgreSQLTableStructure( + std::shared_ptr connection, const String & postgres_table_name, bool use_nulls); + DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions); } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 6fc128aa976..305696c234a 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -206,9 +206,8 @@ public: static std::string clickhouseToPostgresArray(const Array & array_field, const DataTypePtr & data_type) { auto nested = typeid_cast(data_type.get())->getNestedType(); - ColumnPtr nested_column(createNested(nested)); - auto array_column{ColumnArray::create(nested_column)}; - const_cast(array_column.get())->insert(array_field); + auto array_column = ColumnArray::create(createNested(nested)); + array_column->insert(array_field); WriteBufferFromOwnString ostr; data_type->serializeAsText(*array_column, 0, ostr, FormatSettings{}); @@ -217,7 +216,7 @@ public: } - static ColumnPtr createNested(DataTypePtr nested) + static MutableColumnPtr createNested(DataTypePtr nested) { bool is_nullable = false; if (nested->isNullable()) @@ -227,7 +226,7 @@ public: } WhichDataType which(nested); - ColumnPtr nested_column; + MutableColumnPtr nested_column; if (which.isString() || which.isFixedString()) nested_column = ColumnString::create(); else if (which.isInt8() || which.isInt16()) nested_column = ColumnInt16::create(); else if (which.isUInt8() || which.isUInt16()) nested_column = ColumnUInt16::create(); @@ -263,7 +262,7 @@ public: throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Type conversion not supported"); if (is_nullable) - return ColumnNullable::create(nested_column, ColumnUInt8::create(nested_column->size(), 0)); + return ColumnNullable::create(std::move(nested_column), ColumnUInt8::create(nested_column->size(), 0)); return nested_column; } diff --git a/src/Storages/StoragePostgreSQL.h b/src/Storages/StoragePostgreSQL.h index 8fc7a93b579..8aebae5896b 100644 --- a/src/Storages/StoragePostgreSQL.h +++ b/src/Storages/StoragePostgreSQL.h @@ -17,7 +17,6 @@ namespace DB class PostgreSQLConnection; using PostgreSQLConnectionPtr = std::shared_ptr; -using ConnectionPtr = std::shared_ptr; class StoragePostgreSQL final : public ext::shared_ptr_helper, public IStorage { From b1b09ed5ae5ede2609de4a882e94a565a5e68b14 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 23:14:51 +0300 Subject: [PATCH 206/264] Fix SQL injection --- src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index a12778159b3..f81ea1ebe20 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -21,6 +21,7 @@ namespace DB namespace ErrorCodes { extern const int UNKNOWN_TABLE; + extern const int BAD_ARGUMENTS; } @@ -29,6 +30,13 @@ std::shared_ptr fetchPostgreSQLTableStructure( { auto columns = NamesAndTypesList(); + if (postgres_table_name.find('\'') != std::string::npos + || postgres_table_name.find('\\') != std::string::npos) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "PostgreSQL table name cannot contain single quote or backslash characters, passed {}", + postgres_table_name); + } + std::string query = fmt::format( "SELECT attname AS name, format_type(atttypid, atttypmod) AS type, " "attnotnull AS not_null, attndims AS dims " @@ -53,7 +61,7 @@ std::shared_ptr fetchPostgreSQLTableStructure( stream.complete(); tx.commit(); } - catch (pqxx::undefined_table const &) + catch (const pqxx::undefined_table &) { throw Exception(fmt::format( "PostgreSQL table {}.{} does not exist", From c16ecb0bdd30c45321b77186f9098a6fdaaad711 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 23:18:55 +0300 Subject: [PATCH 207/264] It feels more correct --- src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp index f81ea1ebe20..a302ec20594 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp @@ -85,8 +85,9 @@ DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint DataTypePtr res; /// Get rid of trailing '[]' for arrays - if (dimensions && type.ends_with("[]")) - type.resize(type.size() - 2); + if (dimensions) + while (type.ends_with("[]")) + type.resize(type.size() - 2); if (type == "smallint") res = std::make_shared(); @@ -127,7 +128,7 @@ DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint res = std::make_shared(); if (is_nullable) res = std::make_shared(res); - while (dimensions--) + while (--dimensions) res = std::make_shared(res); return res; From ef319bfa7559854e352e3336426c6fb8abe8c6bc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 15 Jan 2021 23:33:04 +0300 Subject: [PATCH 208/264] Slightly better --- .../PostgreSQL/DatabasePostgreSQL.cpp | 2 +- ....cpp => fetchPostgreSQLTableStructure.cpp} | 112 +++++++++--------- ...eSQL.h => fetchPostgreSQLTableStructure.h} | 2 - .../TableFunctionPostgreSQL.cpp | 2 +- 4 files changed, 58 insertions(+), 60 deletions(-) rename src/Databases/PostgreSQL/{FetchFromPostgreSQL.cpp => fetchPostgreSQLTableStructure.cpp} (96%) rename src/Databases/PostgreSQL/{FetchFromPostgreSQL.h => fetchPostgreSQLTableStructure.h} (76%) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index a41b87759f3..0c7cf01767b 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp similarity index 96% rename from src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp rename to src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index a302ec20594..bb1c09073b3 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_LIBPQXX @@ -25,6 +25,61 @@ namespace ErrorCodes } +static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions) +{ + DataTypePtr res; + + /// Get rid of trailing '[]' for arrays + if (dimensions) + while (type.ends_with("[]")) + type.resize(type.size() - 2); + + if (type == "smallint") + res = std::make_shared(); + else if (type == "integer") + res = std::make_shared(); + else if (type == "bigint") + res = std::make_shared(); + else if (type == "real") + res = std::make_shared(); + else if (type == "double precision") + res = std::make_shared(); + else if (type == "serial") + res = std::make_shared(); + else if (type == "bigserial") + res = std::make_shared(); + else if (type.starts_with("timestamp")) + res = std::make_shared(); + else if (type == "date") + res = std::make_shared(); + else if (type.starts_with("numeric")) + { + /// Numeric and decimal will both end up here as numeric. + res = DataTypeFactory::instance().get(type); + uint32_t precision = getDecimalPrecision(*res); + uint32_t scale = getDecimalScale(*res); + + if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + } + + if (!res) + res = std::make_shared(); + if (is_nullable) + res = std::make_shared(res); + while (--dimensions) + res = std::make_shared(res); + + return res; +} + + std::shared_ptr fetchPostgreSQLTableStructure( std::shared_ptr connection, const String & postgres_table_name, bool use_nulls) { @@ -79,61 +134,6 @@ std::shared_ptr fetchPostgreSQLTableStructure( return std::make_shared(columns); } - -DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions) -{ - DataTypePtr res; - - /// Get rid of trailing '[]' for arrays - if (dimensions) - while (type.ends_with("[]")) - type.resize(type.size() - 2); - - if (type == "smallint") - res = std::make_shared(); - else if (type == "integer") - res = std::make_shared(); - else if (type == "bigint") - res = std::make_shared(); - else if (type == "real") - res = std::make_shared(); - else if (type == "double precision") - res = std::make_shared(); - else if (type == "serial") - res = std::make_shared(); - else if (type == "bigserial") - res = std::make_shared(); - else if (type.starts_with("timestamp")) - res = std::make_shared(); - else if (type == "date") - res = std::make_shared(); - else if (type.starts_with("numeric")) - { - /// Numeric and decimal will both end up here as numeric. - res = DataTypeFactory::instance().get(type); - uint32_t precision = getDecimalPrecision(*res); - uint32_t scale = getDecimalScale(*res); - - if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - } - - if (!res) - res = std::make_shared(); - if (is_nullable) - res = std::make_shared(res); - while (--dimensions) - res = std::make_shared(res); - - return res; -} - } #endif diff --git a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h similarity index 76% rename from src/Databases/PostgreSQL/FetchFromPostgreSQL.h rename to src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index ada8a5f8920..bbbb379541b 100644 --- a/src/Databases/PostgreSQL/FetchFromPostgreSQL.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -14,8 +14,6 @@ namespace DB std::shared_ptr fetchPostgreSQLTableStructure( std::shared_ptr connection, const String & postgres_table_name, bool use_nulls); -DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullable, uint16_t dimensions); - } #endif diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index b6b6c40786c..a96492012a6 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -9,7 +9,7 @@ #include #include #include "registerTableFunctions.h" -#include +#include #include From 979d23208ee80c341537ac783bd44fd640608c3d Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Sat, 16 Jan 2021 00:15:06 +0300 Subject: [PATCH 209/264] speed up some perf tests (for other machines) --- tests/performance/any_anyLast.xml | 8 ++++---- tests/performance/date_time_short.xml | 16 ++++++++-------- tests/performance/general_purpose_hashes.xml | 2 +- tests/performance/string_sort.xml | 2 +- tests/performance/website.xml | 2 +- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/performance/any_anyLast.xml b/tests/performance/any_anyLast.xml index 659df4a22ac..05045a93b12 100644 --- a/tests/performance/any_anyLast.xml +++ b/tests/performance/any_anyLast.xml @@ -1,6 +1,6 @@ - SELECT any(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000000) - SELECT anyLast(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000000) - SELECT any(number * 2) as n, n * 3 FROM numbers(500000000) - SELECT any(number * round(toInt64(number), -2)) FROM numbers(500000000) + SELECT any(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(200000000) + SELECT anyLast(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(200000000) + SELECT any(number * 2) as n, n * 3 FROM numbers(200000000) + SELECT any(number * round(toInt64(number), -2)) FROM numbers(200000000) diff --git a/tests/performance/date_time_short.xml b/tests/performance/date_time_short.xml index c677909d9c2..826e1619ab7 100644 --- a/tests/performance/date_time_short.xml +++ b/tests/performance/date_time_short.xml @@ -33,15 +33,15 @@ - SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {date_transform}(t, '{time_zone}')) - SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, {date_transform}(t)) + SELECT count() FROM numbers(50000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {date_transform}(t, '{time_zone}')) + SELECT count() FROM numbers(50000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, {date_transform}(t)) - SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toUnixTimestamp(t, '{time_zone}')) - - SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, toUnixTimestamp(toUInt16(t))) + SELECT count() FROM numbers(50000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toUnixTimestamp(t, '{time_zone}')) + + SELECT count() FROM numbers(50000000) WHERE NOT ignore(toDate('2017-01-01') + number % 1000 + rand() % 10 AS t, toUnixTimestamp(toUInt16(t))) - SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {binary_function}(t, 1)) - SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toStartOfInterval(t, INTERVAL 1 month)) - SELECT count() FROM numbers(100000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, date_trunc('month', t)) + SELECT count() FROM numbers(50000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, {binary_function}(t, 1)) + SELECT count() FROM numbers(50000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, toStartOfInterval(t, INTERVAL 1 month)) + SELECT count() FROM numbers(50000000) WHERE NOT ignore(toDateTime('2017-01-01 00:00:00') + number % 100000000 + rand() % 100000 AS t, date_trunc('month', t)) diff --git a/tests/performance/general_purpose_hashes.xml b/tests/performance/general_purpose_hashes.xml index c54e1c2fb02..f34554360cf 100644 --- a/tests/performance/general_purpose_hashes.xml +++ b/tests/performance/general_purpose_hashes.xml @@ -41,7 +41,7 @@ table - numbers(100000000) + numbers(10000000) numbers_mt(500000000) diff --git a/tests/performance/string_sort.xml b/tests/performance/string_sort.xml index 23ec8cf3144..335ccbe38bb 100644 --- a/tests/performance/string_sort.xml +++ b/tests/performance/string_sort.xml @@ -43,7 +43,7 @@ - + diff --git a/tests/performance/website.xml b/tests/performance/website.xml index ae497295ddb..06148850470 100644 --- a/tests/performance/website.xml +++ b/tests/performance/website.xml @@ -35,7 +35,7 @@ SELECT SearchPhrase, uniq(UserID) AS u FROM {table} WHERE SearchPhrase != '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10 SELECT SearchEngineID, SearchPhrase, count() AS c FROM {table} WHERE SearchPhrase != '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10 SELECT UserID, count() FROM {table} GROUP BY UserID ORDER BY count() DESC LIMIT 10 -SELECT UserID, SearchPhrase, count() FROM {table} GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10 +SELECT UserID, SearchPhrase, count() FROM hits_10m_single GROUP BY UserID, SearchPhrase ORDER BY count() DESC LIMIT 10 SELECT UserID, SearchPhrase, count() FROM hits_10m_single GROUP BY UserID, SearchPhrase LIMIT 10 SELECT UserID, toMinute(EventTime) AS m, SearchPhrase, count() FROM hits_10m_single GROUP BY UserID, m, SearchPhrase ORDER BY count() DESC LIMIT 10 SELECT count() FROM hits_100m_single WHERE UserID = 12345678901234567890 From 67f207d8cafcaa3be3488d1b67f05b79b96b2250 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 00:18:51 +0300 Subject: [PATCH 210/264] More safe connection string --- src/CMakeLists.txt | 4 ++++ src/Databases/DatabaseFactory.cpp | 8 +++----- .../PostgreSQLDictionarySource.cpp | 3 +-- .../PostgreSQL/PostgreSQLConnection.cpp | 16 ++++++++++++++- .../PostgreSQL/PostgreSQLConnection.h | 20 +++++++++---------- src/Storages/StoragePostgreSQL.cpp | 13 ++++++------ .../TableFunctionPostgreSQL.cpp | 12 +++++------ 7 files changed, 45 insertions(+), 31 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 364d79d7d8c..eae9037dc26 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -96,6 +96,10 @@ if (USE_HDFS) add_headers_and_sources(dbms Storages/HDFS) endif() +if (USE_LIBPQXX) + add_headers_and_sources(dbms Storages/PostgreSQL) +endif() + list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index c184f4c9dbe..c61472426a8 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -203,12 +203,10 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String use_table_cache = safeGetLiteralValue(engine_args[4], engine_name); auto parsed_host_port = parseAddress(host_port, 5432); - String connection_str; - connection_str = fmt::format("dbname={} host={} port={} user={} password={}", - postgres_database_name, parsed_host_port.first, std::to_string(parsed_host_port.second), - username, password); + /// no connection is made here - auto connection = std::make_shared(connection_str); + auto connection = std::make_shared( + postgres_database_name, parsed_host_port.first, parsed_host_port.second, username, password); return std::make_shared( context, metadata_path, engine_define, database_name, postgres_database_name, connection, use_table_cache); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index c36402e450d..aa852404750 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -172,13 +172,12 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) { #if USE_LIBPQXX const auto config_prefix = root_config_prefix + ".postgresql"; - auto connection_str = fmt::format("dbname={} host={} port={} user={} password={}", + auto connection = std::make_shared( config.getString(fmt::format("{}.db", config_prefix), ""), config.getString(fmt::format("{}.host", config_prefix), ""), config.getUInt(fmt::format("{}.port", config_prefix), 0), config.getString(fmt::format("{}.user", config_prefix), ""), config.getString(fmt::format("{}.password", config_prefix), "")); - auto connection = std::make_shared(connection_str); return std::make_unique( dict_struct, config, config_prefix, connection, sample_block); diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.cpp b/src/Storages/PostgreSQL/PostgreSQLConnection.cpp index a95f99ee1e1..668550ec721 100644 --- a/src/Storages/PostgreSQL/PostgreSQLConnection.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLConnection.cpp @@ -4,12 +4,14 @@ #if USE_LIBPQXX #include +#include +#include namespace DB { -PostgreSQLConnection::ConnectionPtr conn() +PostgreSQLConnection::ConnectionPtr PostgreSQLConnection::conn() { checkUpdateConnection(); return connection; @@ -21,6 +23,18 @@ void PostgreSQLConnection::checkUpdateConnection() connection = std::make_unique(connection_str); } +std::string PostgreSQLConnection::formatConnectionString( + std::string dbname, std::string host, UInt16 port, std::string user, std::string password) +{ + WriteBufferFromOwnString out; + out << "dbname=" << quote << dbname + << " host=" << quote << host + << " port=" << port + << " user=" << quote << user + << " password=" << quote << password; + return out.str(); +} + } #endif diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.h b/src/Storages/PostgreSQL/PostgreSQLConnection.h index 3fa67bf7a1b..04ba1909803 100644 --- a/src/Storages/PostgreSQL/PostgreSQLConnection.h +++ b/src/Storages/PostgreSQL/PostgreSQLConnection.h @@ -6,6 +6,7 @@ #if USE_LIBPQXX #include +#include namespace DB @@ -18,15 +19,15 @@ class PostgreSQLConnection using ConnectionPtr = std::shared_ptr; public: + PostgreSQLConnection(std::string dbname, std::string host, UInt16 port, std::string user, std::string password) + : connection_str(formatConnectionString(std::move(dbname), std::move(host), port, std::move(user), std::move(password))) {} + PostgreSQLConnection(const std::string & connection_str_) : connection_str(connection_str_) {} + PostgreSQLConnection(const PostgreSQLConnection &) = delete; PostgreSQLConnection operator =(const PostgreSQLConnection &) = delete; - ConnectionPtr conn() - { - checkUpdateConnection(); - return connection; - } + ConnectionPtr conn(); std::string & conn_str() { return connection_str; } @@ -34,11 +35,10 @@ private: ConnectionPtr connection; std::string connection_str; - void checkUpdateConnection() - { - if (!connection || !connection->is_open()) - connection = std::make_unique(connection_str); - } + static std::string formatConnectionString( + std::string dbname, std::string host, UInt16 port, std::string user, std::string password); + + void checkUpdateConnection(); }; using PostgreSQLConnectionPtr = std::shared_ptr; diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 305696c234a..78d3351e2f0 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -302,14 +302,13 @@ void registerStoragePostgreSQL(StorageFactory & factory) auto parsed_host_port = parseAddress(engine_args[0]->as().value.safeGet(), 5432); const String & remote_table = engine_args[2]->as().value.safeGet(); - String connection_str; - connection_str = fmt::format("dbname={} host={} port={} user={} password={}", - engine_args[1]->as().value.safeGet(), - parsed_host_port.first, std::to_string(parsed_host_port.second), - engine_args[3]->as().value.safeGet(), - engine_args[4]->as().value.safeGet()); + auto connection = std::make_shared( + engine_args[1]->as().value.safeGet(), + parsed_host_port.first, + parsed_host_port.second, + engine_args[3]->as().value.safeGet(), + engine_args[4]->as().value.safeGet()); - auto connection = std::make_shared(connection_str); return StoragePostgreSQL::create( args.table_id, remote_table, connection, args.columns, args.constraints, args.context); }, diff --git a/src/TableFunctions/TableFunctionPostgreSQL.cpp b/src/TableFunctions/TableFunctionPostgreSQL.cpp index a96492012a6..eefdff1fa87 100644 --- a/src/TableFunctions/TableFunctionPostgreSQL.cpp +++ b/src/TableFunctions/TableFunctionPostgreSQL.cpp @@ -65,12 +65,12 @@ void TableFunctionPostgreSQL::parseArguments(const ASTPtr & ast_function, const auto parsed_host_port = parseAddress(args[0]->as().value.safeGet(), 5432); remote_table_name = args[2]->as().value.safeGet(); - connection_str = fmt::format("dbname={} host={} port={} user={} password={}", - args[1]->as().value.safeGet(), - parsed_host_port.first, std::to_string(parsed_host_port.second), - args[3]->as().value.safeGet(), - args[4]->as().value.safeGet()); - connection = std::make_shared(connection_str); + connection = std::make_shared( + args[1]->as().value.safeGet(), + parsed_host_port.first, + parsed_host_port.second, + args[3]->as().value.safeGet(), + args[4]->as().value.safeGet()); } From 7705420220cdd774d4f8d7736053bed91263ab42 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 00:23:58 +0300 Subject: [PATCH 211/264] Better --- src/CMakeLists.txt | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index eae9037dc26..4207d371c09 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -81,6 +81,7 @@ endif() if (USE_LIBPQXX) add_headers_and_sources(dbms Databases/PostgreSQL) + add_headers_and_sources(dbms Storages/PostgreSQL) endif() if (USE_ROCKSDB) @@ -96,10 +97,6 @@ if (USE_HDFS) add_headers_and_sources(dbms Storages/HDFS) endif() -if (USE_LIBPQXX) - add_headers_and_sources(dbms Storages/PostgreSQL) -endif() - list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) From 593b6f5c83f599e3ffe82aced4af8e05fa3a7c9c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 00:25:58 +0300 Subject: [PATCH 212/264] Add assert --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 0c7cf01767b..944675f860e 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -382,6 +382,7 @@ ASTPtr DatabasePostgreSQL::getCreateTableQueryImpl(const String & table_name, co storage_engine_arguments->children.resize(storage_engine_arguments->children.size() - 1); /// Add table_name to engine arguments + assert(storage_engine_arguments->children.size() >= 2); storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, std::make_shared(table_id.table_name)); return create_table_query; From 112746e8ca3409a761afce68cd2fbd2fe8ad6dc3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 16 Jan 2021 00:42:21 +0300 Subject: [PATCH 213/264] Update version_date.tsv after release 20.8.12.2 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 1279da09905..cf6778317cc 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -19,6 +19,7 @@ v20.9.5.5-stable 2020-11-13 v20.9.4.76-stable 2020-10-29 v20.9.3.45-stable 2020-10-09 v20.9.2.20-stable 2020-09-22 +v20.8.12.2-lts 2021-01-16 v20.8.11.17-lts 2020-12-25 v20.8.10.13-lts 2020-12-24 v20.8.9.6-lts 2020-12-10 From 7135faf672089b5f03ba2b465f5f5f4ed3fcdd2f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 14 Jan 2021 22:45:04 +0300 Subject: [PATCH 214/264] Cover distributed send with different headers --- .../configs/remote_servers.xml | 18 +++++- .../test.py | 60 +++++++++++++++++-- 2 files changed, 70 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml b/tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml index 2211d599b25..5b7188368f9 100644 --- a/tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml +++ b/tests/integration/test_insert_distributed_async_send/configs/remote_servers.xml @@ -1,6 +1,6 @@ - + false @@ -12,7 +12,21 @@ 9000 - + + + + + n1 + 9000 + + + + + n2 + 9000 + + + diff --git a/tests/integration/test_insert_distributed_async_send/test.py b/tests/integration/test_insert_distributed_async_send/test.py index 55a9c90fefd..7f6a2887c3b 100644 --- a/tests/integration/test_insert_distributed_async_send/test.py +++ b/tests/integration/test_insert_distributed_async_send/test.py @@ -31,16 +31,16 @@ def start_cluster(): cluster.shutdown() -def create_tables(): +def create_tables(remote_cluster_name): for _, instance in list(cluster.instances.items()): - instance.query('CREATE TABLE data (key Int, value String) Engine=Memory()') - instance.query(""" + instance.query('CREATE TABLE data (key Int, value String) Engine=MergeTree() ORDER BY key') + instance.query(f""" CREATE TABLE dist AS data Engine=Distributed( - insert_distributed_async_send_cluster, + {remote_cluster_name}, currentDatabase(), data, - rand() + key ) """) # only via SYSTEM FLUSH DISTRIBUTED @@ -69,7 +69,7 @@ def get_node(batch): def bootstrap(batch): drop_tables() - create_tables() + create_tables('insert_distributed_async_send_cluster_two_replicas') return insert_data(get_node(batch)) def get_path_to_dist_batch(file='2.bin'): @@ -163,3 +163,51 @@ def test_insert_distributed_async_send_corrupted_small(batch): node.exec_in_container(['bash', '-c', f'mv {path} /tmp/bin && head -c {from_original_size} /tmp/bin > {path} && head -c {zeros_size} /dev/zero >> {path}']) check_dist_after_corruption(False, batch) + +@batch_params +def test_insert_distributed_async_send_different_header(batch): + """ + Check INSERT Into Distributed() with different headers in *.bin + If batching will not distinguish headers underlying table will never receive the data. + """ + + drop_tables() + create_tables('insert_distributed_async_send_cluster_two_shards') + + node = get_node(batch) + node.query("INSERT INTO dist VALUES (0, '')", settings={ + 'prefer_localhost_replica': 0, + }) + node.query('ALTER TABLE dist MODIFY COLUMN value Nullable(String)') + node.query("INSERT INTO dist VALUES (2, '')", settings={ + 'prefer_localhost_replica': 0, + }) + + if batch: + # first batch with Nullable(String) + n1.query('ALTER TABLE data MODIFY COLUMN value Nullable(String)', settings={ + 'mutations_sync': 1, + }) + # but only one batch will be sent + with pytest.raises(QueryRuntimeException, match=r"DB::Exception: Cannot convert: String to Nullable\(String\)\. Stack trace:"): + node.query('SYSTEM FLUSH DISTRIBUTED dist') + assert int(n1.query('SELECT count() FROM data')) == 1 + # second batch with String + n1.query('ALTER TABLE data MODIFY COLUMN value String', settings={ + 'mutations_sync': 1, + }) + node.query('SYSTEM FLUSH DISTRIBUTED dist') + assert int(n1.query('SELECT count() FROM data')) == 2 + else: + # first send with String + with pytest.raises(QueryRuntimeException, match=r"DB::Exception: Cannot convert: Nullable\(String\) to String\. Stack trace:"): + node.query('SYSTEM FLUSH DISTRIBUTED dist') + assert int(n1.query('SELECT count() FROM data')) == 1 + # second send with Nullable(String) + n1.query('ALTER TABLE data MODIFY COLUMN value Nullable(String)', settings={ + 'mutations_sync': 1, + }) + node.query('SYSTEM FLUSH DISTRIBUTED dist') + assert int(n1.query('SELECT count() FROM data')) == 2 + + assert int(n2.query('SELECT count() FROM data')) == 0 From cf085b0687d72ec9e9f3a49be0f4084e83b5d420 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 15 Jan 2021 23:07:25 +0300 Subject: [PATCH 215/264] Split RemoteQueryExecutorReadContext into module part --- src/Client/MultiplexedConnections.h | 2 +- src/Common/FiberStack.h | 1 + .../RemoteQueryExecutorReadContext.cpp | 234 +++++++++++++++++ .../RemoteQueryExecutorReadContext.h | 239 ++---------------- src/DataStreams/ya.make | 1 + 5 files changed, 258 insertions(+), 219 deletions(-) create mode 100644 src/DataStreams/RemoteQueryExecutorReadContext.cpp diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 46312ae339d..2ab2b60570e 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -106,7 +106,7 @@ private: /// in separate thread. mutable std::mutex cancel_mutex; - friend class RemoteQueryExecutorReadContext; + friend struct RemoteQueryExecutorRoutine; }; } diff --git a/src/Common/FiberStack.h b/src/Common/FiberStack.h index b54b1a7de10..c509540cc9e 100644 --- a/src/Common/FiberStack.h +++ b/src/Common/FiberStack.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/DataStreams/RemoteQueryExecutorReadContext.cpp new file mode 100644 index 00000000000..d7be1218196 --- /dev/null +++ b/src/DataStreams/RemoteQueryExecutorReadContext.cpp @@ -0,0 +1,234 @@ +#if defined(OS_LINUX) + +#include +#include +#include +#include +#include + +namespace DB +{ + +struct RemoteQueryExecutorRoutine +{ + MultiplexedConnections & connections; + RemoteQueryExecutorReadContext & read_context; + + struct ReadCallback + { + RemoteQueryExecutorReadContext & read_context; + Fiber & fiber; + + void operator()(Poco::Net::Socket & socket) + { + try + { + read_context.setSocket(socket); + } + catch (DB::Exception & e) + { + e.addMessage(" while reading from socket ({})", socket.peerAddress().toString()); + throw; + } + + read_context.is_read_in_progress.store(true, std::memory_order_relaxed); + fiber = std::move(fiber).resume(); + read_context.is_read_in_progress.store(false, std::memory_order_relaxed); + } + }; + + Fiber operator()(Fiber && sink) const + { + try + { + while (true) + { + read_context.packet = connections.receivePacketUnlocked(ReadCallback{read_context, sink}); + sink = std::move(sink).resume(); + } + } + catch (const boost::context::detail::forced_unwind &) + { + /// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited + /// It should not be caught or it will segfault. + /// Other exceptions must be caught + throw; + } + catch (...) + { + read_context.exception = std::current_exception(); + } + + return std::move(sink); + } +}; + +namespace ErrorCodes +{ + extern const int CANNOT_READ_FROM_SOCKET; + extern const int CANNOT_OPEN_FILE; + extern const int SOCKET_TIMEOUT; +} + +RemoteQueryExecutorReadContext::RemoteQueryExecutorReadContext(MultiplexedConnections & connections_) + : connections(connections_) +{ + epoll_fd = epoll_create(2); + if (-1 == epoll_fd) + throwFromErrno("Cannot create epoll descriptor", ErrorCodes::CANNOT_OPEN_FILE); + + if (-1 == pipe2(pipe_fd, O_NONBLOCK)) + throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE); + + { + epoll_event socket_event; + socket_event.events = EPOLLIN | EPOLLPRI; + socket_event.data.fd = pipe_fd[0]; + + if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, pipe_fd[0], &socket_event)) + throwFromErrno("Cannot add pipe descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + } + + { + epoll_event timer_event; + timer_event.events = EPOLLIN | EPOLLPRI; + timer_event.data.fd = timer.getDescriptor(); + + if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, timer_event.data.fd, &timer_event)) + throwFromErrno("Cannot add timer descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + } + + auto routine = RemoteQueryExecutorRoutine{connections, *this}; + fiber = boost::context::fiber(std::allocator_arg_t(), stack, std::move(routine)); +} + +void RemoteQueryExecutorReadContext::setSocket(Poco::Net::Socket & socket) +{ + int fd = socket.impl()->sockfd(); + if (fd == socket_fd) + return; + + epoll_event socket_event; + socket_event.events = EPOLLIN | EPOLLPRI; + socket_event.data.fd = fd; + + if (socket_fd != -1) + { + if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_DEL, socket_fd, &socket_event)) + throwFromErrno("Cannot remove socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + } + + socket_fd = fd; + + if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, socket_fd, &socket_event)) + throwFromErrno("Cannot add socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + + receive_timeout = socket.impl()->getReceiveTimeout(); +} + +bool RemoteQueryExecutorReadContext::checkTimeout() const +{ + try + { + return checkTimeoutImpl(); + } + catch (DB::Exception & e) + { + if (last_used_socket) + e.addMessage(" while reading from socket ({})", last_used_socket->peerAddress().toString()); + throw; + } +} + +bool RemoteQueryExecutorReadContext::checkTimeoutImpl() const +{ + epoll_event events[3]; + events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; + + /// Wait for epoll_fd will not block if it was polled externally. + int num_events = epoll_wait(epoll_fd, events, 3, 0); + if (num_events == -1) + throwFromErrno("Failed to epoll_wait", ErrorCodes::CANNOT_READ_FROM_SOCKET); + + bool is_socket_ready = false; + bool is_pipe_alarmed = false; + bool has_timer_alarm = false; + + for (int i = 0; i < num_events; ++i) + { + if (events[i].data.fd == socket_fd) + is_socket_ready = true; + if (events[i].data.fd == timer.getDescriptor()) + has_timer_alarm = true; + if (events[i].data.fd == pipe_fd[0]) + is_pipe_alarmed = true; + } + + if (is_pipe_alarmed) + return false; + + if (has_timer_alarm && !is_socket_ready) + { + /// Socket receive timeout. Drain it in case or error, or it may be hide by timeout exception. + timer.drain(); + throw NetException("Timeout exceeded", ErrorCodes::SOCKET_TIMEOUT); + } + + return true; +} + +void RemoteQueryExecutorReadContext::setTimer() const +{ + /// Did not get packet yet. Init timeout for the next async reading. + timer.reset(); + + if (receive_timeout.totalMicroseconds()) + timer.setRelative(receive_timeout); +} + +bool RemoteQueryExecutorReadContext::resumeRoutine() +{ + if (is_read_in_progress.load(std::memory_order_relaxed) && !checkTimeout()) + return false; + + { + std::lock_guard guard(fiber_lock); + if (!fiber) + return false; + + fiber = std::move(fiber).resume(); + } + + if (exception) + std::rethrow_exception(std::move(exception)); + + return true; +} + +void RemoteQueryExecutorReadContext::cancel() +{ + std::lock_guard guard(fiber_lock); + /// It is safe to just destroy fiber - we are not in the process of reading from socket. + boost::context::fiber to_destroy = std::move(fiber); + + /// Send something to pipe to cancel executor waiting. + uint64_t buf = 0; + while (-1 == write(pipe_fd[1], &buf, sizeof(buf))) + { + if (errno == EAGAIN) + break; + + if (errno != EINTR) + throwFromErrno("Cannot write to pipe", ErrorCodes::CANNOT_READ_FROM_SOCKET); + } +} + +RemoteQueryExecutorReadContext::~RemoteQueryExecutorReadContext() +{ + /// socket_fd is closed by Poco::Net::Socket + /// timer_fd is closed by TimerDescriptor + close(epoll_fd); +} + +} +#endif diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.h b/src/DataStreams/RemoteQueryExecutorReadContext.h index 6d7099899ae..0eb90d73924 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.h +++ b/src/DataStreams/RemoteQueryExecutorReadContext.h @@ -2,26 +2,27 @@ #if defined(OS_LINUX) -#include +#include +#include #include #include #include +#include +#include + +namespace Poco::Net +{ +class Socket; +} namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_READ_FROM_SOCKET; - extern const int CANNOT_OPEN_FILE; - extern const int SOCKET_TIMEOUT; -} +class MultiplexedConnections; class RemoteQueryExecutorReadContext { public: - using Self = RemoteQueryExecutorReadContext; - std::atomic_bool is_read_in_progress = false; Packet packet; @@ -45,221 +46,23 @@ public: int epoll_fd; int pipe_fd[2]; - explicit RemoteQueryExecutorReadContext(MultiplexedConnections & connections_) : connections(connections_) - { - epoll_fd = epoll_create(2); - if (-1 == epoll_fd) - throwFromErrno("Cannot create epoll descriptor", ErrorCodes::CANNOT_OPEN_FILE); + explicit RemoteQueryExecutorReadContext(MultiplexedConnections & connections_); + ~RemoteQueryExecutorReadContext(); - if (-1 == pipe2(pipe_fd, O_NONBLOCK)) - throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE); + bool checkTimeout() const; + bool checkTimeoutImpl() const; - { - epoll_event socket_event; - socket_event.events = EPOLLIN | EPOLLPRI; - socket_event.data.fd = pipe_fd[0]; + void setSocket(Poco::Net::Socket & socket); + void setTimer() const; - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, pipe_fd[0], &socket_event)) - throwFromErrno("Cannot add pipe descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); - } - - { - epoll_event timer_event; - timer_event.events = EPOLLIN | EPOLLPRI; - timer_event.data.fd = timer.getDescriptor(); - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, timer_event.data.fd, &timer_event)) - throwFromErrno("Cannot add timer descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); - } - - auto routine = Routine{connections, *this}; - fiber = boost::context::fiber(std::allocator_arg_t(), stack, std::move(routine)); - } - - void setSocket(Poco::Net::Socket & socket) - { - int fd = socket.impl()->sockfd(); - if (fd == socket_fd) - return; - - epoll_event socket_event; - socket_event.events = EPOLLIN | EPOLLPRI; - socket_event.data.fd = fd; - - if (socket_fd != -1) - { - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_DEL, socket_fd, &socket_event)) - throwFromErrno("Cannot remove socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); - } - - socket_fd = fd; - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, socket_fd, &socket_event)) - throwFromErrno("Cannot add socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); - - receive_timeout = socket.impl()->getReceiveTimeout(); - } - - bool checkTimeout() const - { - try - { - return checkTimeoutImpl(); - } - catch (DB::Exception & e) - { - if (last_used_socket) - e.addMessage(" while reading from socket ({})", last_used_socket->peerAddress().toString()); - throw; - } - } - - bool checkTimeoutImpl() const - { - epoll_event events[3]; - events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; - - /// Wait for epoll_fd will not block if it was polled externally. - int num_events = epoll_wait(epoll_fd, events, 3, 0); - if (num_events == -1) - throwFromErrno("Failed to epoll_wait", ErrorCodes::CANNOT_READ_FROM_SOCKET); - - bool is_socket_ready = false; - bool is_pipe_alarmed = false; - bool has_timer_alarm = false; - - for (int i = 0; i < num_events; ++i) - { - if (events[i].data.fd == socket_fd) - is_socket_ready = true; - if (events[i].data.fd == timer.getDescriptor()) - has_timer_alarm = true; - if (events[i].data.fd == pipe_fd[0]) - is_pipe_alarmed = true; - } - - if (is_pipe_alarmed) - return false; - - if (has_timer_alarm && !is_socket_ready) - { - /// Socket receive timeout. Drain it in case or error, or it may be hide by timeout exception. - timer.drain(); - throw NetException("Timeout exceeded", ErrorCodes::SOCKET_TIMEOUT); - } - - return true; - } - - void setTimer() const - { - /// Did not get packet yet. Init timeout for the next async reading. - timer.reset(); - - if (receive_timeout.totalMicroseconds()) - timer.setRelative(receive_timeout); - } - - bool resumeRoutine() - { - if (is_read_in_progress.load(std::memory_order_relaxed) && !checkTimeout()) - return false; - - { - std::lock_guard guard(fiber_lock); - if (!fiber) - return false; - - fiber = std::move(fiber).resume(); - } - - if (exception) - std::rethrow_exception(std::move(exception)); - - return true; - } - - void cancel() - { - std::lock_guard guard(fiber_lock); - /// It is safe to just destroy fiber - we are not in the process of reading from socket. - boost::context::fiber to_destroy = std::move(fiber); - - /// Send something to pipe to cancel executor waiting. - uint64_t buf = 0; - while (-1 == write(pipe_fd[1], &buf, sizeof(buf))) - { - if (errno == EAGAIN) - break; - - if (errno != EINTR) - throwFromErrno("Cannot write to pipe", ErrorCodes::CANNOT_READ_FROM_SOCKET); - } - } - - ~RemoteQueryExecutorReadContext() - { - /// socket_fd is closed by Poco::Net::Socket - /// timer_fd is closed by TimerDescriptor - close(epoll_fd); - } - - struct Routine - { - MultiplexedConnections & connections; - Self & read_context; - - struct ReadCallback - { - Self & read_context; - Fiber & fiber; - - void operator()(Poco::Net::Socket & socket) - { - try - { - read_context.setSocket(socket); - } - catch (DB::Exception & e) - { - e.addMessage(" while reading from socket ({})", socket.peerAddress().toString()); - throw; - } - - read_context.is_read_in_progress.store(true, std::memory_order_relaxed); - fiber = std::move(fiber).resume(); - read_context.is_read_in_progress.store(false, std::memory_order_relaxed); - } - }; - - Fiber operator()(Fiber && sink) const - { - try - { - while (true) - { - read_context.packet = connections.receivePacketUnlocked(ReadCallback{read_context, sink}); - sink = std::move(sink).resume(); - } - } - catch (const boost::context::detail::forced_unwind &) - { - /// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited - /// It should not be caught or it will segfault. - /// Other exceptions must be caught - throw; - } - catch (...) - { - read_context.exception = std::current_exception(); - } - - return std::move(sink); - } - }; + bool resumeRoutine(); + void cancel(); }; + } + #else + namespace DB { class RemoteQueryExecutorReadContext diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index 8648f233f26..0af72f25d3d 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -39,6 +39,7 @@ SRCS( RemoteBlockInputStream.cpp RemoteBlockOutputStream.cpp RemoteQueryExecutor.cpp + RemoteQueryExecutorReadContext.cpp SizeLimits.cpp SquashingBlockInputStream.cpp SquashingBlockOutputStream.cpp From ee45c122eacdd3c172fae47f8bb0002d54f3091e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 15 Jan 2021 23:14:49 +0300 Subject: [PATCH 216/264] Fix leaking of pipe fd for async_socket_for_remote --- src/DataStreams/RemoteQueryExecutorReadContext.cpp | 8 ++++++-- src/DataStreams/RemoteQueryExecutorReadContext.h | 4 ++-- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/DataStreams/RemoteQueryExecutorReadContext.cpp index d7be1218196..bc47b049407 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/DataStreams/RemoteQueryExecutorReadContext.cpp @@ -226,8 +226,12 @@ void RemoteQueryExecutorReadContext::cancel() RemoteQueryExecutorReadContext::~RemoteQueryExecutorReadContext() { /// socket_fd is closed by Poco::Net::Socket - /// timer_fd is closed by TimerDescriptor - close(epoll_fd); + if (epoll_fd != -1) + close(epoll_fd); + if (pipe_fd[0] != -1) + close(pipe_fd[0]); + if (pipe_fd[1] != -1) + close(pipe_fd[1]); } } diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.h b/src/DataStreams/RemoteQueryExecutorReadContext.h index 0eb90d73924..89dc2813a9a 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.h +++ b/src/DataStreams/RemoteQueryExecutorReadContext.h @@ -43,8 +43,8 @@ public: /// We put those descriptors into our own epoll_fd which is used by external executor. TimerDescriptor timer{CLOCK_MONOTONIC, 0}; int socket_fd = -1; - int epoll_fd; - int pipe_fd[2]; + int epoll_fd = -1; + int pipe_fd[2] = { -1, -1 }; explicit RemoteQueryExecutorReadContext(MultiplexedConnections & connections_); ~RemoteQueryExecutorReadContext(); From 11ffa6293dca26581422094b7b5e621da1bb521a Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Jan 2021 08:04:24 +0000 Subject: [PATCH 217/264] Fix integration tests --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index bb1c09073b3..ec23cfc8794 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -73,7 +73,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl res = std::make_shared(); if (is_nullable) res = std::make_shared(res); - while (--dimensions) + while (dimensions--) res = std::make_shared(res); return res; From 31593e200019ea8e07420b713f5997dde1b5eb61 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 11:10:55 +0300 Subject: [PATCH 218/264] Fix Arcadia --- src/Interpreters/ya.make | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index d6671b0973c..77ca6bc0e14 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -158,8 +158,8 @@ SRCS( interpretSubquery.cpp join_common.cpp loadMetadata.cpp - replaceAliasColumnsInQuery.cpp processColumnTransformers.cpp + replaceAliasColumnsInQuery.cpp sortBlock.cpp ) From 45380d45c8140f513b3ea4470cadb5865d4ed08b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 11:15:43 +0300 Subject: [PATCH 219/264] Fix potential (safe) SQL-injection --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index 944675f860e..b4ecf3b83a4 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -99,6 +99,13 @@ std::unordered_set DatabasePostgreSQL::fetchTablesList() const bool DatabasePostgreSQL::checkPostgresTable(const String & table_name) const { + if (table_name.find('\'') != std::string::npos + || table_name.find('\\') != std::string::npos) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "PostgreSQL table name cannot contain single quote or backslash characters, passed {}", table_name); + } + pqxx::nontransaction tx(*connection->conn()); try From b25e334313106513105654d1ae82ad6e0efd5ab8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 11:30:35 +0300 Subject: [PATCH 220/264] Add assert --- src/Storages/StoragePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 78d3351e2f0..2cc2a15e573 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -212,6 +212,7 @@ public: data_type->serializeAsText(*array_column, 0, ostr, FormatSettings{}); /// ostr is guaranteed to be at least '[]', i.e. size is at least 2 and 2 only if ostr.str() == '[]' + assert(ostr.str().size() >= 2); return '{' + std::string(ostr.str().begin() + 1, ostr.str().end() - 1) + '}'; } From 4ee9c57190e0daa2324b40217a818d527adf59af Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 11:43:59 +0300 Subject: [PATCH 221/264] Add TODO --- src/Storages/StoragePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 2cc2a15e573..78ec8c34e41 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -167,6 +167,7 @@ public: /// Cannot just use serializeAsText for array data type even though it converts perfectly /// any dimension number array into text format, because it incloses in '[]' and for postgres it must be '{}'. + /// Check if array[...] syntax from PostgreSQL will be applicable. void parseArray(const Field & array_field, const DataTypePtr & data_type, WriteBuffer & ostr) { const auto * array_type = typeid_cast(data_type.get()); From 69cafd4d7fdac8b957bba0f82a07ca2585927542 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 11:48:50 +0300 Subject: [PATCH 222/264] Add suggestion from Kita --- src/Core/ExternalResultDescription.cpp | 48 ++++++++++++++------------ 1 file changed, 25 insertions(+), 23 deletions(-) diff --git a/src/Core/ExternalResultDescription.cpp b/src/Core/ExternalResultDescription.cpp index 7c1982a1787..3f6191523a9 100644 --- a/src/Core/ExternalResultDescription.cpp +++ b/src/Core/ExternalResultDescription.cpp @@ -36,51 +36,53 @@ void ExternalResultDescription::init(const Block & sample_block_) DataTypePtr type_not_nullable = removeNullable(elem.type); const IDataType * type = type_not_nullable.get(); - if (typeid_cast(type)) + WhichDataType which(type); + + if (which.isUInt8()) types.emplace_back(ValueType::vtUInt8, is_nullable); - else if (typeid_cast(type)) + else if (which.isUInt16()) types.emplace_back(ValueType::vtUInt16, is_nullable); - else if (typeid_cast(type)) + else if (which.isUInt32()) types.emplace_back(ValueType::vtUInt32, is_nullable); - else if (typeid_cast(type)) + else if (which.isUInt64()) types.emplace_back(ValueType::vtUInt64, is_nullable); - else if (typeid_cast(type)) + else if (which.isInt8()) types.emplace_back(ValueType::vtInt8, is_nullable); - else if (typeid_cast(type)) + else if (which.isInt16()) types.emplace_back(ValueType::vtInt16, is_nullable); - else if (typeid_cast(type)) + else if (which.isInt32()) types.emplace_back(ValueType::vtInt32, is_nullable); - else if (typeid_cast(type)) + else if (which.isInt64()) types.emplace_back(ValueType::vtInt64, is_nullable); - else if (typeid_cast(type)) + else if (which.isFloat32()) types.emplace_back(ValueType::vtFloat32, is_nullable); - else if (typeid_cast(type)) + else if (which.isFloat64()) types.emplace_back(ValueType::vtFloat64, is_nullable); - else if (typeid_cast(type)) + else if (which.isString()) types.emplace_back(ValueType::vtString, is_nullable); - else if (typeid_cast(type)) + else if (which.isDate()) types.emplace_back(ValueType::vtDate, is_nullable); - else if (typeid_cast(type)) + else if (which.isDateTime()) types.emplace_back(ValueType::vtDateTime, is_nullable); - else if (typeid_cast(type)) + else if (which.isUUID()) types.emplace_back(ValueType::vtUUID, is_nullable); - else if (typeid_cast(type)) + else if (which.isEnum8()) types.emplace_back(ValueType::vtString, is_nullable); - else if (typeid_cast(type)) + else if (which.isEnum16()) types.emplace_back(ValueType::vtString, is_nullable); - else if (typeid_cast(type)) + else if (which.isDateTime64()) types.emplace_back(ValueType::vtDateTime64, is_nullable); - else if (typeid_cast *>(type)) + else if (which.isDecimal32()) types.emplace_back(ValueType::vtDecimal32, is_nullable); - else if (typeid_cast *>(type)) + else if (which.isDecimal64()) types.emplace_back(ValueType::vtDecimal64, is_nullable); - else if (typeid_cast *>(type)) + else if (which.isDecimal128()) types.emplace_back(ValueType::vtDecimal128, is_nullable); - else if (typeid_cast *>(type)) + else if (which.isDecimal256()) types.emplace_back(ValueType::vtDecimal256, is_nullable); - else if (typeid_cast(type)) + else if (which.isArray()) types.emplace_back(ValueType::vtArray, is_nullable); - else if (typeid_cast(type)) + else if (which.isFixedString()) types.emplace_back(ValueType::vtFixedString, is_nullable); else throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE}; From e32b1e3fe60227da86f1a8a6afdef4220cf1e55d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 16 Jan 2021 11:53:28 +0300 Subject: [PATCH 223/264] Update StorageTinyLog.cpp --- src/Storages/StorageTinyLog.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index f10eb218715..06e2c21b1a8 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -307,11 +307,11 @@ void TinyLogBlockOutputStream::writeData(const NameAndTypePair & name_and_type, /// Use different WrittenStreams set, or we get nullptr for them in `serializeBinaryBulkWithMultipleStreams` WrittenStreams prefix_written_streams; settings.getter = createStreamGetter(name_and_type, prefix_written_streams); - type.serializeBinaryBulkStatePrefix(settings, serialize_states[name]); + type->serializeBinaryBulkStatePrefix(settings, serialize_states[name]); } settings.getter = createStreamGetter(name_and_type, written_streams); - type.serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); + type->serializeBinaryBulkWithMultipleStreams(column, 0, 0, settings, serialize_states[name]); } From 015e392bb7fa43989f4d58592759d94e9410e05d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 11:58:23 +0300 Subject: [PATCH 224/264] Disable TestFlows due to low reliability --- tests/testflows/regression.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/testflows/regression.py b/tests/testflows/regression.py index 20ba26318ba..6e19e4e49e1 100755 --- a/tests/testflows/regression.py +++ b/tests/testflows/regression.py @@ -16,8 +16,8 @@ def regression(self, local, clickhouse_binary_path, stress=None, parallel=None): # Feature(test=load("example.regression", "regression"))(**args) # Feature(test=load("ldap.regression", "regression"))(**args) - for i in range(10): - Feature(test=load("rbac.regression", "regression"))(**args) +# for i in range(10): +# Feature(test=load("rbac.regression", "regression"))(**args) # Feature(test=load("aes_encryption.regression", "regression"))(**args) if main(): From 2222741e33fc2619f062b9fc45a8b4978485fffb Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 16 Jan 2021 09:17:04 +0000 Subject: [PATCH 225/264] Fix style check --- src/Databases/PostgreSQL/DatabasePostgreSQL.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index b4ecf3b83a4..722b9c64edb 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -23,6 +23,7 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; extern const int UNKNOWN_TABLE; extern const int TABLE_IS_DROPPED; From 1b901b368f508b587cfb213eba6a4273e72ef7c9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 16 Jan 2021 09:46:25 +0300 Subject: [PATCH 226/264] Add missing type check in dictHas for COMPLEX_KEY_SSD_CACHE layout --- src/Dictionaries/SSDComplexKeyCacheDictionary.cpp | 2 ++ tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql | 3 +++ 2 files changed, 5 insertions(+) diff --git a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp index 58b117b3596..b23529eac7d 100644 --- a/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp +++ b/src/Dictionaries/SSDComplexKeyCacheDictionary.cpp @@ -1644,6 +1644,8 @@ void SSDComplexKeyCacheDictionary::has( const DataTypes & key_types, PaddedPODArray & out) const { + dict_struct.validateKeyTypes(key_types); + const auto now = std::chrono::system_clock::now(); std::unordered_map> not_found_keys; diff --git a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql index fbfd6900555..70e1d2bed29 100644 --- a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql +++ b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql @@ -42,6 +42,9 @@ LAYOUT(COMPLEX_KEY_SSD_CACHE(FILE_SIZE 8192 PATH '/var/lib/clickhouse/clickhouse SELECT 'TEST_SMALL'; SELECT 'VALUE FROM RAM BUFFER'; +-- NUMBER_OF_ARGUMENTS_DOESNT_MATCH +SELECT dictHas('database_for_dict.ssd_dict', 'a', tuple('1')); -- { serverError 42 } + SELECT dictGetUInt64('database_for_dict.ssd_dict', 'a', tuple('1', toInt32(3))); SELECT dictGetInt32('database_for_dict.ssd_dict', 'b', tuple('1', toInt32(3))); SELECT dictGetString('database_for_dict.ssd_dict', 'c', tuple('1', toInt32(3))); From 4efc7a7dc3b08ffbb0e454deed41e615152fa30a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 16 Jan 2021 13:07:58 +0300 Subject: [PATCH 227/264] Update MergeTreeIOSettings.h --- src/Storages/MergeTree/MergeTreeIOSettings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index db2a84b4363..d82aa7dd7c2 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -14,6 +14,7 @@ struct MergeTreeReaderSettings /// If save_marks_in_cache is false, then, if marks are not in cache, /// we will load them but won't save in the cache, to avoid evicting other data. bool save_marks_in_cache = false; + /// Convert old-style nested (single arrays with same prefix, `n.a`, `n.b`...) to subcolumns of data type Nested. bool convert_nested_to_subcolumns = false; }; From a5a19de8784808ef77ea9ae4c023fae2c8a480b7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 16 Jan 2021 13:22:25 +0300 Subject: [PATCH 228/264] Update DistributedBlockOutputStream.cpp --- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index e85cb227310..70e3ef0f17a 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -352,6 +352,7 @@ DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobRep /// Forward user settings job.local_context = std::make_unique(context); + /// InterpreterInsertQuery is modifying the AST, but the same AST is also used to insert to remote shards. auto copy_query_ast = query_ast->clone(); InterpreterInsertQuery interp(copy_query_ast, *job.local_context); From 489fa1e0833cf0dfb73a1e6551930b8f949b8734 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 16 Jan 2021 13:30:15 +0300 Subject: [PATCH 229/264] Fixes and translation to Ru --- docs/en/operations/settings/index.md | 2 +- .../sql-reference/statements/select/index.md | 6 +++++- docs/ru/operations/settings/index.md | 1 + .../sql-reference/statements/select/index.md | 18 +++++++++++++++++- 4 files changed, 24 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 87c8a9bf1b9..57d20f52899 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -25,7 +25,7 @@ Ways to configure settings, in order of priority: - When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`. - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). - - Make settings right in the query, in the `SETTINGS` clause. The setting value is applied only to that query and is reset to default or previous value after the query is executed. + - Make settings right in the query, in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select) clause. The setting value is applied only to that query and is reset to default or previous value after the query is executed. Settings that can only be made in the server config file are not covered in this section. diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 73ae1392350..9c8ac01db35 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -268,7 +268,11 @@ SELECT * REPLACE(i + 1 AS i) EXCEPT (j) APPLY(sum) from columns_transformers; ## SETTINGS in SELECT Query {#settings-in-select} -There are multiple ways to make settings, see [here](../../../operations/settings/index.md). One of them is to specify necessary settings right in the SELECT query. The setting value is applied only to this query and is reset to default or previous value after the query is executed. +You can specify the necessary settings right in the `SELECT` query. The setting value is applied only to this query and is reset to default or previous value after the query is executed. + +To specify several settings, use several `SETTINGS` clauses. + +Other ways to make settings, see [here](../../../operations/settings/index.md). **Example** diff --git a/docs/ru/operations/settings/index.md b/docs/ru/operations/settings/index.md index c24b7053c46..0fc76d16f7c 100644 --- a/docs/ru/operations/settings/index.md +++ b/docs/ru/operations/settings/index.md @@ -24,6 +24,7 @@ toc_title: Introduction - При запуске консольного клиента ClickHouse в не интерактивном режиме установите параметр запуска `--setting=value`. - При использовании HTTP API передавайте cgi-параметры (`URL?setting_1=value&setting_2=value...`). + - В самом запросе укажите необходимые настройки в секции [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select). Значения таких настроек действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. Настройки, которые можно задать только в конфигурационном файле сервера, в разделе не рассматриваются. diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index bf4ae44a6f1..9af4d051451 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -23,6 +23,7 @@ SELECT [DISTINCT] expr_list [ORDER BY expr_list] [WITH FILL] [FROM expr] [TO expr] [STEP expr] [LIMIT [offset_value, ]n BY columns] [LIMIT [n, ]m] [WITH TIES] +[SETTINGS ...] [UNION ALL ...] [INTO OUTFILE filename] [FORMAT format] @@ -161,4 +162,19 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of Подробнее смотрите в разделе «Настройки». Присутствует возможность использовать внешнюю сортировку (с сохранением временных данных на диск) и внешнюю агрегацию. -{## [Оригинальная статья](https://clickhouse.tech/docs/en/sql-reference/statements/select/) ##} +## SETTINGS в запросе SELECT {#settings-in-select} + +Вы можете задать значения необходимых настроек непосредственно в запросе `SELECT`, в секции `SETTINGS`. Значение настройки действует только в рамках данного запроса, а после его выполнения сбрасывается до предыдущего значения или значения по умолчанию. + +Чтобы задать значения нескольких настроек, используйте несколько отдельных секций `SETTINGS`. + +Другие способы задания настроек описаны [здесь](../../../operations/settings/index.md). + +**Пример** + +``` sql +SELECT * FROM some_table SETTINGS optimize_read_in_order=1 SETTINGS cast_keep_nullable=1; +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/) + From e67e4588e75728d473b14a7ac74f7cff0f18ad53 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 16 Jan 2021 15:26:21 +0300 Subject: [PATCH 230/264] Update PostgreSQLConnection.h --- src/Storages/PostgreSQL/PostgreSQLConnection.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLConnection.h b/src/Storages/PostgreSQL/PostgreSQLConnection.h index 04ba1909803..ae79a3436e0 100644 --- a/src/Storages/PostgreSQL/PostgreSQLConnection.h +++ b/src/Storages/PostgreSQL/PostgreSQLConnection.h @@ -5,7 +5,7 @@ #endif #if USE_LIBPQXX -#include +#include // Y_IGNORE #include From 029302d7667912d69d6952cd84f7337f5c4bdccf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 17:09:44 +0300 Subject: [PATCH 231/264] Merge with master --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 95b0cbde264..5a2c17a7b98 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -646,7 +646,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const { if (path_to_data_file.empty()) { - String candidate_path = getFullRelativePath() + IDataType::getFileNameForStream(part_column.name, substream_path) + ".bin"; + String candidate_path = getFullRelativePath() + IDataType::getFileNameForStream(part_column, substream_path) + ".bin"; /// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns and column_name.dict.null.bin file. if (volume->getDisk()->exists(candidate_path) && volume->getDisk()->getFileSize(candidate_path) != 0) From a2c77c0343805b7f76f3d4b75030c69a6f2b96d1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 17:13:52 +0300 Subject: [PATCH 232/264] Fix possible aliasing issues --- src/DataTypes/DataTypeArray.cpp | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index 79e0196c9f1..aa1d20d409e 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -161,9 +161,14 @@ namespace auto & sizes_data = assert_cast(*column_sizes).getData(); sizes_data.resize(offsets_data.size()); - sizes_data[0] = offsets_data[0]; - for (size_t i = 1; i < offsets_data.size(); ++i) - sizes_data[i] = offsets_data[i] - offsets_data[i - 1]; + + IColumn::Offset prev_offset = 0; + for (size_t i = 0, size = offsets_data.size(); i < size; ++i) + { + auto current_offset = offsets_data[i]; + sizes_data[i] = current_offset - prev_offset; + prev_offset = current_offset; + } return column_sizes; } @@ -180,9 +185,13 @@ namespace auto & offsets_data = assert_cast(*column_offsets).getData(); offsets_data.resize(sizes_data.size()); - offsets_data[0] = sizes_data[0]; + + IColumn::Offset prev_offset = 0; for (size_t i = 0; i < sizes_data.size(); ++i) - offsets_data[i] = offsets_data[i - 1] + sizes_data[i]; + { + offsets_data[i] = prev_offset; + prev_offset += sizes_data[i]; + } return column_offsets; } From 6cd352fbe3f3b59bbebcdc954737753571a5c874 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 18:08:21 +0300 Subject: [PATCH 233/264] Remove unneeded change from #18977 --- programs/local/LocalServer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 22e72be27f2..5a8d35e204d 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -273,9 +273,10 @@ try global_context->setCurrentDatabase(default_database); applyCmdOptions(*global_context); - String path = global_context->getPath(); - if (!path.empty()) + if (config().has("path")) { + String path = global_context->getPath(); + /// Lock path directory before read status.emplace(path + "status", StatusFile::write_full_info); From 54f604c41f892f0cdf4b16f177945d7b84c619c3 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 16 Jan 2021 18:29:43 +0300 Subject: [PATCH 234/264] Update DataTypeArray.cpp --- src/DataTypes/DataTypeArray.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeArray.cpp b/src/DataTypes/DataTypeArray.cpp index aa1d20d409e..3ad84a8fcd7 100644 --- a/src/DataTypes/DataTypeArray.cpp +++ b/src/DataTypes/DataTypeArray.cpp @@ -187,10 +187,10 @@ namespace offsets_data.resize(sizes_data.size()); IColumn::Offset prev_offset = 0; - for (size_t i = 0; i < sizes_data.size(); ++i) + for (size_t i = 0, size = sizes_data.size(); i < size; ++i) { - offsets_data[i] = prev_offset; prev_offset += sizes_data[i]; + offsets_data[i] = prev_offset; } return column_offsets; From 950bfb3ec645a386e82c080c2fe3af09af1c9aee Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 18:33:34 +0300 Subject: [PATCH 235/264] Allow change max_server_memory_usage without restart --- programs/server/Server.cpp | 62 ++++++++++++++++++------------------ src/Common/MemoryTracker.cpp | 6 ++++ src/Common/MemoryTracker.h | 2 ++ 3 files changed, 39 insertions(+), 31 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 2f8029fc39c..94cd6854f78 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -692,6 +692,37 @@ int Server::main(const std::vector & /*args*/) { Settings::checkNoSettingNamesAtTopLevel(*config, config_path); + /// Limit on total memory usage + size_t max_server_memory_usage = config->getUInt64("max_server_memory_usage", 0); + + double max_server_memory_usage_to_ram_ratio = config->getDouble("max_server_memory_usage_to_ram_ratio", 0.9); + size_t default_max_server_memory_usage = memory_amount * max_server_memory_usage_to_ram_ratio; + + if (max_server_memory_usage == 0) + { + max_server_memory_usage = default_max_server_memory_usage; + LOG_INFO(log, "Setting max_server_memory_usage was set to {}" + " ({} available * {:.2f} max_server_memory_usage_to_ram_ratio)", + formatReadableSizeWithBinarySuffix(max_server_memory_usage), + formatReadableSizeWithBinarySuffix(memory_amount), + max_server_memory_usage_to_ram_ratio); + } + else if (max_server_memory_usage > default_max_server_memory_usage) + { + max_server_memory_usage = default_max_server_memory_usage; + LOG_INFO(log, "Setting max_server_memory_usage was lowered to {}" + " because the system has low amount of memory. The amount was" + " calculated as {} available" + " * {:.2f} max_server_memory_usage_to_ram_ratio", + formatReadableSizeWithBinarySuffix(max_server_memory_usage), + formatReadableSizeWithBinarySuffix(memory_amount), + max_server_memory_usage_to_ram_ratio); + } + + total_memory_tracker.setHardLimit(max_server_memory_usage); + total_memory_tracker.setDescription("(total)"); + total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + // FIXME logging-related things need synchronization -- see the 'Logger * log' saved // in a lot of places. For now, disable updating log configuration without server restart. //setTextLog(global_context->getTextLog()); @@ -780,37 +811,6 @@ int Server::main(const std::vector & /*args*/) global_context->getMergeTreeSettings().sanityCheck(settings); global_context->getReplicatedMergeTreeSettings().sanityCheck(settings); - /// Limit on total memory usage - size_t max_server_memory_usage = config().getUInt64("max_server_memory_usage", 0); - - double max_server_memory_usage_to_ram_ratio = config().getDouble("max_server_memory_usage_to_ram_ratio", 0.9); - size_t default_max_server_memory_usage = memory_amount * max_server_memory_usage_to_ram_ratio; - - if (max_server_memory_usage == 0) - { - max_server_memory_usage = default_max_server_memory_usage; - LOG_INFO(log, "Setting max_server_memory_usage was set to {}" - " ({} available * {:.2f} max_server_memory_usage_to_ram_ratio)", - formatReadableSizeWithBinarySuffix(max_server_memory_usage), - formatReadableSizeWithBinarySuffix(memory_amount), - max_server_memory_usage_to_ram_ratio); - } - else if (max_server_memory_usage > default_max_server_memory_usage) - { - max_server_memory_usage = default_max_server_memory_usage; - LOG_INFO(log, "Setting max_server_memory_usage was lowered to {}" - " because the system has low amount of memory. The amount was" - " calculated as {} available" - " * {:.2f} max_server_memory_usage_to_ram_ratio", - formatReadableSizeWithBinarySuffix(max_server_memory_usage), - formatReadableSizeWithBinarySuffix(memory_amount), - max_server_memory_usage_to_ram_ratio); - } - - total_memory_tracker.setOrRaiseHardLimit(max_server_memory_usage); - total_memory_tracker.setDescription("(total)"); - total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0); Poco::ThreadPool server_pool(3, config().getUInt("max_connections", 1024)); diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index e5682cf37d0..d9bbf8ffa96 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -316,6 +316,12 @@ void MemoryTracker::set(Int64 to) } +void MemoryTracker::setHardLimit(Int64 value) +{ + hard_limit.store(value, std::memory_order_relaxed); +} + + void MemoryTracker::setOrRaiseHardLimit(Int64 value) { /// This is just atomic set to maximum. diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 961c8722afa..641ebe238c9 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -96,6 +96,8 @@ public: return peak.load(std::memory_order_relaxed); } + void setHardLimit(Int64 value); + /** Set limit if it was not set. * Otherwise, set limit to new value, if new value is greater than previous limit. */ From cd865dcf4243699c3e93250aa9091e898f94b41b Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Sat, 16 Jan 2021 22:34:13 +0300 Subject: [PATCH 236/264] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/operations/settings/index.md | 2 +- docs/ru/operations/settings/index.md | 2 +- docs/ru/sql-reference/statements/select/index.md | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 57d20f52899..d38c98f51cb 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -25,7 +25,7 @@ Ways to configure settings, in order of priority: - When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`. - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). - - Make settings right in the query, in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select) clause. The setting value is applied only to that query and is reset to default or previous value after the query is executed. + - Make settings in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select) clause of the SELECT query. The setting value is applied only to that query and is reset to default or previous value after the query is executed. Settings that can only be made in the server config file are not covered in this section. diff --git a/docs/ru/operations/settings/index.md b/docs/ru/operations/settings/index.md index 0fc76d16f7c..2ef1d4730a3 100644 --- a/docs/ru/operations/settings/index.md +++ b/docs/ru/operations/settings/index.md @@ -24,7 +24,7 @@ toc_title: Introduction - При запуске консольного клиента ClickHouse в не интерактивном режиме установите параметр запуска `--setting=value`. - При использовании HTTP API передавайте cgi-параметры (`URL?setting_1=value&setting_2=value...`). - - В самом запросе укажите необходимые настройки в секции [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select). Значения таких настроек действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. + - Укажите необходимые настройки в секции [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select) запроса SELECT. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. Настройки, которые можно задать только в конфигурационном файле сервера, в разделе не рассматриваются. diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 9af4d051451..d0a0c25cd96 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -164,7 +164,7 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of ## SETTINGS в запросе SELECT {#settings-in-select} -Вы можете задать значения необходимых настроек непосредственно в запросе `SELECT`, в секции `SETTINGS`. Значение настройки действует только в рамках данного запроса, а после его выполнения сбрасывается до предыдущего значения или значения по умолчанию. +Вы можете задать значения необходимых настроек непосредственно в запросе `SELECT` в секции `SETTINGS`. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. Чтобы задать значения нескольких настроек, используйте несколько отдельных секций `SETTINGS`. From 909873b45c7005652d9f714c2f482cc21c8ae250 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 16 Jan 2021 23:54:39 +0300 Subject: [PATCH 237/264] Prepare Fuzzer to run with different builds --- docker/test/fuzzer/run-fuzzer.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 889d7308ed8..9c2bf371344 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -8,6 +8,7 @@ stage=${stage:-} script_dir="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" echo "$script_dir" repo_dir=ch +BINARY_TO_DOWNLOAD=${BINARY_TO_DOWNLOAD:="clang-11_debug_none_bundled_unsplitted_disable_False_binary"} function clone { @@ -35,7 +36,7 @@ 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-11_debug_none_bundled_unsplitted_disable_False_binary/clickhouse" + wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/$BINARY_TO_DOWNLOAD/clickhouse" chmod +x clickhouse ln -s ./clickhouse ./clickhouse-server ln -s ./clickhouse ./clickhouse-client From bf522d0eea76cd4273ad8d4d7e675c21c6022a48 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 17 Jan 2021 00:11:58 +0300 Subject: [PATCH 238/264] Update InterpreterSelectQuery.cpp --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8cc4b08c61b..fcf5a242417 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -234,7 +234,7 @@ static void checkAccessRightsForSelect( if (access->isGranted(AccessType::SELECT, table_id.database_name, table_id.table_name, column.name)) return; } - throw Exception(context.getUserName() + " : Not enough privileges. " + throw Exception(context.getUserName() + ": Not enough privileges. " "To execute this query it's necessary to have grant SELECT for at least one column on " + table_id.getFullTableName(), ErrorCodes::ACCESS_DENIED); } From 858f07c7962fa69d8b699ff86c09bb3d9cdddd01 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jan 2021 12:38:44 +0300 Subject: [PATCH 239/264] Update comment for query AST cloning during inesrt into multiple local shards Refs: https://github.com/ClickHouse/ClickHouse/pull/18264#discussion_r558839456 --- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 70e3ef0f17a..f5a2e48da14 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -352,7 +352,12 @@ DistributedBlockOutputStream::runWritingJob(DistributedBlockOutputStream::JobRep /// Forward user settings job.local_context = std::make_unique(context); - /// InterpreterInsertQuery is modifying the AST, but the same AST is also used to insert to remote shards. + /// Copying of the query AST is required to avoid race, + /// in case of INSERT into multiple local shards. + /// + /// Since INSERT into local node uses AST, + /// and InterpreterInsertQuery::execute() is modifying it, + /// to resolve tables (in InterpreterInsertQuery::getTable()) auto copy_query_ast = query_ast->clone(); InterpreterInsertQuery interp(copy_query_ast, *job.local_context); From 2955e25e834be43fea966126e0d5e95431e1b048 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jan 2021 12:42:19 +0300 Subject: [PATCH 240/264] Fix inserted blocks accounting for insert_distributed_one_random_shard=1 It is tricky due to block splitting Refs: https://github.com/ClickHouse/ClickHouse/pull/18294 --- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index f5a2e48da14..1c470acef20 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -145,6 +145,7 @@ void DistributedBlockOutputStream::writeAsync(const Block & block) if (random_shard_insert) { writeAsyncImpl(block, storage.getRandomShardIndex(cluster->getShardsInfo())); + ++inserted_blocks; } else { From b725e1d131fc5ba58e37797cacab35b1e4fd1e4a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jan 2021 12:48:51 +0300 Subject: [PATCH 241/264] DistributedBlockOutputStream: Remove superfluous brackets for string construction --- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 1c470acef20..c22ccd31c6d 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -614,7 +614,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: /// tmp directory is used to ensure atomicity of transactions /// and keep monitor thread out from reading incomplete data - std::string first_file_tmp_path{}; + std::string first_file_tmp_path; auto reservation = storage.getStoragePolicy()->reserveAndCheck(block.bytes()); const auto disk = reservation->getDisk(); From a6631287a7434a69885fa49193a4a3a284ccdbe8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jan 2021 12:50:37 +0300 Subject: [PATCH 242/264] DistributedBlockOutputStream: add more comments --- src/Storages/Distributed/DistributedBlockOutputStream.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/src/Storages/Distributed/DistributedBlockOutputStream.cpp index c22ccd31c6d..f4711399400 100644 --- a/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -655,6 +655,8 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: NativeBlockOutputStream stream{compress, DBMS_TCP_PROTOCOL_VERSION, block.cloneEmpty()}; /// Prepare the header. + /// See also readDistributedHeader() in DirectoryMonitor (for reading side) + /// /// We wrap the header into a string for compatibility with older versions: /// a shard will able to read the header partly and ignore other parts based on its version. WriteBufferFromOwnString header_buf; From be253e3d6863c1f5eed87b3f3432437a8d1051f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 14:59:35 +0300 Subject: [PATCH 243/264] Skip test for ANTLR --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index cfbac463932..c2e8d1263f1 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -309,7 +309,8 @@ "01632_max_partitions_to_read", "01638_div_mod_ambiguities", "01642_if_nullable_regression", - "01643_system_suspend" + "01643_system_suspend", + "01475_read_subcolumns_storages" ], "parallel": [ From e482c5ea1584e7cfaca1161d064fd5c574cd47f5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 15:40:14 +0300 Subject: [PATCH 244/264] Fix TSan --- src/Common/MemoryTracker.cpp | 15 +++++++++------ src/Common/MemoryTracker.h | 4 ++-- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index d9bbf8ffa96..d037142fbfb 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -145,8 +145,9 @@ void MemoryTracker::alloc(Int64 size) */ Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed); - if (metric != CurrentMetrics::end()) - CurrentMetrics::add(metric, size); + auto metric_loaded = metric.load(std::memory_order_relaxed); + if (metric_loaded != CurrentMetrics::end()) + CurrentMetrics::add(metric_loaded, size); Int64 current_hard_limit = hard_limit.load(std::memory_order_relaxed); Int64 current_profiler_limit = profiler_limit.load(std::memory_order_relaxed); @@ -286,8 +287,9 @@ void MemoryTracker::free(Int64 size) if (auto * loaded_next = parent.load(std::memory_order_relaxed)) loaded_next->free(size); - if (metric != CurrentMetrics::end()) - CurrentMetrics::sub(metric, accounted_size); + auto metric_loaded = metric.load(std::memory_order_relaxed); + if (metric_loaded != CurrentMetrics::end()) + CurrentMetrics::sub(metric_loaded, accounted_size); } @@ -302,8 +304,9 @@ void MemoryTracker::resetCounters() void MemoryTracker::reset() { - if (metric != CurrentMetrics::end()) - CurrentMetrics::sub(metric, amount.load(std::memory_order_relaxed)); + auto metric_loaded = metric.load(std::memory_order_relaxed); + if (metric_loaded != CurrentMetrics::end()) + CurrentMetrics::sub(metric_loaded, amount.load(std::memory_order_relaxed)); resetCounters(); } diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 641ebe238c9..9475b555975 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -53,7 +53,7 @@ private: std::atomic parent {}; /// You could specify custom metric to track memory usage. - CurrentMetrics::Metric metric = CurrentMetrics::end(); + std::atomic metric = CurrentMetrics::end(); /// This description will be used as prefix into log messages (if isn't nullptr) std::atomic description_ptr = nullptr; @@ -134,7 +134,7 @@ public: /// The memory consumption could be shown in realtime via CurrentMetrics counter void setMetric(CurrentMetrics::Metric metric_) { - metric = metric_; + metric.store(metric_, std::memory_order_relaxed); } void setDescription(const char * description) From 32ffa99461656b92ae9adf1b536e5e814d1582f6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jan 2021 18:36:13 +0300 Subject: [PATCH 245/264] Disable curl for mariadb-connector-c (it is not required) --- contrib/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 20818767b99..6593eb9812f 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -120,7 +120,6 @@ if (USE_INTERNAL_LDAP_LIBRARY) endif () # Should go before: -# - mariadb-connector-c # - aws-s3-cmake # - sentry-native add_subdirectory (curl-cmake) @@ -142,6 +141,7 @@ function(mysql_support) set(ZLIB_LIBRARY ${ZLIB_LIBRARIES}) set(WITH_EXTERNAL_ZLIB ON) endif() + set(WITH_CURL OFF) add_subdirectory (mariadb-connector-c) endfunction() if (ENABLE_MYSQL AND USE_INTERNAL_MYSQL_LIBRARY) From d61a6379fc59db7ae130a15f466b9ed9c6c91d91 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 17 Jan 2021 18:38:29 +0300 Subject: [PATCH 246/264] Update comment for curl dependency for aws It is not true since #13628 --- contrib/CMakeLists.txt | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 6593eb9812f..20b4fad0437 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -119,11 +119,6 @@ if (USE_INTERNAL_LDAP_LIBRARY) add_subdirectory (openldap-cmake) endif () -# Should go before: -# - aws-s3-cmake -# - sentry-native -add_subdirectory (curl-cmake) - function(mysql_support) set(CLIENT_PLUGIN_CACHING_SHA2_PASSWORD STATIC) set(CLIENT_PLUGIN_SHA256_PASSWORD STATIC) @@ -288,6 +283,10 @@ if (USE_CASSANDRA) add_subdirectory (cassandra) endif() +# Should go before: +# - sentry-native +add_subdirectory (curl-cmake) + if (USE_SENTRY) add_subdirectory (sentry-native) endif() From 17b2ca2374a0b810069ee57c9e27c2b6f79f603b Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 17 Jan 2021 20:05:46 +0300 Subject: [PATCH 247/264] Fixes --- docs/en/sql-reference/statements/select/index.md | 6 ++---- docs/ru/sql-reference/statements/select/index.md | 4 +--- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 9c8ac01db35..7c13772ffdf 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -270,14 +270,12 @@ SELECT * REPLACE(i + 1 AS i) EXCEPT (j) APPLY(sum) from columns_transformers; You can specify the necessary settings right in the `SELECT` query. The setting value is applied only to this query and is reset to default or previous value after the query is executed. -To specify several settings, use several `SETTINGS` clauses. - -Other ways to make settings, see [here](../../../operations/settings/index.md). +Other ways to make settings see [here](../../../operations/settings/index.md). **Example** ``` sql -SELECT * FROM some_table SETTINGS optimize_read_in_order=1 SETTINGS cast_keep_nullable=1; +SELECT * FROM some_table SETTINGS optimize_read_in_order=1, cast_keep_nullable=1; ``` [Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/) diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index d0a0c25cd96..c37e82ae0be 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -166,14 +166,12 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of Вы можете задать значения необходимых настроек непосредственно в запросе `SELECT` в секции `SETTINGS`. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. -Чтобы задать значения нескольких настроек, используйте несколько отдельных секций `SETTINGS`. - Другие способы задания настроек описаны [здесь](../../../operations/settings/index.md). **Пример** ``` sql -SELECT * FROM some_table SETTINGS optimize_read_in_order=1 SETTINGS cast_keep_nullable=1; +SELECT * FROM some_table SETTINGS optimize_read_in_order=1, cast_keep_nullable=1; ``` [Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/) From 19dc5239fab94369d32c99688667b269b6fa970f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 21:14:57 +0300 Subject: [PATCH 248/264] Add more Fuzzer tasks --- tests/ci/ci_config.json | 50 ++++++++++++++++++++++++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index b2e93d39848..44b35d61601 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -609,7 +609,7 @@ "with_coverage": false } }, - "AST fuzzer": { + "AST fuzzer (debug)": { "required_build_properties": { "compiler": "clang-11", "package_type": "binary", @@ -621,6 +621,54 @@ "with_coverage": false } }, + "AST fuzzer (ASan)": { + "required_build_properties": { + "compiler": "clang-11", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "AST fuzzer (MSan)": { + "required_build_properties": { + "compiler": "clang-11", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "memory", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "AST fuzzer (TSan)": { + "required_build_properties": { + "compiler": "clang-11", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "thread", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, + "AST fuzzer (UBSan)": { + "required_build_properties": { + "compiler": "clang-11", + "package_type": "binary", + "build_type": "relwithdebuginfo", + "sanitizer": "undefined", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, "Release": { "required_build_properties": { "compiler": "clang-11", From 98a8a20b7066852ff559bd9eb660cc0d03abbe5e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 21:30:02 +0300 Subject: [PATCH 249/264] Fix MSan error in rocksdb #19213 --- base/glibc-compatibility/musl/sched_getcpu.c | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/base/glibc-compatibility/musl/sched_getcpu.c b/base/glibc-compatibility/musl/sched_getcpu.c index 4ec5eaf6796..57b8b416043 100644 --- a/base/glibc-compatibility/musl/sched_getcpu.c +++ b/base/glibc-compatibility/musl/sched_getcpu.c @@ -4,6 +4,12 @@ #include "syscall.h" #include "atomic.h" +#if defined(__has_feature) +#if __has_feature(memory_sanitizer) +#include +#endif +#endif + #ifdef VDSO_GETCPU_SYM static void *volatile vdso_func; @@ -37,6 +43,13 @@ int sched_getcpu(void) #endif r = __syscall(SYS_getcpu, &cpu, 0, 0); - if (!r) return cpu; + if (!r) { +#if defined(__has_feature) +#if __has_feature(memory_sanitizer) + __msan_unpoison(&cpu, sizeof(cpu)); +#endif +#endif + return cpu; + } return __syscall_ret(r); } From ce6dc40888de09919013809c34af3c59c05e87a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 21:35:53 +0300 Subject: [PATCH 250/264] Fix MSan report in Kerberos library --- contrib/krb5 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/krb5 b/contrib/krb5 index 90ff6f4f8c6..5149dea4e2b 160000 --- a/contrib/krb5 +++ b/contrib/krb5 @@ -1 +1 @@ -Subproject commit 90ff6f4f8c695d6bf1aaba78a9b8942be92141c2 +Subproject commit 5149dea4e2be0f67707383d2682b897c14631374 From c5ac0762e4a483601e78f6a14b95649e4bd4ce7e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 21:59:31 +0300 Subject: [PATCH 251/264] Speed up aggregate function sum --- src/AggregateFunctions/AggregateFunctionSum.h | 69 +++++++++++-------- 1 file changed, 41 insertions(+), 28 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index ecc47eecab4..1038c8107a5 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -29,61 +29,74 @@ struct AggregateFunctionSumData template void NO_INLINE addMany(const Value * __restrict ptr, size_t count) { - /// Compiler cannot unroll this loop, do it manually. - /// (at least for floats, most likely due to the lack of -fassociative-math) - - /// Something around the number of SSE registers * the number of elements fit in register. - constexpr size_t unroll_count = 128 / sizeof(T); - T partial_sums[unroll_count]{}; - const auto * end = ptr + count; - const auto * unrolled_end = ptr + (count / unroll_count * unroll_count); - while (ptr < unrolled_end) + if constexpr (std::is_floating_point_v) { + /// Compiler cannot unroll this loop, do it manually. + /// (at least for floats, most likely due to the lack of -fassociative-math) + + /// Something around the number of SSE registers * the number of elements fit in register. + constexpr size_t unroll_count = 128 / sizeof(T); + T partial_sums[unroll_count]{}; + + const auto * unrolled_end = ptr + (count / unroll_count * unroll_count); + + while (ptr < unrolled_end) + { + for (size_t i = 0; i < unroll_count; ++i) + partial_sums[i] += ptr[i]; + ptr += unroll_count; + } + for (size_t i = 0; i < unroll_count; ++i) - partial_sums[i] += ptr[i]; - ptr += unroll_count; + sum += partial_sums[i]; } - for (size_t i = 0; i < unroll_count; ++i) - sum += partial_sums[i]; - + /// clang cannot vectorize the loop if accumulator is class member instead of local variable. + T local_sum{}; while (ptr < end) { - sum += *ptr; + local_sum += *ptr; ++ptr; } + sum += local_sum; } template void NO_INLINE addManyNotNull(const Value * __restrict ptr, const UInt8 * __restrict null_map, size_t count) { - constexpr size_t unroll_count = 128 / sizeof(T); - T partial_sums[unroll_count]{}; - const auto * end = ptr + count; - const auto * unrolled_end = ptr + (count / unroll_count * unroll_count); - while (ptr < unrolled_end) + if constexpr (std::is_floating_point_v) { + constexpr size_t unroll_count = 128 / sizeof(T); + T partial_sums[unroll_count]{}; + + const auto * unrolled_end = ptr + (count / unroll_count * unroll_count); + + while (ptr < unrolled_end) + { + for (size_t i = 0; i < unroll_count; ++i) + if (!null_map[i]) + partial_sums[i] += ptr[i]; + ptr += unroll_count; + null_map += unroll_count; + } + for (size_t i = 0; i < unroll_count; ++i) - if (!null_map[i]) - partial_sums[i] += ptr[i]; - ptr += unroll_count; - null_map += unroll_count; + sum += partial_sums[i]; } - for (size_t i = 0; i < unroll_count; ++i) - sum += partial_sums[i]; - + T local_sum{}; while (ptr < end) { if (!*null_map) - sum += *ptr; + local_sum += *ptr; ++ptr; ++null_map; } + sum += local_sum; } void merge(const AggregateFunctionSumData & rhs) From c02f2d45ad5234c1a2815746cd958627dbd036eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 23:38:59 +0300 Subject: [PATCH 252/264] Update Dragonbox --- contrib/dragonbox | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/dragonbox b/contrib/dragonbox index b2751c65c05..923705af6fd 160000 --- a/contrib/dragonbox +++ b/contrib/dragonbox @@ -1 +1 @@ -Subproject commit b2751c65c0592c0239aec3becd53d0ea2fde9329 +Subproject commit 923705af6fd953aa948fc175f6020b15f7359838 From 2c96fc53f5ee41d35bc5371a085896ebf23b0bae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 23:54:00 +0300 Subject: [PATCH 253/264] Make symbolizers available in fuzzer Docker image --- docker/test/fuzzer/Dockerfile | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile index 04d533eb601..57daba9cfd6 100644 --- a/docker/test/fuzzer/Dockerfile +++ b/docker/test/fuzzer/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-fuzzer . -FROM ubuntu:18.04 +FROM yandex/clickhouse-test-base ENV LANG=C.UTF-8 ENV TZ=Europe/Moscow @@ -7,11 +7,7 @@ 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 \ From c595a3deaa4517016297a7906f65c3a5c168cf87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 17 Jan 2021 23:57:30 +0300 Subject: [PATCH 254/264] Update images.json --- docker/images.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/images.json b/docker/images.json index 6ab2d287b74..303bd159ce4 100644 --- a/docker/images.json +++ b/docker/images.json @@ -137,7 +137,8 @@ "docker/test/stateless", "docker/test/stateless_unbundled", "docker/test/stateless_pytest", - "docker/test/integration/base" + "docker/test/integration/base", + "docker/test/fuzzer" ] }, "docker/packager/unbundled": { From e967b6bf49ff43aaaf6808b34234c2456bd24be5 Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Mon, 18 Jan 2021 12:16:32 +0800 Subject: [PATCH 255/264] add a %Q replacement for formatDateTime --- .../sql-reference/functions/date-time-functions.md | 3 ++- .../sql-reference/functions/date-time-functions.md | 13 +++++++------ src/Functions/formatDateTime.cpp | 11 +++++++++++ ...quarter_modificator_for_formatDateTime.reference | 4 ++++ ...01655_quarter_modificator_for_formatDateTime.sql | 4 ++++ 5 files changed, 28 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.reference create mode 100644 tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 628c321adee..9de780fb596 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -622,7 +622,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %C | year divided by 100 and truncated to integer (00-99) | 20 | | %d | day of the month, zero-padded (01-31) | 02 | | %D | Short MM/DD/YY date, equivalent to %m/%d/%y | 01/02/18 | -| %e | day of the month, space-padded ( 1-31) | 2 | +| %e | day of the month, space-padded ( 1-31) | 2 | | %F | short YYYY-MM-DD date, equivalent to %Y-%m-%d | 2018-01-02 | | %G | four-digit year format for ISO week number, calculated from the week-based year [defined by the ISO 8601](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) standard, normally useful only with %V | 2018 | | %g | two-digit year format, aligned to ISO 8601, abbreviated from four-digit notation | 18 | @@ -633,6 +633,7 @@ Using replacement fields, you can define a pattern for the resulting string. “ | %M | minute (00-59) | 33 | | %n | new-line character (‘’) | | | %p | AM or PM designation | PM | +| %Q | Quarter (1-4) | 1 | | %R | 24-hour HH:MM time, equivalent to %H:%M | 22:33 | | %S | second (00-59) | 44 | | %t | horizontal-tab character (’) | | diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index 00dab5ee680..df365ee9c1c 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -613,24 +613,25 @@ formatDateTime(Time, Format\[, Timezone\]) | %C | 年除以100并截断为整数(00-99) | 20 | | %d | 月中的一天,零填充(01-31) | 02 | | %D | 短MM/DD/YY日期,相当于%m/%d/%y | 01/02/2018 | -| %e | 月中的一天,空格填充(1-31) | 2 | +| %e | 月中的一天,空格填充( 1-31) | 2 | | %F | 短YYYY-MM-DD日期,相当于%Y-%m-%d | 2018-01-02 | | %G | ISO周号的四位数年份格式, 从基于周的年份[由ISO 8601定义](https://en.wikipedia.org/wiki/ISO_8601#Week_dates) 标准计算得出,通常仅对%V有用 | 2018 | | %g | 两位数的年份格式,与ISO 8601一致,四位数表示法的缩写 | 18 | | %H | 24小时格式(00-23) | 22 | -| %I | 小时12h格式(01-12) | 10 | -| %j | 一年(001-366) | 002 | +| %I | 12小时格式(01-12) | 10 | +| %j | 一年中的一天 (001-366) | 002 | | %m | 月份为十进制数(01-12) | 01 | | %M | 分钟(00-59) | 33 | | %n | 换行符(") | | | %p | AM或PM指定 | PM | +| %Q | 季度(1-4) | 1 | | %R | 24小时HH:MM时间,相当于%H:%M | 22:33 | -| %S | 第二(00-59) | 44 | +| %S | 秒 (00-59) | 44 | | %t | 水平制表符(’) | | | %T | ISO8601时间格式(HH:MM:SS),相当于%H:%M:%S | 22:33:44 | -| %u | ISO8601平日as编号,星期一为1(1-7) | 2 | +| %u | ISO8601工作日为数字,星期一为1(1-7) | 2 | | %V | ISO8601周编号(01-53) | 01 | -| %w | 周日为十进制数,周日为0(0-6) | 2 | +| %w | 工作日为十进制数,周日为0(0-6) | 2 | | %y | 年份,最后两位数字(00-99) | 18 | | %Y | 年 | 2018 | | %% | %符号 | % | diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 362390a2cbc..fd909ed6fce 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -272,6 +272,11 @@ private: writeNumber2(target + 3, ToMinuteImpl::execute(source, timezone)); writeNumber2(target + 6, ToSecondImpl::execute(source, timezone)); } + + static void quarter(char * target, Time source, const DateLUTImpl & timezone) + { + *target += ToQuarterImpl::execute(source, timezone); + } }; public: @@ -621,6 +626,12 @@ public: result.append("0000"); break; + // Quarter (1-4) + case 'Q': + instructions.template emplace_back(&Action::quarter, 1); + result.append("0"); + break; + /// Time components. If the argument is Date, not a DateTime, then this components will have default value. // Minute (00-59) diff --git a/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.reference b/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.reference new file mode 100644 index 00000000000..94ebaf90016 --- /dev/null +++ b/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.reference @@ -0,0 +1,4 @@ +1 +2 +3 +4 diff --git a/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql b/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql new file mode 100644 index 00000000000..35a6f6e2ab7 --- /dev/null +++ b/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql @@ -0,0 +1,4 @@ +SELECT formatDateTime(toDate('2010-01-04'), '%Q'); +SELECT formatDateTime(toDate('2010-04-30'), '%Q'); +SELECT formatDateTime(toDate('2010-07-31'), '%Q'); +SELECT formatDateTime(toDate('2010-10-07'), '%Q'); \ No newline at end of file From b90b38fd4ead358fdfa9da6d28b63013756abd3e Mon Sep 17 00:00:00 2001 From: jianmei zhang Date: Mon, 18 Jan 2021 12:20:15 +0800 Subject: [PATCH 256/264] add a new line --- .../01655_quarter_modificator_for_formatDateTime.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql b/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql index 35a6f6e2ab7..0d8a764082e 100644 --- a/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql +++ b/tests/queries/0_stateless/01655_quarter_modificator_for_formatDateTime.sql @@ -1,4 +1,4 @@ SELECT formatDateTime(toDate('2010-01-04'), '%Q'); SELECT formatDateTime(toDate('2010-04-30'), '%Q'); SELECT formatDateTime(toDate('2010-07-31'), '%Q'); -SELECT formatDateTime(toDate('2010-10-07'), '%Q'); \ No newline at end of file +SELECT formatDateTime(toDate('2010-10-07'), '%Q'); From 7ebe04aed0675626cbfb43bba6a6a4532a4bc4c7 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 18 Jan 2021 13:59:56 +0300 Subject: [PATCH 257/264] Update README.md --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index 97d6eb605e6..61adf4c9cc0 100644 --- a/README.md +++ b/README.md @@ -17,3 +17,4 @@ ClickHouse® is an open-source column-oriented database management system that a ## Upcoming Events * [SF Bay Area ClickHouse Virtual Office Hours (online)](https://www.meetup.com/San-Francisco-Bay-Area-ClickHouse-Meetup/events/274273549/) on 20 January 2020. +* [Chinese ClickHouse Meetup (online)](http://hdxu.cn/8KxZE) on 6 February 2020. From cafc6a492dd255299838a28b8095a2daab3c1344 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Mon, 18 Jan 2021 14:00:24 +0300 Subject: [PATCH 258/264] Update jit_large_requests.xml --- tests/performance/jit_large_requests.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/performance/jit_large_requests.xml b/tests/performance/jit_large_requests.xml index fe7d4346152..f0e1583e5fa 100644 --- a/tests/performance/jit_large_requests.xml +++ b/tests/performance/jit_large_requests.xml @@ -28,8 +28,8 @@ number, number FROM - system.numbers - LIMIT 100000000 + system.numbers_mt + LIMIT 200000000 SELECT @@ -41,7 +41,7 @@ SETTINGS compile_expressions = 0; - + SELECT COUNT() FROM From 897d51b6e707945d3c2186911cdeff1f6ac9be2b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 18 Jan 2021 14:38:31 +0300 Subject: [PATCH 259/264] Usability improvement of clickhouse-test --- tests/clickhouse-test | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index da7d055f250..e095339ec76 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -293,6 +293,8 @@ def run_tests_array(all_tests_with_params): try: sys.stdout.flush() sys.stdout.write("{0:72}".format(name + ": ")) + # This flush is needed so you can see the test name of the long running test before it will finish. + sys.stdout.flush() if args.skip and any(s in name for s in args.skip): print(MSG_SKIPPED + " - skip") From c99b5a78a5bfc3f70b7dfc3df839cfcdd9c1781a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 18 Jan 2021 15:06:51 +0300 Subject: [PATCH 260/264] Update version_date.tsv after release 21.1.2.15 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index cf6778317cc..fc1cf7c1b67 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v21.1.2.15-stable 2021-01-18 v20.12.5.14-stable 2020-12-28 v20.12.4.5-stable 2020-12-24 v20.12.3.3-stable 2020-12-09 From d8a9c969c6d98cf3eea7b393c4c885acc5a61e1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 18 Jan 2021 16:07:12 +0300 Subject: [PATCH 261/264] Add changelog for version 21.1 --- CHANGELOG.md | 3621 ++------------------------ docs/en/whats-new/changelog/2020.md | 3376 ++++++++++++++++++++++++ docs/en/whats-new/changelog/index.md | 2 +- 3 files changed, 3656 insertions(+), 3343 deletions(-) create mode 100644 docs/en/whats-new/changelog/2020.md diff --git a/CHANGELOG.md b/CHANGELOG.md index 2ec48bcd584..fbaf92ce659 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3376 +1,313 @@ -### ClickHouse release 20.12 - -### ClickHouse release v20.12.4.5-stable, 2020-12-24 - -#### Bug Fix - -* Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual IPv4/IPv6 stack; - Fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes; Possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). -* Fixed key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). -* Fixed unique key convert crash in `MaterializeMySQL` database engine. This fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) and fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fixed the issue when some tables not synchronized to ClickHouse from MySQL caused by the fact that convertion MySQL prefix index wasn't supported for MaterializeMySQL. This fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) and fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed the issue when query optimization was producing wrong result if query contains `ARRAY JOIN`. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). -* Fixed possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). -* Fixed empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). - - -### ClickHouse release v20.12.3.3-stable, 2020-12-13 +622 PRs added between v20.12.5.14-stable and v21.1.2.15-stable. +curl: (22) The requested URL returned error: 404 Not Found +Failed to download 'https://api.github.com/repos/ClickHouse/ClickHouse/pulls/18701' to 'pr18701.json'. Contents: ''. +curl: (28) Failed to connect to api.github.com port 443: Connection timed out +Failed to download 'https://api.github.com/repos/ClickHouse/ClickHouse/pulls/17832' to 'pr17832.json'. Contents: ''. +/home/milovidov/.local/lib/python3.8/site-packages/fuzzywuzzy/fuzz.py:11: UserWarning: Using slow pure-python SequenceMatcher. Install python-Levenshtein to remove this warning + warnings.warn('Using slow pure-python SequenceMatcher. Install python-Levenshtein to remove this warning') +### ClickHouse release v21.1.2.15-stable FIXME as compared to v20.12.5.14-stable #### Backward Incompatible Change -* Enable `use_compact_format_in_distributed_parts_names` by default (see the documentation for the reference). [#16728](https://github.com/ClickHouse/ClickHouse/pull/16728) ([Azat Khuzhin](https://github.com/azat)). -* Accept user settings related to file formats (e.g. `format_csv_delimiter`) in the `SETTINGS` clause when creating a table that uses `File` engine, and use these settings in all `INSERT`s and `SELECT`s. The file format settings changed in the current user session, or in the `SETTINGS` clause of a DML query itself, no longer affect the query. [#16591](https://github.com/ClickHouse/ClickHouse/pull/16591) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Remove `sumburConsistentHash` function. This closes [#18120](https://github.com/ClickHouse/ClickHouse/issues/18120). [#18656](https://github.com/ClickHouse/ClickHouse/pull/18656) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Check settings constraints for profile settings from config. Server will fail to start if users.xml contain settings that do not meet corresponding constraints. [#18486](https://github.com/ClickHouse/ClickHouse/pull/18486) ([tavplubix](https://github.com/tavplubix)). +* Restrict `ALTER MODIFY SETTING` from changing storage settings that affects data parts (`write_final_mark` and `enable_mixed_granularity_parts`). [#18306](https://github.com/ClickHouse/ClickHouse/pull/18306) ([Amos Bird](https://github.com/amosbird)). +* Set `insert_quorum_parallel` to 1 by default. It is significantly more convenient to use than "sequential" quorum inserts. But if you rely to sequential consistency, you should set the setting back to zero. [#17567](https://github.com/ClickHouse/ClickHouse/pull/17567) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* The setting `input_format_null_as_default` is enabled by default. [#17525](https://github.com/ClickHouse/ClickHouse/pull/17525) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Removed aggregate functions `timeSeriesGroupSum`, `timeSeriesGroupRateSum` because a friend of mine said they never worked. This fixes [#16869](https://github.com/ClickHouse/ClickHouse/issues/16869). If you have luck using these functions, write a email to clickhouse-feedback@yandex-team.com. [#17423](https://github.com/ClickHouse/ClickHouse/pull/17423) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Prohibit toUnixTimestamp(Date()) (before it just returns UInt16 representation of Date). [#17376](https://github.com/ClickHouse/ClickHouse/pull/17376) ([Azat Khuzhin](https://github.com/azat)). +* Allow using extended integer types (`Int128`, `Int256`, `UInt256`) in `avg` and `avgWeighted` functions. Also allow using different types (integer, decimal, floating point) for value and for weight in `avgWeighted` function. This is a backward-incompatible change: now the `avg` and `avgWeighted` functions always return `Float64` (as documented). Before this change the return type for `Decimal` arguments was also `Decimal`. [#15419](https://github.com/ClickHouse/ClickHouse/pull/15419) ([Mike](https://github.com/myrrc)). #### New Feature -* add `*.xz` compression/decompression support.It enables using `*.xz` in `file()` function. This closes [#8828](https://github.com/ClickHouse/ClickHouse/issues/8828). [#16578](https://github.com/ClickHouse/ClickHouse/pull/16578) ([Abi Palagashvili](https://github.com/fibersel)). -* Introduce the query `ALTER TABLE ... DROP|DETACH PART 'part_name'`. [#15511](https://github.com/ClickHouse/ClickHouse/pull/15511) ([nvartolomei](https://github.com/nvartolomei)). -* Added new ALTER UPDATE/DELETE IN PARTITION syntax. [#13403](https://github.com/ClickHouse/ClickHouse/pull/13403) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Allow formatting named tuples as JSON objects when using JSON input/output formats, controlled by the `output_format_json_named_tuples_as_objects` setting, disabled by default. [#17175](https://github.com/ClickHouse/ClickHouse/pull/17175) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Add a possibility to input enum value as it's id in TSV and CSV formats by default. [#16834](https://github.com/ClickHouse/ClickHouse/pull/16834) ([Kruglov Pavel](https://github.com/Avogar)). -* Add COLLATE support for Nullable, LowCardinality, Array and Tuple, where nested type is String. Also refactor the code associated with collations in ColumnString.cpp. [#16273](https://github.com/ClickHouse/ClickHouse/pull/16273) ([Kruglov Pavel](https://github.com/Avogar)). -* New `tcpPort` function returns TCP port listened by this server. [#17134](https://github.com/ClickHouse/ClickHouse/pull/17134) ([Ivan](https://github.com/abyss7)). -* Add new math functions: `acosh`, `asinh`, `atan2`, `atanh`, `cosh`, `hypot`, `log1p`, `sinh`. [#16636](https://github.com/ClickHouse/ClickHouse/pull/16636) ([Konstantin Malanchev](https://github.com/hombit)). -* Possibility to distribute the merges between different replicas. Introduces the `execute_merges_on_single_replica_time_threshold` mergetree setting. [#16424](https://github.com/ClickHouse/ClickHouse/pull/16424) ([filimonov](https://github.com/filimonov)). -* Add setting `aggregate_functions_null_for_empty` for SQL standard compatibility. This option will rewrite all aggregate functions in a query, adding -OrNull suffix to them. Implements [10273](https://github.com/ClickHouse/ClickHouse/issues/10273). [#16123](https://github.com/ClickHouse/ClickHouse/pull/16123) ([flynn](https://github.com/ucasFL)). -* Updated DateTime, DateTime64 parsing to accept string Date literal format. [#16040](https://github.com/ClickHouse/ClickHouse/pull/16040) ([Maksim Kita](https://github.com/kitaisreal)). -* Make it possible to change the path to history file in `clickhouse-client` using the `--history_file` parameter. [#15960](https://github.com/ClickHouse/ClickHouse/pull/15960) ([Maksim Kita](https://github.com/kitaisreal)). +* Function `position` now supports `position(needle in haystack)` synax for SQL compatibility. This closes [#18701](https://github.com/ClickHouse/ClickHouse/issues/18701). ... [#18779](https://github.com/ClickHouse/ClickHouse/pull/18779) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* Now we have a new storage setting `max_partitions_to_read` for tables in the MergeTree family. It limits the max number of partitions that can be accessed in one query. A user setting `force_max_partition_limit` is also added to enforce this constraint. [#18712](https://github.com/ClickHouse/ClickHouse/pull/18712) ([Amos Bird](https://github.com/amosbird)). +* Add `query_id` column to `system.part_log` for inserted parts. closes [#10097](https://github.com/ClickHouse/ClickHouse/issues/10097). [#18644](https://github.com/ClickHouse/ClickHouse/pull/18644) ([flynn](https://github.com/ucasFL)). +* Implemented `REPLACE TABLE` and `CREATE OR REPLACE TABLE` queries. [#18521](https://github.com/ClickHouse/ClickHouse/pull/18521) ([tavplubix](https://github.com/tavplubix)). +* - IP Dictionary supports key fetching. Resolves [#18241](https://github.com/ClickHouse/ClickHouse/issues/18241). [#18480](https://github.com/ClickHouse/ClickHouse/pull/18480) ([vdimir](https://github.com/vdimir)). +* Allow create table as select with columns specification. Example `CREATE TABLE t1 (x String) ENGINE = Memory AS SELECT 1;`. [#18060](https://github.com/ClickHouse/ClickHouse/pull/18060) ([Maksim Kita](https://github.com/kitaisreal)). +* Support `SHOW SETTINGS` statement to show parameters in system.settings. `SHOW CHANGED SETTINGS` and `LIKE/ILIKE` clause are also supported. [#18056](https://github.com/ClickHouse/ClickHouse/pull/18056) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* Added `arrayMin`, `arrayMax`, `arrayAvg` aggregation functions. [#18032](https://github.com/ClickHouse/ClickHouse/pull/18032) ([Maksim Kita](https://github.com/kitaisreal)). +* Implemented `ATTACH TABLE name FROM 'path/to/data/' (col1 Type1, ...` query. It creates new table with provided structure and attaches table data from provided directory in `user_files`. [#17903](https://github.com/ClickHouse/ClickHouse/pull/17903) ([tavplubix](https://github.com/tavplubix)). +* Add settings `min_compress_block_size` and `max_compress_block_size` to MergeTreeSettings, which have higher priority than the global settings and take effect when they are set. close [13890](https://github.com/ClickHouse/ClickHouse/issues/13890). [#17867](https://github.com/ClickHouse/ClickHouse/pull/17867) ([flynn](https://github.com/ucasFL)). +* Add bitmap64 feature. [#17858](https://github.com/ClickHouse/ClickHouse/pull/17858) ([Andy Yang](https://github.com/andyyzh)). +* Extended `OPTIMIZE ... DEDUPLICATE` syntax to allow explicit (or implicit with asterisk/column transformers) list of columns to check for duplicates on. ... [#17846](https://github.com/ClickHouse/ClickHouse/pull/17846) ([Vasily Nemkov](https://github.com/Enmk)). +* Added functions `toMJD`, `fromMJD`, `toMJDOrNull`, and `fromMJDOrNull`. These functions convert between Proleptic Gregorian calendar date and Modified Julian Day number. [#17750](https://github.com/ClickHouse/ClickHouse/pull/17750) ([PHO](https://github.com/depressed-pho)). +* Add ability to use custom TLD list in functions `firstSignificantSubdomainCustom`, `cutToFirstSignificantSubdomainCustom`. [#17748](https://github.com/ClickHouse/ClickHouse/pull/17748) ([Azat Khuzhin](https://github.com/azat)). +* Add support for PROXYv1 protocol to wrap native TCP interface. Allow quotas to be keyed by proxy-forwarded IP address (applied for PROXYv1 address and for X-Forwarded-For from HTTP interface). This is useful when you provide access to ClickHouse only via trusted proxy (e.g. CloudFlare) but want to account user resources by their original IP addresses. This fixes [#17268](https://github.com/ClickHouse/ClickHouse/issues/17268). [#17707](https://github.com/ClickHouse/ClickHouse/pull/17707) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now clickhouse-client supports opening EDITOR to edit commands. `Alt-Shift-E`. [#17665](https://github.com/ClickHouse/ClickHouse/pull/17665) ([Amos Bird](https://github.com/amosbird)). +* Add function `encodeXMLComponent` to escape characters to place string into XML text node or attribute. [#17659](https://github.com/ClickHouse/ClickHouse/pull/17659) ([nauta](https://github.com/nautaa)). +* Introduce `DETACH TABLE/VIEW ... PERMANENTLY` syntax, so that after restarting the table does not reappear back automatically (only by explicit request). The table can still be attached back using the short syntax ATTACH TABLE. Implements [#5555](https://github.com/ClickHouse/ClickHouse/issues/5555). Fixes [#13850](https://github.com/ClickHouse/ClickHouse/issues/13850). [#17642](https://github.com/ClickHouse/ClickHouse/pull/17642) ([filimonov](https://github.com/filimonov)). +* Add asynchronous metrics on total amount of rows, bytes and parts in MergeTree tables. This fix [#11714](https://github.com/ClickHouse/ClickHouse/issues/11714). [#17639](https://github.com/ClickHouse/ClickHouse/pull/17639) ([flynn](https://github.com/ucasFL)). +* related: [#16176](https://github.com/ClickHouse/ClickHouse/issues/16176) Usage: ``` set limit = 10; set offset = 20; ``` this two settings will affect SELECT query as if it is added like ``` select * from ($your_original_select_query) tmp limit xxx offset xxx; ```. [#17633](https://github.com/ClickHouse/ClickHouse/pull/17633) ([hexiaoting](https://github.com/hexiaoting)). +* * IP Dictionary supports `IPv4` / `IPv6` types directly. [#17571](https://github.com/ClickHouse/ClickHouse/pull/17571) ([vdimir](https://github.com/vdimir)). +* add ```*.zst``` compression/decompression support.It enables using ```*.zst``` in ```file()``` function and ```Content-encoding: zstd``` in http client.This closes [#16791 ](https://github.com/ClickHouse/ClickHouse/issues/16791). [#17144](https://github.com/ClickHouse/ClickHouse/pull/17144) ([Abi Palagashvili](https://github.com/fibersel)). +* Add a setting optimize_on_insert. When enabled, do the same transformation for INSERTed block of data as if merge was done on this block (e.g. Replacing, Collapsing, Aggregating...). This setting will be enabled as default. This can influence Materialized View and MaterializeMySQL behaviour (see detailed description). This closes [#10683](https://github.com/ClickHouse/ClickHouse/issues/10683). [#16954](https://github.com/ClickHouse/ClickHouse/pull/16954) ([Kruglov Pavel](https://github.com/Avogar)). +* Added `mannWitneyUTest`, `studentTTest` and `welchTTest` aggregate functions. Refactored RankCorr a bit. [#16883](https://github.com/ClickHouse/ClickHouse/pull/16883) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Provide a new aggregator combinator : `-SimpleState` to build SimpleAggregateFunction types via query. It's useful for defining MaterializedView of AggregatingMergeTree engine, and will benefit projections too. [#16853](https://github.com/ClickHouse/ClickHouse/pull/16853) ([Amos Bird](https://github.com/amosbird)). +* ... [#16819](https://github.com/ClickHouse/ClickHouse/pull/16819) ([Aleksandrov Vladimir](https://github.com/invis87)). +* Use https://github.com/lemire/fast_float to parse floating point numbers. [#16787](https://github.com/ClickHouse/ClickHouse/pull/16787) ([Maksim Kita](https://github.com/kitaisreal)). +* Added function `accurateCastOrNull`. This closes [#10290](https://github.com/ClickHouse/ClickHouse/issues/10290). Add type conversions in `x IN (subquery)` expressions. This closes [#10266](https://github.com/ClickHouse/ClickHouse/issues/10266). [#16724](https://github.com/ClickHouse/ClickHouse/pull/16724) ([Maksim Kita](https://github.com/kitaisreal)). +* Kerberos Authenticaiton for HDFS. [#16621](https://github.com/ClickHouse/ClickHouse/pull/16621) ([Ilya Golshtein](https://github.com/ilejn)). +* Implement `UNION DISTINCT` and treat the plain `UNION` clause as `UNION DISTINCT` by default. Add a setting `union_default_mode` that allows to treat it as `UNION ALL` or require explicit mode specification. [#16338](https://github.com/ClickHouse/ClickHouse/pull/16338) ([flynn](https://github.com/ucasFL)). +* Added `queries-file` parameter for clickhouse-client and clickhouse-local. [#15930](https://github.com/ClickHouse/ClickHouse/pull/15930) ([Maksim Kita](https://github.com/kitaisreal)). +* Add new datatype Map for supporting storage k:v . related to issue: [#1841](https://github.com/ClickHouse/ClickHouse/issues/1841) 1st version for Map only support String type of key and value. Later I will implement Int and other type for key and value. [#15806](https://github.com/ClickHouse/ClickHouse/pull/15806) ([hexiaoting](https://github.com/hexiaoting)). +* Implement gRPC protocol in ClickHouse. [#15111](https://github.com/ClickHouse/ClickHouse/pull/15111) ([Vitaly Baranov](https://github.com/vitlibar)). +* - parallel parsing was rewritten to processors - parallel formatting was implemented. [#11617](https://github.com/ClickHouse/ClickHouse/pull/11617) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Added functions `Simhash`, `Minhash`, `bitHammingDistance`, `tupleHammingDistance`. [#7649](https://github.com/ClickHouse/ClickHouse/pull/7649) ([flynn](https://github.com/ucasFL)). #### Bug Fix -* Fix the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([Amos Bird](https://github.com/amosbird)). -* Fixed `Function not implemented` error when executing `RENAME` query in `Atomic` database with ClickHouse running on Windows Subsystem for Linux. Fixes [#17661](https://github.com/ClickHouse/ClickHouse/issues/17661). [#17664](https://github.com/ClickHouse/ClickHouse/pull/17664) ([tavplubix](https://github.com/tavplubix)). +* - Split RemoteQueryExecutorReadContext into module part - Fix leaking of pipe fd for `async_socket_for_remote`. [#19153](https://github.com/ClickHouse/ClickHouse/pull/19153) ([Azat Khuzhin](https://github.com/azat)). +* Fix infinite reading from file in `ORC` format (was introduced in [#10580](https://github.com/ClickHouse/ClickHouse/issues/10580)). Fixes [#19095](https://github.com/ClickHouse/ClickHouse/issues/19095). [#19134](https://github.com/ClickHouse/ClickHouse/pull/19134) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix bug in merge tree data writer which can lead to marks with bigger size than fixed granularity size. Fixes [#18913](https://github.com/ClickHouse/ClickHouse/issues/18913). [#19123](https://github.com/ClickHouse/ClickHouse/pull/19123) ([alesapin](https://github.com/alesapin)). +* Fix startup bug when clickhouse was not able to read compression codec from `LowCardinality(Nullable(...))` and throws exception `Attempt to read after EOF`. Fixes [#18340](https://github.com/ClickHouse/ClickHouse/issues/18340). [#19101](https://github.com/ClickHouse/ClickHouse/pull/19101) ([alesapin](https://github.com/alesapin)). +* Simplify the implementation of `tupleHammingDistance`. Support for tuples of any equal length. Fixes [#19029](https://github.com/ClickHouse/ClickHouse/issues/19029). [#19084](https://github.com/ClickHouse/ClickHouse/pull/19084) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Restrict `MODIFY TTL` queries for `MergeTree` tables created in old syntax. Previously the query succeeded, but actually it had no effect. [#19064](https://github.com/ClickHouse/ClickHouse/pull/19064) ([Anton Popov](https://github.com/CurtizJ)). +* Make sure `groupUniqArray` returns correct type for argument of Enum type. This closes [#17875](https://github.com/ClickHouse/ClickHouse/issues/17875). [#19019](https://github.com/ClickHouse/ClickHouse/pull/19019) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible error `Expected single dictionary argument for function` if use function `ignore` with `LowCardinality` argument. Fixes [#14275](https://github.com/ClickHouse/ClickHouse/issues/14275). [#19016](https://github.com/ClickHouse/ClickHouse/pull/19016) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix inserting of `LowCardinality` column to table with `TinyLog` engine. Fixes [#18629](https://github.com/ClickHouse/ClickHouse/issues/18629). [#19010](https://github.com/ClickHouse/ClickHouse/pull/19010) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Join tries to materialize const columns, but our code waits for them in other places. [#18982](https://github.com/ClickHouse/ClickHouse/pull/18982) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Disable `optimize_move_functions_out_of_any` because optimization is not always correct. This closes [#18051](https://github.com/ClickHouse/ClickHouse/issues/18051). This closes [#18973](https://github.com/ClickHouse/ClickHouse/issues/18973). [#18981](https://github.com/ClickHouse/ClickHouse/pull/18981) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible exception `QueryPipeline stream: different number of columns` caused by merging of query plan's `Expression` steps. Fixes [#18190](https://github.com/ClickHouse/ClickHouse/issues/18190). [#18980](https://github.com/ClickHouse/ClickHouse/pull/18980) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed very rare deadlock at shutdown. [#18977](https://github.com/ClickHouse/ClickHouse/pull/18977) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect behavior when `ALTER TABLE ... DROP PART 'part_name'` query removes all deduplication blocks for the whole partition. Fixes [#18874](https://github.com/ClickHouse/ClickHouse/issues/18874). [#18969](https://github.com/ClickHouse/ClickHouse/pull/18969) ([alesapin](https://github.com/alesapin)). +* Fix error `Task was not found in task queue` (possible only for remote queries, with `async_socket_for_remote = 1`). [#18964](https://github.com/ClickHouse/ClickHouse/pull/18964) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix bug when mutation with some escaped text (like `ALTER ... UPDATE e = CAST('foo', 'Enum8(\'foo\' = 1')` serialized incorrectly. Fixes [#18878](https://github.com/ClickHouse/ClickHouse/issues/18878). [#18944](https://github.com/ClickHouse/ClickHouse/pull/18944) ([alesapin](https://github.com/alesapin)). +* Attach partition should reset the mutation. [#18804](https://github.com/ClickHouse/ClickHouse/issues/18804). [#18935](https://github.com/ClickHouse/ClickHouse/pull/18935) ([fastio](https://github.com/fastio)). +* Fix issue with `bitmapOrCardinality` that may lead to nullptr dereference. This closes [#18911](https://github.com/ClickHouse/ClickHouse/issues/18911). [#18912](https://github.com/ClickHouse/ClickHouse/pull/18912) ([sundyli](https://github.com/sundy-li)). +* Fix possible hang at shutdown in clickhouse-local. This fixes [#18891](https://github.com/ClickHouse/ClickHouse/issues/18891). [#18893](https://github.com/ClickHouse/ClickHouse/pull/18893) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* change the sorting key of events_list from timestamp to (timestamp, event_index). [#18884](https://github.com/ClickHouse/ClickHouse/pull/18884) ([Fuwang Hu](https://github.com/fuwhu)). +* Queries for external databases (MySQL, ODBC, JDBC) were incorrectly rewritten if there was an expression in form of `x IN table`. This fixes [#9756](https://github.com/ClickHouse/ClickHouse/issues/9756). [#18876](https://github.com/ClickHouse/ClickHouse/pull/18876) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix use after free bug in rocksdb. [#18862](https://github.com/ClickHouse/ClickHouse/pull/18862) ([sundyli](https://github.com/sundy-li)). +* - Fix never worked `fsync_part_directory`/`fsync_after_insert`/`in_memory_parts_insert_sync`. [#18845](https://github.com/ClickHouse/ClickHouse/pull/18845) ([Azat Khuzhin](https://github.com/azat)). +* Fix *If combinator with unary function and Nullable types. [#18806](https://github.com/ClickHouse/ClickHouse/pull/18806) ([Azat Khuzhin](https://github.com/azat)). +* Asynchronous distributed INSERTs can be rejected by the server if the setting `network_compression_method` is globally set to non-default value. This fixes [#18741](https://github.com/ClickHouse/ClickHouse/issues/18741). [#18776](https://github.com/ClickHouse/ClickHouse/pull/18776) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Attempt to read after eof` error when trying to `CAST` `NULL` from `Nullable(String)` to `Nullable(Decimal(P, S))`. Now function `CAST` returns `NULL` when it cannot parse decimal from nullable string. Fixes [#7690](https://github.com/ClickHouse/ClickHouse/issues/7690). [#18718](https://github.com/ClickHouse/ClickHouse/pull/18718) ([Winter Zhang](https://github.com/zhang2014)). +* Fix Logger with unmatched arg size. [#18717](https://github.com/ClickHouse/ClickHouse/pull/18717) ([sundyli](https://github.com/sundy-li)). +* Fix removing of empty parts in `ReplicatedMergeTree` tables, created with old syntax. Fixes [#18582](https://github.com/ClickHouse/ClickHouse/issues/18582). [#18614](https://github.com/ClickHouse/ClickHouse/pull/18614) ([Anton Popov](https://github.com/CurtizJ)). +* Fix previous bug when date overflow with different values. Strict Date value limit to "2106-02-07", cast date > "2106-02-07" to value 0. [#18565](https://github.com/ClickHouse/ClickHouse/pull/18565) ([hexiaoting](https://github.com/hexiaoting)). +* Add FixedString Data type support. I'll get this exception "Code: 50, e.displayText() = DB::Exception: Unsupported type FixedString(1)" when replicating data from MySQL to ClickHouse. This patch fixes bug [#18450](https://github.com/ClickHouse/ClickHouse/issues/18450) Also fixes [#6556](https://github.com/ClickHouse/ClickHouse/issues/6556). [#18553](https://github.com/ClickHouse/ClickHouse/pull/18553) ([awesomeleo](https://github.com/awesomeleo)). +* Fix possible `Pipeline stuck` error while using `ORDER BY` after subquery with `RIGHT` or `FULL` join. [#18550](https://github.com/ClickHouse/ClickHouse/pull/18550) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix bug which may lead to `ALTER` queries hung after corresponding mutation kill. Found by thread fuzzer. [#18518](https://github.com/ClickHouse/ClickHouse/pull/18518) ([alesapin](https://github.com/alesapin)). +* Disable write with AIO during merges because it can lead to extremely rare data corruption of primary key columns during merge. [#18481](https://github.com/ClickHouse/ClickHouse/pull/18481) ([alesapin](https://github.com/alesapin)). +* Proper support for 12AM in `parseDateTimeBestEffort` function. This fixes [#18402](https://github.com/ClickHouse/ClickHouse/issues/18402). [#18449](https://github.com/ClickHouse/ClickHouse/pull/18449) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). +* Fixed `value is too short` error when executing `toType(...)` functions (`toDate`, `toUInt32`, etc) with argument of type `Nullable(String)`. Now such functions return `NULL` on parsing errors instead of throwing exception. Fixes [#7673](https://github.com/ClickHouse/ClickHouse/issues/7673). [#18445](https://github.com/ClickHouse/ClickHouse/pull/18445) ([tavplubix](https://github.com/tavplubix)). +* Fix the unexpected behaviour of `SHOW TABLES`. [#18431](https://github.com/ClickHouse/ClickHouse/pull/18431) ([fastio](https://github.com/fastio)). +* Fix -SimpleState combinator generates incompatible arugment type and return type. [#18404](https://github.com/ClickHouse/ClickHouse/pull/18404) ([Amos Bird](https://github.com/amosbird)). +* Fix possible race condition in concurrent usage of `Set` or `Join` tables and selects from `system.tables`. [#18385](https://github.com/ClickHouse/ClickHouse/pull/18385) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Restrict merges from wide to compact parts. In case of vertical merge it led to broken result part. [#18381](https://github.com/ClickHouse/ClickHouse/pull/18381) ([Anton Popov](https://github.com/CurtizJ)). +* Fix filling table `system.settings_profile_elements`. This PR fixes [#18231](https://github.com/ClickHouse/ClickHouse/issues/18231). [#18379](https://github.com/ClickHouse/ClickHouse/pull/18379) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix index analysis of binary functions with constant argument which leads to wrong query results. This fixes [#18364](https://github.com/ClickHouse/ClickHouse/issues/18364). [#18373](https://github.com/ClickHouse/ClickHouse/pull/18373) ([Amos Bird](https://github.com/amosbird)). +* Fix possible crashes in aggregate functions with combinator `Distinct`, while using two-level aggregation. Fixes [#17682](https://github.com/ClickHouse/ClickHouse/issues/17682). [#18365](https://github.com/ClickHouse/ClickHouse/pull/18365) ([Anton Popov](https://github.com/CurtizJ)). +* - Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual IPv4/IPv6 stack; - Fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes; Possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). +* `SELECT count() FROM table` now can be executed if only one any column can be selected from the `table`. This PR fixes [#10639](https://github.com/ClickHouse/ClickHouse/issues/10639). [#18233](https://github.com/ClickHouse/ClickHouse/pull/18233) ([Vitaly Baranov](https://github.com/vitlibar)). +* `SELECT JOIN` now requires the `SELECT` privilege on each of the joined tables. This PR fixes [#17654](https://github.com/ClickHouse/ClickHouse/issues/17654). [#18232](https://github.com/ClickHouse/ClickHouse/pull/18232) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix possible incomplete query result while reading from `MergeTree*` in case of read backoff (message ` MergeTreeReadPool: Will lower number of threads` in logs). Was introduced in [#16423](https://github.com/ClickHouse/ClickHouse/issues/16423). Fixes [#18137](https://github.com/ClickHouse/ClickHouse/issues/18137). [#18216](https://github.com/ClickHouse/ClickHouse/pull/18216) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). +* fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) fix unique key convert crash in MaterializeMySQL database engine. [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). +* Related to [#17466](https://github.com/ClickHouse/ClickHouse/issues/17466). [#18188](https://github.com/ClickHouse/ClickHouse/pull/18188) ([hexiaoting](https://github.com/hexiaoting)). +* Fix inserting a row with default value in case of parsing error in the last column. Fixes [#17712](https://github.com/ClickHouse/ClickHouse/issues/17712). [#18182](https://github.com/ClickHouse/ClickHouse/pull/18182) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* Fix `Unknown setting profile` error on attempt to set settings profile. [#18167](https://github.com/ClickHouse/ClickHouse/pull/18167) ([tavplubix](https://github.com/tavplubix)). +* Fix error when query `MODIFY COLUMN ... REMOVE TTL` doesn't actually remove column TTL. [#18130](https://github.com/ClickHouse/ClickHouse/pull/18130) ([alesapin](https://github.com/alesapin)). +* Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix corruption in librdkafka snappy decompression (was a problem only for gcc10 builds, but official builds uses clang already, so at least recent official releases are not affected). [#18053](https://github.com/ClickHouse/ClickHouse/pull/18053) ([Azat Khuzhin](https://github.com/azat)). +* Fix comparison of `DateTime64` and `Date`. Fixes [#13804](https://github.com/ClickHouse/ClickHouse/issues/13804) and [#11222](https://github.com/ClickHouse/ClickHouse/issues/11222). ... [#18050](https://github.com/ClickHouse/ClickHouse/pull/18050) ([Vasily Nemkov](https://github.com/Enmk)). +* fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) support convert MySQL prefix index for MaterializeMySQL CC: @tavplubix. [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). +* When server log rotation was configured using `logger.size` parameter with numeric value larger than 2^32, the logs were not rotated properly. This is fixed. [#17905](https://github.com/ClickHouse/ClickHouse/pull/17905) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix comparison of `DateTime64` and `Date`. Fixes [#13804](https://github.com/ClickHouse/ClickHouse/issues/13804) and [#11222](https://github.com/ClickHouse/ClickHouse/issues/11222). ... [#17895](https://github.com/ClickHouse/ClickHouse/pull/17895) ([Vasily Nemkov](https://github.com/Enmk)). +* Trivial query optimization was producing wrong result if query contains ARRAY JOIN (so query is actually non trivial). [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). +* Fix max_distributed_connections (affects `prefer_localhost_replica=1` and `max_threads!=max_distributed_connections`). [#17848](https://github.com/ClickHouse/ClickHouse/pull/17848) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). +* fix incorrect initialize `max_compress_block_size` of MergeTreeWriterSettings with `min_compress_block_size`. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). * Do not restore parts from WAL if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). -* fix incorrect initialization of `max_compress_block_size` of MergeTreeWriterSettings with `min_compress_block_size`. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). * Exception message about max table size to drop was displayed incorrectly. [#17764](https://github.com/ClickHouse/ClickHouse/pull/17764) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed possible segfault when there is not enough space when inserting into `Distributed` table. [#17737](https://github.com/ClickHouse/ClickHouse/pull/17737) ([tavplubix](https://github.com/tavplubix)). +* Fixed segfault when there is not enough space when inserting into `Distributed` table. [#17737](https://github.com/ClickHouse/ClickHouse/pull/17737) ([tavplubix](https://github.com/tavplubix)). * Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). +* Fixed `Function not implemented` error when executing `RENAME` query in `Atomic` database with ClickHouse running on Windows Subsystem for Linux. Fixes [#17661](https://github.com/ClickHouse/ClickHouse/issues/17661). [#17664](https://github.com/ClickHouse/ClickHouse/pull/17664) ([tavplubix](https://github.com/tavplubix)). * In might be determined incorrectly if cluster is circular- (cross-) replicated or not when executing `ON CLUSTER` query due to race condition when `pool_size` > 1. It's fixed. [#17640](https://github.com/ClickHouse/ClickHouse/pull/17640) ([tavplubix](https://github.com/tavplubix)). +* Fix empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). * Exception `fmt::v7::format_error` can be logged in background for MergeTree tables. This fixes [#17613](https://github.com/ClickHouse/ClickHouse/issues/17613). [#17615](https://github.com/ClickHouse/ClickHouse/pull/17615) ([alexey-milovidov](https://github.com/alexey-milovidov)). * When clickhouse-client is used in interactive mode with multiline queries, single line comment was erronously extended till the end of query. This fixes [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix alter query hang when the corresponding mutation was killed on the different replica. Fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* Fix issue when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). +* Fix bug when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). * Fix `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). +* Avoid server abnormal termination in case of too low memory limits (`max_memory_usage=1`/`max_untracked_memory=1`). [#17453](https://github.com/ClickHouse/ClickHouse/pull/17453) ([Azat Khuzhin](https://github.com/azat)). * Fix duplicates after `DISTINCT` which were possible because of incorrect optimization. Fixes [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed high CPU usage in background tasks of *MergeTree tables. [#17416](https://github.com/ClickHouse/ClickHouse/pull/17416) ([tavplubix](https://github.com/tavplubix)). * Fix crash while reading from `JOIN` table with `LowCardinality` types. Fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* fix `toInt256(inf)` stack overflow. Int256 is an experimental feature. Closed [#17235](https://github.com/ClickHouse/ClickHouse/issues/17235). [#17257](https://github.com/ClickHouse/ClickHouse/pull/17257) ([flynn](https://github.com/ucasFL)). -* Fix possible `Unexpected packet Data received from client` error logged for Distributed queries with `LIMIT`. [#17254](https://github.com/ClickHouse/ClickHouse/pull/17254) ([Azat Khuzhin](https://github.com/azat)). -* Fix set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246). [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). +* fixes [#16835](https://github.com/ClickHouse/ClickHouse/issues/16835) try fix miss match header with MySQL SHOW statement. [#17366](https://github.com/ClickHouse/ClickHouse/pull/17366) ([Winter Zhang](https://github.com/zhang2014)). +* Fix indeterministic functions with predicate optimizer. This fixes [#17244](https://github.com/ClickHouse/ClickHouse/issues/17244). [#17273](https://github.com/ClickHouse/ClickHouse/pull/17273) ([Winter Zhang](https://github.com/zhang2014)). +* Fix possible `Unexpected packet Data received from client` error for Distributed queries with `LIMIT`. [#17254](https://github.com/ClickHouse/ClickHouse/pull/17254) ([Azat Khuzhin](https://github.com/azat)). +* Fix set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246) . [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). +* Fix [#15235](https://github.com/ClickHouse/ClickHouse/issues/15235). When clickhouse-copier handle non-partitioned table, throws segfault error. [#17248](https://github.com/ClickHouse/ClickHouse/pull/17248) ([Qi Chen](https://github.com/kaka11chen)). +* Fixed possible not-working mutations for parts stored on S3 disk. [#17227](https://github.com/ClickHouse/ClickHouse/pull/17227) ([Pavel Kovalenko](https://github.com/Jokser)). * Fix possible wrong index analysis when the types of the index comparison are different. This fixes [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). -* Fix ColumnConst comparison which leads to crash. This fixed [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* Multiple fixed for MaterializeMySQL (experimental feature). Fixes [#16923](https://github.com/ClickHouse/ClickHouse/issues/16923) Fixes [#15883](https://github.com/ClickHouse/ClickHouse/issues/15883) Fix MaterializeMySQL SYNC failure when the modify MySQL binlog_checksum. [#17091](https://github.com/ClickHouse/ClickHouse/pull/17091) ([Winter Zhang](https://github.com/zhang2014)). -* Fix bug when `ON CLUSTER` queries may hang forever for non-leader ReplicatedMergeTreeTables. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* Fixed crash on `CREATE TABLE ... AS some_table` query when `some_table` was created `AS table_function()` Fixes [#16944](https://github.com/ClickHouse/ClickHouse/issues/16944). [#17072](https://github.com/ClickHouse/ClickHouse/pull/17072) ([tavplubix](https://github.com/tavplubix)). -* Bug unfinished implementation for funciton fuzzBits, related issue: [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* Fix LLVM's libunwind in the case when CFA register is RAX. This is the [bug](https://bugs.llvm.org/show_bug.cgi?id=48186) in [LLVM's libunwind](https://github.com/llvm/llvm-project/tree/master/libunwind). We already have workarounds for this bug. [#17046](https://github.com/ClickHouse/ClickHouse/pull/17046) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* Fix `optimize_distributed_group_by_sharding_key` setting (that is disabled by default) for query with OFFSET only. [#16996](https://github.com/ClickHouse/ClickHouse/pull/16996) ([Azat Khuzhin](https://github.com/azat)). -* Fix for Merge tables over Distributed tables with JOIN. [#16993](https://github.com/ClickHouse/ClickHouse/pull/16993) ([Azat Khuzhin](https://github.com/azat)). -* Fixed wrong result in big integers (128, 256 bit) when casting from double. Big integers support is experimental. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). -* Fix possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* Blame info was not calculated correctly in `clickhouse-git-import`. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Bug fix for funciton fuzzBits, related issue: [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). +* - Fix optimize_distributed_group_by_sharding_key for query with OFFSET only. [#16996](https://github.com/ClickHouse/ClickHouse/pull/16996) ([Azat Khuzhin](https://github.com/azat)). +* Fix Merge(Distributed()) with JOIN. [#16993](https://github.com/ClickHouse/ClickHouse/pull/16993) ([Azat Khuzhin](https://github.com/azat)). * Fix order by optimization with monotonous functions. Fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). +* Fix incorrect comparison of types `DateTime64` with different scales. Fixes [#16655](https://github.com/ClickHouse/ClickHouse/issues/16655) ... [#16952](https://github.com/ClickHouse/ClickHouse/pull/16952) ([Vasily Nemkov](https://github.com/Enmk)). * Fix optimization of group by with enabled setting `optimize_aggregators_of_group_by_keys` and joins. Fixes [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). -* Fix possible error `Illegal type of argument` for queries with `ORDER BY`. Fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix strange code in InterpreterShowAccessQuery. [#16866](https://github.com/ClickHouse/ClickHouse/pull/16866) ([tavplubix](https://github.com/tavplubix)). -* Prevent clickhouse server crashes when using the function `timeSeriesGroupSum`. The function is removed from newer ClickHouse releases. [#16865](https://github.com/ClickHouse/ClickHouse/pull/16865) ([filimonov](https://github.com/filimonov)). -* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash when using `any` without any arguments. This is for [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803) . cc @azat. [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* If no memory can be allocated while writing table metadata on disk, broken metadata file can be written. [#16772](https://github.com/ClickHouse/ClickHouse/pull/16772) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix trivial query optimization with partition predicate. [#16767](https://github.com/ClickHouse/ClickHouse/pull/16767) ([Azat Khuzhin](https://github.com/azat)). -* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* TODO. [#16866](https://github.com/ClickHouse/ClickHouse/pull/16866) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when clickhouse-server doesn't send `close` request to ZooKeeper server. [#16837](https://github.com/ClickHouse/ClickHouse/pull/16837) ([alesapin](https://github.com/alesapin)). +* Fix optimize_trivial_count_query with partition predicate. [#16767](https://github.com/ClickHouse/ClickHouse/pull/16767) ([Azat Khuzhin](https://github.com/azat)). * Return number of affected rows for INSERT queries via MySQL protocol. Previously ClickHouse used to always return 0, it's fixed. Fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). -* Fix remote query failure when using 'if' suffix aggregate function. Fixes [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) Fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). * Fix inconsistent behavior caused by `select_sequential_consistency` for optimized trivial count query and system.tables. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). - -#### Improvement - -* Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm. [#16895](https://github.com/ClickHouse/ClickHouse/pull/16895) ([Anton Popov](https://github.com/CurtizJ)). -* Enable compact format of directories for asynchronous sends in Distributed tables: `use_compact_format_in_distributed_parts_names` is set to 1 by default. [#16788](https://github.com/ClickHouse/ClickHouse/pull/16788) ([Azat Khuzhin](https://github.com/azat)). -* Abort multipart upload if no data was written to S3. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* Mask password in data_path in the system.distribution_queue. [#16727](https://github.com/ClickHouse/ClickHouse/pull/16727) ([Azat Khuzhin](https://github.com/azat)). -* Throw error when use column transformer replaces non existing column. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). -* Turn off parallel parsing when there is no enough memory for all threads to work simultaneously. Also there could be exceptions like "Memory limit exceeded" when somebody will try to insert extremely huge rows (> min_chunk_bytes_for_parallel_parsing), because each piece to parse has to be independent set of strings (one or more). [#16721](https://github.com/ClickHouse/ClickHouse/pull/16721) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* Correct grammar in error message in JSONEachRow, JSONCompactEachRow, and RegexpRow input formats. [#17205](https://github.com/ClickHouse/ClickHouse/pull/17205) ([nico piderman](https://github.com/sneako)). -* Set default `host` and `port` parameters for `SOURCE(CLICKHOUSE(...))` to current instance and set default `user` value to `'default'`. [#16997](https://github.com/ClickHouse/ClickHouse/pull/16997) ([vdimir](https://github.com/vdimir)). -* Throw an informative error message when doing `ATTACH/DETACH TABLE `. Before this PR, `detach table ` works but leads to an ill-formed in-memory metadata. [#16885](https://github.com/ClickHouse/ClickHouse/pull/16885) ([Amos Bird](https://github.com/amosbird)). -* Add cutToFirstSignificantSubdomainWithWWW(). [#16845](https://github.com/ClickHouse/ClickHouse/pull/16845) ([Azat Khuzhin](https://github.com/azat)). -* Server refused to startup with exception message if wrong config is given (`metric_log`.`collect_interval_milliseconds` is missing). [#16815](https://github.com/ClickHouse/ClickHouse/pull/16815) ([Ivan](https://github.com/abyss7)). -* Better exception message when configuration for distributed DDL is absent. This fixes [#5075](https://github.com/ClickHouse/ClickHouse/issues/5075). [#16769](https://github.com/ClickHouse/ClickHouse/pull/16769) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Usability improvement: better suggestions in syntax error message when `CODEC` expression is misplaced in `CREATE TABLE` query. This fixes [#12493](https://github.com/ClickHouse/ClickHouse/issues/12493). [#16768](https://github.com/ClickHouse/ClickHouse/pull/16768) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove empty directories for async INSERT at start of Distributed engine. [#16729](https://github.com/ClickHouse/ClickHouse/pull/16729) ([Azat Khuzhin](https://github.com/azat)). -* Workaround for use S3 with nginx server as proxy. Nginx currenty does not accept urls with empty path like `http://domain.com?delete`, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like `http://domain.com/?delete`. [#16709](https://github.com/ClickHouse/ClickHouse/pull/16709) ([ianton-ru](https://github.com/ianton-ru)). -* Allow `reinterpretAs*` functions to work for integers and floats of the same size. Implements [16640](https://github.com/ClickHouse/ClickHouse/issues/16640). [#16657](https://github.com/ClickHouse/ClickHouse/pull/16657) ([flynn](https://github.com/ucasFL)). -* Now, `` configuration can be changed in `config.xml` and reloaded without server startup. [#16627](https://github.com/ClickHouse/ClickHouse/pull/16627) ([Amos Bird](https://github.com/amosbird)). -* Support SNI in https connections to remote resources. This will allow to connect to Cloudflare servers that require SNI. This fixes [#10055](https://github.com/ClickHouse/ClickHouse/issues/10055). [#16252](https://github.com/ClickHouse/ClickHouse/pull/16252) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make it possible to connect to `clickhouse-server` secure endpoint which requires SNI. This is possible when `clickhouse-server` is hosted behind TLS proxy. [#16938](https://github.com/ClickHouse/ClickHouse/pull/16938) ([filimonov](https://github.com/filimonov)). -* Fix possible stack overflow if a loop of materialized views is created. This closes [#15732](https://github.com/ClickHouse/ClickHouse/issues/15732). [#16048](https://github.com/ClickHouse/ClickHouse/pull/16048) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Simplify the implementation of background tasks processing for the MergeTree table engines family. There should be no visible changes for user. [#15983](https://github.com/ClickHouse/ClickHouse/pull/15983) ([alesapin](https://github.com/alesapin)). -* Improvement for MaterializeMySQL (experimental feature). Throw exception about right sync privileges when MySQL sync user has error privileges. [#15977](https://github.com/ClickHouse/ClickHouse/pull/15977) ([TCeason](https://github.com/TCeason)). -* Made `indexOf()` use BloomFilter. [#14977](https://github.com/ClickHouse/ClickHouse/pull/14977) ([achimbab](https://github.com/achimbab)). - -#### Performance Improvement - -* Use Floyd-Rivest algorithm, it is the best for the ClickHouse use case of partial sorting. Bechmarks are in https://github.com/danlark1/miniselect and [here](https://drive.google.com/drive/folders/1DHEaeXgZuX6AJ9eByeZ8iQVQv0ueP8XM). [#16825](https://github.com/ClickHouse/ClickHouse/pull/16825) ([Danila Kutenin](https://github.com/danlark1)). -* Now `ReplicatedMergeTree` tree engines family uses a separate thread pool for replicated fetches. Size of the pool limited by setting `background_fetches_pool_size` which can be tuned with a server restart. The default value of the setting is 3 and it means that the maximum amount of parallel fetches is equal to 3 (and it allows to utilize 10G network). Fixes #520. [#16390](https://github.com/ClickHouse/ClickHouse/pull/16390) ([alesapin](https://github.com/alesapin)). -* Fixed uncontrolled growth of the state of `quantileTDigest`. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). -* Add `VIEW` subquery description to `EXPLAIN`. Limit push down optimisation for `VIEW`. Add local replicas of `Distributed` to query plan. [#14936](https://github.com/ClickHouse/ClickHouse/pull/14936) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix optimize_read_in_order/optimize_aggregation_in_order with max_threads > 0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* Fix performance of reading from `Merge` tables over huge number of `MergeTree` tables. Fixes [#7748](https://github.com/ClickHouse/ClickHouse/issues/7748). [#16988](https://github.com/ClickHouse/ClickHouse/pull/16988) ([Anton Popov](https://github.com/CurtizJ)). -* Now we can safely prune partitions with exact match. Useful case: Suppose table is partitioned by `intHash64(x) % 100` and the query has condition on `intHash64(x) % 100` verbatim, not on x. [#16253](https://github.com/ClickHouse/ClickHouse/pull/16253) ([Amos Bird](https://github.com/amosbird)). - -#### Experimental Feature - -* Add `EmbeddedRocksDB` table engine (can be used for dictionaries). [#15073](https://github.com/ClickHouse/ClickHouse/pull/15073) ([sundyli](https://github.com/sundy-li)). - -#### Build/Testing/Packaging Improvement - -* Improvements in test coverage building images. [#17233](https://github.com/ClickHouse/ClickHouse/pull/17233) ([alesapin](https://github.com/alesapin)). -* Update embedded timezone data to version 2020d (also update cctz to the latest master). [#17204](https://github.com/ClickHouse/ClickHouse/pull/17204) ([filimonov](https://github.com/filimonov)). -* Fix UBSan report in Poco. This closes [#12719](https://github.com/ClickHouse/ClickHouse/issues/12719). [#16765](https://github.com/ClickHouse/ClickHouse/pull/16765) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Do not instrument 3rd-party libraries with UBSan. [#16764](https://github.com/ClickHouse/ClickHouse/pull/16764) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan report in cache dictionaries. This closes [#12641](https://github.com/ClickHouse/ClickHouse/issues/12641). [#16763](https://github.com/ClickHouse/ClickHouse/pull/16763) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan report when trying to convert infinite floating point number to integer. This closes [#14190](https://github.com/ClickHouse/ClickHouse/issues/14190). [#16677](https://github.com/ClickHouse/ClickHouse/pull/16677) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -## ClickHouse release 20.11 - -### ClickHouse release v20.11.6.6-stable, 2020-12-24 - -#### Bug Fix - -* Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual `IPv4/IPv6 stack` and fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes. This possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). -* Fixed key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). -* Fixed unique key convert crash in `MaterializeMySQL` database engine. This fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) and fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fixed the issue when some tables not synchronized to ClickHouse from MySQL caused by the fact that convertion MySQL prefix index wasn't supported for MaterializeMySQL. This fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) and fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed the issue when query optimization was producing wrong result if query contains `ARRAY JOIN`. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). -* Fix possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). -* Do not restore parts from WAL if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). -* Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* Fixed inconsistent behaviour of `optimize_trivial_count_query` with partition predicate. [#17644](https://github.com/ClickHouse/ClickHouse/pull/17644) ([Azat Khuzhin](https://github.com/azat)). -* Fixed empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). -* Fixed the behaviour when xxception `fmt::v7::format_error` can be logged in background for MergeTree tables. This fixes [#17613](https://github.com/ClickHouse/ClickHouse/issues/17613). [#17615](https://github.com/ClickHouse/ClickHouse/pull/17615) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the behaviour when clickhouse-client is used in interactive mode with multiline queries and single line comment was erronously extended till the end of query. This fixes [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed alter query hang when the corresponding mutation was killed on the different replica. This fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* Fixed bug when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* Fixed `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed duplicates after `DISTINCT` which were possible because of incorrect optimization. This fixes [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed crash while reading from `JOIN` table with `LowCardinality` types. This fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246) . [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). -* Fixed possible wrong index analysis when the types of the index comparison are different. This fixes [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). -* Fixed `ColumnConst` comparison which leads to crash. This fixes [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* Fixed bug when `ON CLUSTER` queries may hang forever for non-leader `ReplicatedMergeTreeTables`. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* Fixed fuzzer-found bug in funciton `fuzzBits`. This fixes [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* Fixed wrong result in big integers (128, 256 bit) when casting from double. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). -* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* Fixed possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* Blame info was not calculated correctly in `clickhouse-git-import`. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed order by optimization with monotonous functions. Fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed optimization of group by with enabled setting `optimize_aggregators_of_group_by_keys` and joins. This fixes [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). -* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* Fixed possible error `Illegal type of argument` for queries with `ORDER BY`. This fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Abort multipart upload if no data was written to WriteBufferFromS3. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fixed crash when using `any` without any arguments. This fixes [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* Fixed the behaviour when ClickHouse used to always return 0 insted of a number of affected rows for `INSERT` queries via MySQL protocol. This fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed uncontrolled growth of TDigest. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). -* Fixed remote query failure when using suffix `if` in Aggregate function. This fixes [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed inconsistent behavior caused by `select_sequential_consistency` for optimized trivial count query and system.tables. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). * Throw error when use ColumnTransformer replace non exist column. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). - - -### ClickHouse release v20.11.3.3-stable, 2020-11-13 - -#### Bug Fix - -* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse release v20.11.2.1, 2020-11-11 - -#### Backward Incompatible Change - -* If some `profile` was specified in `distributed_ddl` config section, then this profile could overwrite settings of `default` profile on server startup. It's fixed, now settings of distributed DDL queries should not affect global server settings. [#16635](https://github.com/ClickHouse/ClickHouse/pull/16635) ([tavplubix](https://github.com/tavplubix)). -* Restrict to use of non-comparable data types (like `AggregateFunction`) in keys (Sorting key, Primary key, Partition key, and so on). [#16601](https://github.com/ClickHouse/ClickHouse/pull/16601) ([alesapin](https://github.com/alesapin)). -* Remove `ANALYZE` and `AST` queries, and make the setting `enable_debug_queries` obsolete since now it is the part of full featured `EXPLAIN` query. [#16536](https://github.com/ClickHouse/ClickHouse/pull/16536) ([Ivan](https://github.com/abyss7)). -* Aggregate functions `boundingRatio`, `rankCorr`, `retention`, `timeSeriesGroupSum`, `timeSeriesGroupRateSum`, `windowFunnel` were erroneously made case-insensitive. Now their names are made case sensitive as designed. Only functions that are specified in SQL standard or made for compatibility with other DBMS or functions similar to those should be case-insensitive. [#16407](https://github.com/ClickHouse/ClickHouse/pull/16407) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make `rankCorr` function return nan on insufficient data [#16124](https://github.com/ClickHouse/ClickHouse/issues/16124). [#16135](https://github.com/ClickHouse/ClickHouse/pull/16135) ([hexiaoting](https://github.com/hexiaoting)). -* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). - -#### New Feature - -* Added support of LDAP as a user directory for locally non-existent users. [#12736](https://github.com/ClickHouse/ClickHouse/pull/12736) ([Denis Glazachev](https://github.com/traceon)). -* Add `system.replicated_fetches` table which shows currently running background fetches. [#16428](https://github.com/ClickHouse/ClickHouse/pull/16428) ([alesapin](https://github.com/alesapin)). -* Added setting `date_time_output_format`. [#15845](https://github.com/ClickHouse/ClickHouse/pull/15845) ([Maksim Kita](https://github.com/kitaisreal)). -* Added minimal web UI to ClickHouse. [#16158](https://github.com/ClickHouse/ClickHouse/pull/16158) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allows to read/write Single protobuf message at once (w/o length-delimiters). [#15199](https://github.com/ClickHouse/ClickHouse/pull/15199) ([filimonov](https://github.com/filimonov)). -* Added initial OpenTelemetry support. ClickHouse now accepts OpenTelemetry traceparent headers over Native and HTTP protocols, and passes them downstream in some cases. The trace spans for executed queries are saved into the `system.opentelemetry_span_log` table. [#14195](https://github.com/ClickHouse/ClickHouse/pull/14195) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Allow specify primary key in column list of `CREATE TABLE` query. This is needed for compatibility with other SQL dialects. [#15823](https://github.com/ClickHouse/ClickHouse/pull/15823) ([Maksim Kita](https://github.com/kitaisreal)). -* Implement `OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES}` in SELECT query with ORDER BY. This is the SQL-standard way to specify `LIMIT`. [#15855](https://github.com/ClickHouse/ClickHouse/pull/15855) ([hexiaoting](https://github.com/hexiaoting)). -* `errorCodeToName` function - return variable name of the error (useful for analyzing query_log and similar). `system.errors` table - shows how many times errors has been happened (respects `system_events_show_zero_values`). [#16438](https://github.com/ClickHouse/ClickHouse/pull/16438) ([Azat Khuzhin](https://github.com/azat)). -* Added function `untuple` which is a special function which can introduce new columns to the SELECT list by expanding a named tuple. [#16242](https://github.com/ClickHouse/ClickHouse/pull/16242) ([Nikolai Kochetov](https://github.com/KochetovNicolai), [Amos Bird](https://github.com/amosbird)). -* Now we can provide identifiers via query parameters. And these parameters can be used as table objects or columns. [#16594](https://github.com/ClickHouse/ClickHouse/pull/16594) ([Amos Bird](https://github.com/amosbird)). -* Added big integers (UInt256, Int128, Int256) and UUID data types support for MergeTree BloomFilter index. Big integers is an experimental feature. [#16642](https://github.com/ClickHouse/ClickHouse/pull/16642) ([Maksim Kita](https://github.com/kitaisreal)). -* Add `farmFingerprint64` function (non-cryptographic string hashing). [#16570](https://github.com/ClickHouse/ClickHouse/pull/16570) ([Jacob Hayes](https://github.com/JacobHayes)). -* Add `log_queries_min_query_duration_ms`, only queries slower than the value of this setting will go to `query_log`/`query_thread_log` (i.e. something like `slow_query_log` in mysql). [#16529](https://github.com/ClickHouse/ClickHouse/pull/16529) ([Azat Khuzhin](https://github.com/azat)). -* Ability to create a docker image on the top of `Alpine`. Uses precompiled binary and glibc components from ubuntu 20.04. [#16479](https://github.com/ClickHouse/ClickHouse/pull/16479) ([filimonov](https://github.com/filimonov)). -* Added `toUUIDOrNull`, `toUUIDOrZero` cast functions. [#16337](https://github.com/ClickHouse/ClickHouse/pull/16337) ([Maksim Kita](https://github.com/kitaisreal)). -* Add `max_concurrent_queries_for_all_users` setting, see [#6636](https://github.com/ClickHouse/ClickHouse/issues/6636) for use cases. [#16154](https://github.com/ClickHouse/ClickHouse/pull/16154) ([nvartolomei](https://github.com/nvartolomei)). -* Add a new option `print_query_id` to clickhouse-client. It helps generate arbitrary strings with the current query id generated by the client. Also print query id in clickhouse-client by default. [#15809](https://github.com/ClickHouse/ClickHouse/pull/15809) ([Amos Bird](https://github.com/amosbird)). -* Add `tid` and `logTrace` functions. This closes [#9434](https://github.com/ClickHouse/ClickHouse/issues/9434). [#15803](https://github.com/ClickHouse/ClickHouse/pull/15803) ([flynn](https://github.com/ucasFL)). -* Add function `formatReadableTimeDelta` that format time delta to human readable string ... [#15497](https://github.com/ClickHouse/ClickHouse/pull/15497) ([Filipe Caixeta](https://github.com/filipecaixeta)). -* Added `disable_merges` option for volumes in multi-disk configuration. [#13956](https://github.com/ClickHouse/ClickHouse/pull/13956) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### Experimental Feature - -* New functions `encrypt`, `aes_encrypt_mysql`, `decrypt`, `aes_decrypt_mysql`. These functions are working slowly, so we consider it as an experimental feature. [#11844](https://github.com/ClickHouse/ClickHouse/pull/11844) ([Vasily Nemkov](https://github.com/Enmk)). - -#### Bug Fix - -* Mask password in data_path in the `system.distribution_queue`. [#16727](https://github.com/ClickHouse/ClickHouse/pull/16727) ([Azat Khuzhin](https://github.com/azat)). -* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* The setting `max_parallel_replicas` worked incorrectly if the queried table has no sampling. This fixes [#5733](https://github.com/ClickHouse/ClickHouse/issues/5733). [#16675](https://github.com/ClickHouse/ClickHouse/pull/16675) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix optimize_read_in_order/optimize_aggregation_in_order with max_threads > 0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* Calculation of `DEFAULT` expressions was involving possible name collisions (that was very unlikely to encounter). This fixes [#9359](https://github.com/ClickHouse/ClickHouse/issues/9359). [#16612](https://github.com/ClickHouse/ClickHouse/pull/16612) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `query_thread_log.query_duration_ms` unit. [#16563](https://github.com/ClickHouse/ClickHouse/pull/16563) ([Azat Khuzhin](https://github.com/azat)). -* Fix a bug when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine. `MaterializeMySQL` is an experimental feature. [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* Specifically crafted argument of `round` function with `Decimal` was leading to integer division by zero. This fixes [#13338](https://github.com/ClickHouse/ClickHouse/issues/13338). [#16451](https://github.com/ClickHouse/ClickHouse/pull/16451) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix dictGet in sharding_key (and similar places, i.e. when the function context is stored permanently). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). -* Fix the exception thrown in `clickhouse-local` when trying to execute `OPTIMIZE` command. Fixes [#16076](https://github.com/ClickHouse/ClickHouse/issues/16076). [#16192](https://github.com/ClickHouse/ClickHouse/pull/16192) ([filimonov](https://github.com/filimonov)). -* Fixes [#15780](https://github.com/ClickHouse/ClickHouse/issues/15780) regression, e.g. `indexOf([1, 2, 3], toLowCardinality(1))` now is prohibited but it should not be. [#16038](https://github.com/ClickHouse/ClickHouse/pull/16038) ([Mike](https://github.com/myrrc)). -* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). -* Now exception will be thrown when `ALTER MODIFY COLUMN ... DEFAULT ...` has incompatible default with column type. Fixes [#15854](https://github.com/ClickHouse/ClickHouse/issues/15854). [#15858](https://github.com/ClickHouse/ClickHouse/pull/15858) ([alesapin](https://github.com/alesapin)). -* Fixed IPv4CIDRToRange/IPv6CIDRToRange functions to accept const IP-column values. [#15856](https://github.com/ClickHouse/ClickHouse/pull/15856) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). +* Fix crash in case of not equi-join ON expression in RIGH|FULL JOIN. [#15162](https://github.com/ClickHouse/ClickHouse/pull/15162) ([Artem Zuikov](https://github.com/4ertus2)). #### Improvement -* Treat `INTERVAL '1 hour'` as equivalent to `INTERVAL 1 HOUR`, to be compatible with Postgres and similar. This fixes [#15637](https://github.com/ClickHouse/ClickHouse/issues/15637). [#15978](https://github.com/ClickHouse/ClickHouse/pull/15978) ([flynn](https://github.com/ucasFL)). -* Enable parsing enum values by their numeric ids for CSV, TSV and JSON input formats. [#15685](https://github.com/ClickHouse/ClickHouse/pull/15685) ([vivarum](https://github.com/vivarum)). -* Better read task scheduling for JBOD architecture and `MergeTree` storage. New setting `read_backoff_min_concurrency` which serves as the lower limit to the number of reading threads. [#16423](https://github.com/ClickHouse/ClickHouse/pull/16423) ([Amos Bird](https://github.com/amosbird)). -* Add missing support for `LowCardinality` in `Avro` format. [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* Workaround for use `S3` with nginx server as proxy. Nginx currenty does not accept urls with empty path like `http://domain.com?delete`, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like `http://domain.com/?delete`. [#16814](https://github.com/ClickHouse/ClickHouse/pull/16814) ([ianton-ru](https://github.com/ianton-ru)). -* Better diagnostics on parse errors in input data. Provide row number on `Cannot read all data` errors. [#16644](https://github.com/ClickHouse/ClickHouse/pull/16644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make the behaviour of `minMap` and `maxMap` more desireable. It will not skip zero values in the result. Fixes [#16087](https://github.com/ClickHouse/ClickHouse/issues/16087). [#16631](https://github.com/ClickHouse/ClickHouse/pull/16631) ([Ildus Kurbangaliev](https://github.com/ildus)). -* Better update of ZooKeeper configuration in runtime. [#16630](https://github.com/ClickHouse/ClickHouse/pull/16630) ([sundyli](https://github.com/sundy-li)). -* Apply SETTINGS clause as early as possible. It allows to modify more settings in the query. This closes [#3178](https://github.com/ClickHouse/ClickHouse/issues/3178). [#16619](https://github.com/ClickHouse/ClickHouse/pull/16619) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now `event_time_microseconds` field stores in Decimal64, not UInt64. [#16617](https://github.com/ClickHouse/ClickHouse/pull/16617) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Now paratmeterized functions can be used in `APPLY` column transformer. [#16589](https://github.com/ClickHouse/ClickHouse/pull/16589) ([Amos Bird](https://github.com/amosbird)). -* Improve scheduling of background task which removes data of dropped tables in `Atomic` databases. `Atomic` databases do not create broken symlink to table data directory if table actually has no data directory. [#16584](https://github.com/ClickHouse/ClickHouse/pull/16584) ([tavplubix](https://github.com/tavplubix)). -* Subqueries in `WITH` section (CTE) can reference previous subqueries in `WITH` section by their name. [#16575](https://github.com/ClickHouse/ClickHouse/pull/16575) ([Amos Bird](https://github.com/amosbird)). -* Add current_database into `system.query_thread_log`. [#16558](https://github.com/ClickHouse/ClickHouse/pull/16558) ([Azat Khuzhin](https://github.com/azat)). -* Allow to fetch parts that are already committed or outdated in the current instance into the detached directory. It's useful when migrating tables from another cluster and having N to 1 shards mapping. It's also consistent with the current fetchPartition implementation. [#16538](https://github.com/ClickHouse/ClickHouse/pull/16538) ([Amos Bird](https://github.com/amosbird)). -* Multiple improvements for `RabbitMQ`: Fixed bug for [#16263](https://github.com/ClickHouse/ClickHouse/issues/16263). Also minimized event loop lifetime. Added more efficient queues setup. [#16426](https://github.com/ClickHouse/ClickHouse/pull/16426) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix debug assertion in `quantileDeterministic` function. In previous version it may also transfer up to two times more data over the network. Although no bug existed. This fixes [#15683](https://github.com/ClickHouse/ClickHouse/issues/15683). [#16410](https://github.com/ClickHouse/ClickHouse/pull/16410) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add `TablesToDropQueueSize` metric. It's equal to number of dropped tables, that are waiting for background data removal. [#16364](https://github.com/ClickHouse/ClickHouse/pull/16364) ([tavplubix](https://github.com/tavplubix)). -* Better diagnostics when client has dropped connection. In previous versions, `Attempt to read after EOF` and `Broken pipe` exceptions were logged in server. In new version, it's information message `Client has dropped the connection, cancel the query.`. [#16329](https://github.com/ClickHouse/ClickHouse/pull/16329) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add total_rows/total_bytes (from system.tables) support for Set/Join table engines. [#16306](https://github.com/ClickHouse/ClickHouse/pull/16306) ([Azat Khuzhin](https://github.com/azat)). -* Now it's possible to specify `PRIMARY KEY` without `ORDER BY` for MergeTree table engines family. Closes [#15591](https://github.com/ClickHouse/ClickHouse/issues/15591). [#16284](https://github.com/ClickHouse/ClickHouse/pull/16284) ([alesapin](https://github.com/alesapin)). -* If there is no tmp folder in the system (chroot, misconfigutation etc) `clickhouse-local` will create temporary subfolder in the current directory. [#16280](https://github.com/ClickHouse/ClickHouse/pull/16280) ([filimonov](https://github.com/filimonov)). -* Add support for nested data types (like named tuple) as sub-types. Fixes [#15587](https://github.com/ClickHouse/ClickHouse/issues/15587). [#16262](https://github.com/ClickHouse/ClickHouse/pull/16262) ([Ivan](https://github.com/abyss7)). -* Support for `database_atomic_wait_for_drop_and_detach_synchronously`/`NO DELAY`/`SYNC` for `DROP DATABASE`. [#16127](https://github.com/ClickHouse/ClickHouse/pull/16127) ([Azat Khuzhin](https://github.com/azat)). -* Add `allow_nondeterministic_optimize_skip_unused_shards` (to allow non deterministic like `rand()` or `dictGet()` in sharding key). [#16105](https://github.com/ClickHouse/ClickHouse/pull/16105) ([Azat Khuzhin](https://github.com/azat)). -* Fix `memory_profiler_step`/`max_untracked_memory` for queries via HTTP (test included). Fix the issue that adjusting this value globally in xml config does not help either, since those settings are not applied anyway, only default (4MB) value is [used](https://github.com/ClickHouse/ClickHouse/blob/17731245336d8c84f75e4c0894c5797ed7732190/src/Common/ThreadStatus.h#L104). Fix `query_id` for the most root ThreadStatus of the http query (by initializing QueryScope after reading query_id). [#16101](https://github.com/ClickHouse/ClickHouse/pull/16101) ([Azat Khuzhin](https://github.com/azat)). -* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Fix rare issue when `clickhouse-client` may abort on exit due to loading of suggestions. This fixes [#16035](https://github.com/ClickHouse/ClickHouse/issues/16035). [#16047](https://github.com/ClickHouse/ClickHouse/pull/16047) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add support of `cache` layout for `Redis` dictionaries with complex key. [#15985](https://github.com/ClickHouse/ClickHouse/pull/15985) ([Anton Popov](https://github.com/CurtizJ)). -* Fix query hang (endless loop) in case of misconfiguration (`connections_with_failover_max_tries` set to 0). [#15876](https://github.com/ClickHouse/ClickHouse/pull/15876) ([Azat Khuzhin](https://github.com/azat)). -* Change level of some log messages from information to debug, so information messages will not appear for every query. This closes [#5293](https://github.com/ClickHouse/ClickHouse/issues/5293). [#15816](https://github.com/ClickHouse/ClickHouse/pull/15816) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove `MemoryTrackingInBackground*` metrics to avoid potentially misleading results. This fixes [#15684](https://github.com/ClickHouse/ClickHouse/issues/15684). [#15813](https://github.com/ClickHouse/ClickHouse/pull/15813) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add reconnects to `zookeeper-dump-tree` tool. [#15711](https://github.com/ClickHouse/ClickHouse/pull/15711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow explicitly specify columns list in `CREATE TABLE table AS table_function(...)` query. Fixes [#9249](https://github.com/ClickHouse/ClickHouse/issues/9249) Fixes [#14214](https://github.com/ClickHouse/ClickHouse/issues/14214). [#14295](https://github.com/ClickHouse/ClickHouse/pull/14295) ([tavplubix](https://github.com/tavplubix)). +* Explicitly set uid / gid of clickhouse user & group to the fixed values (101) in clickhouse-server images. [#19096](https://github.com/ClickHouse/ClickHouse/pull/19096) ([filimonov](https://github.com/filimonov)). +* Two new settings (by analogy with MergeTree family) has been added: - `fsync_after_insert` - Do fsync for every inserted. Will decreases performance of inserts. - `fsync_directories` - Do fsync for temporary directory (that is used for async INSERT only) after all operations (writes, renames, etc.). [#18864](https://github.com/ClickHouse/ClickHouse/pull/18864) ([Azat Khuzhin](https://github.com/azat)). +* `SYSTEM KILL` command started to work in Docker. This closes [#18847](https://github.com/ClickHouse/ClickHouse/issues/18847). [#18848](https://github.com/ClickHouse/ClickHouse/pull/18848) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Expand macros in the zk path when executing fetchPartition. [#18839](https://github.com/ClickHouse/ClickHouse/pull/18839) ([fastio](https://github.com/fastio)). +* Apply `ALTER TABLE ON CLUSTER MODIFY SETTING ...` to all replicas. Because we don't replicate such alter commands. [#18789](https://github.com/ClickHouse/ClickHouse/pull/18789) ([Amos Bird](https://github.com/amosbird)). +* Allow column transformer `EXCEPT` to accept a string as regular expression matcher. This resolves [#18685](https://github.com/ClickHouse/ClickHouse/issues/18685) . [#18699](https://github.com/ClickHouse/ClickHouse/pull/18699) ([Amos Bird](https://github.com/amosbird)). +* Another fix of using SimpleAggregateFunction in SummingMergeTree. This fixes [#18676](https://github.com/ClickHouse/ClickHouse/issues/18676) . [#18677](https://github.com/ClickHouse/ClickHouse/pull/18677) ([Amos Bird](https://github.com/amosbird)). +* Fix SimpleAggregateFunction in SummingMergeTree. Now it works like AggregateFunction. In previous versions values were summed together regardless to the aggregate function. This fixes [#18564](https://github.com/ClickHouse/ClickHouse/issues/18564) . [#8052](https://github.com/ClickHouse/ClickHouse/issues/8052). [#18637](https://github.com/ClickHouse/ClickHouse/pull/18637) ([Amos Bird](https://github.com/amosbird)). +* PODArray: Avoid call to memcpy with (nullptr, 0) arguments (Fix UBSan report). This fixes [#18525](https://github.com/ClickHouse/ClickHouse/issues/18525). [#18526](https://github.com/ClickHouse/ClickHouse/pull/18526) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed assertion error inside allocator in case when last argument of function bar is NaN. Now simple ClickHouse's exception is being thrown. This fixes [#17876](https://github.com/ClickHouse/ClickHouse/issues/17876). [#18520](https://github.com/ClickHouse/ClickHouse/pull/18520) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* support syntax `EXISTS DATABASE name`. [#18458](https://github.com/ClickHouse/ClickHouse/pull/18458) ([Du Chuan](https://github.com/spongedu)). +* Fix bug: no newline after exception message in some tools. [#18444](https://github.com/ClickHouse/ClickHouse/pull/18444) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add ability to modify primary and partition key column type from `LowCardinality(Type)` to `Type` and vice versa. Also add an ability to modify primary key column type from `EnumX ` to `IntX` type. Fixes [#5604](https://github.com/ClickHouse/ClickHouse/issues/5604). [#18362](https://github.com/ClickHouse/ClickHouse/pull/18362) ([alesapin](https://github.com/alesapin)). +* Support builtin function `isIPv4String` && `isIPv6String` like [MySQL](https://github.com/ClickHouse/ClickHouse/compare/master...spongedu:support_is_ipv4?expand=1). [#18349](https://github.com/ClickHouse/ClickHouse/pull/18349) ([Du Chuan](https://github.com/spongedu)). +* Fix potential server crash during Buffer rollback (that is impossible in current ClickHouse version). [#18329](https://github.com/ClickHouse/ClickHouse/pull/18329) ([Azat Khuzhin](https://github.com/azat)). +* related to [#18133](https://github.com/ClickHouse/ClickHouse/issues/18133). [#18309](https://github.com/ClickHouse/ClickHouse/pull/18309) ([hexiaoting](https://github.com/hexiaoting)). +* Add a new setting `insert_distributed_one_random_shard = 1` to allow insertion into multi-sharded distributed table without any distributed key. [#18294](https://github.com/ClickHouse/ClickHouse/pull/18294) ([Amos Bird](https://github.com/amosbird)). +* Allow to parse Array fields from CSV if it is represented as a string containing array that was serialized as nested CSV. Example: `"[""Hello"", ""world"", ""42"""" TV""]"` will parse as `['Hello', 'world', '42" TV']`. Allow to parse array in CSV in a string without enclosing braces. Example: `"'Hello', 'world', '42"" TV'"` will parse as `['Hello', 'world', '42" TV']`. [#18271](https://github.com/ClickHouse/ClickHouse/pull/18271) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make better adaptive granularity calculation for merge tree wide parts. [#18223](https://github.com/ClickHouse/ClickHouse/pull/18223) ([alesapin](https://github.com/alesapin)). +* Now clickhouse-install could work on Mac. The problem was that there is no procfs on this platform. [#18201](https://github.com/ClickHouse/ClickHouse/pull/18201) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Better hints for `SHOW ...` query syntax. [#18183](https://github.com/ClickHouse/ClickHouse/pull/18183) ([Du Chuan](https://github.com/spongedu)). +* Array aggregation `arrayMin`, `arrayMax`, `arraySum`, `arrayAvg` support for `Int128`, `Int256`, `UInt256`. [#18147](https://github.com/ClickHouse/ClickHouse/pull/18147) ([Maksim Kita](https://github.com/kitaisreal)). +* All queries of type `Decimal * Float` or vice versa are allowed, including aggregate ones (e.g. `SELECT sum(decimal_field * 1.1)` or `SELECT dec_col * float_col`), the result type is Float32 or Float64. [#18145](https://github.com/ClickHouse/ClickHouse/pull/18145) ([Mike](https://github.com/myrrc)). +* `EXPLAIN AST` now support queries other then `SELECT`. [#18136](https://github.com/ClickHouse/ClickHouse/pull/18136) ([taiyang-li](https://github.com/taiyang-li)). +* Add `disk` to Set and Join storage settings. [#18112](https://github.com/ClickHouse/ClickHouse/pull/18112) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). +* Now the table function `merge()` requires the current user to have the `SELECT` privilege on each table it receives data from. This PR fixes [#16964](https://github.com/ClickHouse/ClickHouse/issues/16964). [#18104](https://github.com/ClickHouse/ClickHouse/pull/18104) ([Vitaly Baranov](https://github.com/vitlibar)). +* Support `SHOW CREATE VIEW name` syntax like [MySQL](https://dev.mysql.com/doc/refman/5.7/en/show-create-view.html). [#18095](https://github.com/ClickHouse/ClickHouse/pull/18095) ([Du Chuan](https://github.com/spongedu)). +* Fix dead list watches removal for TestKeeperStorage. [#18065](https://github.com/ClickHouse/ClickHouse/pull/18065) ([alesapin](https://github.com/alesapin)). +* Temporary tables are visible in the system tables `system.tables` and `system.columns` now only in those session where they have been created. The internal database `_temporary_and_external_tables` is now hidden in those system tables; temporary tables are shown as tables with empty database with the `is_temporary` flag set instead. [#18014](https://github.com/ClickHouse/ClickHouse/pull/18014) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix clickhouse-client rendering issue when the size of terminal window changes. [#18009](https://github.com/ClickHouse/ClickHouse/pull/18009) ([Amos Bird](https://github.com/amosbird)). +* Decrease log verbosity of the events when the client drops the connection from WARNING to INFORMATION. [#18005](https://github.com/ClickHouse/ClickHouse/pull/18005) ([filimonov](https://github.com/filimonov)). +* Now the table function `merge()` requires the current user to have the `SELECT` privilege on each table it receives data from. This PR fixes [#16964](https://github.com/ClickHouse/ClickHouse/issues/16964). [#17983](https://github.com/ClickHouse/ClickHouse/pull/17983) ([Vitaly Baranov](https://github.com/vitlibar)). +* Forcibly removing empty or bad metadata files from filesystem for DiskS3. S3 is an experimental feature. [#17935](https://github.com/ClickHouse/ClickHouse/pull/17935) ([Pavel Kovalenko](https://github.com/Jokser)). +* Adaptive choose of single/multi part upload in WriteBufferFromS3. Single part upload is controlled by a new setting 'max_single_part_upload_size'. [#17934](https://github.com/ClickHouse/ClickHouse/pull/17934) ([Pavel Kovalenko](https://github.com/Jokser)). +* Ability to set custom metadata when putting S3 object. [#17909](https://github.com/ClickHouse/ClickHouse/pull/17909) ([Pavel Kovalenko](https://github.com/Jokser)). +* `allow_introspection_functions=0` prohibits usage of introspection functions but doesn't prohibit giving grants for them anymore (the grantee will need to set `allow_introspection_functions=1` for himself to be able to use that grant). Similarly `allow_ddl=0` prohibits usage of DDL commands but doesn't prohibit giving grants for them anymore. [#17908](https://github.com/ClickHouse/ClickHouse/pull/17908) ([Vitaly Baranov](https://github.com/vitlibar)). +* Support for async tasks in `PipelineExecutor`. Initial support of async sockets for remote queries. [#17868](https://github.com/ClickHouse/ClickHouse/pull/17868) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Hints for column names. [#17112](https://github.com/ClickHouse/ClickHouse/issues/17112). [#17857](https://github.com/ClickHouse/ClickHouse/pull/17857) ([fastio](https://github.com/fastio)). +* Add diagnostic information when two merge tables try to read each other's data. [#17854](https://github.com/ClickHouse/ClickHouse/pull/17854) ([徐炘](https://github.com/weeds085490)). +* Add metrics(Parts, PartsActive, PartsInactive) for part number in MergeTree in clickhouse. [#17838](https://github.com/ClickHouse/ClickHouse/pull/17838) ([徐炘](https://github.com/weeds085490)). +* - Let the possibility to override timeout value for running script using the ClickHouse docker image. [#17818](https://github.com/ClickHouse/ClickHouse/pull/17818) ([Guillaume Tassery](https://github.com/YiuRULE)). +* Improvement of Web UI: do not add empty query to browser history. [#17770](https://github.com/ClickHouse/ClickHouse/pull/17770) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improves the path concatenation of zookeeper paths inside DDLWorker. [#17767](https://github.com/ClickHouse/ClickHouse/pull/17767) ([Bharat Nallan](https://github.com/bharatnc)). +* Check system log tables' engine definition grammatically to prevent some configuration errors. Notes that this grammar check is not semantical, that means such mistakes as non-existent columns / expression functions would be not found out util the table is created. [#17739](https://github.com/ClickHouse/ClickHouse/pull/17739) ([Du Chuan](https://github.com/spongedu)). +* system.query_log now has extensive information to achieve better query analysis. [#17726](https://github.com/ClickHouse/ClickHouse/pull/17726) ([Amos Bird](https://github.com/amosbird)). +* Removed exception throwing at table initialization if there was no connection (it will be reconnecting in the background). [#17709](https://github.com/ClickHouse/ClickHouse/pull/17709) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not ignore server memory limits during Buffer flush. [#17646](https://github.com/ClickHouse/ClickHouse/pull/17646) ([Azat Khuzhin](https://github.com/azat)). +* Switch to patched version of RocksDB (from ClickHouse-Extras). [#17643](https://github.com/ClickHouse/ClickHouse/pull/17643) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* This fixes [#17457](https://github.com/ClickHouse/ClickHouse/issues/17457). [#17641](https://github.com/ClickHouse/ClickHouse/pull/17641) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow to reload symbols from debug file. This PR also fixes a build-id issue. [#17637](https://github.com/ClickHouse/ClickHouse/pull/17637) ([Amos Bird](https://github.com/amosbird)). +* Don't throw "Too many parts" error in the middle of INSERT query. [#17566](https://github.com/ClickHouse/ClickHouse/pull/17566) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow query parameters in UPDATE statement of ALTER query. Fixes [#10976](https://github.com/ClickHouse/ClickHouse/issues/10976). [#17563](https://github.com/ClickHouse/ClickHouse/pull/17563) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Query obfuscator: avoid usage of some SQL keywords for identifier names. [#17526](https://github.com/ClickHouse/ClickHouse/pull/17526) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Export current max ddl entry executed by DDLWorker. It's useful to check if DDLWorker hangs somewhere. [#17464](https://github.com/ClickHouse/ClickHouse/pull/17464) ([Amos Bird](https://github.com/amosbird)). +* Export asynchronous metrics of all servers current threads. It's useful to track down issues like https://github.com/ClickHouse-Extras/poco/pull/28. [#17463](https://github.com/ClickHouse/ClickHouse/pull/17463) ([Amos Bird](https://github.com/amosbird)). +* Return dynamic columns like MATERIALIZED / ALIAS for wildcard query when switches `asterisk_include_materialized_columns` and `asterisk_include_alias_columns` are turned on. [#17462](https://github.com/ClickHouse/ClickHouse/pull/17462) ([Ken Chen](https://github.com/chenziliang)). +* Add functions countMatches/countMatchesCaseInsensitive. [#17459](https://github.com/ClickHouse/ClickHouse/pull/17459) ([Azat Khuzhin](https://github.com/azat)). +* Allow specifying [TTL](https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl) to remove old entries from [system log tables](https://clickhouse.tech/docs/en/operations/system-tables/), using the `` attribute in `config.xml`. [#17438](https://github.com/ClickHouse/ClickHouse/pull/17438) ([Du Chuan](https://github.com/spongedu)). +* Now queries coming to the server via MySQL and PostgreSQL protocols have distinctive interface types (which can be seen in the `interface` column of the table`system.query_log`): `4` for MySQL, and `5` for PostgreSQL, instead of formerly used `1` which is now used for the native protocol only. [#17437](https://github.com/ClickHouse/ClickHouse/pull/17437) ([Vitaly Baranov](https://github.com/vitlibar)). +* Simplify Sys/V init script. It was not working on Ubuntu 12.04. [#17428](https://github.com/ClickHouse/ClickHouse/pull/17428) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Multiple improvements in `./clickhouse install` script. [#17421](https://github.com/ClickHouse/ClickHouse/pull/17421) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix parsing of SETTINGS clause of the INSERT ... SELECT ... SETTINGS query. [#17414](https://github.com/ClickHouse/ClickHouse/pull/17414) ([Azat Khuzhin](https://github.com/azat)). +* Replaced `malloc` with `new`, so that the `MemoryTracker` takes this memory into account. [#17412](https://github.com/ClickHouse/ClickHouse/pull/17412) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add eof check in receiveHello to prevent getting `Attempt to read after eof` exception. [#17365](https://github.com/ClickHouse/ClickHouse/pull/17365) ([Kruglov Pavel](https://github.com/Avogar)). +* Implement `countSubstrings()`/`countSubstringsCaseInsensitive()`/`countSubstringsCaseInsensitiveUTF8()` (Count the number of substring occurrences). [#17347](https://github.com/ClickHouse/ClickHouse/pull/17347) ([Azat Khuzhin](https://github.com/azat)). +* Allow to use `optimize_move_to_prewhere` optimization with compact parts, when sizes of columns are unknown. [#17330](https://github.com/ClickHouse/ClickHouse/pull/17330) ([Anton Popov](https://github.com/CurtizJ)). +* Improved minimal Web UI: add history; add sharing support; avoid race condition of different requests; add request in-flight and ready indicators; add favicon; detect Ctrl+Enter if textarea is not in focus. [#17293](https://github.com/ClickHouse/ClickHouse/pull/17293) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid possible stack overflow in bigint conversion. Big integers are experimental. [#17269](https://github.com/ClickHouse/ClickHouse/pull/17269) ([flynn](https://github.com/ucasFL)). +* Now set indices will work with `GLOBAL IN`. This fixes [#17232](https://github.com/ClickHouse/ClickHouse/issues/17232) , [#5576](https://github.com/ClickHouse/ClickHouse/issues/5576) . [#17253](https://github.com/ClickHouse/ClickHouse/pull/17253) ([Amos Bird](https://github.com/amosbird)). +* - Add limit for http redirects in request to S3 storage ('s3_max_redirects'). [#17220](https://github.com/ClickHouse/ClickHouse/pull/17220) ([ianton-ru](https://github.com/ianton-ru)). +* - Add configuration for multi zookeeper clusters. [#17070](https://github.com/ClickHouse/ClickHouse/pull/17070) ([fastio](https://github.com/fastio)). +* When `-OrNull` combinator combined `-If`, `-Merge`, `-MergeState`, `-State` combinators, we should put `-OrNull` in front. [#16935](https://github.com/ClickHouse/ClickHouse/pull/16935) ([flynn](https://github.com/ucasFL)). +* Support HTTP proxy and HTTPS S3 endpoint configuration. [#16861](https://github.com/ClickHouse/ClickHouse/pull/16861) ([Pavel Kovalenko](https://github.com/Jokser)). +* Added proper authentication using environment, `~/.aws` and `AssumeRole` for S3 client. [#16856](https://github.com/ClickHouse/ClickHouse/pull/16856) ([Vladimir Chebotarev](https://github.com/excitoon)). +* - New IP Dictionary implementation with lower memory consumption, improved performance for some cases, and fixed bugs. [#16804](https://github.com/ClickHouse/ClickHouse/pull/16804) ([vdimir](https://github.com/vdimir)). +* Add more OpenTelemetry spans. Add an example of how to export the span data to Zipkin. [#16535](https://github.com/ClickHouse/ClickHouse/pull/16535) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* - Added `verification_cooldown` parameter in LDAP server connection configuration to allow caching of successful "bind" attempts for configurable period of time. [#15988](https://github.com/ClickHouse/ClickHouse/pull/15988) ([Denis Glazachev](https://github.com/traceon)). +* Avoid deadlock when executing INSERT SELECT into itself from a table with `TinyLog` or `Log` table engines. This closes [#6802](https://github.com/ClickHouse/ClickHouse/issues/6802). This closes [#18691](https://github.com/ClickHouse/ClickHouse/issues/18691). This closes [#16812](https://github.com/ClickHouse/ClickHouse/issues/16812). This closes [#14570](https://github.com/ClickHouse/ClickHouse/issues/14570). [#15260](https://github.com/ClickHouse/ClickHouse/pull/15260) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* fix [#9117](https://github.com/ClickHouse/ClickHouse/issues/9117). [#15127](https://github.com/ClickHouse/ClickHouse/pull/15127) ([flynn](https://github.com/ucasFL)). +* * Completely eliminate callbacks and locks for acquiring them. * Keys are not divided into "not found" and "expired", but stored in the same map during query. [#14958](https://github.com/ClickHouse/ClickHouse/pull/14958) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Allow using `Atomic` engine for nested database of `MaterializeMySQL` engine. [#14849](https://github.com/ClickHouse/ClickHouse/pull/14849) ([tavplubix](https://github.com/tavplubix)). +* Implement new parser based on ANTLR4 runtime and generated from EBNF grammar. [#11298](https://github.com/ClickHouse/ClickHouse/pull/11298) ([Ivan](https://github.com/abyss7)). #### Performance Improvement -* Do not merge parts across partitions in SELECT FINAL. [#15938](https://github.com/ClickHouse/ClickHouse/pull/15938) ([Kruglov Pavel](https://github.com/Avogar)). -* Improve performance of `-OrNull` and `-OrDefault` aggregate functions. [#16661](https://github.com/ClickHouse/ClickHouse/pull/16661) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improve performance of `quantileMerge`. In previous versions it was obnoxiously slow. This closes [#1463](https://github.com/ClickHouse/ClickHouse/issues/1463). [#16643](https://github.com/ClickHouse/ClickHouse/pull/16643) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improve performance of logical functions a little. [#16347](https://github.com/ClickHouse/ClickHouse/pull/16347) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improved performance of merges assignment in MergeTree table engines. Shouldn't be visible for the user. [#16191](https://github.com/ClickHouse/ClickHouse/pull/16191) ([alesapin](https://github.com/alesapin)). -* Speedup hashed/sparse_hashed dictionary loading by preallocating the hash table. [#15454](https://github.com/ClickHouse/ClickHouse/pull/15454) ([Azat Khuzhin](https://github.com/azat)). -* Now trivial count optimization becomes slightly non-trivial. Predicates that contain exact partition expr can be optimized too. This also fixes [#11092](https://github.com/ClickHouse/ClickHouse/issues/11092) which returns wrong count when `max_parallel_replicas > 1`. [#15074](https://github.com/ClickHouse/ClickHouse/pull/15074) ([Amos Bird](https://github.com/amosbird)). +* Add `--no-system-table` option for `clickhouse-local` to run without system tables. This avoids initialization of `DateLUT` that may take noticeable amount of time (tens of milliseconds) at startup. [#18899](https://github.com/ClickHouse/ClickHouse/pull/18899) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Replace `PODArray` with `PODArrayWithStackMemory` in `AggregateFunctionWindowFunnelData` to improvement `windowFunnel` function performance. [#18817](https://github.com/ClickHouse/ClickHouse/pull/18817) ([flynn](https://github.com/ucasFL)). +* Don't send empty blocks to shards on synchronous INSERT into Distributed table. This closes [#14571](https://github.com/ClickHouse/ClickHouse/issues/14571). [#18775](https://github.com/ClickHouse/ClickHouse/pull/18775) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimized read for StorageMemory. [#18052](https://github.com/ClickHouse/ClickHouse/pull/18052) ([Maksim Kita](https://github.com/kitaisreal)). +* Using dragonbox algorithm for float to string conversion instead of ryu. [#17831](https://github.com/ClickHouse/ClickHouse/pull/17831) ([Maksim Kita](https://github.com/kitaisreal)). +* Speedup `IPv6CIDRToRange` implementation. [#17569](https://github.com/ClickHouse/ClickHouse/pull/17569) ([vdimir](https://github.com/vdimir)). +* Add `remerge_sort_lowered_memory_bytes_ratio` setting (If memory usage after remerge does not reduced by this ratio, remerge will be disabled). [#17539](https://github.com/ClickHouse/ClickHouse/pull/17539) ([Azat Khuzhin](https://github.com/azat)). +* Improve performance of AggregatingMergeTree w/ SimpleAggregateFunction(String) in PK. [#17109](https://github.com/ClickHouse/ClickHouse/pull/17109) ([Azat Khuzhin](https://github.com/azat)). +* Now the `-If` combinator is devirtualized, and `count` is properly vectorized. This is for https://github.com/ClickHouse/ClickHouse/pull/17041. [#17043](https://github.com/ClickHouse/ClickHouse/pull/17043) ([Amos Bird](https://github.com/amosbird)). +* Fix performance of reading from `Merge` tables over huge number of `MergeTree` tables. Fixes [#7748](https://github.com/ClickHouse/ClickHouse/issues/7748). [#16988](https://github.com/ClickHouse/ClickHouse/pull/16988) ([Anton Popov](https://github.com/CurtizJ)). +* Improved performance of function `repeat`. [#16937](https://github.com/ClickHouse/ClickHouse/pull/16937) ([satanson](https://github.com/satanson)). +* Slightly improved performance of float parsing. [#16809](https://github.com/ClickHouse/ClickHouse/pull/16809) ([Maksim Kita](https://github.com/kitaisreal)). +* Add possibility to skip merged partitions for OPTIMIZE TABLE ... FINAL. [#15939](https://github.com/ClickHouse/ClickHouse/pull/15939) ([Kruglov Pavel](https://github.com/Avogar)). #### Build/Testing/Packaging Improvement -* Add flaky check for stateless tests. It will detect potentially flaky functional tests in advance, before they are merged. [#16238](https://github.com/ClickHouse/ClickHouse/pull/16238) ([alesapin](https://github.com/alesapin)). -* Use proper version for `croaring` instead of amalgamation. [#16285](https://github.com/ClickHouse/ClickHouse/pull/16285) ([sundyli](https://github.com/sundy-li)). -* Improve generation of build files for `ya.make` build system (Arcadia). [#16700](https://github.com/ClickHouse/ClickHouse/pull/16700) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add MySQL BinLog file check tool for `MaterializeMySQL` database engine. `MaterializeMySQL` is an experimental feature. [#16223](https://github.com/ClickHouse/ClickHouse/pull/16223) ([Winter Zhang](https://github.com/zhang2014)). -* Check for executable bit on non-executable files. People often accidentially commit executable files from Windows. [#15843](https://github.com/ClickHouse/ClickHouse/pull/15843) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Check for `#pragma once` in headers. [#15818](https://github.com/ClickHouse/ClickHouse/pull/15818) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix illegal code style `&vector[idx]` in libhdfs3. This fixes libcxx debug build. See also https://github.com/ClickHouse-Extras/libhdfs3/pull/8 . [#15815](https://github.com/ClickHouse/ClickHouse/pull/15815) ([Amos Bird](https://github.com/amosbird)). -* Fix build of one miscellaneous example tool on Mac OS. Note that we don't build examples on Mac OS in our CI (we build only ClickHouse binary), so there is zero chance it will not break again. This fixes [#15804](https://github.com/ClickHouse/ClickHouse/issues/15804). [#15808](https://github.com/ClickHouse/ClickHouse/pull/15808) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Simplify Sys/V init script. [#14135](https://github.com/ClickHouse/ClickHouse/pull/14135) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added `boost::program_options` to `db_generator` in order to increase its usability. This closes [#15940](https://github.com/ClickHouse/ClickHouse/issues/15940). [#15973](https://github.com/ClickHouse/ClickHouse/pull/15973) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - - -## ClickHouse release 20.10 - -### ClickHouse release v20.10.7.4-stable, 2020-12-24 - -#### Bug Fix - -* Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual `IPv4/IPv6` stack and fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes. This possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). -* Fix key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). -* Fixed unique key convert crash in `MaterializeMySQL` database engine. This fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) and fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fixed the issue when some tables not synchronized to ClickHouse from MySQL caused by the fact that convertion MySQL prefix index wasn't supported for MaterializeMySQL. This fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) and fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). -* Fix possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). -* Do not restore parts from `WAL` if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). -* Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* Fixed empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). -* Fixed the behaviour when `clickhouse-client` is used in interactive mode with multiline queries and single line comment was erronously extended till the end of query. This fixes [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `ALTER` query hang when the corresponding mutation was killed on the different replica. This fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* Fixed bug when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* Fixed `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed duplicates after `DISTINCT` which were possible because of incorrect optimization. Fixes [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed crash while reading from `JOIN` table with `LowCardinality` types. This fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246) . [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). -* Fixed `ColumnConst` comparison which leads to crash. This fixed [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* Fixed bug when `ON CLUSTER` queries may hang forever for non-leader `ReplicatedMergeTreeTables`. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* Fixed fuzzer-found bug in function `fuzzBits`. This fixes [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* Fixed wrong result in big integers (128, 256 bit) when casting from double. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). -* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* Fixed possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* Blame info was not calculated correctly in `clickhouse-git-import`. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed order by optimization with monotonous functions. This fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). -* Fixrf optimization of group by with enabled setting `optimize_aggregators_of_group_by_keys` and joins. This fixes [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). -* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* Fixrf possible error `Illegal type of argument` for queries with `ORDER BY`. This fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Abort multipart upload if no data was written to `WriteBufferFromS3`. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fixed crash when using `any` without any arguments. This fixes [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* Fixed the behaviour when ClickHouse used to always return 0 insted of a number of affected rows for `INSERT` queries via MySQL protocol. This fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed uncontrolled growth of `TDigest`. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). -* Fixed remote query failure when using suffix `if` in Aggregate function. This fixes [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). - - -### ClickHouse release v20.10.4.1-stable, 2020-11-13 - -#### Bug Fix - -* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* This will fix optimize_read_in_order/optimize_aggregation_in_order with max_threads>0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). - -#### Improvement - -* Workaround for use S3 with nginx server as proxy. Nginx currenty does not accept urls with empty path like http://domain.com?delete, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like http://domain.com/?delete. [#16813](https://github.com/ClickHouse/ClickHouse/pull/16813) ([ianton-ru](https://github.com/ianton-ru)). - - -### ClickHouse release v20.10.3.30, 2020-10-28 - -#### Backward Incompatible Change - -* Make `multiple_joins_rewriter_version` obsolete. Remove first version of joins rewriter. [#15472](https://github.com/ClickHouse/ClickHouse/pull/15472) ([Artem Zuikov](https://github.com/4ertus2)). -* Change default value of `format_regexp_escaping_rule` setting (it's related to `Regexp` format) to `Raw` (it means - read whole subpattern as a value) to make the behaviour more like to what users expect. [#15426](https://github.com/ClickHouse/ClickHouse/pull/15426) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add support for nested multiline comments `/* comment /* comment */ */` in SQL. This conforms to the SQL standard. [#14655](https://github.com/ClickHouse/ClickHouse/pull/14655) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added MergeTree settings (`max_replicated_merges_with_ttl_in_queue` and `max_number_of_merges_with_ttl_in_pool`) to control the number of merges with TTL in the background pool and replicated queue. This change breaks compatibility with older versions only if you use delete TTL. Otherwise, replication will stay compatible. You can avoid incompatibility issues if you update all shard replicas at once or execute `SYSTEM STOP TTL MERGES` until you finish the update of all replicas. If you'll get an incompatible entry in the replication queue, first of all, execute `SYSTEM STOP TTL MERGES` and after `ALTER TABLE ... DETACH PARTITION ...` the partition where incompatible TTL merge was assigned. Attach it back on a single replica. [#14490](https://github.com/ClickHouse/ClickHouse/pull/14490) ([alesapin](https://github.com/alesapin)). -* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). - -#### New Feature - -* Background data recompression. Add the ability to specify `TTL ... RECOMPRESS codec_name` for MergeTree table engines family. [#14494](https://github.com/ClickHouse/ClickHouse/pull/14494) ([alesapin](https://github.com/alesapin)). -* Add parallel quorum inserts. This closes [#15601](https://github.com/ClickHouse/ClickHouse/issues/15601). [#15601](https://github.com/ClickHouse/ClickHouse/pull/15601) ([Latysheva Alexandra](https://github.com/alexelex)). -* Settings for additional enforcement of data durability. Useful for non-replicated setups. [#11948](https://github.com/ClickHouse/ClickHouse/pull/11948) ([Anton Popov](https://github.com/CurtizJ)). -* When duplicate block is written to replica where it does not exist locally (has not been fetched from replicas), don't ignore it and write locally to achieve the same effect as if it was successfully replicated. [#11684](https://github.com/ClickHouse/ClickHouse/pull/11684) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now we support `WITH AS (subquery) ... ` to introduce named subqueries in the query context. This closes [#2416](https://github.com/ClickHouse/ClickHouse/issues/2416). This closes [#4967](https://github.com/ClickHouse/ClickHouse/issues/4967). [#14771](https://github.com/ClickHouse/ClickHouse/pull/14771) ([Amos Bird](https://github.com/amosbird)). -* Introduce `enable_global_with_statement` setting which propagates the first select's `WITH` statements to other select queries at the same level, and makes aliases in `WITH` statements visible to subqueries. [#15451](https://github.com/ClickHouse/ClickHouse/pull/15451) ([Amos Bird](https://github.com/amosbird)). -* Secure inter-cluster query execution (with initial_user as current query user). [#13156](https://github.com/ClickHouse/ClickHouse/pull/13156) ([Azat Khuzhin](https://github.com/azat)). [#15551](https://github.com/ClickHouse/ClickHouse/pull/15551) ([Azat Khuzhin](https://github.com/azat)). -* Add the ability to remove column properties and table TTLs. Introduced queries `ALTER TABLE MODIFY COLUMN col_name REMOVE what_to_remove` and `ALTER TABLE REMOVE TTL`. Both operations are lightweight and executed at the metadata level. [#14742](https://github.com/ClickHouse/ClickHouse/pull/14742) ([alesapin](https://github.com/alesapin)). -* Added format `RawBLOB`. It is intended for input or output a single value without any escaping and delimiters. This closes [#15349](https://github.com/ClickHouse/ClickHouse/issues/15349). [#15364](https://github.com/ClickHouse/ClickHouse/pull/15364) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add the `reinterpretAsUUID` function that allows to convert a big-endian byte string to UUID. [#15480](https://github.com/ClickHouse/ClickHouse/pull/15480) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Implement `force_data_skipping_indices` setting. [#15642](https://github.com/ClickHouse/ClickHouse/pull/15642) ([Azat Khuzhin](https://github.com/azat)). -* Add a setting `output_format_pretty_row_numbers` to numerate the result in Pretty formats. This closes [#15350](https://github.com/ClickHouse/ClickHouse/issues/15350). [#15443](https://github.com/ClickHouse/ClickHouse/pull/15443) ([flynn](https://github.com/ucasFL)). -* Added query obfuscation tool. It allows to share more queries for better testing. This closes [#15268](https://github.com/ClickHouse/ClickHouse/issues/15268). [#15321](https://github.com/ClickHouse/ClickHouse/pull/15321) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add table function `null('structure')`. [#14797](https://github.com/ClickHouse/ClickHouse/pull/14797) ([vxider](https://github.com/Vxider)). -* Added `formatReadableQuantity` function. It is useful for reading big numbers by human. [#14725](https://github.com/ClickHouse/ClickHouse/pull/14725) ([Artem Hnilov](https://github.com/BooBSD)). -* Add format `LineAsString` that accepts a sequence of lines separated by newlines, every line is parsed as a whole as a single String field. [#14703](https://github.com/ClickHouse/ClickHouse/pull/14703) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)), [#13846](https://github.com/ClickHouse/ClickHouse/pull/13846) ([hexiaoting](https://github.com/hexiaoting)). -* Add `JSONStrings` format which output data in arrays of strings. [#14333](https://github.com/ClickHouse/ClickHouse/pull/14333) ([hcz](https://github.com/hczhcz)). -* Add support for "Raw" column format for `Regexp` format. It allows to simply extract subpatterns as a whole without any escaping rules. [#15363](https://github.com/ClickHouse/ClickHouse/pull/15363) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow configurable `NULL` representation for `TSV` output format. It is controlled by the setting `output_format_tsv_null_representation` which is `\N` by default. This closes [#9375](https://github.com/ClickHouse/ClickHouse/issues/9375). Note that the setting only controls output format and `\N` is the only supported `NULL` representation for `TSV` input format. [#14586](https://github.com/ClickHouse/ClickHouse/pull/14586) ([Kruglov Pavel](https://github.com/Avogar)). -* Support Decimal data type for `MaterializeMySQL`. `MaterializeMySQL` is an experimental feature. [#14535](https://github.com/ClickHouse/ClickHouse/pull/14535) ([Winter Zhang](https://github.com/zhang2014)). -* Add new feature: `SHOW DATABASES LIKE 'xxx'`. [#14521](https://github.com/ClickHouse/ClickHouse/pull/14521) ([hexiaoting](https://github.com/hexiaoting)). -* Added a script to import (arbitrary) git repository to ClickHouse as a sample dataset. [#14471](https://github.com/ClickHouse/ClickHouse/pull/14471) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now insert statements can have asterisk (or variants) with column transformers in the column list. [#14453](https://github.com/ClickHouse/ClickHouse/pull/14453) ([Amos Bird](https://github.com/amosbird)). -* New query complexity limit settings `max_rows_to_read_leaf`, `max_bytes_to_read_leaf` for distributed queries to limit max rows/bytes read on the leaf nodes. Limit is applied for local reads only, *excluding* the final merge stage on the root node. [#14221](https://github.com/ClickHouse/ClickHouse/pull/14221) ([Roman Khavronenko](https://github.com/hagen1778)). -* Allow user to specify settings for `ReplicatedMergeTree*` storage in `` section of config file. It works similarly to `` section. For `ReplicatedMergeTree*` storages settings from `` and `` are applied together, but settings from `` has higher priority. Added `system.replicated_merge_tree_settings` table. [#13573](https://github.com/ClickHouse/ClickHouse/pull/13573) ([Amos Bird](https://github.com/amosbird)). -* Add `mapPopulateSeries` function. [#13166](https://github.com/ClickHouse/ClickHouse/pull/13166) ([Ildus Kurbangaliev](https://github.com/ildus)). -* Supporting MySQL types: `decimal` (as ClickHouse `Decimal`) and `datetime` with sub-second precision (as `DateTime64`). [#11512](https://github.com/ClickHouse/ClickHouse/pull/11512) ([Vasily Nemkov](https://github.com/Enmk)). -* Introduce `event_time_microseconds` field to `system.text_log`, `system.trace_log`, `system.query_log` and `system.query_thread_log` tables. [#14760](https://github.com/ClickHouse/ClickHouse/pull/14760) ([Bharat Nallan](https://github.com/bharatnc)). -* Add `event_time_microseconds` to `system.asynchronous_metric_log` & `system.metric_log` tables. [#14514](https://github.com/ClickHouse/ClickHouse/pull/14514) ([Bharat Nallan](https://github.com/bharatnc)). -* Add `query_start_time_microseconds` field to `system.query_log` & `system.query_thread_log` tables. [#14252](https://github.com/ClickHouse/ClickHouse/pull/14252) ([Bharat Nallan](https://github.com/bharatnc)). - -#### Bug Fix - -* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `executable` dictionary source hang. In previous versions, when using some formats (e.g. `JSONEachRow`) data was not feed to a child process before it outputs at least something. This closes [#1697](https://github.com/ClickHouse/ClickHouse/issues/1697). This closes [#2455](https://github.com/ClickHouse/ClickHouse/issues/2455). [#14525](https://github.com/ClickHouse/ClickHouse/pull/14525) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). -* Fix async Distributed INSERT with prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). -* Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). -* Fix segfault in some cases of wrong aggregation in lambdas. [#16082](https://github.com/ClickHouse/ClickHouse/pull/16082) ([Anton Popov](https://github.com/CurtizJ)). -* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). -* `MaterializeMySQL` (experimental feature): Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). -* Allow to use `direct` layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). -* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). -* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). -* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). -* `MaterializeMySQL` (experimental feature): Fix crash on create database failure. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) - Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). -* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixes [#12513](https://github.com/ClickHouse/ClickHouse/issues/12513): difference expressions with same alias when query is reanalyzed. [#15886](https://github.com/ClickHouse/ClickHouse/pull/15886) ([Winter Zhang](https://github.com/zhang2014)). -* Fix possible very rare deadlocks in RBAC implementation. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). -* `MaterializeMySQL` (experimental feature): Fix `select count()` inaccuracy. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). -* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). -* Fix drop of materialized view with inner table in Atomic database (hangs all subsequent DROP TABLE due to hang of the worker thread, due to recursive DROP TABLE for inner table of MV). [#15743](https://github.com/ClickHouse/ClickHouse/pull/15743) ([Azat Khuzhin](https://github.com/azat)). -* Possibility to move part to another disk/volume if the first attempt was failed. [#15723](https://github.com/ClickHouse/ClickHouse/pull/15723) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). -* Fix rare race condition in dictionaries and tables from MySQL. [#15686](https://github.com/ClickHouse/ClickHouse/pull/15686) ([alesapin](https://github.com/alesapin)). -* Fix (benign) race condition in AMQP-CPP. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). -* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Proper error handling during insert into MergeTree with S3. MergeTree over S3 is an experimental feature. [#15657](https://github.com/ClickHouse/ClickHouse/pull/15657) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fixed bug with S3 table function: region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Subtract `ReadonlyReplica` metric when detach readonly tables. [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). -* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). -* Query will finish faster in case of exception. Cancel execution on remote replicas if exception happens. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). -* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `Database doesn't exist.` in queries with IN and Distributed table when there's no database on initiator. [#15538](https://github.com/ClickHouse/ClickHouse/pull/15538) ([Artem Zuikov](https://github.com/4ertus2)). -* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). -* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). -* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). -* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). -* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). -* Fixes [#15365](https://github.com/ClickHouse/ClickHouse/issues/15365): attach a database with MySQL engine throws exception (no query context). [#15384](https://github.com/ClickHouse/ClickHouse/pull/15384) ([Winter Zhang](https://github.com/zhang2014)). -* Fix the case of multiple occurrences of column transformers in a select query. [#15378](https://github.com/ClickHouse/ClickHouse/pull/15378) ([Amos Bird](https://github.com/amosbird)). -* Fixed compression in `S3` storage. [#15376](https://github.com/ClickHouse/ClickHouse/pull/15376) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fix bug where queries like `SELECT toStartOfDay(today())` fail complaining about empty time_zone argument. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). -* Fix rare race condition on server startup when system logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). -* Fix hang of queries with a lot of subqueries to same table of `MySQL` engine. Previously, if there were more than 16 subqueries to same `MySQL` table in query, it hang forever. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). -* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix 'Unknown identifier' in GROUP BY when query has JOIN over Merge table. [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix instance crash when using `joinGet` with `LowCardinality` types. This fixes [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). -* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). -* Adjust Decimal field size in MySQL column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). -* Fixes `Data compressed with different methods` in `join_algorithm='auto'`. Keep LowCardinality as type for left table join key in `join_algorithm='partial_merge'`. [#15088](https://github.com/ClickHouse/ClickHouse/pull/15088) ([Artem Zuikov](https://github.com/4ertus2)). -* Update `jemalloc` to fix `percpu_arena` with affinity mask. [#15035](https://github.com/ClickHouse/ClickHouse/pull/15035) ([Azat Khuzhin](https://github.com/azat)). [#14957](https://github.com/ClickHouse/ClickHouse/pull/14957) ([Azat Khuzhin](https://github.com/azat)). -* We already use padded comparison between String and FixedString (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333). This PR applies the same logic to field comparison which corrects the usage of FixedString as primary keys. This fixes [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). -* If function `bar` was called with specifically crafted arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in Docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). -* Fix crash in RIGHT or FULL JOIN with join_algorith='auto' when memory limit exceeded and we should change HashJoin with MergeJoin. [#15002](https://github.com/ClickHouse/ClickHouse/pull/15002) ([Artem Zuikov](https://github.com/4ertus2)). -* Now settings `number_of_free_entries_in_pool_to_execute_mutation` and `number_of_free_entries_in_pool_to_lower_max_size_of_merge` can be equal to `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). -* Fix to make predicate push down work when subquery contains `finalizeAggregation` function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* `MaterializeMySQL` (experimental feature): Fixed `.metadata.tmp File exists` error. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). -* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix SIGSEGV for an attempt to INSERT into StorageFile with file descriptor. [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). -* Fixed segfault in `cache` dictionary [#14837](https://github.com/ClickHouse/ClickHouse/issues/14837). [#14879](https://github.com/ClickHouse/ClickHouse/pull/14879) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* `MaterializeMySQL` (experimental feature): Fixed bug in parsing MySQL binlog events, which causes `Attempt to read after eof` and `Packet payload is not fully read` in `MaterializeMySQL` database engine. [#14852](https://github.com/ClickHouse/ClickHouse/pull/14852) ([Winter Zhang](https://github.com/zhang2014)). -* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). -* `Replace` column transformer should replace identifiers with cloned ASTs. This fixes [#14695](https://github.com/ClickHouse/ClickHouse/issues/14695) . [#14734](https://github.com/ClickHouse/ClickHouse/pull/14734) ([Amos Bird](https://github.com/amosbird)). -* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). -* Fix bug when `ALTER UPDATE` mutation with `Nullable` column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix function `has` with `LowCardinality` of `Nullable`. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([Mike](https://github.com/myrrc)). -* Cleanup data directory after Zookeeper exceptions during CreateQuery for StorageReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix rare segfaults in functions with combinator `-Resample`, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). -* Fix a bug when converting `Nullable(String)` to Enum. Introduced by [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745). This fixes [#14435](https://github.com/ClickHouse/ClickHouse/issues/14435). [#14530](https://github.com/ClickHouse/ClickHouse/pull/14530) ([Amos Bird](https://github.com/amosbird)). -* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix `currentDatabase()` function cannot be used in `ON CLUSTER` ddl query. [#14211](https://github.com/ClickHouse/ClickHouse/pull/14211) ([Winter Zhang](https://github.com/zhang2014)). -* `MaterializeMySQL` (experimental feature): Fixed `Packet payload is not fully read` error in `MaterializeMySQL` database engine. [#14696](https://github.com/ClickHouse/ClickHouse/pull/14696) ([BohuTANG](https://github.com/BohuTANG)). - -#### Improvement - -* Enable `Atomic` database engine by default for newly created databases. [#15003](https://github.com/ClickHouse/ClickHouse/pull/15003) ([tavplubix](https://github.com/tavplubix)). -* Add the ability to specify specialized codecs like `Delta`, `T64`, etc. for columns with subtypes. Implements [#12551](https://github.com/ClickHouse/ClickHouse/issues/12551), fixes [#11397](https://github.com/ClickHouse/ClickHouse/issues/11397), fixes [#4609](https://github.com/ClickHouse/ClickHouse/issues/4609). [#15089](https://github.com/ClickHouse/ClickHouse/pull/15089) ([alesapin](https://github.com/alesapin)). -* Dynamic reload of zookeeper config. [#14678](https://github.com/ClickHouse/ClickHouse/pull/14678) ([sundyli](https://github.com/sundy-li)). -* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Now `joinGet` supports multi-key lookup. Continuation of [#12418](https://github.com/ClickHouse/ClickHouse/issues/12418). [#13015](https://github.com/ClickHouse/ClickHouse/pull/13015) ([Amos Bird](https://github.com/amosbird)). -* Wait for `DROP/DETACH TABLE` to actually finish if `NO DELAY` or `SYNC` is specified for `Atomic` database. [#15448](https://github.com/ClickHouse/ClickHouse/pull/15448) ([tavplubix](https://github.com/tavplubix)). -* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). -* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). -* The function `now` allows an argument with timezone. This closes [15264](https://github.com/ClickHouse/ClickHouse/issues/15264). [#15285](https://github.com/ClickHouse/ClickHouse/pull/15285) ([flynn](https://github.com/ucasFL)). -* Do not allow connections to ClickHouse server until all scripts in `/docker-entrypoint-initdb.d/` are executed. [#15244](https://github.com/ClickHouse/ClickHouse/pull/15244) ([Aleksei Kozharin](https://github.com/alekseik1)). -* Added `optimize` setting to `EXPLAIN PLAN` query. If enabled, query plan level optimisations are applied. Enabled by default. [#15201](https://github.com/ClickHouse/ClickHouse/pull/15201) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Proper exception message for wrong number of arguments of CAST. This closes [#13992](https://github.com/ClickHouse/ClickHouse/issues/13992). [#15029](https://github.com/ClickHouse/ClickHouse/pull/15029) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add option to disable TTL move on data part insert. [#15000](https://github.com/ClickHouse/ClickHouse/pull/15000) ([Pavel Kovalenko](https://github.com/Jokser)). -* Ignore key constraints when doing mutations. Without this pull request, it's not possible to do mutations when `force_index_by_date = 1` or `force_primary_key = 1`. [#14973](https://github.com/ClickHouse/ClickHouse/pull/14973) ([Amos Bird](https://github.com/amosbird)). -* Allow to drop Replicated table if previous drop attempt was failed due to ZooKeeper session expiration. This fixes [#11891](https://github.com/ClickHouse/ClickHouse/issues/11891). [#14926](https://github.com/ClickHouse/ClickHouse/pull/14926) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed excessive settings constraint violation when running SELECT with SETTINGS from a distributed table. [#14876](https://github.com/ClickHouse/ClickHouse/pull/14876) ([Amos Bird](https://github.com/amosbird)). -* Provide a `load_balancing_first_offset` query setting to explicitly state what the first replica is. It's used together with `FIRST_OR_RANDOM` load balancing strategy, which allows to control replicas workload. [#14867](https://github.com/ClickHouse/ClickHouse/pull/14867) ([Amos Bird](https://github.com/amosbird)). -* Show subqueries for `SET` and `JOIN` in `EXPLAIN` result. [#14856](https://github.com/ClickHouse/ClickHouse/pull/14856) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Allow using multi-volume storage configuration in storage `Distributed`. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). -* Construct `query_start_time` and `query_start_time_microseconds` from the same timespec. [#14831](https://github.com/ClickHouse/ClickHouse/pull/14831) ([Bharat Nallan](https://github.com/bharatnc)). -* Support for disabling persistency for `StorageJoin` and `StorageSet`, this feature is controlled by setting `disable_set_and_join_persistency`. And this PR solved issue [#6318](https://github.com/ClickHouse/ClickHouse/issues/6318). [#14776](https://github.com/ClickHouse/ClickHouse/pull/14776) ([vxider](https://github.com/Vxider)). -* Now `COLUMNS` can be used to wrap over a list of columns and apply column transformers afterwards. [#14775](https://github.com/ClickHouse/ClickHouse/pull/14775) ([Amos Bird](https://github.com/amosbird)). -* Add `merge_algorithm` to `system.merges` table to improve merging inspections. [#14705](https://github.com/ClickHouse/ClickHouse/pull/14705) ([Amos Bird](https://github.com/amosbird)). -* Fix potential memory leak caused by zookeeper exists watch. [#14693](https://github.com/ClickHouse/ClickHouse/pull/14693) ([hustnn](https://github.com/hustnn)). -* Allow parallel execution of distributed DDL. [#14684](https://github.com/ClickHouse/ClickHouse/pull/14684) ([Azat Khuzhin](https://github.com/azat)). -* Add `QueryMemoryLimitExceeded` event counter. This closes [#14589](https://github.com/ClickHouse/ClickHouse/issues/14589). [#14647](https://github.com/ClickHouse/ClickHouse/pull/14647) ([fastio](https://github.com/fastio)). -* Fix some trailing whitespaces in query formatting. [#14595](https://github.com/ClickHouse/ClickHouse/pull/14595) ([Azat Khuzhin](https://github.com/azat)). -* ClickHouse treats partition expr and key expr differently. Partition expr is used to construct an minmax index containing related columns, while primary key expr is stored as an expr. Sometimes user might partition a table at coarser levels, such as `partition by i / 1000`. However, binary operators are not monotonic and this PR tries to fix that. It might also benifit other use cases. [#14513](https://github.com/ClickHouse/ClickHouse/pull/14513) ([Amos Bird](https://github.com/amosbird)). -* Add an option to skip access checks for `DiskS3`. `s3` disk is an experimental feature. [#14497](https://github.com/ClickHouse/ClickHouse/pull/14497) ([Pavel Kovalenko](https://github.com/Jokser)). -* Speed up server shutdown process if there are ongoing S3 requests. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). -* `SYSTEM RELOAD CONFIG` now throws an exception if failed to reload and continues using the previous users.xml. The background periodic reloading also continues using the previous users.xml if failed to reload. [#14492](https://github.com/ClickHouse/ClickHouse/pull/14492) ([Vitaly Baranov](https://github.com/vitlibar)). -* For INSERTs with inline data in VALUES format in the script mode of `clickhouse-client`, support semicolon as the data terminator, in addition to the new line. Closes [#12288](https://github.com/ClickHouse/ClickHouse/issues/12288). [#13192](https://github.com/ClickHouse/ClickHouse/pull/13192) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - -#### Performance Improvement - -* Enable compact parts by default for small parts. This will allow to process frequent inserts slightly more efficiently (4..100 times). [#11913](https://github.com/ClickHouse/ClickHouse/pull/11913) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improve `quantileTDigest` performance. This fixes [#2668](https://github.com/ClickHouse/ClickHouse/issues/2668). [#15542](https://github.com/ClickHouse/ClickHouse/pull/15542) ([Kruglov Pavel](https://github.com/Avogar)). -* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). -* Faster 256-bit multiplication. [#15418](https://github.com/ClickHouse/ClickHouse/pull/15418) ([Artem Zuikov](https://github.com/4ertus2)). -* Improve performance of 256-bit types using (u)int64_t as base type for wide integers. Original wide integers use 8-bit types as base. [#14859](https://github.com/ClickHouse/ClickHouse/pull/14859) ([Artem Zuikov](https://github.com/4ertus2)). -* Explicitly use a temporary disk to store vertical merge temporary data. [#15639](https://github.com/ClickHouse/ClickHouse/pull/15639) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). -* Use one S3 DeleteObjects request instead of multiple DeleteObject in a loop. No any functionality changes, so covered by existing tests like integration/test_log_family_s3. [#15238](https://github.com/ClickHouse/ClickHouse/pull/15238) ([ianton-ru](https://github.com/ianton-ru)). -* Fix `DateTime DateTime` mistakenly choosing the slow generic implementation. This fixes [#15153](https://github.com/ClickHouse/ClickHouse/issues/15153). [#15178](https://github.com/ClickHouse/ClickHouse/pull/15178) ([Amos Bird](https://github.com/amosbird)). -* Improve performance of GROUP BY key of type `FixedString`. [#15034](https://github.com/ClickHouse/ClickHouse/pull/15034) ([Amos Bird](https://github.com/amosbird)). -* Only `mlock` code segment when starting clickhouse-server. In previous versions, all mapped regions were locked in memory, including debug info. Debug info is usually splitted to a separate file but if it isn't, it led to +2..3 GiB memory usage. [#14929](https://github.com/ClickHouse/ClickHouse/pull/14929) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* ClickHouse binary become smaller due to link time optimization. - -#### Build/Testing/Packaging Improvement - -* Now we use clang-11 for production ClickHouse build. [#15239](https://github.com/ClickHouse/ClickHouse/pull/15239) ([alesapin](https://github.com/alesapin)). -* Now we use clang-11 to build ClickHouse in CI. [#14846](https://github.com/ClickHouse/ClickHouse/pull/14846) ([alesapin](https://github.com/alesapin)). -* Switch binary builds (Linux, Darwin, AArch64, FreeDSD) to clang-11. [#15622](https://github.com/ClickHouse/ClickHouse/pull/15622) ([Ilya Yatsishin](https://github.com/qoega)). -* Now all test images use `llvm-symbolizer-11`. [#15069](https://github.com/ClickHouse/ClickHouse/pull/15069) ([alesapin](https://github.com/alesapin)). -* Allow to build with llvm-11. [#15366](https://github.com/ClickHouse/ClickHouse/pull/15366) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Switch from `clang-tidy-10` to `clang-tidy-11`. [#14922](https://github.com/ClickHouse/ClickHouse/pull/14922) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Use LLVM's experimental pass manager by default. [#15608](https://github.com/ClickHouse/ClickHouse/pull/15608) ([Danila Kutenin](https://github.com/danlark1)). -* Don't allow any C++ translation unit to build more than 10 minutes or to use more than 10 GB or memory. This fixes [#14925](https://github.com/ClickHouse/ClickHouse/issues/14925). [#15060](https://github.com/ClickHouse/ClickHouse/pull/15060) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make performance test more stable and representative by splitting test runs and profile runs. [#15027](https://github.com/ClickHouse/ClickHouse/pull/15027) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Attempt to make performance test more reliable. It is done by remapping the executable memory of the process on the fly with `madvise` to use transparent huge pages - it can lower the number of iTLB misses which is the main source of instabilities in performance tests. [#14685](https://github.com/ClickHouse/ClickHouse/pull/14685) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Convert to python3. This closes [#14886](https://github.com/ClickHouse/ClickHouse/issues/14886). [#15007](https://github.com/ClickHouse/ClickHouse/pull/15007) ([Azat Khuzhin](https://github.com/azat)). -* Fail early in functional tests if server failed to respond. This closes [#15262](https://github.com/ClickHouse/ClickHouse/issues/15262). [#15267](https://github.com/ClickHouse/ClickHouse/pull/15267) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow to run AArch64 version of clickhouse-server without configs. This facilitates [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174). [#15266](https://github.com/ClickHouse/ClickHouse/pull/15266) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improvements in CI docker images: get rid of ZooKeeper and single script for test configs installation. [#15215](https://github.com/ClickHouse/ClickHouse/pull/15215) ([alesapin](https://github.com/alesapin)). -* Fix CMake options forwarding in fast test script. Fixes error in [#14711](https://github.com/ClickHouse/ClickHouse/issues/14711). [#15155](https://github.com/ClickHouse/ClickHouse/pull/15155) ([alesapin](https://github.com/alesapin)). -* Added a script to perform hardware benchmark in a single command. [#15115](https://github.com/ClickHouse/ClickHouse/pull/15115) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Splitted huge test `test_dictionaries_all_layouts_and_sources` into smaller ones. [#15110](https://github.com/ClickHouse/ClickHouse/pull/15110) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Maybe fix MSan report in base64 (on servers with AVX-512). This fixes [#14006](https://github.com/ClickHouse/ClickHouse/issues/14006). [#15030](https://github.com/ClickHouse/ClickHouse/pull/15030) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Reformat and cleanup code in all integration test *.py files. [#14864](https://github.com/ClickHouse/ClickHouse/pull/14864) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix MaterializeMySQL empty transaction unstable test case found in CI. [#14854](https://github.com/ClickHouse/ClickHouse/pull/14854) ([Winter Zhang](https://github.com/zhang2014)). -* Attempt to speed up build a little. [#14808](https://github.com/ClickHouse/ClickHouse/pull/14808) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Speed up build a little by removing unused headers. [#14714](https://github.com/ClickHouse/ClickHouse/pull/14714) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix build failure in OSX. [#14761](https://github.com/ClickHouse/ClickHouse/pull/14761) ([Winter Zhang](https://github.com/zhang2014)). -* Enable ccache by default in cmake if it's found in OS. [#14575](https://github.com/ClickHouse/ClickHouse/pull/14575) ([alesapin](https://github.com/alesapin)). -* Control CI builds configuration from the ClickHouse repository. [#14547](https://github.com/ClickHouse/ClickHouse/pull/14547) ([alesapin](https://github.com/alesapin)). -* In CMake files: - Moved some options' descriptions' parts to comments above. - Replace 0 -> `OFF`, 1 -> `ON` in `option`s default values. - Added some descriptions and links to docs to the options. - Replaced `FUZZER` option (there is another option `ENABLE_FUZZING` which also enables same functionality). - Removed `ENABLE_GTEST_LIBRARY` option as there is `ENABLE_TESTS`. See the full description in PR: [#14711](https://github.com/ClickHouse/ClickHouse/pull/14711) ([Mike](https://github.com/myrrc)). -* Make binary a bit smaller (~50 Mb for debug version). [#14555](https://github.com/ClickHouse/ClickHouse/pull/14555) ([Artem Zuikov](https://github.com/4ertus2)). -* Use std::filesystem::path in ConfigProcessor for concatenating file paths. [#14558](https://github.com/ClickHouse/ClickHouse/pull/14558) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix debug assertion in `bitShiftLeft()` when called with negative big integer. [#14697](https://github.com/ClickHouse/ClickHouse/pull/14697) ([Artem Zuikov](https://github.com/4ertus2)). - - -## ClickHouse release 20.9 - -### ClickHouse release v20.9.5.5-stable, 2020-11-13 - -#### Bug Fix - -* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). - - -### ClickHouse release v20.9.4.76-stable (2020-10-29) - -#### Bug Fix - -* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). -* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). -* Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. I'm suprised how this bug can lurk for so long.... [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). -* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). -* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). -* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). -* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). -* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). -* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). -* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). -* Fix `select count()` inaccuracy for MaterializeMySQL. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). -* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). -* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). -* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). - -#### Improvement - -* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16160](https://github.com/ClickHouse/ClickHouse/pull/16160) ([tavplubix](https://github.com/tavplubix)). - - -### ClickHouse release v20.9.3.45-stable (2020-10-09) - -#### Bug Fix - -* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix race condition in AMQP-CPP. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). -* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). -* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). -* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). -* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). -* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). -* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). -* Fix bug where queries like `SELECT toStartOfDay(today())` fail complaining about empty time_zone argument. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). -* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). -* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix instance crash when using joinGet with LowCardinality types. This fixes [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). -* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). -* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). -* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). -* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). - -#### Improvement - -* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse release v20.9.2.20, 2020-09-22 - -#### Backward Incompatible Change - -* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). - -#### New Feature - -* Added column transformers `EXCEPT`, `REPLACE`, `APPLY`, which can be applied to the list of selected columns (after `*` or `COLUMNS(...)`). For example, you can write `SELECT * EXCEPT(URL) REPLACE(number + 1 AS number)`. Another example: `select * apply(length) apply(max) from wide_string_table` to find out the maxium length of all string columns. [#14233](https://github.com/ClickHouse/ClickHouse/pull/14233) ([Amos Bird](https://github.com/amosbird)). -* Added an aggregate function `rankCorr` which computes a rank correlation coefficient. [#11769](https://github.com/ClickHouse/ClickHouse/pull/11769) ([antikvist](https://github.com/antikvist)) [#14411](https://github.com/ClickHouse/ClickHouse/pull/14411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Added table function `view` which turns a subquery into a table object. This helps passing queries around. For instance, it can be used in remote/cluster table functions. [#12567](https://github.com/ClickHouse/ClickHouse/pull/12567) ([Amos Bird](https://github.com/amosbird)). - -#### Bug Fix - -* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed inconsistent comparison with primary key of type `FixedString` on index analysis if they're compered with a string of less size. This fixes [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). -* Fix bug which leads to wrong merges assignment if table has partitions with a single part. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). -* If function `bar` was called with specifically crafted arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fixed `.metadata.tmp File exists` error when using `MaterializeMySQL` database engine. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). -* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix SIGSEGV for an attempt to INSERT into StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). -* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). -* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). -* Fix possibly incorrect result of function `has` when LowCardinality and Nullable types are involved. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([Mike](https://github.com/myrrc)). -* Cleanup data directory after Zookeeper exceptions during CREATE query for tables with ReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix rare segfaults in functions with combinator `-Resample`, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). -* Check for array size overflow in `topK` aggregate function. Without this check the user may send a query with carefully crafted parameters that will lead to server crash. This closes [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Proxy restart/start/stop/reload of SysVinit to systemd (if it is used). [#14460](https://github.com/ClickHouse/ClickHouse/pull/14460) ([Azat Khuzhin](https://github.com/azat)). -* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix crash during `ALTER` query for table which was created `AS table_function`. Fixes [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). -* Fix exception during ALTER LIVE VIEW query with REFRESH command. LIVE VIEW is an experimental feature. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter. [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). -* Better check for tuple size in SSD cache complex key external dictionaries. This fixes [#13981](https://github.com/ClickHouse/ClickHouse/issues/13981). [#14313](https://github.com/ClickHouse/ClickHouse/pull/14313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Disallows `CODEC` on `ALIAS` column type. Fixes [#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix GRANT ALL statement when executed on a non-global level. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix arrayJoin() capturing in lambda (exception with logical error message was thrown). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). - -#### Experimental Feature - -* Added `db-generator` tool for random database generation by given SELECT queries. It may faciliate reproducing issues when there is only incomplete bug report from the user. [#14442](https://github.com/ClickHouse/ClickHouse/pull/14442) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) [#10973](https://github.com/ClickHouse/ClickHouse/issues/10973) ([ZeDRoman](https://github.com/ZeDRoman)). - -#### Improvement - -* Allow using multi-volume storage configuration in storage Distributed. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). -* Disallow empty time_zone argument in `toStartOf*` type of functions. [#14509](https://github.com/ClickHouse/ClickHouse/pull/14509) ([Bharat Nallan](https://github.com/bharatnc)). -* MySQL handler returns `OK` for queries like `SET @@var = value`. Such statement is ignored. It is needed because some MySQL drivers send `SET @@` query for setup after handshake https://github.com/ClickHouse/ClickHouse/issues/9336#issuecomment-686222422 . [#14469](https://github.com/ClickHouse/ClickHouse/pull/14469) ([BohuTANG](https://github.com/BohuTANG)). -* Now TTLs will be applied during merge if they were not previously materialized. [#14438](https://github.com/ClickHouse/ClickHouse/pull/14438) ([alesapin](https://github.com/alesapin)). -* Now `clickhouse-obfuscator` supports UUID type as proposed in [#13163](https://github.com/ClickHouse/ClickHouse/issues/13163). [#14409](https://github.com/ClickHouse/ClickHouse/pull/14409) ([dimarub2000](https://github.com/dimarub2000)). -* Added new setting `system_events_show_zero_values` as proposed in [#11384](https://github.com/ClickHouse/ClickHouse/issues/11384). [#14404](https://github.com/ClickHouse/ClickHouse/pull/14404) ([dimarub2000](https://github.com/dimarub2000)). -* Implicitly convert primary key to not null in `MaterializeMySQL` (Same as `MySQL`). Fixes [#14114](https://github.com/ClickHouse/ClickHouse/issues/14114). [#14397](https://github.com/ClickHouse/ClickHouse/pull/14397) ([Winter Zhang](https://github.com/zhang2014)). -* Replace wide integers (256 bit) from boost multiprecision with implementation from https://github.com/cerevra/int. 256bit integers are experimental. [#14229](https://github.com/ClickHouse/ClickHouse/pull/14229) ([Artem Zuikov](https://github.com/4ertus2)). -* Add default compression codec for parts in `system.part_log` with the name `default_compression_codec`. [#14116](https://github.com/ClickHouse/ClickHouse/pull/14116) ([alesapin](https://github.com/alesapin)). -* Add precision argument for `DateTime` type. It allows to use `DateTime` name instead of `DateTime64`. [#13761](https://github.com/ClickHouse/ClickHouse/pull/13761) ([Winter Zhang](https://github.com/zhang2014)). -* Added requirepass authorization for `Redis` external dictionary. [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). -* Improvements in `RabbitMQ` engine: added connection and channels failure handling, proper commits, insert failures handling, better exchanges, queue durability and queue resume opportunity, new queue settings. Fixed tests. [#12761](https://github.com/ClickHouse/ClickHouse/pull/12761) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - -#### Performance Improvement - -* Optimize queries with LIMIT/LIMIT BY/ORDER BY for distributed with GROUP BY sharding_key (under `optimize_skip_unused_shards` and `optimize_distributed_group_by_sharding_key`). [#10373](https://github.com/ClickHouse/ClickHouse/pull/10373) ([Azat Khuzhin](https://github.com/azat)). -* Creating sets for multiple `JOIN` and `IN` in parallel. It may slightly improve performance for queries with several different `IN subquery` expressions. [#14412](https://github.com/ClickHouse/ClickHouse/pull/14412) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Improve Kafka engine performance by providing independent thread for each consumer. Separate thread pool for streaming engines (like Kafka). [#13939](https://github.com/ClickHouse/ClickHouse/pull/13939) ([fastio](https://github.com/fastio)). - -#### Build/Testing/Packaging Improvement - -* Lower binary size in debug build by removing debug info from `Functions`. This is needed only for one internal project in Yandex who is using very old linker. [#14549](https://github.com/ClickHouse/ClickHouse/pull/14549) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Prepare for build with clang 11. [#14455](https://github.com/ClickHouse/ClickHouse/pull/14455) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix the logic in backport script. In previous versions it was triggered for any labels of 100% red color. It was strange. [#14433](https://github.com/ClickHouse/ClickHouse/pull/14433) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Integration tests use default base config. All config changes are explicit with main_configs, user_configs and dictionaries parameters for instance. [#13647](https://github.com/ClickHouse/ClickHouse/pull/13647) ([Ilya Yatsishin](https://github.com/qoega)). - - - -## ClickHouse release 20.8 - -### ClickHouse release v20.8.10.13-lts, 2020-12-24 - -#### Bug Fix - -* When server log rotation was configured using `logger.size` parameter with numeric value larger than 2^32, the logs were not rotated properly. [#17905](https://github.com/ClickHouse/ClickHouse/pull/17905) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fixed incorrect initialization of `max_compress_block_size` in MergeTreeWriterSettings with `min_compress_block_size`. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). -* Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* Fixed `ALTER` query hang when the corresponding mutation was killed on the different replica. This fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* Fixed a bug when mark cache size was underestimated by ClickHouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). -* Fixed `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed `ColumnConst` comparison which leads to crash. This fixed [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). -* Fixed bug when `ON CLUSTER` queries may hang forever for non-leader ReplicatedMergeTreeTables. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). -* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). -* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). -* Fixed possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). -* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). -* Fixed possible error `Illegal type of argument` for queries with `ORDER BY`. Fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Abort multipart upload if no data was written to WriteBufferFromS3. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fixed crash when using `any` without any arguments. This fixes [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). -* Fixed `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed inconsistent behaviour of `optimize_read_in_order/optimize_aggregation_in_order` with max_threads > 0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). -* Fixed the issue when query optimization was producing wrong result if query contains `ARRAY JOIN`. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). -* Query is finished faster in case of exception. Cancel execution on remote replicas if exception happens. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). - - -### ClickHouse release v20.8.6.6-lts, 2020-11-13 - -#### Bug Fix - -* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). -* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). -* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). -* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). - - -### ClickHouse release v20.8.5.45-lts, 2020-10-29 - -#### Bug Fix - -* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). -* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). -* Fix a possible memory leak during `GROUP BY` with string keys, caused by an error in `TwoLevelStringHashTable` implementation. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). -* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). -* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). -* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). -* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). -* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). -* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). -* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). -* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). -* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). -* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). -* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). - -#### Improvement - -* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). -* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16159](https://github.com/ClickHouse/ClickHouse/pull/16159) ([tavplubix](https://github.com/tavplubix)). - - -### ClickHouse release v20.8.4.11-lts, 2020-10-09 - -#### Bug Fix - -* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). -* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). -* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). -* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). -* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). -* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). -* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). -* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). -* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix instance crash when using joinGet with LowCardinality types. This fixes [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). -* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). -* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). -* We already use padded comparison between String and FixedString (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333). This PR applies the same logic to field comparison which corrects the usage of FixedString as primary keys. This fixes [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). -* If function `bar` was called with specifically crafted arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). -* Now settings `number_of_free_entries_in_pool_to_execute_mutation` and `number_of_free_entries_in_pool_to_lower_max_size_of_merge` can be equal to `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). -* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fixed `.metadata.tmp File exists` error when using `MaterializeMySQL` database engine. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). -* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). -* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - -#### Improvement - -* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse release v20.8.3.18-stable, 2020-09-18 - -#### Bug Fix - -* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix SIGSEGV for an attempt to INSERT into StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). -* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). -* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). -* Added the checker as neither calling `lc->isNullable()` nor calling `ls->getDictionaryPtr()->isNullable()` would return the correct result. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([myrrc](https://github.com/myrrc)). -* Cleanup data directory after Zookeeper exceptions during CreateQuery for StorageReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix rare segfaults in functions with combinator -Resample, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). - -#### Improvement - -* Speed up server shutdown process if there are ongoing S3 requests. [#14858](https://github.com/ClickHouse/ClickHouse/pull/14858) ([Pavel Kovalenko](https://github.com/Jokser)). -* Allow using multi-volume storage configuration in storage Distributed. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). -* Speed up server shutdown process if there are ongoing S3 requests. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). -* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse release v20.8.2.3-stable, 2020-09-08 - -#### Backward Incompatible Change - -* Now `OPTIMIZE FINAL` query doesn't recalculate TTL for parts that were added before TTL was created. Use `ALTER TABLE ... MATERIALIZE TTL` once to calculate them, after that `OPTIMIZE FINAL` will evaluate TTL's properly. This behavior never worked for replicated tables. [#14220](https://github.com/ClickHouse/ClickHouse/pull/14220) ([alesapin](https://github.com/alesapin)). -* Extend `parallel_distributed_insert_select` setting, adding an option to run `INSERT` into local table. The setting changes type from `Bool` to `UInt64`, so the values `false` and `true` are no longer supported. If you have these values in server configuration, the server will not start. Please replace them with `0` and `1`, respectively. [#14060](https://github.com/ClickHouse/ClickHouse/pull/14060) ([Azat Khuzhin](https://github.com/azat)). -* Remove support for the `ODBCDriver` input/output format. This was a deprecated format once used for communication with the ClickHouse ODBC driver, now long superseded by the `ODBCDriver2` format. Resolves [#13629](https://github.com/ClickHouse/ClickHouse/issues/13629). [#13847](https://github.com/ClickHouse/ClickHouse/pull/13847) ([hexiaoting](https://github.com/hexiaoting)). -* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). - -#### New Feature - -* Add the ability to specify `Default` compression codec for columns that correspond to settings specified in `config.xml`. Implements: [#9074](https://github.com/ClickHouse/ClickHouse/issues/9074). [#14049](https://github.com/ClickHouse/ClickHouse/pull/14049) ([alesapin](https://github.com/alesapin)). -* Support Kerberos authentication in Kafka, using `krb5` and `cyrus-sasl` libraries. [#12771](https://github.com/ClickHouse/ClickHouse/pull/12771) ([Ilya Golshtein](https://github.com/ilejn)). -* Add function `normalizeQuery` that replaces literals, sequences of literals and complex aliases with placeholders. Add function `normalizedQueryHash` that returns identical 64bit hash values for similar queries. It helps to analyze query log. This closes [#11271](https://github.com/ClickHouse/ClickHouse/issues/11271). [#13816](https://github.com/ClickHouse/ClickHouse/pull/13816) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add `time_zones` table. [#13880](https://github.com/ClickHouse/ClickHouse/pull/13880) ([Bharat Nallan](https://github.com/bharatnc)). -* Add function `defaultValueOfTypeName` that returns the default value for a given type. [#13877](https://github.com/ClickHouse/ClickHouse/pull/13877) ([hcz](https://github.com/hczhcz)). -* Add `countDigits(x)` function that count number of decimal digits in integer or decimal column. Add `isDecimalOverflow(d, [p])` function that checks if the value in Decimal column is out of its (or specified) precision. [#14151](https://github.com/ClickHouse/ClickHouse/pull/14151) ([Artem Zuikov](https://github.com/4ertus2)). -* Add `quantileExactLow` and `quantileExactHigh` implementations with respective aliases for `medianExactLow` and `medianExactHigh`. [#13818](https://github.com/ClickHouse/ClickHouse/pull/13818) ([Bharat Nallan](https://github.com/bharatnc)). -* Added `date_trunc` function that truncates a date/time value to a specified date/time part. [#13888](https://github.com/ClickHouse/ClickHouse/pull/13888) ([Vladimir Golovchenko](https://github.com/vladimir-golovchenko)). -* Add new optional section `` to the main config. [#13425](https://github.com/ClickHouse/ClickHouse/pull/13425) ([Vitaly Baranov](https://github.com/vitlibar)). -* Add `ALTER SAMPLE BY` statement that allows to change table sample clause. [#13280](https://github.com/ClickHouse/ClickHouse/pull/13280) ([Amos Bird](https://github.com/amosbird)). -* Function `position` now supports optional `start_pos` argument. [#13237](https://github.com/ClickHouse/ClickHouse/pull/13237) ([vdimir](https://github.com/vdimir)). - -#### Bug Fix - -* Fix visible data clobbering by progress bar in client in interactive mode. This fixes [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) and [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) and [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) and fixes [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed incorrect sorting order if `LowCardinality` column when sorting by multiple columns. This fixes [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Check for array size overflow in `topK` aggregate function. Without this check the user may send a query with carefully crafted parameters that will lead to server crash. This closes [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug which can lead to wrong merges assignment if table has partitions with a single part. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). -* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix crash during `ALTER` query for table which was created `AS table_function`. Fixes [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). -* Fix exception during ALTER LIVE VIEW query with REFRESH command. Live view is an experimental feature. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). -* Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter. [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). -* Fix segfault in `clickhouse-odbc-bridge` during schema fetch from some external sources. This PR fixes [#13861](https://github.com/ClickHouse/ClickHouse/issues/13861). [#14267](https://github.com/ClickHouse/ClickHouse/pull/14267) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix crash in mark inclusion search introduced in [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277). [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). -* Fix creation of tables with named tuples. This fixes [#13027](https://github.com/ClickHouse/ClickHouse/issues/13027). [#14143](https://github.com/ClickHouse/ClickHouse/pull/14143) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix formatting of minimal negative decimal numbers. This fixes [#14111](https://github.com/ClickHouse/ClickHouse/issues/14111). [#14119](https://github.com/ClickHouse/ClickHouse/pull/14119) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix `DistributedFilesToInsert` metric (zeroed when it should not). [#14095](https://github.com/ClickHouse/ClickHouse/pull/14095) ([Azat Khuzhin](https://github.com/azat)). -* Fix `pointInPolygon` with const 2d array as polygon. [#14079](https://github.com/ClickHouse/ClickHouse/pull/14079) ([Alexey Ilyukhov](https://github.com/livace)). -* Fixed wrong mount point in extra info for `Poco::Exception: no space left on device`. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). -* Fix GRANT ALL statement when executed on a non-global level. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix parser to reject create table as table function with engine. [#13940](https://github.com/ClickHouse/ClickHouse/pull/13940) ([hcz](https://github.com/hczhcz)). -* Fix wrong results in select queries with `DISTINCT` keyword and subqueries with UNION ALL in case `optimize_duplicate_order_by_and_distinct` setting is enabled. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed potential deadlock when renaming `Distributed` table. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). -* Fix incorrect sorting for `FixedString` columns when sorting by multiple columns. Fixes [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix potentially imprecise result of `topK`/`topKWeighted` merge (with non-default parameters). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). -* Fix reading from MergeTree table with INDEX of type SET fails when comparing against NULL. This fixes [#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). -* Fix `arrayJoin` capturing in lambda (LOGICAL_ERROR). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). -* Add step overflow check in function `range`. [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). -* Fixed `Directory not empty` error when concurrently executing `DROP DATABASE` and `CREATE TABLE`. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add range check for `h3KRing` function. This fixes [#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix race condition between DETACH and background merges. Parts may revive after detach. This is continuation of [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) that did not fix the issue but introduced a test that started to fail in very rare cases, demonstrating the issue. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix logging Settings.Names/Values when log_queries_min_type > QUERY_START. [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). -* Fixes `/replicas_status` endpoint response status code when verbose=1. [#13722](https://github.com/ClickHouse/ClickHouse/pull/13722) ([javi santana](https://github.com/javisantana)). -* Fix incorrect message in `clickhouse-server.init` while checking user and group. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). -* Do not optimize any(arrayJoin()) -> arrayJoin() under `optimize_move_functions_out_of_any` setting. [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). -* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix typo in error message about `The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting`. [#13678](https://github.com/ClickHouse/ClickHouse/pull/13678) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). -* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix secondary indices corruption in compact parts. Compact parts are experimental feature. [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). -* Fix premature `ON CLUSTER` timeouts for queries that must be executed on a single replica. Fixes [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). -* Fix wrong code in function `netloc`. This fixes [#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix possible race in `StorageMemory`. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix missing or excessive headers in `TSV/CSVWithNames` formats in HTTP protocol. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). -* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix access to `redis` dictionary after connection was dropped once. It may happen with `cache` and `direct` dictionary layouts. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). -* Removed wrong auth access check when using ClickHouseDictionarySource to query remote tables. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). -* Properly distinguish subqueries in some cases for common subexpression elimination. [#8333](https://github.com/ClickHouse/ClickHouse/issues/8333). [#8367](https://github.com/ClickHouse/ClickHouse/pull/8367) ([Amos Bird](https://github.com/amosbird)). - -#### Improvement - -* Disallows `CODEC` on `ALIAS` column type. Fixes [#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). -* When waiting for a dictionary update to complete, use the timeout specified by `query_wait_timeout_milliseconds` setting instead of a hard-coded value. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Add setting `min_index_granularity_bytes` that protects against accidentally creating a table with very low `index_granularity_bytes` setting. [#14139](https://github.com/ClickHouse/ClickHouse/pull/14139) ([Bharat Nallan](https://github.com/bharatnc)). -* Now it's possible to fetch partitions from clusters that use different ZooKeeper: `ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'zk-name:/path-in-zookeeper'`. It's useful for shipping data to new clusters. [#14155](https://github.com/ClickHouse/ClickHouse/pull/14155) ([Amos Bird](https://github.com/amosbird)). -* Slightly better performance of Memory table if it was constructed from a huge number of very small blocks (that's unlikely). Author of the idea: [Mark Papadakis](https://github.com/markpapadakis). Closes [#14043](https://github.com/ClickHouse/ClickHouse/issues/14043). [#14056](https://github.com/ClickHouse/ClickHouse/pull/14056) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Conditional aggregate functions (for example: `avgIf`, `sumIf`, `maxIf`) should return `NULL` when miss rows and use nullable arguments. [#13964](https://github.com/ClickHouse/ClickHouse/pull/13964) ([Winter Zhang](https://github.com/zhang2014)). -* Increase limit in -Resample combinator to 1M. [#13947](https://github.com/ClickHouse/ClickHouse/pull/13947) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Corrected an error in AvroConfluent format that caused the Kafka table engine to stop processing messages when an abnormally small, malformed, message was received. [#13941](https://github.com/ClickHouse/ClickHouse/pull/13941) ([Gervasio Varela](https://github.com/gervarela)). -* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Better error message for null value of `TabSeparated` format. [#13906](https://github.com/ClickHouse/ClickHouse/pull/13906) ([jiang tao](https://github.com/tomjiang1987)). -* Function `arrayCompact` will compare NaNs bitwise if the type of array elements is Float32/Float64. In previous versions NaNs were always not equal if the type of array elements is Float32/Float64 and were always equal if the type is more complex, like Nullable(Float64). This closes [#13857](https://github.com/ClickHouse/ClickHouse/issues/13857). [#13868](https://github.com/ClickHouse/ClickHouse/pull/13868) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix data race in `lgamma` function. This race was caught only in `tsan`, no side effects a really happened. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Avoid too slow queries when arrays are manipulated as fields. Throw exception instead. [#13753](https://github.com/ClickHouse/ClickHouse/pull/13753) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added Redis requirepass authorization (for redis dictionary source). [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). -* Add MergeTree Write-Ahead-Log (WAL) dump tool. WAL is an experimental feature. [#13640](https://github.com/ClickHouse/ClickHouse/pull/13640) ([BohuTANG](https://github.com/BohuTANG)). -* In previous versions `lcm` function may produce assertion violation in debug build if called with specifically crafted arguments. This fixes [#13368](https://github.com/ClickHouse/ClickHouse/issues/13368). [#13510](https://github.com/ClickHouse/ClickHouse/pull/13510) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Provide monotonicity for `toDate/toDateTime` functions in more cases. Monotonicity information is used for index analysis (more complex queries will be able to use index). Now the input arguments are saturated more naturally and provides better monotonicity. [#13497](https://github.com/ClickHouse/ClickHouse/pull/13497) ([Amos Bird](https://github.com/amosbird)). -* Support compound identifiers for custom settings. Custom settings is an integration point of ClickHouse codebase with other codebases (no benefits for ClickHouse itself) [#13496](https://github.com/ClickHouse/ClickHouse/pull/13496) ([Vitaly Baranov](https://github.com/vitlibar)). -* Move parts from DiskLocal to DiskS3 in parallel. `DiskS3` is an experimental feature. [#13459](https://github.com/ClickHouse/ClickHouse/pull/13459) ([Pavel Kovalenko](https://github.com/Jokser)). -* Enable mixed granularity parts by default. [#13449](https://github.com/ClickHouse/ClickHouse/pull/13449) ([alesapin](https://github.com/alesapin)). -* Proper remote host checking in S3 redirects (security-related thing). [#13404](https://github.com/ClickHouse/ClickHouse/pull/13404) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Add `QueryTimeMicroseconds`, `SelectQueryTimeMicroseconds` and `InsertQueryTimeMicroseconds` to system.events. [#13336](https://github.com/ClickHouse/ClickHouse/pull/13336) ([ianton-ru](https://github.com/ianton-ru)). -* Fix debug assertion when Decimal has too large negative exponent. Fixes [#13188](https://github.com/ClickHouse/ClickHouse/issues/13188). [#13228](https://github.com/ClickHouse/ClickHouse/pull/13228) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added cache layer for DiskS3 (cache to local disk mark and index files). `DiskS3` is an experimental feature. [#13076](https://github.com/ClickHouse/ClickHouse/pull/13076) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fix readline so it dumps history to file now. [#13600](https://github.com/ClickHouse/ClickHouse/pull/13600) ([Amos Bird](https://github.com/amosbird)). -* Create `system` database with `Atomic` engine by default (a preparation to enable `Atomic` database engine by default everywhere). [#13680](https://github.com/ClickHouse/ClickHouse/pull/13680) ([tavplubix](https://github.com/tavplubix)). - -#### Performance Improvement - -* Slightly optimize very short queries with `LowCardinality`. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). -* Enable parallel INSERTs for table engines `Null`, `Memory`, `Distributed` and `Buffer` when the setting `max_insert_threads` is set. [#14120](https://github.com/ClickHouse/ClickHouse/pull/14120) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fail fast if `max_rows_to_read` limit is exceeded on parts scan. The motivation behind this change is to skip ranges scan for all selected parts if it is clear that `max_rows_to_read` is already exceeded. The change is quite noticeable for queries over big number of parts. [#13677](https://github.com/ClickHouse/ClickHouse/pull/13677) ([Roman Khavronenko](https://github.com/hagen1778)). -* Slightly improve performance of aggregation by UInt8/UInt16 keys. [#13099](https://github.com/ClickHouse/ClickHouse/pull/13099) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Optimize `has()`, `indexOf()` and `countEqual()` functions for `Array(LowCardinality(T))` and constant right arguments. [#12550](https://github.com/ClickHouse/ClickHouse/pull/12550) ([myrrc](https://github.com/myrrc)). -* When performing trivial `INSERT SELECT` queries, automatically set `max_threads` to 1 or `max_insert_threads`, and set `max_block_size` to `min_insert_block_size_rows`. Related to [#5907](https://github.com/ClickHouse/ClickHouse/issues/5907). [#12195](https://github.com/ClickHouse/ClickHouse/pull/12195) ([flynn](https://github.com/ucasFL)). - -#### Experimental Feature - -* ClickHouse can work as MySQL replica - it is implemented by `MaterializeMySQL` database engine. Implements [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#10851](https://github.com/ClickHouse/ClickHouse/pull/10851) ([Winter Zhang](https://github.com/zhang2014)). -* Add types `Int128`, `Int256`, `UInt256` and related functions for them. Extend Decimals with Decimal256 (precision up to 76 digits). New types are under the setting `allow_experimental_bigint_types`. It is working extremely slow and bad. The implementation is incomplete. Please don't use this feature. [#13097](https://github.com/ClickHouse/ClickHouse/pull/13097) ([Artem Zuikov](https://github.com/4ertus2)). - -#### Build/Testing/Packaging Improvement - -* Added `clickhouse install` script, that is useful if you only have a single binary. [#13528](https://github.com/ClickHouse/ClickHouse/pull/13528) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow to run `clickhouse` binary without configuration. [#13515](https://github.com/ClickHouse/ClickHouse/pull/13515) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Enable check for typos in code with `codespell`. [#13513](https://github.com/ClickHouse/ClickHouse/pull/13513) [#13511](https://github.com/ClickHouse/ClickHouse/pull/13511) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Enable Shellcheck in CI as a linter of .sh tests. This closes [#13168](https://github.com/ClickHouse/ClickHouse/issues/13168). [#13530](https://github.com/ClickHouse/ClickHouse/pull/13530) [#13529](https://github.com/ClickHouse/ClickHouse/pull/13529) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add a CMake option to fail configuration instead of auto-reconfiguration, enabled by default. [#13687](https://github.com/ClickHouse/ClickHouse/pull/13687) ([Konstantin](https://github.com/podshumok)). -* Expose version of embedded tzdata via TZDATA_VERSION in system.build_options. [#13648](https://github.com/ClickHouse/ClickHouse/pull/13648) ([filimonov](https://github.com/filimonov)). -* Improve generation of system.time_zones table during build. Closes [#14209](https://github.com/ClickHouse/ClickHouse/issues/14209). [#14215](https://github.com/ClickHouse/ClickHouse/pull/14215) ([filimonov](https://github.com/filimonov)). -* Build ClickHouse with the most fresh tzdata from package repository. [#13623](https://github.com/ClickHouse/ClickHouse/pull/13623) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add the ability to write js-style comments in skip_list.json. [#14159](https://github.com/ClickHouse/ClickHouse/pull/14159) ([alesapin](https://github.com/alesapin)). -* Ensure that there is no copy-pasted GPL code. [#13514](https://github.com/ClickHouse/ClickHouse/pull/13514) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Switch tests docker images to use test-base parent. [#14167](https://github.com/ClickHouse/ClickHouse/pull/14167) ([Ilya Yatsishin](https://github.com/qoega)). -* Adding retry logic when bringing up docker-compose cluster; Increasing COMPOSE_HTTP_TIMEOUT. [#14112](https://github.com/ClickHouse/ClickHouse/pull/14112) ([vzakaznikov](https://github.com/vzakaznikov)). -* Enabled `system.text_log` in stress test to find more bugs. [#13855](https://github.com/ClickHouse/ClickHouse/pull/13855) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Testflows LDAP module: adding missing certificates and dhparam.pem for openldap4. [#13780](https://github.com/ClickHouse/ClickHouse/pull/13780) ([vzakaznikov](https://github.com/vzakaznikov)). -* ZooKeeper cannot work reliably in unit tests in CI infrastructure. Using unit tests for ZooKeeper interaction with real ZooKeeper is bad idea from the start (unit tests are not supposed to verify complex distributed systems). We already using integration tests for this purpose and they are better suited. [#13745](https://github.com/ClickHouse/ClickHouse/pull/13745) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added docker image for style check. Added style check that all docker and docker compose files are located in docker directory. [#13724](https://github.com/ClickHouse/ClickHouse/pull/13724) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix cassandra build on Mac OS. [#13708](https://github.com/ClickHouse/ClickHouse/pull/13708) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix link error in shared build. [#13700](https://github.com/ClickHouse/ClickHouse/pull/13700) ([Amos Bird](https://github.com/amosbird)). -* Updating LDAP user authentication suite to check that it works with RBAC. [#13656](https://github.com/ClickHouse/ClickHouse/pull/13656) ([vzakaznikov](https://github.com/vzakaznikov)). -* Removed `-DENABLE_CURL_CLIENT` for `contrib/aws`. [#13628](https://github.com/ClickHouse/ClickHouse/pull/13628) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Increasing health-check timeouts for ClickHouse nodes and adding support to dump docker-compose logs if unhealthy containers found. [#13612](https://github.com/ClickHouse/ClickHouse/pull/13612) ([vzakaznikov](https://github.com/vzakaznikov)). -* Make sure [#10977](https://github.com/ClickHouse/ClickHouse/issues/10977) is invalid. [#13539](https://github.com/ClickHouse/ClickHouse/pull/13539) ([Amos Bird](https://github.com/amosbird)). -* Skip PR's from robot-clickhouse. [#13489](https://github.com/ClickHouse/ClickHouse/pull/13489) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Move Dockerfiles from integration tests to `docker/test` directory. docker_compose files are available in `runner` docker container. Docker images are built in CI and not in integration tests. [#13448](https://github.com/ClickHouse/ClickHouse/pull/13448) ([Ilya Yatsishin](https://github.com/qoega)). - - -## ClickHouse release 20.7 - -### ClickHouse release v20.7.2.30-stable, 2020-08-31 - -#### Backward Incompatible Change - -* Function `modulo` (operator `%`) with at least one floating point number as argument will calculate remainder of division directly on floating point numbers without converting both arguments to integers. It makes behaviour compatible with most of DBMS. This also applicable for Date and DateTime data types. Added alias `mod`. This closes [#7323](https://github.com/ClickHouse/ClickHouse/issues/7323). [#12585](https://github.com/ClickHouse/ClickHouse/pull/12585) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Deprecate special printing of zero Date/DateTime values as `0000-00-00` and `0000-00-00 00:00:00`. [#12442](https://github.com/ClickHouse/ClickHouse/pull/12442) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* The function `groupArrayMoving*` was not working for distributed queries. It's result was calculated within incorrect data type (without promotion to the largest type). The function `groupArrayMovingAvg` was returning integer number that was inconsistent with the `avg` function. This fixes [#12568](https://github.com/ClickHouse/ClickHouse/issues/12568). [#12622](https://github.com/ClickHouse/ClickHouse/pull/12622) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add sanity check for MergeTree settings. If the settings are incorrect, the server will refuse to start or to create a table, printing detailed explanation to the user. [#13153](https://github.com/ClickHouse/ClickHouse/pull/13153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Protect from the cases when user may set `background_pool_size` to value lower than `number_of_free_entries_in_pool_to_execute_mutation` or `number_of_free_entries_in_pool_to_lower_max_size_of_merge`. In these cases ALTERs won't work or the maximum size of merge will be too limited. It will throw exception explaining what to do. This closes [#10897](https://github.com/ClickHouse/ClickHouse/issues/10897). [#12728](https://github.com/ClickHouse/ClickHouse/pull/12728) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). - -#### New Feature - -* Polygon dictionary type that provides efficient "reverse geocoding" lookups - to find the region by coordinates in a dictionary of many polygons (world map). It is using carefully optimized algorithm with recursive grids to maintain low CPU and memory usage. [#9278](https://github.com/ClickHouse/ClickHouse/pull/9278) ([achulkov2](https://github.com/achulkov2)). -* Added support of LDAP authentication for preconfigured users ("Simple Bind" method). [#11234](https://github.com/ClickHouse/ClickHouse/pull/11234) ([Denis Glazachev](https://github.com/traceon)). -* Introduce setting `alter_partition_verbose_result` which outputs information about touched parts for some types of `ALTER TABLE ... PARTITION ...` queries (currently `ATTACH` and `FREEZE`). Closes [#8076](https://github.com/ClickHouse/ClickHouse/issues/8076). [#13017](https://github.com/ClickHouse/ClickHouse/pull/13017) ([alesapin](https://github.com/alesapin)). -* Add `bayesAB` function for bayesian-ab-testing. [#12327](https://github.com/ClickHouse/ClickHouse/pull/12327) ([achimbab](https://github.com/achimbab)). -* Added `system.crash_log` table into which stack traces for fatal errors are collected. This table should be empty. [#12316](https://github.com/ClickHouse/ClickHouse/pull/12316) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added http headers `X-ClickHouse-Database` and `X-ClickHouse-Format` which may be used to set default database and output format. [#12981](https://github.com/ClickHouse/ClickHouse/pull/12981) ([hcz](https://github.com/hczhcz)). -* Add `minMap` and `maxMap` functions support to `SimpleAggregateFunction`. [#12662](https://github.com/ClickHouse/ClickHouse/pull/12662) ([Ildus Kurbangaliev](https://github.com/ildus)). -* Add setting `allow_non_metadata_alters` which restricts to execute `ALTER` queries which modify data on disk. Disabled be default. Closes [#11547](https://github.com/ClickHouse/ClickHouse/issues/11547). [#12635](https://github.com/ClickHouse/ClickHouse/pull/12635) ([alesapin](https://github.com/alesapin)). -* A function `formatRow` is added to support turning arbitrary expressions into a string via given format. It's useful for manipulating SQL outputs and is quite versatile combined with the `columns` function. [#12574](https://github.com/ClickHouse/ClickHouse/pull/12574) ([Amos Bird](https://github.com/amosbird)). -* Add `FROM_UNIXTIME` function for compatibility with MySQL, related to [12149](https://github.com/ClickHouse/ClickHouse/issues/12149). [#12484](https://github.com/ClickHouse/ClickHouse/pull/12484) ([flynn](https://github.com/ucasFL)). -* Allow Nullable types as keys in MergeTree tables if `allow_nullable_key` table setting is enabled. Closes [#5319](https://github.com/ClickHouse/ClickHouse/issues/5319). [#12433](https://github.com/ClickHouse/ClickHouse/pull/12433) ([Amos Bird](https://github.com/amosbird)). -* Integration with [COS](https://intl.cloud.tencent.com/product/cos). [#12386](https://github.com/ClickHouse/ClickHouse/pull/12386) ([fastio](https://github.com/fastio)). -* Add `mapAdd` and `mapSubtract` functions for adding/subtracting key-mapped values. [#11735](https://github.com/ClickHouse/ClickHouse/pull/11735) ([Ildus Kurbangaliev](https://github.com/ildus)). - -#### Bug Fix - -* Fix premature `ON CLUSTER` timeouts for queries that must be executed on a single replica. Fixes [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). -* Fix crash in mark inclusion search introduced in [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277). [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). -* Fix race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* Fix visible data clobbering by progress bar in client in interactive mode. This fixes [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) and [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) and [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) and fixes [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed incorrect sorting order for `LowCardinality` columns when ORDER BY multiple columns is used. This fixes [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Removed hardcoded timeout, which wrongly overruled `query_wait_timeout_milliseconds` setting for cache-dictionary. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed wrong mount point in extra info for `Poco::Exception: no space left on device`. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). -* Fix wrong query optimization of select queries with `DISTINCT` keyword when subqueries also have `DISTINCT` in case `optimize_duplicate_order_by_and_distinct` setting is enabled. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed potential deadlock when renaming `Distributed` table. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). -* Fix incorrect sorting for `FixedString` columns when ORDER BY multiple columns is used. Fixes [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix potentially lower precision of `topK`/`topKWeighted` aggregations (with non-default parameters). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). -* Fix reading from MergeTree table with INDEX of type SET fails when compared against NULL. This fixes [#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). -* Fix step overflow in function `range()`. [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). -* Fixed `Directory not empty` error when concurrently executing `DROP DATABASE` and `CREATE TABLE`. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add range check for `h3KRing` function. This fixes [#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix race condition between DETACH and background merges. Parts may revive after detach. This is continuation of [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) that did not fix the issue but introduced a test that started to fail in very rare cases, demonstrating the issue. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix logging Settings.Names/Values when `log_queries_min_type` greater than `QUERY_START`. [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). -* Fix incorrect message in `clickhouse-server.init` while checking user and group. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). -* Do not optimize `any(arrayJoin())` to `arrayJoin()` under `optimize_move_functions_out_of_any`. [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). -* Fixed possible deadlock in concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). -* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix secondary indices corruption in compact parts (compact parts is an experimental feature). [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). -* Fix wrong code in function `netloc`. This fixes [#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix invalid return type for comparison of tuples with `NULL` elements. Fixes [#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix wrong optimization caused `aggregate function any(x) is found inside another aggregate function in query` error with `SET optimize_move_functions_out_of_any = 1` and aliases inside `any()`. [#13419](https://github.com/ClickHouse/ClickHouse/pull/13419) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix possible race in `StorageMemory`. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix empty output for `Arrow` and `Parquet` formats in case if query return zero rows. It was done because empty output is not valid for this formats. [#13399](https://github.com/ClickHouse/ClickHouse/pull/13399) ([hcz](https://github.com/hczhcz)). -* Fix select queries with constant columns and prefix of primary key in `ORDER BY` clause. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). -* Fix `PrettyCompactMonoBlock` for clickhouse-local. Fix extremes/totals with `PrettyCompactMonoBlock`. Fixes [#7746](https://github.com/ClickHouse/ClickHouse/issues/7746). [#13394](https://github.com/ClickHouse/ClickHouse/pull/13394) ([Azat Khuzhin](https://github.com/azat)). -* Fixed deadlock in system.text_log. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). It is a part of [#12339](https://github.com/ClickHouse/ClickHouse/issues/12339). This fixes [#12325](https://github.com/ClickHouse/ClickHouse/issues/12325). [#13386](https://github.com/ClickHouse/ClickHouse/pull/13386) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed `File(TSVWithNames*)` (header was written multiple times), fixed `clickhouse-local --format CSVWithNames*` (lacks header, broken after [#12197](https://github.com/ClickHouse/ClickHouse/issues/12197)), fixed `clickhouse-local --format CSVWithNames*` with zero rows (lacks header). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). -* Fix segfault when function `groupArrayMovingSum` deserializes empty state. Fixes [#13339](https://github.com/ClickHouse/ClickHouse/issues/13339). [#13341](https://github.com/ClickHouse/ClickHouse/pull/13341) ([alesapin](https://github.com/alesapin)). -* Throw error on `arrayJoin()` function in `JOIN ON` section. [#13330](https://github.com/ClickHouse/ClickHouse/pull/13330) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix crash in `LEFT ASOF JOIN` with `join_use_nulls=1`. [#13291](https://github.com/ClickHouse/ClickHouse/pull/13291) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix possible error `Totals having transform was already added to pipeline` in case of a query from delayed replica. [#13290](https://github.com/ClickHouse/ClickHouse/pull/13290) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* The server may crash if user passed specifically crafted arguments to the function `h3ToChildren`. This fixes [#13275](https://github.com/ClickHouse/ClickHouse/issues/13275). [#13277](https://github.com/ClickHouse/ClickHouse/pull/13277) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with `NaN` values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix assertion in KeyCondition when primary key contains expression with monotonic function and query contains comparison with constant whose type is different. This fixes [#12465](https://github.com/ClickHouse/ClickHouse/issues/12465). [#13251](https://github.com/ClickHouse/ClickHouse/pull/13251) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Return passed number for numbers with MSB set in function roundUpToPowerOfTwoOrZero(). It prevents potential errors in case of overflow of array sizes. [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). -* Fix function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix DateTime64 conversion functions with constant argument. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)). -* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix access to `redis` dictionary after connection was dropped once. It may happen with `cache` and `direct` dictionary layouts. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). -* Fix wrong index analysis with functions. It could lead to some data parts being skipped when reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* Fix error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix crash which was possible for queries with `ORDER BY` tuple and small `LIMIT`. Fixes [#12623](https://github.com/ClickHouse/ClickHouse/issues/12623). [#13009](https://github.com/ClickHouse/ClickHouse/pull/13009) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix `Block structure mismatch` error for queries with `UNION` and `JOIN`. Fixes [#12602](https://github.com/ClickHouse/ClickHouse/issues/12602). [#12989](https://github.com/ClickHouse/ClickHouse/pull/12989) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Corrected `merge_with_ttl_timeout` logic which did not work well when expiration affected more than one partition over one time interval. (Authored by @excitoon). [#12982](https://github.com/ClickHouse/ClickHouse/pull/12982) ([Alexander Kazakov](https://github.com/Akazz)). -* Fix columns duplication for range hashed dictionary created from DDL query. This fixes [#10605](https://github.com/ClickHouse/ClickHouse/issues/10605). [#12857](https://github.com/ClickHouse/ClickHouse/pull/12857) ([alesapin](https://github.com/alesapin)). -* Fix unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix rare bug when `ALTER DELETE` and `ALTER MODIFY COLUMN` queries executed simultaneously as a single mutation. Bug leads to an incorrect amount of rows in `count.txt` and as a consequence incorrect data in part. Also, fix a small bug with simultaneous `ALTER RENAME COLUMN` and `ALTER ADD COLUMN`. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). -* Wrong credentials being used when using `clickhouse` dictionary source to query remote tables. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). -* Fix `CAST(Nullable(String), Enum())`. [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). -* Fix performance with large tuples, which are interpreted as functions in `IN` section. The case when user writes `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* Fix memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* Fix wrong optimization `optimize_move_functions_out_of_any=1` in case of `any(func())`. [#12664](https://github.com/ClickHouse/ClickHouse/pull/12664) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* Fix SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* Add support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* CREATE USER IF NOT EXISTS now doesn't throw exception if the user exists. This fixes [#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)). -* Exception `There is no supertype...` can be thrown during `ALTER ... UPDATE` in unexpected cases (e.g. when subtracting from UInt64 column). This fixes [#7306](https://github.com/ClickHouse/ClickHouse/issues/7306). This fixes [#4165](https://github.com/ClickHouse/ClickHouse/issues/4165). [#12633](https://github.com/ClickHouse/ClickHouse/pull/12633) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix possible `Pipeline stuck` error for queries with external sorting. Fixes [#12617](https://github.com/ClickHouse/ClickHouse/issues/12617). [#12618](https://github.com/ClickHouse/ClickHouse/pull/12618) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix error `Output of TreeExecutor is not sorted` for `OPTIMIZE DEDUPLICATE`. Fixes [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572). [#12613](https://github.com/ClickHouse/ClickHouse/pull/12613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix the issue when alias on result of function `any` can be lost during query optimization. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). -* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* Now ClickHouse will recalculate checksums for parts when file `checksums.txt` is absent. Broken since [#9827](https://github.com/ClickHouse/ClickHouse/issues/9827). [#12545](https://github.com/ClickHouse/ClickHouse/pull/12545) ([alesapin](https://github.com/alesapin)). -* Fix bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* Fixing race condition in live view tables which could cause data duplication. LIVE VIEW is an experimental feature. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). -* Fix backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash in JOIN with dictionary when we are joining over expression of dictionary key: `t JOIN dict ON expr(dict.id) = t.id`. Disable dictionary join optimisation for this case. [#12458](https://github.com/ClickHouse/ClickHouse/pull/12458) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix overflow when very large LIMIT or OFFSET is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* kafka: fix SIGSEGV if there is a message with error in the middle of the batch. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). - -#### Improvement - -* Keep smaller amount of logs in ZooKeeper. Avoid excessive growing of ZooKeeper nodes in case of offline replicas when having many servers/tables/inserts. [#13100](https://github.com/ClickHouse/ClickHouse/pull/13100) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now exceptions forwarded to the client if an error happened during ALTER or mutation. Closes [#11329](https://github.com/ClickHouse/ClickHouse/issues/11329). [#12666](https://github.com/ClickHouse/ClickHouse/pull/12666) ([alesapin](https://github.com/alesapin)). -* Add `QueryTimeMicroseconds`, `SelectQueryTimeMicroseconds` and `InsertQueryTimeMicroseconds` to `system.events`, along with system.metrics, processes, query_log, etc. [#13028](https://github.com/ClickHouse/ClickHouse/pull/13028) ([ianton-ru](https://github.com/ianton-ru)). -* Added `SelectedRows` and `SelectedBytes` to `system.events`, along with system.metrics, processes, query_log, etc. [#12638](https://github.com/ClickHouse/ClickHouse/pull/12638) ([ianton-ru](https://github.com/ianton-ru)). -* Added `current_database` information to `system.query_log`. [#12652](https://github.com/ClickHouse/ClickHouse/pull/12652) ([Amos Bird](https://github.com/amosbird)). -* Allow `TabSeparatedRaw` as input format. [#12009](https://github.com/ClickHouse/ClickHouse/pull/12009) ([hcz](https://github.com/hczhcz)). -* Now `joinGet` supports multi-key lookup. [#12418](https://github.com/ClickHouse/ClickHouse/pull/12418) ([Amos Bird](https://github.com/amosbird)). -* Allow `*Map` aggregate functions to work on Arrays with NULLs. Fixes [#13157](https://github.com/ClickHouse/ClickHouse/issues/13157). [#13225](https://github.com/ClickHouse/ClickHouse/pull/13225) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid overflow in parsing of DateTime values that will lead to negative unix timestamp in their timezone (for example, `1970-01-01 00:00:00` in Moscow). Saturate to zero instead. This fixes [#3470](https://github.com/ClickHouse/ClickHouse/issues/3470). This fixes [#4172](https://github.com/ClickHouse/ClickHouse/issues/4172). [#12443](https://github.com/ClickHouse/ClickHouse/pull/12443) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* AvroConfluent: Skip Kafka tombstone records - Support skipping broken records [#13203](https://github.com/ClickHouse/ClickHouse/pull/13203) ([Andrew Onyshchuk](https://github.com/oandrew)). -* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix data race in `lgamma` function. This race was caught only in `tsan`, no side effects really happened. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix a 'Week'-interval formatting for ATTACH/ALTER/CREATE QUOTA-statements. [#13417](https://github.com/ClickHouse/ClickHouse/pull/13417) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). -* Now broken parts are also reported when encountered in compact part processing. Compact parts is an experimental feature. [#13282](https://github.com/ClickHouse/ClickHouse/pull/13282) ([Amos Bird](https://github.com/amosbird)). -* Fix assert in `geohashesInBox`. This fixes [#12554](https://github.com/ClickHouse/ClickHouse/issues/12554). [#13229](https://github.com/ClickHouse/ClickHouse/pull/13229) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix assert in `parseDateTimeBestEffort`. This fixes [#12649](https://github.com/ClickHouse/ClickHouse/issues/12649). [#13227](https://github.com/ClickHouse/ClickHouse/pull/13227) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Minor optimization in Processors/PipelineExecutor: breaking out of a loop because it makes sense to do so. [#13058](https://github.com/ClickHouse/ClickHouse/pull/13058) ([Mark Papadakis](https://github.com/markpapadakis)). -* Support TRUNCATE table without TABLE keyword. [#12653](https://github.com/ClickHouse/ClickHouse/pull/12653) ([Winter Zhang](https://github.com/zhang2014)). -* Fix explain query format overwrite by default. This fixes [#12541](https://github.com/ClickHouse/ClickHouse/issues/12432). [#12541](https://github.com/ClickHouse/ClickHouse/pull/12541) ([BohuTANG](https://github.com/BohuTANG)). -* Allow to set JOIN kind and type in more standad way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). -* Changes default value for `multiple_joins_rewriter_version` to 2. It enables new multiple joins rewriter that knows about column names. [#12469](https://github.com/ClickHouse/ClickHouse/pull/12469) ([Artem Zuikov](https://github.com/4ertus2)). -* Add several metrics for requests to S3 storages. [#12464](https://github.com/ClickHouse/ClickHouse/pull/12464) ([ianton-ru](https://github.com/ianton-ru)). -* Use correct default secure port for clickhouse-benchmark with `--secure` argument. This fixes [#11044](https://github.com/ClickHouse/ClickHouse/issues/11044). [#12440](https://github.com/ClickHouse/ClickHouse/pull/12440) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Rollback insertion errors in `Log`, `TinyLog`, `StripeLog` engines. In previous versions insertion error lead to inconsisent table state (this works as documented and it is normal for these table engines). This fixes [#12402](https://github.com/ClickHouse/ClickHouse/issues/12402). [#12426](https://github.com/ClickHouse/ClickHouse/pull/12426) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Implement `RENAME DATABASE` and `RENAME DICTIONARY` for `Atomic` database engine - Add implicit `{uuid}` macro, which can be used in ZooKeeper path for `ReplicatedMergeTree`. It works with `CREATE ... ON CLUSTER ...` queries. Set `show_table_uuid_in_table_create_query_if_not_nil` to `true` to use it. - Make `ReplicatedMergeTree` engine arguments optional, `/clickhouse/tables/{uuid}/{shard}/` and `{replica}` are used by default. Closes [#12135](https://github.com/ClickHouse/ClickHouse/issues/12135). - Minor fixes. - These changes break backward compatibility of `Atomic` database engine. Previously created `Atomic` databases must be manually converted to new format. Atomic database is an experimental feature. [#12343](https://github.com/ClickHouse/ClickHouse/pull/12343) ([tavplubix](https://github.com/tavplubix)). -* Separated `AWSAuthV4Signer` into different logger, removed excessive `AWSClient: AWSClient` from log messages. [#12320](https://github.com/ClickHouse/ClickHouse/pull/12320) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Better exception message in disk access storage. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). -* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* Fix error message about adaptive granularity. [#12624](https://github.com/ClickHouse/ClickHouse/pull/12624) ([alesapin](https://github.com/alesapin)). -* Fix SETTINGS parse after FORMAT. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). -* If MergeTree table does not contain ORDER BY or PARTITION BY, it was possible to request ALTER to CLEAR all the columns and ALTER will stuck. Fixed [#7941](https://github.com/ClickHouse/ClickHouse/issues/7941). [#12382](https://github.com/ClickHouse/ClickHouse/pull/12382) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid re-loading completion from the history file after each query (to avoid history overlaps with other client sessions). [#13086](https://github.com/ClickHouse/ClickHouse/pull/13086) ([Azat Khuzhin](https://github.com/azat)). - -#### Performance Improvement - -* Lower memory usage for some operations up to 2 times. [#12424](https://github.com/ClickHouse/ClickHouse/pull/12424) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Optimize PK lookup for queries that match exact PK range. [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277) ([Ivan Babrou](https://github.com/bobrik)). -* Slightly optimize very short queries with `LowCardinality`. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). -* Slightly improve performance of aggregation by UInt8/UInt16 keys. [#13091](https://github.com/ClickHouse/ClickHouse/pull/13091) and [#13055](https://github.com/ClickHouse/ClickHouse/pull/13055) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Push down `LIMIT` step for query plan (inside subqueries). [#13016](https://github.com/ClickHouse/ClickHouse/pull/13016) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Parallel primary key lookup and skipping index stages on parts, as described in [#11564](https://github.com/ClickHouse/ClickHouse/issues/11564). [#12589](https://github.com/ClickHouse/ClickHouse/pull/12589) ([Ivan Babrou](https://github.com/bobrik)). -* Converting String-type arguments of function "if" and "transform" into enum if `set optimize_if_transform_strings_to_enum = 1`. [#12515](https://github.com/ClickHouse/ClickHouse/pull/12515) ([Artem Zuikov](https://github.com/4ertus2)). -* Replaces monotonic functions with its argument in `ORDER BY` if `set optimize_monotonous_functions_in_order_by=1`. [#12467](https://github.com/ClickHouse/ClickHouse/pull/12467) ([Artem Zuikov](https://github.com/4ertus2)). -* Add order by optimization that rewrites `ORDER BY x, f(x)` with `ORDER by x` if `set optimize_redundant_functions_in_order_by = 1`. [#12404](https://github.com/ClickHouse/ClickHouse/pull/12404) ([Artem Zuikov](https://github.com/4ertus2)). -* Allow pushdown predicate when subquery contains `WITH` clause. This fixes [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). -* Improve performance of reading from compact parts. Compact parts is an experimental feature. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* Attempt to implement streaming optimization in `DiskS3`. DiskS3 is an experimental feature. [#12434](https://github.com/ClickHouse/ClickHouse/pull/12434) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### Build/Testing/Packaging Improvement - -* Use `shellcheck` for sh tests linting. [#13200](https://github.com/ClickHouse/ClickHouse/pull/13200) [#13207](https://github.com/ClickHouse/ClickHouse/pull/13207) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add script which set labels for pull requests in GitHub hook. [#13183](https://github.com/ClickHouse/ClickHouse/pull/13183) ([alesapin](https://github.com/alesapin)). -* Remove some of recursive submodules. See [#13378](https://github.com/ClickHouse/ClickHouse/issues/13378). [#13379](https://github.com/ClickHouse/ClickHouse/pull/13379) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Ensure that all the submodules are from proper URLs. Continuation of [#13379](https://github.com/ClickHouse/ClickHouse/issues/13379). This fixes [#13378](https://github.com/ClickHouse/ClickHouse/issues/13378). [#13397](https://github.com/ClickHouse/ClickHouse/pull/13397) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added support for user-declared settings, which can be accessed from inside queries. This is needed when ClickHouse engine is used as a component of another system. [#13013](https://github.com/ClickHouse/ClickHouse/pull/13013) ([Vitaly Baranov](https://github.com/vitlibar)). -* Added testing for RBAC functionality of INSERT privilege in TestFlows. Expanded tables on which SELECT is being tested. Added Requirements to match new table engine tests. [#13340](https://github.com/ClickHouse/ClickHouse/pull/13340) ([MyroTk](https://github.com/MyroTk)). -* Fix timeout error during server restart in the stress test. [#13321](https://github.com/ClickHouse/ClickHouse/pull/13321) ([alesapin](https://github.com/alesapin)). -* Now fast test will wait server with retries. [#13284](https://github.com/ClickHouse/ClickHouse/pull/13284) ([alesapin](https://github.com/alesapin)). -* Function `materialize()` (the function for ClickHouse testing) will work for NULL as expected - by transforming it to non-constant column. [#13212](https://github.com/ClickHouse/ClickHouse/pull/13212) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix libunwind build in AArch64. This fixes [#13204](https://github.com/ClickHouse/ClickHouse/issues/13204). [#13208](https://github.com/ClickHouse/ClickHouse/pull/13208) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Even more retries in zkutil gtest to prevent test flakiness. [#13165](https://github.com/ClickHouse/ClickHouse/pull/13165) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Small fixes to the RBAC TestFlows. [#13152](https://github.com/ClickHouse/ClickHouse/pull/13152) ([vzakaznikov](https://github.com/vzakaznikov)). -* Fixing `00960_live_view_watch_events_live.py` test. [#13108](https://github.com/ClickHouse/ClickHouse/pull/13108) ([vzakaznikov](https://github.com/vzakaznikov)). -* Improve cache purge in documentation deploy script. [#13107](https://github.com/ClickHouse/ClickHouse/pull/13107) ([alesapin](https://github.com/alesapin)). -* Rewrote some orphan tests to gtest. Removed useless includes from tests. [#13073](https://github.com/ClickHouse/ClickHouse/pull/13073) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Added tests for RBAC functionality of `SELECT` privilege in TestFlows. [#13061](https://github.com/ClickHouse/ClickHouse/pull/13061) ([Ritaank Tiwari](https://github.com/ritaank)). -* Rerun some tests in fast test check. [#12992](https://github.com/ClickHouse/ClickHouse/pull/12992) ([alesapin](https://github.com/alesapin)). -* Fix MSan error in "rdkafka" library. This closes [#12990](https://github.com/ClickHouse/ClickHouse/issues/12990). Updated `rdkafka` to version 1.5 (master). [#12991](https://github.com/ClickHouse/ClickHouse/pull/12991) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan report in base64 if tests were run on server with AVX-512. This fixes [#12318](https://github.com/ClickHouse/ClickHouse/issues/12318). Author: @qoega. [#12441](https://github.com/ClickHouse/ClickHouse/pull/12441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan report in HDFS library. This closes [#12330](https://github.com/ClickHouse/ClickHouse/issues/12330). [#12453](https://github.com/ClickHouse/ClickHouse/pull/12453) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Check an ability that we able to restore the backup from an old version to the new version. This closes [#8979](https://github.com/ClickHouse/ClickHouse/issues/8979). [#12959](https://github.com/ClickHouse/ClickHouse/pull/12959) ([alesapin](https://github.com/alesapin)). -* Do not build helper_container image inside integrational tests. Build docker container in CI and use pre-built helper_container in integration tests. [#12953](https://github.com/ClickHouse/ClickHouse/pull/12953) ([Ilya Yatsishin](https://github.com/qoega)). -* Add a test for `ALTER TABLE CLEAR COLUMN` query for primary key columns. [#12951](https://github.com/ClickHouse/ClickHouse/pull/12951) ([alesapin](https://github.com/alesapin)). -* Increased timeouts in testflows tests. [#12949](https://github.com/ClickHouse/ClickHouse/pull/12949) ([vzakaznikov](https://github.com/vzakaznikov)). -* Fix build of test under Mac OS X. This closes [#12767](https://github.com/ClickHouse/ClickHouse/issues/12767). [#12772](https://github.com/ClickHouse/ClickHouse/pull/12772) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Connector-ODBC updated to mysql-connector-odbc-8.0.21. [#12739](https://github.com/ClickHouse/ClickHouse/pull/12739) ([Ilya Yatsishin](https://github.com/qoega)). -* Adding RBAC syntax tests in TestFlows. [#12642](https://github.com/ClickHouse/ClickHouse/pull/12642) ([vzakaznikov](https://github.com/vzakaznikov)). -* Improve performance of TestKeeper. This will speedup tests with heavy usage of Replicated tables. [#12505](https://github.com/ClickHouse/ClickHouse/pull/12505) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now we check that server is able to start after stress tests run. This fixes [#12473](https://github.com/ClickHouse/ClickHouse/issues/12473). [#12496](https://github.com/ClickHouse/ClickHouse/pull/12496) ([alesapin](https://github.com/alesapin)). -* Update fmtlib to master (7.0.1). [#12446](https://github.com/ClickHouse/ClickHouse/pull/12446) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add docker image for fast tests. [#12294](https://github.com/ClickHouse/ClickHouse/pull/12294) ([alesapin](https://github.com/alesapin)). -* Rework configuration paths for integration tests. [#12285](https://github.com/ClickHouse/ClickHouse/pull/12285) ([Ilya Yatsishin](https://github.com/qoega)). -* Add compiler option to control that stack frames are not too large. This will help to run the code in fibers with small stack size. [#11524](https://github.com/ClickHouse/ClickHouse/pull/11524) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Update gitignore-files. [#13447](https://github.com/ClickHouse/ClickHouse/pull/13447) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). - - -## ClickHouse release 20.6 - -### ClickHouse release v20.6.3.28-stable - -#### Backward Incompatible Change - -* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). - -#### New Feature - -* Added an initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Added storage `RabbitMQ`. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Implemented PostgreSQL-like `ILIKE` operator for [#11710](https://github.com/ClickHouse/ClickHouse/issues/11710). [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). -* Supported RIGHT and FULL JOIN with `SET join_algorithm = 'partial_merge'`. Only ALL strictness is allowed (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). -* Added a function `initializeAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). -* Supported `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). -* Added function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). -* Support format `ORC` for output (was supported only for input). [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). - -#### Bug Fix - -* Fixed `aggregate function any(x) is found inside another aggregate function in query` error with `SET optimize_move_functions_out_of_any = 1` and aliases inside `any()`. [#13419](https://github.com/ClickHouse/ClickHouse/pull/13419) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed `PrettyCompactMonoBlock` for clickhouse-local. Fixed extremes/totals with `PrettyCompactMonoBlock`. This fixes [#7746](https://github.com/ClickHouse/ClickHouse/issues/7746). [#13394](https://github.com/ClickHouse/ClickHouse/pull/13394) ([Azat Khuzhin](https://github.com/azat)). -* Fixed possible error `Totals having transform was already added to pipeline` in case of a query from delayed replica. [#13290](https://github.com/ClickHouse/ClickHouse/pull/13290) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* The server may crash if user passed specifically crafted arguments to the function `h3ToChildren`. This fixes [#13275](https://github.com/ClickHouse/ClickHouse/issues/13275). [#13277](https://github.com/ClickHouse/ClickHouse/pull/13277) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with NaN values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `DateTime64` conversion functions with constant argument. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)). -* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed rare bug when `ALTER DELETE` and `ALTER MODIFY COLUMN` queries executed simultaneously as a single mutation. Bug leads to an incorrect amount of rows in `count.txt` and as a consequence incorrect data in part. Also, fix a small bug with simultaneous `ALTER RENAME COLUMN` and `ALTER ADD COLUMN`. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). -* Fixed `CAST(Nullable(String), Enum())`. [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). -* Fixed a performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed memory tracking for `input_format_parallel_parsing` (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* Fixed bloom filter index with const expression. This fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* `CREATE USER IF NOT EXISTS` now doesn't throw exception if the user exists. This fixes [#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)). -* Better exception message in disk access storage. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). -* The function `groupArrayMoving*` was not working for distributed queries. It's result was calculated within incorrect data type (without promotion to the largest type). The function `groupArrayMovingAvg` was returning integer number that was inconsistent with the `avg` function. This fixes [#12568](https://github.com/ClickHouse/ClickHouse/issues/12568). [#12622](https://github.com/ClickHouse/ClickHouse/pull/12622) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* Fixing race condition in live view tables which could cause data duplication. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). -* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed SETTINGS parse after FORMAT. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). -* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed overflow when very large `LIMIT` or `OFFSET` is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed possible segfault if `StorageMerge`. This fixes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* Reverted change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). -* Additional check for arguments of bloom filter index. This fixes [#11408](https://github.com/ClickHouse/ClickHouse/issues/11408). [#12388](https://github.com/ClickHouse/ClickHouse/pull/12388) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid exception when negative or floating point constant is used in WHERE condition for indexed tables. This fixes [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allowed to `CLEAR` column even if there are depending `DEFAULT` expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `TOTALS/ROLLUP/CUBE` for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed error message and exit codes for `ALTER RENAME COLUMN` queries, when `RENAME` is not allowed. Fixes [#12301](https://github.com/ClickHouse/ClickHouse/issues/12301) and [#12303](https://github.com/ClickHouse/ClickHouse/issues/12303). [#12335](https://github.com/ClickHouse/ClickHouse/pull/12335) ([alesapin](https://github.com/alesapin)). -* Fixed very rare race condition in `ReplicatedMergeTreeQueue`. [#12315](https://github.com/ClickHouse/ClickHouse/pull/12315) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* When using codec `Delta` or `DoubleDelta` with non fixed width types, exception with code `LOGICAL_ERROR` was returned instead of exception with code `BAD_ARGUMENTS` (we ensure that exceptions with code logical error never happen). This fixes [#12110](https://github.com/ClickHouse/ClickHouse/issues/12110). [#12308](https://github.com/ClickHouse/ClickHouse/pull/12308) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). -* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `TTL` after renaming column, on which depends TTL expression. [#12304](https://github.com/ClickHouse/ClickHouse/pull/12304) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed SIGSEGV if there is an message with error in the middle of the batch in `Kafka` Engine. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). -* Fixed the situation when some threads might randomly hang for a few seconds during `DNS` cache updating. [#12296](https://github.com/ClickHouse/ClickHouse/pull/12296) ([tavplubix](https://github.com/tavplubix)). -* Fixed typo in setting name. [#12292](https://github.com/ClickHouse/ClickHouse/pull/12292) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Show error after `TrieDictionary` failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). -* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Implement conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the behaviour on reaching redirect limit in request to `S3` storage. [#12256](https://github.com/ClickHouse/ClickHouse/pull/12256) ([ianton-ru](https://github.com/ianton-ru)). -* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). -* Cap max_memory_usage* limits to the process resident memory. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). -* Fix dictGet arguments check during `GROUP BY` injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). -* Fixed the behaviour when `SummingMergeTree` engine sums up columns from partition key. Added an exception in case of explicit definition of columns to sum which intersects with partition key columns. This fixes [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Don't split the dictionary source's table name into schema and table name itself if ODBC connection doesn't support schema. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed bad code in redundant ORDER BY optimization. The bug was introduced in [#10067](https://github.com/ClickHouse/ClickHouse/issues/10067). [#12148](https://github.com/ClickHouse/ClickHouse/pull/12148) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Normalize "pid" file handling. In previous versions the server may refuse to start if it was killed without proper shutdown and if there is another process that has the same pid as previously runned server. Also pid file may be removed in unsuccessful server startup even if there is another server running. This fixes [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). -* Avoid "There is no query" exception for materialized views with joins or with subqueries attached to system logs (system.query_log, metric_log, etc) or to engine=Buffer underlying table. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). -* Fixed handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). -* Format `Parquet` now properly works with `LowCardinality` and `LowCardinality(Nullable)` types. Fixes [#12086](https://github.com/ClickHouse/ClickHouse/issues/12086), [#8406](https://github.com/ClickHouse/ClickHouse/issues/8406). [#12108](https://github.com/ClickHouse/ClickHouse/pull/12108) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed SIGSEGV in StorageKafka on DROP TABLE. [#12075](https://github.com/ClickHouse/ClickHouse/pull/12075) ([Azat Khuzhin](https://github.com/azat)). -* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### Improvement - -* Allowed to set `JOIN` kind and type in more standard way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). -* lifetime_rows/lifetime_bytes for Buffer engine. [#12421](https://github.com/ClickHouse/ClickHouse/pull/12421) ([Azat Khuzhin](https://github.com/azat)). -* Write the detail exception message to the client instead of 'MySQL server has gone away'. [#12383](https://github.com/ClickHouse/ClickHouse/pull/12383) ([BohuTANG](https://github.com/BohuTANG)). -* Allows to change a charset which is used for printing grids borders. Available charsets are following: UTF-8, ASCII. Setting `output_format_pretty_grid_charset` enables this feature. [#12372](https://github.com/ClickHouse/ClickHouse/pull/12372) ([Sabyanin Maxim](https://github.com/s-mx)). -* Supported MySQL 'SELECT DATABASE()' [#9336](https://github.com/ClickHouse/ClickHouse/issues/9336) 2. Add MySQL replacement query integration test. [#12314](https://github.com/ClickHouse/ClickHouse/pull/12314) ([BohuTANG](https://github.com/BohuTANG)). -* Added `KILL QUERY [connection_id]` for the MySQL client/driver to cancel the long query, issue [#12038](https://github.com/ClickHouse/ClickHouse/issues/12038). [#12152](https://github.com/ClickHouse/ClickHouse/pull/12152) ([BohuTANG](https://github.com/BohuTANG)). -* Added support for `%g` (two digit ISO year) and `%G` (four digit ISO year) substitutions in `formatDateTime` function. [#12136](https://github.com/ClickHouse/ClickHouse/pull/12136) ([vivarum](https://github.com/vivarum)). -* Added 'type' column in system.disks. [#12115](https://github.com/ClickHouse/ClickHouse/pull/12115) ([ianton-ru](https://github.com/ianton-ru)). -* Improved `REVOKE` command: now it requires grant/admin option for only access which will be revoked. For example, to execute `REVOKE ALL ON *.* FROM user1` now it doesn't require to have full access rights granted with grant option. Added command `REVOKE ALL FROM user1` - it revokes all granted roles from `user1`. [#12083](https://github.com/ClickHouse/ClickHouse/pull/12083) ([Vitaly Baranov](https://github.com/vitlibar)). -* Added replica priority for load_balancing (for manual prioritization of the load balancing). [#11995](https://github.com/ClickHouse/ClickHouse/pull/11995) ([Azat Khuzhin](https://github.com/azat)). -* Switched paths in S3 metadata to relative which allows to handle S3 blobs more easily. [#11892](https://github.com/ClickHouse/ClickHouse/pull/11892) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### Performance Improvement - -* Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key (enabled with `optimize_aggregation_in_order` setting, disabled by default). [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). -* Removed injective functions inside `uniq*()` if `set optimize_injective_functions_inside_uniq=1`. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Ruslan Kamalov](https://github.com/kamalov-ruslan)). -* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). -* Implemented single part uploads for DiskS3 (experimental feature). [#12026](https://github.com/ClickHouse/ClickHouse/pull/12026) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### Experimental Feature -* Added new in-memory format of parts in `MergeTree`-family tables, which stores data in memory. Parts are written on disk at first merge. Part will be created in in-memory format if its size in rows or bytes is below thresholds `min_rows_for_compact_part` and `min_bytes_for_compact_part`. Also optional support of Write-Ahead-Log is available, which is enabled by default and is controlled by setting `in_memory_parts_enable_wal`. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). - -#### Build/Testing/Packaging Improvement - -* Implement AST-based query fuzzing mode for clickhouse-client. See [this label](https://github.com/ClickHouse/ClickHouse/issues?q=label%3Afuzz+is%3Aissue) for the list of issues we recently found by fuzzing. Most of them were found by this tool, and a couple by SQLancer and `00746_sql_fuzzy.pl`. [#12111](https://github.com/ClickHouse/ClickHouse/pull/12111) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Add new type of tests based on Testflows framework. [#12090](https://github.com/ClickHouse/ClickHouse/pull/12090) ([vzakaznikov](https://github.com/vzakaznikov)). -* Added S3 HTTPS integration test. [#12412](https://github.com/ClickHouse/ClickHouse/pull/12412) ([Pavel Kovalenko](https://github.com/Jokser)). -* Log sanitizer trap messages from separate thread. This will prevent possible deadlock under thread sanitizer. [#12313](https://github.com/ClickHouse/ClickHouse/pull/12313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now functional and stress tests will be able to run with old version of `clickhouse-test` script. [#12287](https://github.com/ClickHouse/ClickHouse/pull/12287) ([alesapin](https://github.com/alesapin)). -* Remove strange file creation during build in `orc`. [#12258](https://github.com/ClickHouse/ClickHouse/pull/12258) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Place common docker compose files to integration docker container. [#12168](https://github.com/ClickHouse/ClickHouse/pull/12168) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix warnings from CodeQL. `CodeQL` is another static analyzer that we will use along with `clang-tidy` and `PVS-Studio` that we use already. [#12138](https://github.com/ClickHouse/ClickHouse/pull/12138) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Minor CMake fixes for UNBUNDLED build. [#12131](https://github.com/ClickHouse/ClickHouse/pull/12131) ([Matwey V. Kornilov](https://github.com/matwey)). -* Added a showcase of the minimal Docker image without using any Linux distribution. [#12126](https://github.com/ClickHouse/ClickHouse/pull/12126) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Perform an upgrade of system packages in the `clickhouse-server` docker image. [#12124](https://github.com/ClickHouse/ClickHouse/pull/12124) ([Ivan Blinkov](https://github.com/blinkov)). -* Add `UNBUNDLED` flag to `system.build_options` table. Move skip lists for `clickhouse-test` to clickhouse repo. [#12107](https://github.com/ClickHouse/ClickHouse/pull/12107) ([alesapin](https://github.com/alesapin)). -* Regular check by [Anchore Container Analysis](https://docs.anchore.com) security analysis tool that looks for [CVE](https://cve.mitre.org/) in `clickhouse-server` Docker image. Also confirms that `Dockerfile` is buildable. Runs daily on `master` and on pull-requests to `Dockerfile`. [#12102](https://github.com/ClickHouse/ClickHouse/pull/12102) ([Ivan Blinkov](https://github.com/blinkov)). -* Daily check by [GitHub CodeQL](https://securitylab.github.com/tools/codeql) security analysis tool that looks for [CWE](https://cwe.mitre.org/). [#12101](https://github.com/ClickHouse/ClickHouse/pull/12101) ([Ivan Blinkov](https://github.com/blinkov)). -* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). - -## ClickHouse release 20.5 - -### ClickHouse release v20.5.4.40-stable 2020-08-10 - -#### Bug Fix - -* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* Fixed bloom filter index with const expression. This fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed race condition in live view tables which could cause data duplication. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). -* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed overflow when very large LIMIT or OFFSET is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* Reverts change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). -* Avoid exception when negative or floating point constant is used in WHERE condition for indexed tables. This fixes [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow to CLEAR column even if there are depending DEFAULT expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed TOTALS/ROLLUP/CUBE for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed SIGSEGV if there is an message with error in the middle of the batch in `Kafka` Engine. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). -* Fixed the behaviour when `SummingMergeTree` engine sums up columns from partition key. Added an exception in case of explicit definition of columns to sum which intersects with partition key columns. This fixes [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). -* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed crash in JOIN with LowCardinality type with `join_algorithm=partial_merge`. [#12035](https://github.com/ClickHouse/ClickHouse/pull/12035) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). - -#### Performance Improvement - -* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). - -#### Build/Testing/Packaging Improvement - -* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). - - -### ClickHouse release v20.5.2.7-stable 2020-07-02 - -#### Backward Incompatible Change - -* Return non-Nullable result from COUNT(DISTINCT), and `uniq` aggregate functions family. If all passed values are NULL, return zero instead. This improves SQL compatibility. [#11661](https://github.com/ClickHouse/ClickHouse/pull/11661) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added a check for the case when user-level setting is specified in a wrong place. User-level settings should be specified in `users.xml` inside `` section for specific user profile (or in `` for default settings). The server won't start with exception message in log. This fixes [#9051](https://github.com/ClickHouse/ClickHouse/issues/9051). If you want to skip the check, you can either move settings to the appropriate place or add `1` to config.xml. [#11449](https://github.com/ClickHouse/ClickHouse/pull/11449) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* The setting `input_format_with_names_use_header` is enabled by default. It will affect parsing of input formats `-WithNames` and `-WithNamesAndTypes`. [#10937](https://github.com/ClickHouse/ClickHouse/pull/10937) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove `experimental_use_processors` setting. It is enabled by default. [#10924](https://github.com/ClickHouse/ClickHouse/pull/10924) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Update `zstd` to 1.4.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. This change is backward compatible but we list it here in changelog in case you will wonder about these messages. [#10663](https://github.com/ClickHouse/ClickHouse/pull/10663) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added a check for meaningless codecs and a setting `allow_suspicious_codecs` to control this check. This closes [#4966](https://github.com/ClickHouse/ClickHouse/issues/4966). [#10645](https://github.com/ClickHouse/ClickHouse/pull/10645) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Several Kafka setting changes their defaults. See [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388). -* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). - -#### New Feature - -* `TTL DELETE WHERE` and `TTL GROUP BY` for automatic data coarsening and rollup in tables. [#10537](https://github.com/ClickHouse/ClickHouse/pull/10537) ([expl0si0nn](https://github.com/expl0si0nn)). -* Implementation of PostgreSQL wire protocol. [#10242](https://github.com/ClickHouse/ClickHouse/pull/10242) ([Movses](https://github.com/MovElb)). -* Added system tables for users, roles, grants, settings profiles, quotas, row policies; added commands SHOW USER, SHOW [CURRENT|ENABLED] ROLES, SHOW SETTINGS PROFILES. [#10387](https://github.com/ClickHouse/ClickHouse/pull/10387) ([Vitaly Baranov](https://github.com/vitlibar)). -* 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)). -* 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)). -* 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)). -* Add SHOW CLUSTER(S) queries. [#11467](https://github.com/ClickHouse/ClickHouse/pull/11467) ([hexiaoting](https://github.com/hexiaoting)). -* 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)). -* Add `cast_keep_nullable` setting. If set `CAST(something_nullable AS Type)` return `Nullable(Type)`. [#11733](https://github.com/ClickHouse/ClickHouse/pull/11733) ([Artem Zuikov](https://github.com/4ertus2)). -* Added column `position` to `system.columns` table and `column_position` to `system.parts_columns` table. It contains ordinal position of a column in a table starting with 1. This closes [#7744](https://github.com/ClickHouse/ClickHouse/issues/7744). [#11655](https://github.com/ClickHouse/ClickHouse/pull/11655) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* ON CLUSTER support for SYSTEM {FLUSH DISTRIBUTED,STOP/START DISTRIBUTED SEND}. [#11415](https://github.com/ClickHouse/ClickHouse/pull/11415) ([Azat Khuzhin](https://github.com/azat)). -* Add system.distribution_queue table. [#11394](https://github.com/ClickHouse/ClickHouse/pull/11394) ([Azat Khuzhin](https://github.com/azat)). -* Support for all format settings in Kafka, expose some setting on table level, adjust the defaults for better performance. [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388) ([filimonov](https://github.com/filimonov)). -* Add `port` function (to extract port from URL). [#11120](https://github.com/ClickHouse/ClickHouse/pull/11120) ([Azat Khuzhin](https://github.com/azat)). -* Now `dictGet*` functions accept table names. [#11050](https://github.com/ClickHouse/ClickHouse/pull/11050) ([Vitaly Baranov](https://github.com/vitlibar)). -* The `clickhouse-format` tool is now able to format multiple queries when the `-n` argument is used. [#10852](https://github.com/ClickHouse/ClickHouse/pull/10852) ([Darío](https://github.com/dgrr)). -* Possibility to configure proxy-resolver for DiskS3. [#10744](https://github.com/ClickHouse/ClickHouse/pull/10744) ([Pavel Kovalenko](https://github.com/Jokser)). -* Make `pointInPolygon` work with non-constant polygon. PointInPolygon now can take Array(Array(Tuple(..., ...))) as second argument, array of polygon and holes. [#10623](https://github.com/ClickHouse/ClickHouse/pull/10623) ([Alexey Ilyukhov](https://github.com/livace)) [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* Added `move_ttl_info` to `system.parts` in order to provide introspection of move TTL functionality. [#10591](https://github.com/ClickHouse/ClickHouse/pull/10591) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Possibility to work with S3 through proxies. [#10576](https://github.com/ClickHouse/ClickHouse/pull/10576) ([Pavel Kovalenko](https://github.com/Jokser)). -* Add `NCHAR` and `NVARCHAR` synonims for data types. [#11025](https://github.com/ClickHouse/ClickHouse/pull/11025) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Resolved [#7224](https://github.com/ClickHouse/ClickHouse/issues/7224): added `FailedQuery`, `FailedSelectQuery` and `FailedInsertQuery` metrics to `system.events` table. [#11151](https://github.com/ClickHouse/ClickHouse/pull/11151) ([Nikita Orlov](https://github.com/naorlov)). -* Add more `jemalloc` statistics to `system.asynchronous_metrics`, and ensure that we see up-to-date values for them. [#11748](https://github.com/ClickHouse/ClickHouse/pull/11748) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Allow to specify default S3 credentials and custom auth headers. [#11134](https://github.com/ClickHouse/ClickHouse/pull/11134) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). -* Added new functions to import/export DateTime64 as Int64 with various precision: `to-/fromUnixTimestamp64Milli/-Micro/-Nano`. [#10923](https://github.com/ClickHouse/ClickHouse/pull/10923) ([Vasily Nemkov](https://github.com/Enmk)). -* Allow specifying `mongodb://` URI for MongoDB dictionaries. [#10915](https://github.com/ClickHouse/ClickHouse/pull/10915) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* OFFSET keyword can now be used without an affiliated LIMIT clause. [#10802](https://github.com/ClickHouse/ClickHouse/pull/10802) ([Guillaume Tassery](https://github.com/YiuRULE)). -* Added `system.licenses` table. This table contains licenses of third-party libraries that are located in `contrib` directory. This closes [#2890](https://github.com/ClickHouse/ClickHouse/issues/2890). [#10795](https://github.com/ClickHouse/ClickHouse/pull/10795) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* New function function toStartOfSecond(DateTime64) -> DateTime64 that nullifies sub-second part of DateTime64 value. [#10722](https://github.com/ClickHouse/ClickHouse/pull/10722) ([Vasily Nemkov](https://github.com/Enmk)). -* Add new input format `JSONAsString` that accepts a sequence of JSON objects separated by newlines, spaces and/or commas. [#10607](https://github.com/ClickHouse/ClickHouse/pull/10607) ([Kruglov Pavel](https://github.com/Avogar)). -* Allowed to profile memory with finer granularity steps than 4 MiB. Added sampling memory profiler to capture random allocations/deallocations. [#10598](https://github.com/ClickHouse/ClickHouse/pull/10598) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* `SimpleAggregateFunction` now also supports `sumMap`. [#10000](https://github.com/ClickHouse/ClickHouse/pull/10000) ([Ildus Kurbangaliev](https://github.com/ildus)). -* Support `ALTER RENAME COLUMN` for the distributed table engine. Continuation of [#10727](https://github.com/ClickHouse/ClickHouse/issues/10727). Fixes [#10747](https://github.com/ClickHouse/ClickHouse/issues/10747). [#10887](https://github.com/ClickHouse/ClickHouse/pull/10887) ([alesapin](https://github.com/alesapin)). - -#### Bug Fix - -* Fix UBSan report in Decimal parse. This fixes [#7540](https://github.com/ClickHouse/ClickHouse/issues/7540). [#10512](https://github.com/ClickHouse/ClickHouse/pull/10512) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential floating point exception when parsing DateTime64. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix rare crash caused by using `Nullable` column in prewhere condition. [#11895](https://github.com/ClickHouse/ClickHouse/pull/11895) [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608) [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix wrong result of comparison of FixedString with constant String. This fixes [#11393](https://github.com/ClickHouse/ClickHouse/issues/11393). This bug appeared in version 20.4. [#11828](https://github.com/ClickHouse/ClickHouse/pull/11828) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix wrong result for `if` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix using too many threads for queries. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed `Scalar doesn't exist` exception when using `WITH ...` in `SELECT ... FROM merge_tree_table ...` [#11621](https://github.com/ClickHouse/ClickHouse/issues/11621). [#11767](https://github.com/ClickHouse/ClickHouse/pull/11767) ([Amos Bird](https://github.com/amosbird)). -* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* Now replicated fetches will be cancelled during metadata alter. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). -* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). -* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* Fix very rare race condition in SYSTEM SYNC REPLICA. If the replicated table is created and at the same time from the separate connection another client is issuing `SYSTEM SYNC REPLICA` command on that table (this is unlikely, because another client should be aware that the table is created), it's possible to get nullptr dereference. [#11691](https://github.com/ClickHouse/ClickHouse/pull/11691) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `LIMIT n WITH TIES` usage together with `ORDER BY` statement, which contains aliases. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). -* Fix possible `Pipeline stuck` for selects with parallel `FINAL`. Fixes [#11636](https://github.com/ClickHouse/ClickHouse/issues/11636). [#11682](https://github.com/ClickHouse/ClickHouse/pull/11682) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix error which leads to an incorrect state of `system.mutations`. It may show that whole mutation is already done but the server still has `MUTATE_PART` tasks in the replication queue and tries to execute them. This fixes [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). -* Fix syntax hilite in CREATE USER query. [#11664](https://github.com/ClickHouse/ClickHouse/pull/11664) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove trivial count query optimization if row-level security is set. In previous versions the user get total count of records in a table instead filtered. This fixes [#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* Without `-q` option the database does not get created at startup. [#11604](https://github.com/ClickHouse/ClickHouse/pull/11604) ([giordyb](https://github.com/giordyb)). -* Fix error `Block structure mismatch` for queries with sampling reading from `Buffer` table. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix wrong exit code of the clickhouse-client, when `exception.code() % 256 == 0`. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* Fix race conditions in CREATE/DROP of different replicas of ReplicatedMergeTree. Continue to work if the table was not removed completely from ZooKeeper or not created successfully. This fixes [#11432](https://github.com/ClickHouse/ClickHouse/issues/11432). [#11592](https://github.com/ClickHouse/ClickHouse/pull/11592) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed rare segfault in `SHOW CREATE TABLE` Fixes [#11490](https://github.com/ClickHouse/ClickHouse/issues/11490). [#11579](https://github.com/ClickHouse/ClickHouse/pull/11579) ([tavplubix](https://github.com/tavplubix)). -* All queries in HTTP session have had the same query_id. It is fixed. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). -* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix shard_num/replica_num for `` (breaks use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). -* Fix async INSERT into Distributed for prefer_localhost_replica=0 and w/o internal_replication. [#11527](https://github.com/ClickHouse/ClickHouse/pull/11527) ([Azat Khuzhin](https://github.com/azat)). -* Fix memory leak when exception is thrown in the middle of aggregation with `-State` functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `Pipeline stuck` exception for `INSERT SELECT FINAL` where `SELECT` (`max_threads`>1) has multiple streams but `INSERT` has only one (`max_insert_threads`==0). [#11455](https://github.com/ClickHouse/ClickHouse/pull/11455) ([Azat Khuzhin](https://github.com/azat)). -* Fix wrong result in queries like `select count() from t, u`. [#11454](https://github.com/ClickHouse/ClickHouse/pull/11454) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* Fix potential uninitialized memory read in MergeTree shutdown if table was not created successfully. [#11420](https://github.com/ClickHouse/ClickHouse/pull/11420) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash in JOIN over `LowCarinality(T)` and `Nullable(T)`. [#11380](https://github.com/ClickHouse/ClickHouse/issues/11380). [#11414](https://github.com/ClickHouse/ClickHouse/pull/11414) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix error code for wrong `USING` key. [#11373](https://github.com/ClickHouse/ClickHouse/issues/11373). [#11404](https://github.com/ClickHouse/ClickHouse/pull/11404) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed `geohashesInBox` with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* Better errors for `joinGet()` functions. [#11389](https://github.com/ClickHouse/ClickHouse/pull/11389) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Remove redundant lock during parts send in ReplicatedMergeTree. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). -* Fix support for `\G` (vertical output) in clickhouse-client in multiline mode. This closes [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential segfault when using `Lazy` database. [#11348](https://github.com/ClickHouse/ClickHouse/pull/11348) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash in direct selects from `Join` table engine (without JOIN) and wrong nullability. [#11340](https://github.com/ClickHouse/ClickHouse/pull/11340) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Now merges stopped before change metadata in `ALTER` queries. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). -* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix `visitParamExtractRaw` when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix insignificant data race in `clickhouse-copier`. Found by integration tests. [#11313](https://github.com/ClickHouse/ClickHouse/pull/11313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash when `SET DEFAULT ROLE` is called with wrong arguments. This fixes [#10586](https://github.com/ClickHouse/ClickHouse/issues/10586). [#11278](https://github.com/ClickHouse/ClickHouse/pull/11278) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix crash while reading malformed data in `Protobuf` format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed a bug when `cache` dictionary could return default value instead of normal (when there are only expired keys). This affects only string fields. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix error `Block structure mismatch in QueryPipeline` while reading from `VIEW` with constants in inner query. Fixes [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible exception `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Now `primary.idx` will be checked if it's defined in `CREATE` query. [#11199](https://github.com/ClickHouse/ClickHouse/pull/11199) ([alesapin](https://github.com/alesapin)). -* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed `S3` globbing which could fail in case of more than 1000 keys and some backends. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). -* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#11145](https://github.com/ClickHouse/ClickHouse/pull/11145) ([filimonov](https://github.com/filimonov)). -* Fix excessive reserving of threads for simple queries (optimization for reducing the number of threads, which was partly broken after changes in pipeline). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). -* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* Fixed deadlock during server startup after update with changes in structure of system log tables. [#11106](https://github.com/ClickHouse/ClickHouse/pull/11106) ([alesapin](https://github.com/alesapin)). -* Fixed memory leak in registerDiskS3. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fix error `No such name in Block::erase()` when JOIN appears with PREWHERE or `optimize_move_to_prewhere` makes PREWHERE from WHERE. [#11051](https://github.com/ClickHouse/ClickHouse/pull/11051) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixes the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). -* 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)). -* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* Fix incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid sending partially written files by the DistributedBlockOutputStream. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). -* Fix crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). -* Now it's possible to execute multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). -* Fix possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way (which I found) it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quanite*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). -* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* Now constraints are updated if the column participating in `CONSTRAINT` expression was renamed. Fixes [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). -* Fix potential read of uninitialized memory in cache dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make use of `src_type` for correct type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* Get rid of old libunwind patches. https://github.com/ClickHouse-Extras/libunwind/commit/500aa227911bd185a94bfc071d68f4d3b03cb3b1#r39048012 This allows to disable `-fno-omit-frame-pointer` in `clang` builds that improves performance at least by 1% in average. [#10761](https://github.com/ClickHouse/ClickHouse/pull/10761) ([Amos Bird](https://github.com/amosbird)). -* Fix avgWeighted when using floating-point weight over multiple shards. [#10758](https://github.com/ClickHouse/ClickHouse/pull/10758) ([Baudouin Giard](https://github.com/bgiard)). -* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* Fix crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix usage of primary key wrapped into a function with 'FINAL' modifier and 'ORDER BY' optimization. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). -* Fix possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix atomicity of HTTP insert. This fixes [#9666](https://github.com/ClickHouse/ClickHouse/issues/9666). [#10687](https://github.com/ClickHouse/ClickHouse/pull/10687) ([Andrew Onyshchuk](https://github.com/oandrew)). -* Fix multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). -* Fix order of parameters in AggregateTransform constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). -* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* Fix nullptr dereference in StorageBuffer if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). -* Fix optimize_skip_unused_shards with LowCardinality. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). -* Fix segfault in StorageBuffer when exception on server startup. Fixes [#10550](https://github.com/ClickHouse/ClickHouse/issues/10550). [#10609](https://github.com/ClickHouse/ClickHouse/pull/10609) ([tavplubix](https://github.com/tavplubix)). -* On `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). -* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fixes possible crash `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix error `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). -* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* Implemented comparison between DateTime64 and String values (just like for DateTime). [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* Fix index corruption, which may occur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). -* Disable GROUP BY sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for WITH ROLLUP/CUBE/TOTALS. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). -* Fixes: [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263) (after that PR dist send via INSERT had been postponing on each INSERT) Fixes: [#8756](https://github.com/ClickHouse/ClickHouse/issues/8756) (that PR breaks distributed sends with all of the following conditions met (unlikely setup for now I guess): `internal_replication == false`, multiple local shards (activates the hardlinking code) and `distributed_storage_policy` (makes `link(2)` fails on `EXDEV`)). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). -* Fixed error with "max_rows_to_sort" limit. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Get dictionary and check access rights only once per each call of any function reading external dictionaries. [#10928](https://github.com/ClickHouse/ClickHouse/pull/10928) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### Improvement - -* Apply `TTL` for old data, after `ALTER MODIFY TTL` query. This behaviour is controlled by setting `materialize_ttl_after_modify`, which is enabled by default. [#11042](https://github.com/ClickHouse/ClickHouse/pull/11042) ([Anton Popov](https://github.com/CurtizJ)). -* When parsing C-style backslash escapes in string literals, VALUES and various text formats (this is an extension to SQL standard that is endemic for ClickHouse and MySQL), keep backslash if unknown escape sequence is found (e.g. `\%` or `\w`) that will make usage of `LIKE` and `match` regular expressions more convenient (it's enough to write `name LIKE 'used\_cars'` instead of `name LIKE 'used\\_cars'`) and more compatible at the same time. This fixes [#10922](https://github.com/ClickHouse/ClickHouse/issues/10922). [#11208](https://github.com/ClickHouse/ClickHouse/pull/11208) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* When reading Decimal value, cut extra digits after point. This behaviour is more compatible with MySQL and PostgreSQL. This fixes [#10202](https://github.com/ClickHouse/ClickHouse/issues/10202). [#11831](https://github.com/ClickHouse/ClickHouse/pull/11831) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow to DROP replicated table if the metadata in ZooKeeper was already removed and does not exist (this is also the case when using TestKeeper for testing and the server was restarted). Allow to RENAME replicated table even if there is an error communicating with ZooKeeper. This fixes [#10720](https://github.com/ClickHouse/ClickHouse/issues/10720). [#11652](https://github.com/ClickHouse/ClickHouse/pull/11652) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Slightly improve diagnostic of reading decimal from string. This closes [#10202](https://github.com/ClickHouse/ClickHouse/issues/10202). [#11829](https://github.com/ClickHouse/ClickHouse/pull/11829) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix sleep invocation in signal handler. It was sleeping for less amount of time than expected. [#11825](https://github.com/ClickHouse/ClickHouse/pull/11825) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* (Only Linux) OS related performance metrics (for CPU and I/O) will work even without `CAP_NET_ADMIN` capability. [#10544](https://github.com/ClickHouse/ClickHouse/pull/10544) ([Alexander Kazakov](https://github.com/Akazz)). -* Added `hostname` as an alias to function `hostName`. This feature was suggested by Victor Tarnavskiy from Yandex.Metrica. [#11821](https://github.com/ClickHouse/ClickHouse/pull/11821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added support for distributed `DDL` (update/delete/drop partition) on cross replication clusters. [#11703](https://github.com/ClickHouse/ClickHouse/pull/11703) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Emit warning instead of error in server log at startup if we cannot listen one of the listen addresses (e.g. IPv6 is unavailable inside Docker). Note that if server fails to listen all listed addresses, it will refuse to startup as before. This fixes [#4406](https://github.com/ClickHouse/ClickHouse/issues/4406). [#11687](https://github.com/ClickHouse/ClickHouse/pull/11687) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* 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)). -* Skip empty parameters in requested URL. They may appear when you write `http://localhost:8123/?&a=b` or `http://localhost:8123/?a=b&&c=d`. This closes [#10749](https://github.com/ClickHouse/ClickHouse/issues/10749). [#11651](https://github.com/ClickHouse/ClickHouse/pull/11651) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow using `groupArrayArray` and `groupUniqArrayArray` as `SimpleAggregateFunction`. [#11650](https://github.com/ClickHouse/ClickHouse/pull/11650) ([Volodymyr Kuznetsov](https://github.com/ksvladimir)). -* Allow comparison with constant strings by implicit conversions when analysing index conditions on other types. This may close [#11630](https://github.com/ClickHouse/ClickHouse/issues/11630). [#11648](https://github.com/ClickHouse/ClickHouse/pull/11648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* https://github.com/ClickHouse/ClickHouse/pull/7572#issuecomment-642815377 Support config default HTTPHandlers. [#11628](https://github.com/ClickHouse/ClickHouse/pull/11628) ([Winter Zhang](https://github.com/zhang2014)). -* Make more input formats to work with Kafka engine. Fix the issue with premature flushes. Fix the performance issue when `kafka_num_consumers` is greater than number of partitions in topic. [#11599](https://github.com/ClickHouse/ClickHouse/pull/11599) ([filimonov](https://github.com/filimonov)). -* Improve `multiple_joins_rewriter_version=2` logic. Fix unknown columns error for lambda aliases. [#11587](https://github.com/ClickHouse/ClickHouse/pull/11587) ([Artem Zuikov](https://github.com/4ertus2)). -* Better exception message when cannot parse columns declaration list. This closes [#10403](https://github.com/ClickHouse/ClickHouse/issues/10403). [#11537](https://github.com/ClickHouse/ClickHouse/pull/11537) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improve `enable_optimize_predicate_expression=1` logic for VIEW. [#11513](https://github.com/ClickHouse/ClickHouse/pull/11513) ([Artem Zuikov](https://github.com/4ertus2)). -* Adding support for PREWHERE in live view tables. [#11495](https://github.com/ClickHouse/ClickHouse/pull/11495) ([vzakaznikov](https://github.com/vzakaznikov)). -* 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)). -* Better exception message in case when there is shortage of memory mappings. This closes [#11027](https://github.com/ClickHouse/ClickHouse/issues/11027). [#11316](https://github.com/ClickHouse/ClickHouse/pull/11316) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Support (U)Int8, (U)Int16, Date in ASOF JOIN. [#11301](https://github.com/ClickHouse/ClickHouse/pull/11301) ([Artem Zuikov](https://github.com/4ertus2)). -* 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)). -* Set thread names for internal threads of rdkafka library. Make logs from rdkafka available in server logs. [#10983](https://github.com/ClickHouse/ClickHouse/pull/10983) ([Azat Khuzhin](https://github.com/azat)). -* Support for unicode whitespaces in queries. This helps when queries are copy-pasted from Word or from web page. This fixes [#10896](https://github.com/ClickHouse/ClickHouse/issues/10896). [#10903](https://github.com/ClickHouse/ClickHouse/pull/10903) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow large UInt types as the index in function `tupleElement`. [#10874](https://github.com/ClickHouse/ClickHouse/pull/10874) ([hcz](https://github.com/hczhcz)). -* Respect prefer_localhost_replica/load_balancing on INSERT into Distributed. [#10867](https://github.com/ClickHouse/ClickHouse/pull/10867) ([Azat Khuzhin](https://github.com/azat)). -* Introduce `min_insert_block_size_rows_for_materialized_views`, `min_insert_block_size_bytes_for_materialized_views` settings. This settings are similar to `min_insert_block_size_rows` and `min_insert_block_size_bytes`, but applied only for blocks inserted into `MATERIALIZED VIEW`. It helps to control blocks squashing while pushing to MVs and avoid excessive memory usage. [#10858](https://github.com/ClickHouse/ClickHouse/pull/10858) ([Azat Khuzhin](https://github.com/azat)). -* Get rid of exception from replicated queue during server shutdown. Fixes [#10819](https://github.com/ClickHouse/ClickHouse/issues/10819). [#10841](https://github.com/ClickHouse/ClickHouse/pull/10841) ([alesapin](https://github.com/alesapin)). -* Ensure that `varSamp`, `varPop` cannot return negative results due to numerical errors and that `stddevSamp`, `stddevPop` cannot be calculated from negative variance. This fixes [#10532](https://github.com/ClickHouse/ClickHouse/issues/10532). [#10829](https://github.com/ClickHouse/ClickHouse/pull/10829) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Better DNS exception message. This fixes [#10813](https://github.com/ClickHouse/ClickHouse/issues/10813). [#10828](https://github.com/ClickHouse/ClickHouse/pull/10828) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Change HTTP response code in case of some parse errors to 400 Bad Request. This fix [#10636](https://github.com/ClickHouse/ClickHouse/issues/10636). [#10640](https://github.com/ClickHouse/ClickHouse/pull/10640) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Print a message if clickhouse-client is newer than clickhouse-server. [#10627](https://github.com/ClickHouse/ClickHouse/pull/10627) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Adding support for `INSERT INTO [db.]table WATCH` query. [#10498](https://github.com/ClickHouse/ClickHouse/pull/10498) ([vzakaznikov](https://github.com/vzakaznikov)). -* Allow to pass quota_key in clickhouse-client. This closes [#10227](https://github.com/ClickHouse/ClickHouse/issues/10227). [#10270](https://github.com/ClickHouse/ClickHouse/pull/10270) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### Performance Improvement - -* Allow multiple replicas to assign merges, mutations, partition drop, move and replace concurrently. This closes [#10367](https://github.com/ClickHouse/ClickHouse/issues/10367). [#11639](https://github.com/ClickHouse/ClickHouse/pull/11639) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#11795](https://github.com/ClickHouse/ClickHouse/pull/11795) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Optimization of GROUP BY with respect to table sorting key, enabled with `optimize_aggregation_in_order` setting. [#9113](https://github.com/ClickHouse/ClickHouse/pull/9113) ([dimarub2000](https://github.com/dimarub2000)). -* Selects with final are executed in parallel. Added setting `max_final_threads` to limit the number of threads used. [#10463](https://github.com/ClickHouse/ClickHouse/pull/10463) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Improve performance for INSERT queries via `INSERT SELECT` or INSERT with clickhouse-client when small blocks are generated (typical case with parallel parsing). This fixes [#11275](https://github.com/ClickHouse/ClickHouse/issues/11275). Fix the issue that CONSTRAINTs were not working for DEFAULT fields. This fixes [#11273](https://github.com/ClickHouse/ClickHouse/issues/11273). Fix the issue that CONSTRAINTS were ignored for TEMPORARY tables. This fixes [#11274](https://github.com/ClickHouse/ClickHouse/issues/11274). [#11276](https://github.com/ClickHouse/ClickHouse/pull/11276) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Optimization that eliminates min/max/any aggregators of GROUP BY keys in SELECT section, enabled with `optimize_aggregators_of_group_by_keys` setting. [#11667](https://github.com/ClickHouse/ClickHouse/pull/11667) ([xPoSx](https://github.com/xPoSx)). [#11806](https://github.com/ClickHouse/ClickHouse/pull/11806) ([Azat Khuzhin](https://github.com/azat)). -* 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)). -* Sort bigger parts of the left table in MergeJoin. Buffer left blocks in memory. Add `partial_merge_join_left_table_buffer_bytes` setting to manage the left blocks buffers sizes. [#10601](https://github.com/ClickHouse/ClickHouse/pull/10601) ([Artem Zuikov](https://github.com/4ertus2)). -* Remove duplicate ORDER BY and DISTINCT from subqueries, this optimization is enabled with `optimize_duplicate_order_by_and_distinct` [#10067](https://github.com/ClickHouse/ClickHouse/pull/10067) ([Mikhail Malafeev](https://github.com/demo-99)). -* This feature eliminates functions of other keys in GROUP BY section, enabled with `optimize_group_by_function_keys` [#10051](https://github.com/ClickHouse/ClickHouse/pull/10051) ([xPoSx](https://github.com/xPoSx)). -* New optimization that takes arithmetic operations out of aggregate functions, enabled with `optimize_arithmetic_operations_in_aggregate_functions` [#10047](https://github.com/ClickHouse/ClickHouse/pull/10047) ([Ruslan](https://github.com/kamalov-ruslan)). -* Use HTTP client for S3 based on Poco instead of curl. This will improve performance and lower memory usage of s3 storage and table functions. [#11230](https://github.com/ClickHouse/ClickHouse/pull/11230) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fix Kafka performance issue related to reschedules based on limits, which were always applied. [#11149](https://github.com/ClickHouse/ClickHouse/pull/11149) ([filimonov](https://github.com/filimonov)). -* Enable percpu_arena:percpu for jemalloc (This will reduce memory fragmentation due to thread pool). [#11084](https://github.com/ClickHouse/ClickHouse/pull/11084) ([Azat Khuzhin](https://github.com/azat)). -* Optimize memory usage when reading a response from an S3 HTTP client. [#11561](https://github.com/ClickHouse/ClickHouse/pull/11561) ([Pavel Kovalenko](https://github.com/Jokser)). -* Adjust the default Kafka settings for better performance. [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388) ([filimonov](https://github.com/filimonov)). - -#### Experimental Feature - -* Add data type `Point` (Tuple(Float64, Float64)) and `Polygon` (Array(Array(Tuple(Float64, Float64))). [#10678](https://github.com/ClickHouse/ClickHouse/pull/10678) ([Alexey Ilyukhov](https://github.com/livace)). -* Add's a `hasSubstr` function that allows for look for subsequences in arrays. Note: this function is likely to be renamed without further notice. [#11071](https://github.com/ClickHouse/ClickHouse/pull/11071) ([Ryad Zenine](https://github.com/r-zenine)). -* Added OpenCL support and bitonic sort algorithm, which can be used for sorting integer types of data in single column. Needs to be build with flag `-DENABLE_OPENCL=1`. For using bitonic sort algorithm instead of others you need to set `bitonic_sort` for Setting's option `special_sort` and make sure that OpenCL is available. This feature does not improve performance or anything else, it is only provided as an example and for demonstration purposes. It is likely to be removed in near future if there will be no further development in this direction. [#10232](https://github.com/ClickHouse/ClickHouse/pull/10232) ([Ri](https://github.com/margaritiko)). - -#### Build/Testing/Packaging Improvement - -* Enable clang-tidy for programs and utils. [#10991](https://github.com/ClickHouse/ClickHouse/pull/10991) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove dependency on `tzdata`: do not fail if `/usr/share/zoneinfo` directory does not exist. Note that all timezones work in ClickHouse even without tzdata installed in system. [#11827](https://github.com/ClickHouse/ClickHouse/pull/11827) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added MSan and UBSan stress tests. Note that we already have MSan, UBSan for functional tests and "stress" test is another kind of tests. [#10871](https://github.com/ClickHouse/ClickHouse/pull/10871) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Print compiler build id in crash messages. It will make us slightly more certain about what binary has crashed. Added new function `buildId`. [#11824](https://github.com/ClickHouse/ClickHouse/pull/11824) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added a test to ensure that mutations continue to work after FREEZE query. [#11820](https://github.com/ClickHouse/ClickHouse/pull/11820) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Don't allow tests with "fail" substring in their names because it makes looking at the tests results in browser less convenient when you type Ctrl+F and search for "fail". [#11817](https://github.com/ClickHouse/ClickHouse/pull/11817) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Removes unused imports from HTTPHandlerFactory. [#11660](https://github.com/ClickHouse/ClickHouse/pull/11660) ([Bharat Nallan](https://github.com/bharatnc)). -* Added a random sampling of instances where copier is executed. It is needed to avoid `Too many simultaneous queries` error. Also increased timeout and decreased fault probability. [#11573](https://github.com/ClickHouse/ClickHouse/pull/11573) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix missed include. [#11525](https://github.com/ClickHouse/ClickHouse/pull/11525) ([Matwey V. Kornilov](https://github.com/matwey)). -* Speed up build by removing old example programs. Also found some orphan functional test. [#11486](https://github.com/ClickHouse/ClickHouse/pull/11486) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Increase ccache size for builds in CI. [#11450](https://github.com/ClickHouse/ClickHouse/pull/11450) ([alesapin](https://github.com/alesapin)). -* Leave only unit_tests_dbms in deb build. [#11429](https://github.com/ClickHouse/ClickHouse/pull/11429) ([Ilya Yatsishin](https://github.com/qoega)). -* Update librdkafka to version [1.4.2](https://github.com/edenhill/librdkafka/releases/tag/v1.4.2). [#11256](https://github.com/ClickHouse/ClickHouse/pull/11256) ([filimonov](https://github.com/filimonov)). -* Refactor CMake build files. [#11390](https://github.com/ClickHouse/ClickHouse/pull/11390) ([Ivan](https://github.com/abyss7)). -* Fix several flaky integration tests. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). -* Add support for unit tests run with UBSan. [#11345](https://github.com/ClickHouse/ClickHouse/pull/11345) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove redundant timeout from integration test `test_insertion_sync_fails_with_timeout`. [#11343](https://github.com/ClickHouse/ClickHouse/pull/11343) ([alesapin](https://github.com/alesapin)). -* Better check for hung queries in clickhouse-test. [#11321](https://github.com/ClickHouse/ClickHouse/pull/11321) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Emit a warning if server was build in debug or with sanitizers. [#11304](https://github.com/ClickHouse/ClickHouse/pull/11304) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now clickhouse-test check the server aliveness before tests run. [#11285](https://github.com/ClickHouse/ClickHouse/pull/11285) ([alesapin](https://github.com/alesapin)). -* Fix potentially flacky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Repeat test in CI if `curl` invocation was timed out. It is possible due to system hangups for 10+ seconds that are typical in our CI infrastructure. This fixes [#11267](https://github.com/ClickHouse/ClickHouse/issues/11267). [#11268](https://github.com/ClickHouse/ClickHouse/pull/11268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add a test for Join table engine from @donmikel. This closes [#9158](https://github.com/ClickHouse/ClickHouse/issues/9158). [#11265](https://github.com/ClickHouse/ClickHouse/pull/11265) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix several non significant errors in unit tests. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). -* Now parts of linker command for `cctz` library will not be shuffled with other libraries. [#11213](https://github.com/ClickHouse/ClickHouse/pull/11213) ([alesapin](https://github.com/alesapin)). -* Split /programs/server into actual program and library. [#11186](https://github.com/ClickHouse/ClickHouse/pull/11186) ([Ivan](https://github.com/abyss7)). -* Improve build scripts for protobuf & gRPC. [#11172](https://github.com/ClickHouse/ClickHouse/pull/11172) ([Vitaly Baranov](https://github.com/vitlibar)). -* Enable performance test that was not working. [#11158](https://github.com/ClickHouse/ClickHouse/pull/11158) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Create root S3 bucket for tests before any CH instance is started. [#11142](https://github.com/ClickHouse/ClickHouse/pull/11142) ([Pavel Kovalenko](https://github.com/Jokser)). -* Add performance test for non-constant polygons. [#11141](https://github.com/ClickHouse/ClickHouse/pull/11141) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixing `00979_live_view_watch_continuous_aggregates` test. [#11024](https://github.com/ClickHouse/ClickHouse/pull/11024) ([vzakaznikov](https://github.com/vzakaznikov)). -* Add ability to run zookeeper in integration tests over tmpfs. [#11002](https://github.com/ClickHouse/ClickHouse/pull/11002) ([alesapin](https://github.com/alesapin)). -* Wait for odbc-bridge with exponential backoff. Previous wait time of 200 ms was not enough in our CI environment. [#10990](https://github.com/ClickHouse/ClickHouse/pull/10990) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix non-deterministic test. [#10989](https://github.com/ClickHouse/ClickHouse/pull/10989) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added a test for empty external data. [#10926](https://github.com/ClickHouse/ClickHouse/pull/10926) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Database is recreated for every test. This improves separation of tests. [#10902](https://github.com/ClickHouse/ClickHouse/pull/10902) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added more asserts in columns code. [#10833](https://github.com/ClickHouse/ClickHouse/pull/10833) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Better cooperation with sanitizers. Print information about query_id in the message of sanitizer failure. [#10832](https://github.com/ClickHouse/ClickHouse/pull/10832) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix obvious race condition in "Split build smoke test" check. [#10820](https://github.com/ClickHouse/ClickHouse/pull/10820) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix (false) MSan report in MergeTreeIndexFullText. The issue first appeared in [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add MSan suppression for MariaDB Client library. [#10800](https://github.com/ClickHouse/ClickHouse/pull/10800) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* GRPC make couldn't find protobuf files, changed make file by adding the right link. [#10794](https://github.com/ClickHouse/ClickHouse/pull/10794) ([mnkonkova](https://github.com/mnkonkova)). -* Enable extra warnings (`-Weverything`) for base, utils, programs. Note that we already have it for the most of the code. [#10779](https://github.com/ClickHouse/ClickHouse/pull/10779) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Suppressions of warnings from libraries was mistakenly declared as public in [#10396](https://github.com/ClickHouse/ClickHouse/issues/10396). [#10776](https://github.com/ClickHouse/ClickHouse/pull/10776) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Restore a patch that was accidentially deleted in [#10396](https://github.com/ClickHouse/ClickHouse/issues/10396). [#10774](https://github.com/ClickHouse/ClickHouse/pull/10774) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix performance tests errors, part 2. [#10773](https://github.com/ClickHouse/ClickHouse/pull/10773) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix performance test errors. [#10766](https://github.com/ClickHouse/ClickHouse/pull/10766) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Update cross-builds to use clang-10 compiler. [#10724](https://github.com/ClickHouse/ClickHouse/pull/10724) ([Ivan](https://github.com/abyss7)). -* Update instruction to install RPM packages. This was suggested by Denis (TG login @ldviolet) and implemented by Arkady Shejn. [#10707](https://github.com/ClickHouse/ClickHouse/pull/10707) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Trying to fix `tests/queries/0_stateless/01246_insert_into_watch_live_view.py` test. [#10670](https://github.com/ClickHouse/ClickHouse/pull/10670) ([vzakaznikov](https://github.com/vzakaznikov)). -* Fixing and re-enabling 00979_live_view_watch_continuous_aggregates.py test. [#10658](https://github.com/ClickHouse/ClickHouse/pull/10658) ([vzakaznikov](https://github.com/vzakaznikov)). -* Fix OOM in ASan stress test. [#10646](https://github.com/ClickHouse/ClickHouse/pull/10646) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan report (adding zero to nullptr) in HashTable that appeared after migration to clang-10. [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove external call to `ld` (bfd) linker during tzdata processing in compile time. [#10634](https://github.com/ClickHouse/ClickHouse/pull/10634) ([alesapin](https://github.com/alesapin)). -* Allow to use `lld` to link blobs (resources). [#10632](https://github.com/ClickHouse/ClickHouse/pull/10632) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan report in `LZ4` library. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). See also [https://github.com/lz4/lz4/issues/857](https://github.com/lz4/lz4/issues/857) -* Update LZ4 to the latest dev branch. [#10630](https://github.com/ClickHouse/ClickHouse/pull/10630) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added auto-generated machine-readable file with the list of stable versions. [#10628](https://github.com/ClickHouse/ClickHouse/pull/10628) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `capnproto` version check for `capnp::UnalignedFlatArrayMessageReader`. [#10618](https://github.com/ClickHouse/ClickHouse/pull/10618) ([Matwey V. Kornilov](https://github.com/matwey)). -* Lower memory usage in tests. [#10617](https://github.com/ClickHouse/ClickHouse/pull/10617) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixing hard coded timeouts in new live view tests. [#10604](https://github.com/ClickHouse/ClickHouse/pull/10604) ([vzakaznikov](https://github.com/vzakaznikov)). -* Increasing timeout when opening a client in tests/queries/0_stateless/helpers/client.py. [#10599](https://github.com/ClickHouse/ClickHouse/pull/10599) ([vzakaznikov](https://github.com/vzakaznikov)). -* Enable ThinLTO for clang builds, continuation of [#10435](https://github.com/ClickHouse/ClickHouse/pull/10435). [#10585](https://github.com/ClickHouse/ClickHouse/pull/10585) ([Amos Bird](https://github.com/amosbird)). -* Adding fuzzers and preparing for oss-fuzz integration. [#10546](https://github.com/ClickHouse/ClickHouse/pull/10546) ([kyprizel](https://github.com/kyprizel)). -* Fix FreeBSD build. [#10150](https://github.com/ClickHouse/ClickHouse/pull/10150) ([Ivan](https://github.com/abyss7)). -* Add new build for query tests using pytest framework. [#10039](https://github.com/ClickHouse/ClickHouse/pull/10039) ([Ivan](https://github.com/abyss7)). - - -## ClickHouse release v20.4 - -### ClickHouse release v20.4.8.99-stable 2020-08-10 - -#### Bug Fix - -* Fixed error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with NaN values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed possible extra overflow row in data which could appear for queries `WITH TOTALS`. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed memory tracking for `input_format_parallel_parsing` (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* Fixed [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) allow push predicate when subquery contains with clause. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* Removed data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). -* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed crash in JOIN with dictionary when we are joining over expression of dictionary key: `t JOIN dict ON expr(dict.id) = t.id`. Disable dictionary join optimisation for this case. [#12458](https://github.com/ClickHouse/ClickHouse/pull/12458) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). -* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Show error after TrieDictionary failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). -* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Implemented conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). -* Cap max_memory_usage* limits to the process resident memory. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). -* Fixed `dictGet` arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). -* Don't split the dictionary source's table name into schema and table name itself if ODBC connection doesn't support schema. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Normalize "pid" file handling. In previous versions the server may refuse to start if it was killed without proper shutdown and if there is another process that has the same pid as previously runned server. Also pid file may be removed in unsuccessful server startup even if there is another server running. This fixes [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed error `Expected single dictionary argument for function` for function `defaultValueOfArgumentType` with `LowCardinality` type. Fixes [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Parse tables metadata in parallel when loading database. This fixes slow server startup when there are large number of tables. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). -* Make `topK` aggregate function return Enum for Enum types. This fixes [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed incorrect comparison of tuples with `Nullable` columns. Fixes [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed calculation of access rights when allow_introspection_functions=0. [#12031](https://github.com/ClickHouse/ClickHouse/pull/12031) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* A query with function `neighbor` as the only returned expression may return empty result if the function is called with offset `-9223372036854775808`. This fixes [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed calculation of access rights when allow_ddl=0. [#12015](https://github.com/ClickHouse/ClickHouse/pull/12015) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed potential array size overflow in generateRandom that may lead to crash. This fixes [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential floating point exception. This closes [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed wrong setting name in log message at server startup. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `Query parameter was not set` in `Values` format. Fixes [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). -* Keep aliases for substitutions in query (parametrized queries). This fixes [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed bug with no moves when changing storage policy from default one. [#11893](https://github.com/ClickHouse/ClickHouse/pull/11893) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fixed potential floating point exception when parsing `DateTime64`. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed memory accounting via HTTP interface (can be significant with `wait_end_of_query=1`). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). -* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). - -#### Performance Improvement - -* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). - -#### Build/Testing/Packaging Improvement - -* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). - - -### ClickHouse release v20.4.6.53-stable 2020-06-25 - -#### Bug Fix - -* Fix rare crash caused by using `Nullable` column in prewhere condition. Continuation of [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix wrong result of comparison of FixedString with constant String. This fixes [#11393](https://github.com/ClickHouse/ClickHouse/issues/11393). This bug appeared in version 20.4. [#11828](https://github.com/ClickHouse/ClickHouse/pull/11828) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix using too many threads for queries. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* Now replicated fetches will be cancelled during metadata alter. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). -* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `LIMIT n WITH TIES` usage together with `ORDER BY` statement, which contains aliases. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). -* Fix error which leads to an incorrect state of `system.mutations`. It may show that whole mutation is already done but the server still has `MUTATE_PART` tasks in the replication queue and tries to execute them. This fixes [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). -* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove trivial count query optimization if row-level security is set. In previous versions the user get total count of records in a table instead filtered. This fixes [#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* Fix rare crash caused by using `Nullable` column in prewhere condition. (Probably it is connected with [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) somehow). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix error `Block structure mismatch` for queries with sampling reading from `Buffer` table. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix wrong exit code of the clickhouse-client, when exception.code() % 256 = 0. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed rare segfault in `SHOW CREATE TABLE` Fixes [#11490](https://github.com/ClickHouse/ClickHouse/issues/11490). [#11579](https://github.com/ClickHouse/ClickHouse/pull/11579) ([tavplubix](https://github.com/tavplubix)). -* All queries in HTTP session have had the same query_id. It is fixed. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). -* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* Fix shard_num/replica_num for `` (breaks use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). -* Fix race condition which may lead to an exception during table drop. It's a bit tricky and not dangerous at all. If you want an explanation, just notice me in telegram. [#11523](https://github.com/ClickHouse/ClickHouse/pull/11523) ([alesapin](https://github.com/alesapin)). -* Fix memory leak when exception is thrown in the middle of aggregation with -State functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* Get rid of old libunwind patches. https://github.com/ClickHouse-Extras/libunwind/commit/500aa227911bd185a94bfc071d68f4d3b03cb3b1#r39048012 This allows to disable `-fno-omit-frame-pointer` in `clang` builds that improves performance at least by 1% in average. [#10761](https://github.com/ClickHouse/ClickHouse/pull/10761) ([Amos Bird](https://github.com/amosbird)). -* Fix usage of primary key wrapped into a function with 'FINAL' modifier and 'ORDER BY' optimization. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). - -#### Build/Testing/Packaging Improvement - -* Fix several non significant errors in unit tests. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). -* Fix (false) MSan report in MergeTreeIndexFullText. The issue first appeared in [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse release v20.4.5.36-stable 2020-06-10 - -#### Bug Fix - -* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* Fix pointInPolygon with nan as point. Fixes [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* Fix potential uninitialized memory read in MergeTree shutdown if table was not created successfully. [#11420](https://github.com/ClickHouse/ClickHouse/pull/11420) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed geohashesInBox with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Remove redundant lock during parts send in ReplicatedMergeTree. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). -* Fix support for `\G` (vertical output) in clickhouse-client in multiline mode. This closes [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential segfault when using `Lazy` database. [#11348](https://github.com/ClickHouse/ClickHouse/pull/11348) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Now merges stopped before change metadata in `ALTER` queries. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). -* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix visitParamExtractRaw when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix insignificant data race in clickhouse-copier. Found by integration tests. [#11313](https://github.com/ClickHouse/ClickHouse/pull/11313) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash when SET DEFAULT ROLE is called with wrong arguments. This fixes [#10586](https://github.com/ClickHouse/ClickHouse/issues/10586). [#11278](https://github.com/ClickHouse/ClickHouse/pull/11278) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix crash while reading malformed data in Protobuf format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed a bug when cache-dictionary could return default value instead of normal (when there are only expired keys). This affects only string fields. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix error `Block structure mismatch in QueryPipeline` while reading from `VIEW` with constants in inner query. Fixes [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible exception `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed S3 globbing which could fail in case of more than 1000 keys and some backends. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). -* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* Fix Kafka performance issue related to reschedules based on limits, which were always applied. [#11149](https://github.com/ClickHouse/ClickHouse/pull/11149) ([filimonov](https://github.com/filimonov)). -* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#11145](https://github.com/ClickHouse/ClickHouse/pull/11145) ([filimonov](https://github.com/filimonov)). -* Fix excessive reserving of threads for simple queries (optimization for reducing the number of threads, which was partly broken after changes in pipeline). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). -* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). - -#### Build/Testing/Packaging Improvement - -* Fix several flaky integration tests. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). - -### ClickHouse release v20.4.4.18-stable 2020-05-26 - -No changes compared to v20.4.3.16-stable. - -### ClickHouse release v20.4.3.16-stable 2020-05-23 - -#### Bug Fix - -* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* Fixed memory leak in registerDiskS3. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fixed the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). -* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* Fixed very rare potential use-after-free error in `MergeTree` if table was not created successfully. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986), [#10970](https://github.com/ClickHouse/ClickHouse/pull/10970) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed 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)). -* Fixed 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)). -* Fixed incorrect raw data size in `getRawData()` method. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed sending partially written files by the `DistributedBlockOutputStream`. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). -* Fixed crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed the hang which was happening sometimes during `DROP` of `Kafka` table engine. (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). -* Fixed the impossibility of executing multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). -* Fixed possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quantile*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). -* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* Now constraints are updated if the column participating in `CONSTRAINT` expression was renamed. Fixes [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). -* Fixed potential read of uninitialized memory in cache-dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed combinator `-OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* Fixed `nullptr` dereference in `StorageBuffer` if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `optimize_skip_unused_shards` with `LowCardinality`. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). -* Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fixed possible crash when `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* Disable `GROUP BY` sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). -* Fixed [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). -* Added tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added backward compatibility for create bloom filter index. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). - -### ClickHouse release v20.4.2.9, 2020-05-12 - -#### Backward Incompatible Change -* System tables (e.g. system.query_log, system.trace_log, system.metric_log) are using compact data part format for parts smaller than 10 MiB in size. Compact data part format is supported since version 20.3. If you are going to downgrade to version less than 20.3, you should manually delete table data for system logs in `/var/lib/clickhouse/data/system/`. -* When string comparison involves FixedString and compared arguments are of different sizes, do comparison as if smaller string is padded to the length of the larger. This is intented for SQL compatibility if we imagine that FixedString data type corresponds to SQL CHAR. This closes [#9272](https://github.com/ClickHouse/ClickHouse/issues/9272). [#10363](https://github.com/ClickHouse/ClickHouse/pull/10363) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Make SHOW CREATE TABLE multiline. Now it is more readable and more like MySQL. [#10049](https://github.com/ClickHouse/ClickHouse/pull/10049) ([Azat Khuzhin](https://github.com/azat)) -* Added a setting `validate_polygons` that is used in `pointInPolygon` function and enabled by default. [#9857](https://github.com/ClickHouse/ClickHouse/pull/9857) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### New Feature -* Add support for secured connection from ClickHouse to Zookeeper [#10184](https://github.com/ClickHouse/ClickHouse/pull/10184) ([Konstantin Lebedev](https://github.com/xzkostyan)) -* Support custom HTTP handlers. See [#5436](https://github.com/ClickHouse/ClickHouse/issues/5436) for description. [#7572](https://github.com/ClickHouse/ClickHouse/pull/7572) ([Winter Zhang](https://github.com/zhang2014)) -* Add MessagePack Input/Output format. [#9889](https://github.com/ClickHouse/ClickHouse/pull/9889) ([Kruglov Pavel](https://github.com/Avogar)) -* Add Regexp input format. [#9196](https://github.com/ClickHouse/ClickHouse/pull/9196) ([Kruglov Pavel](https://github.com/Avogar)) -* Added output format `Markdown` for embedding tables in markdown documents. [#10317](https://github.com/ClickHouse/ClickHouse/pull/10317) ([Kruglov Pavel](https://github.com/Avogar)) -* Added support for custom settings section in dictionaries. Also fixes issue [#2829](https://github.com/ClickHouse/ClickHouse/issues/2829). [#10137](https://github.com/ClickHouse/ClickHouse/pull/10137) ([Artem Streltsov](https://github.com/kekekekule)) -* Added custom settings support in DDL-queries for `CREATE DICTIONARY` [#10465](https://github.com/ClickHouse/ClickHouse/pull/10465) ([Artem Streltsov](https://github.com/kekekekule)) -* Add simple server-wide memory profiler that will collect allocation contexts when server memory usage becomes higher than the next allocation threshold. [#10444](https://github.com/ClickHouse/ClickHouse/pull/10444) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add setting `always_fetch_merged_part` which restrict replica to merge parts by itself and always prefer dowloading from other replicas. [#10379](https://github.com/ClickHouse/ClickHouse/pull/10379) ([alesapin](https://github.com/alesapin)) -* Add function `JSONExtractKeysAndValuesRaw` which extracts raw data from JSON objects [#10378](https://github.com/ClickHouse/ClickHouse/pull/10378) ([hcz](https://github.com/hczhcz)) -* Add memory usage from OS to `system.asynchronous_metrics`. [#10361](https://github.com/ClickHouse/ClickHouse/pull/10361) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added generic variants for functions `least` and `greatest`. Now they work with arbitrary number of arguments of arbitrary types. This fixes [#4767](https://github.com/ClickHouse/ClickHouse/issues/4767) [#10318](https://github.com/ClickHouse/ClickHouse/pull/10318) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now ClickHouse controls timeouts of dictionary sources on its side. Two new settings added to cache dictionary configuration: `strict_max_lifetime_seconds`, which is `max_lifetime` by default, and `query_wait_timeout_milliseconds`, which is one minute by default. The first settings is also useful with `allow_read_expired_keys` settings (to forbid reading very expired keys). [#10337](https://github.com/ClickHouse/ClickHouse/pull/10337) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Add log_queries_min_type to filter which entries will be written to query_log [#10053](https://github.com/ClickHouse/ClickHouse/pull/10053) ([Azat Khuzhin](https://github.com/azat)) -* Added function `isConstant`. This function checks whether its argument is constant expression and returns 1 or 0. It is intended for development, debugging and demonstration purposes. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* add joinGetOrNull to return NULL when key is missing instead of returning the default value. [#10094](https://github.com/ClickHouse/ClickHouse/pull/10094) ([Amos Bird](https://github.com/amosbird)) -* Consider `NULL` to be equal to `NULL` in `IN` operator, if the option `transform_null_in` is set. [#10085](https://github.com/ClickHouse/ClickHouse/pull/10085) ([achimbab](https://github.com/achimbab)) -* Add `ALTER TABLE ... RENAME COLUMN` for MergeTree table engines family. [#9948](https://github.com/ClickHouse/ClickHouse/pull/9948) ([alesapin](https://github.com/alesapin)) -* Support parallel distributed INSERT SELECT. [#9759](https://github.com/ClickHouse/ClickHouse/pull/9759) ([vxider](https://github.com/Vxider)) -* Add ability to query Distributed over Distributed (w/o `distributed_group_by_no_merge`) ... [#9923](https://github.com/ClickHouse/ClickHouse/pull/9923) ([Azat Khuzhin](https://github.com/azat)) -* Add function `arrayReduceInRanges` which aggregates array elements in given ranges. [#9598](https://github.com/ClickHouse/ClickHouse/pull/9598) ([hcz](https://github.com/hczhcz)) -* Add Dictionary Status on prometheus exporter. [#9622](https://github.com/ClickHouse/ClickHouse/pull/9622) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Add function `arrayAUC` [#8698](https://github.com/ClickHouse/ClickHouse/pull/8698) ([taiyang-li](https://github.com/taiyang-li)) -* Support `DROP VIEW` statement for better TPC-H compatibility. [#9831](https://github.com/ClickHouse/ClickHouse/pull/9831) ([Amos Bird](https://github.com/amosbird)) -* Add 'strict_order' option to windowFunnel() [#9773](https://github.com/ClickHouse/ClickHouse/pull/9773) ([achimbab](https://github.com/achimbab)) -* Support `DATE` and `TIMESTAMP` SQL operators, e.g. `SELECT date '2001-01-01'` [#9691](https://github.com/ClickHouse/ClickHouse/pull/9691) ([Artem Zuikov](https://github.com/4ertus2)) - -#### Experimental Feature -* Added experimental database engine Atomic. It supports non-blocking `DROP` and `RENAME TABLE` queries and atomic `EXCHANGE TABLES t1 AND t2` query [#7512](https://github.com/ClickHouse/ClickHouse/pull/7512) ([tavplubix](https://github.com/tavplubix)) -* Initial support for ReplicatedMergeTree over S3 (it works in suboptimal way) [#10126](https://github.com/ClickHouse/ClickHouse/pull/10126) ([Pavel Kovalenko](https://github.com/Jokser)) - -#### Bug Fix -* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed bug, which caused HTTP requests to get stuck on client closing connection when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)) -* Fix segfault in StorageBuffer when exception is thrown on server startup. Fixes [#10550](https://github.com/ClickHouse/ClickHouse/issues/10550) [#10609](https://github.com/ClickHouse/ClickHouse/pull/10609) ([tavplubix](https://github.com/tavplubix)) -* The query`SYSTEM DROP DNS CACHE` now also drops caches used to check if user is allowed to connect from some IP addresses [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)) -* Fix usage of multiple `IN` operators with an identical set in one query. Fixes [#10539](https://github.com/ClickHouse/ClickHouse/issues/10539) [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)) -* Fix crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix logic for aggregation_memory_efficient_merge_threads setting. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)) -* Fix disappearing totals. Totals could have being filtered if query had `JOIN` or subquery with external `WHERE` condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674) [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655) [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709) [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix index corruption, which may occur in some cases after merging compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)) -* Fix the situation, when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)) -* Fix overflow at beginning of unix epoch for timezones with fractional offset from UTC. Fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better diagnostics for input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204) [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)) -* Fix numeric overflow in `simpleLinearRegression()` over large integers [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)) -* Fix use-after-free in Distributed shutdown, avoid waiting for sending all batches [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)) -* Add CA certificates to clickhouse-server docker image [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)) -* Fix a rare endless loop that might have occurred when using the `addressToLine` function or AggregateFunctionState columns. [#10466](https://github.com/ClickHouse/ClickHouse/pull/10466) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Handle zookeeper "no node error" during distributed query [#10050](https://github.com/ClickHouse/ClickHouse/pull/10050) ([Daniel Chen](https://github.com/Phantomape)) -* Fix bug when server cannot attach table after column's default was altered. [#10441](https://github.com/ClickHouse/ClickHouse/pull/10441) ([alesapin](https://github.com/alesapin)) -* Implicitly cast the default expression type to the column type for the ALIAS columns [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)) -* Don't remove metadata directory if `ATTACH DATABASE` fails [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)) -* Avoid dependency on system tzdata. Fixes loading of `Africa/Casablanca` timezone on CentOS 8. Fixes [#10211](https://github.com/ClickHouse/ClickHouse/issues/10211) [#10425](https://github.com/ClickHouse/ClickHouse/pull/10425) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix some issues if data is inserted with quorum and then gets deleted (DROP PARTITION, TTL, etc.). It led to stuck of INSERTs or false-positive exceptions in SELECTs. Fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946) [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Check the number and type of arguments when creating BloomFilter index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623) [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)) -* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`, otherwise when both settings specified and there are no up-to-date replicas the query will fail (patch from @alex-zaitsev ) [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)) -* Fix the issue when a query with ARRAY JOIN, ORDER BY and LIMIT may return incomplete result. Fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([Vadim Plakhtinskiy](https://github.com/VadimPlh)) -* Add database name to dictionary name after DETACH/ATTACH. Fixes system.dictionaries table and `SYSTEM RELOAD` query [#10415](https://github.com/ClickHouse/ClickHouse/pull/10415) ([Azat Khuzhin](https://github.com/azat)) -* Fix possible incorrect result for extremes in processors pipeline. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix possible segfault when the setting `distributed_group_by_no_merge` is enabled (introduced in 20.3.7.46 by [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131)). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix wrong flattening of `Array(Tuple(...))` data types. Fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259) [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN [#9950](https://github.com/ClickHouse/ClickHouse/pull/9950) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Fix order of columns after Block::sortColumns() [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)) -* Fix possible `Pipeline stuck` error in `ConcatProcessor` which may happen in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Don't make disk reservations for aggregations. Fixes [#9241](https://github.com/ClickHouse/ClickHouse/issues/9241) [#10375](https://github.com/ClickHouse/ClickHouse/pull/10375) ([Azat Khuzhin](https://github.com/azat)) -* Fix wrong behaviour of datetime functions for timezones that has altered between positive and negative offsets from UTC (e.g. Pacific/Kiritimati). Fixes [#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid infinite loop in `dictIsIn` function. Fixes #515 [#10365](https://github.com/ClickHouse/ClickHouse/pull/10365) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Disable GROUP BY sharding_key optimization by default and fix it for WITH ROLLUP/CUBE/TOTALS [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)) -* Check for error code when checking parts and don't mark part as broken if the error is like "not enough memory". Fixes [#6269](https://github.com/ClickHouse/ClickHouse/issues/6269) [#10364](https://github.com/ClickHouse/ClickHouse/pull/10364) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Show information about not loaded dictionaries in system tables. [#10234](https://github.com/ClickHouse/ClickHouse/pull/10234) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix nullptr dereference in StorageBuffer if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed `DROP` vs `OPTIMIZE` race in `ReplicatedMergeTree`. `DROP` could left some garbage in replica path in ZooKeeper if there was concurrent `OPTIMIZE` query. [#10312](https://github.com/ClickHouse/ClickHouse/pull/10312) ([tavplubix](https://github.com/tavplubix)) -* Fix 'Logical error: CROSS JOIN has expressions' error for queries with comma and names joins mix. Fixes [#9910](https://github.com/ClickHouse/ClickHouse/issues/9910) [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix queries with `max_bytes_before_external_group_by`. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix the issue with limiting maximum recursion depth in parser in certain cases. This fixes [#10283](https://github.com/ClickHouse/ClickHouse/issues/10283) This fix may introduce minor incompatibility: long and deep queries via clickhouse-client may refuse to work, and you should adjust settings `max_query_size` and `max_parser_depth` accordingly. [#10295](https://github.com/ClickHouse/ClickHouse/pull/10295) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Allow to use `count(*)` with multiple JOINs. Fixes [#9853](https://github.com/ClickHouse/ClickHouse/issues/9853) [#10291](https://github.com/ClickHouse/ClickHouse/pull/10291) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix 'Cannot add column' error while creating `range_hashed` dictionary using DDL query. Fixes [#10093](https://github.com/ClickHouse/ClickHouse/issues/10093). [#10235](https://github.com/ClickHouse/ClickHouse/pull/10235) ([alesapin](https://github.com/alesapin)) -* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed bug where ClickHouse would throw "Unknown function lambda." error message when user tries to run ALTER UPDATE/DELETE on tables with ENGINE = Replicated*. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)) -* Fixed reasonably rare segfault in StorageSystemTables that happens when SELECT ... FROM system.tables is run on a database with Lazy engine. [#10209](https://github.com/ClickHouse/ClickHouse/pull/10209) ([Alexander Kazakov](https://github.com/Akazz)) -* Fix possible infinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed "generateRandom" function for Date type. This fixes [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). Fix an edge case when dates with year 2106 are inserted to MergeTree tables with old-style partitioning but partitions are named with year 1970. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022) [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082) [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix move-to-prewhere optimization in presense of arrayJoin functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092) [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix issue with separator appearing in SCRAMBLE for native mysql-connector-java (JDBC) [#10140](https://github.com/ClickHouse/ClickHouse/pull/10140) ([BohuTANG](https://github.com/BohuTANG)) -* Fix using the current database for an access checking when the database isn't specified. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix ALTER of tables with compact parts. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)) -* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)) -* Fix `DROP TABLE` invoked for dictionary [#10165](https://github.com/ClickHouse/ClickHouse/pull/10165) ([Azat Khuzhin](https://github.com/azat)) -* Convert blocks if structure does not match when doing `INSERT` into Distributed table [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)) -* The number of rows was logged incorrectly (as sum across all parts) when inserted block is split by parts with partition key. [#10138](https://github.com/ClickHouse/ClickHouse/pull/10138) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add some arguments check and support identifier arguments for MySQL Database Engine [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)) -* Fix incorrect `index_granularity_bytes` check while creating new replica. Fixes [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)) -* Fix bug in `CHECK TABLE` query when table contain skip indices. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)) -* Fix Distributed-over-Distributed with the only one shard in a nested table [#9997](https://github.com/ClickHouse/ClickHouse/pull/9997) ([Azat Khuzhin](https://github.com/azat)) -* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). ... [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix bug in dictionary when local clickhouse server is used as source. It may caused memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)) -* Fixed replicated tables startup when updating from an old ClickHouse version where `/table/replicas/replica_name/metadata` node doesn't exist. Fixes [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)) -* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with mixed single and two-level aggregation from different shards. [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix deadlock when database with materialized view failed attach at start [#10054](https://github.com/ClickHouse/ClickHouse/pull/10054) ([Azat Khuzhin](https://github.com/azat)) -* Fix a segmentation fault that could occur in GROUP BY over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). ... [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Fix wrong results of distributed queries when alias could override qualified column name. Fixes [#9672](https://github.com/ClickHouse/ClickHouse/issues/9672) [#9714](https://github.com/ClickHouse/ClickHouse/issues/9714) [#9972](https://github.com/ClickHouse/ClickHouse/pull/9972) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix possible deadlock in `SYSTEM RESTART REPLICAS` [#9955](https://github.com/ClickHouse/ClickHouse/pull/9955) ([tavplubix](https://github.com/tavplubix)) -* Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965) [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed `DeleteOnDestroy` logic in `ATTACH PART` which could lead to automatic removal of attached part and added few tests [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)) -* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)) -* Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839) [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix parsing multiple hosts set in the CREATE USER command [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)) -* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)) -* Fix `DISTINCT` for Distributed when `optimize_skip_unused_shards` is set. [#9808](https://github.com/ClickHouse/ClickHouse/pull/9808) ([Azat Khuzhin](https://github.com/azat)) -* Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). ... [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)) -* Fix error with qualified names in `distributed_product_mode=\'local\'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756) [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)) -* For INSERT queries shards now do clamp the settings from the initiator to their constraints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)) -* Add some retries when commiting offsets to Kafka broker, since it can reject commit if during `offsets.commit.timeout.ms` there were no enough replicas available for the `__consumer_offsets` topic [#9884](https://github.com/ClickHouse/ClickHouse/pull/9884) ([filimonov](https://github.com/filimonov)) -* Fix Distributed engine behavior when virtual columns of the underlying table used in `WHERE` [#9847](https://github.com/ClickHouse/ClickHouse/pull/9847) ([Azat Khuzhin](https://github.com/azat)) -* Fixed some cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)) -* Fix 'Different expressions with the same alias' error when query has PREWHERE and WHERE on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)) -* Fix calculating grants for introspection functions from the setting `allow_introspection_functions`. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix max_distributed_connections (w/ and w/o Processors) [#9673](https://github.com/ClickHouse/ClickHouse/pull/9673) ([Azat Khuzhin](https://github.com/azat)) -* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). ... [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix 'COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query' error in case of subqueries with COMMA JOIN out of tables lists (i.e. in WHERE). Fixes [#9782](https://github.com/ClickHouse/ClickHouse/issues/9782) [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix server crashing when `optimize_skip_unused_shards` is set and expression for key can't be converted to its field type [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)) -* Fix empty string handling in `splitByString`. [#9767](https://github.com/ClickHouse/ClickHouse/pull/9767) ([hcz](https://github.com/hczhcz)) -* Fix broken `ALTER TABLE DELETE COLUMN` query for compact parts. [#9779](https://github.com/ClickHouse/ClickHouse/pull/9779) ([alesapin](https://github.com/alesapin)) -* Fixed missing `rows_before_limit_at_least` for queries over http (with processors pipeline). Fixes [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730) [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix excessive memory consumption in `ALTER` queries (mutations). This fixes [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) -* Fix possible permanent "Cannot schedule a task" error. [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) -* Fix bug in backquoting in external dictionaries DDL. Fixes [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) -* Fixed data race in `text_log`. It does not correspond to any real bug. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix bug in a replication that doesn't allow replication to work if the user has executed mutations on the previous version. This fixes [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)) -* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. It led to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)) -* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)) -* Fix RIGHT and FULL JOIN with LowCardinality in JOIN keys. [#9610](https://github.com/ClickHouse/ClickHouse/pull/9610) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) -* Allow `ALTER ON CLUSTER` of Distributed tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268) [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)) -* Fix issue when timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743) [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Improvement -* Use time zone when comparing DateTime with string literal. This fixes [#5206](https://github.com/ClickHouse/ClickHouse/issues/5206). [#10515](https://github.com/ClickHouse/ClickHouse/pull/10515) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Print verbose diagnostic info if Decimal value cannot be parsed from text input format. [#10205](https://github.com/ClickHouse/ClickHouse/pull/10205) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add tasks/memory metrics for distributed/buffer schedule pools [#10449](https://github.com/ClickHouse/ClickHouse/pull/10449) ([Azat Khuzhin](https://github.com/azat)) -* Display result as soon as it's ready for SELECT DISTINCT queries in clickhouse-local and HTTP interface. This fixes [#8951](https://github.com/ClickHouse/ClickHouse/issues/8951) [#9559](https://github.com/ClickHouse/ClickHouse/pull/9559) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Allow to use `SAMPLE OFFSET` query instead of `cityHash64(PRIMARY KEY) % N == n` for splitting in `clickhouse-copier`. To use this feature, pass `--experimental-use-sample-offset 1` as a command line argument. [#10414](https://github.com/ClickHouse/ClickHouse/pull/10414) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Allow to parse BOM in TSV if the first column cannot contain BOM in its value. This fixes [#10301](https://github.com/ClickHouse/ClickHouse/issues/10301) [#10424](https://github.com/ClickHouse/ClickHouse/pull/10424) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add Avro nested fields insert support [#10354](https://github.com/ClickHouse/ClickHouse/pull/10354) ([Andrew Onyshchuk](https://github.com/oandrew)) -* Allowed to alter column in non-modifying data mode when the same type is specified. [#10382](https://github.com/ClickHouse/ClickHouse/pull/10382) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Auto `distributed_group_by_no_merge` on GROUP BY sharding key (if `optimize_skip_unused_shards` is set) [#10341](https://github.com/ClickHouse/ClickHouse/pull/10341) ([Azat Khuzhin](https://github.com/azat)) -* Optimize queries with LIMIT/LIMIT BY/ORDER BY for distributed with GROUP BY sharding_key [#10373](https://github.com/ClickHouse/ClickHouse/pull/10373) ([Azat Khuzhin](https://github.com/azat)) -* Added a setting `max_server_memory_usage` to limit total memory usage of the server. The metric `MemoryTracking` is now calculated without a drift. The setting `max_memory_usage_for_all_queries` is now obsolete and does nothing. This closes [#10293](https://github.com/ClickHouse/ClickHouse/issues/10293). [#10362](https://github.com/ClickHouse/ClickHouse/pull/10362) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add config option `system_tables_lazy_load`. If it's set to false, then system tables with logs are loaded at the server startup. [Alexander Burmak](https://github.com/Alex-Burmak), [Svyatoslav Tkhon Il Pak](https://github.com/DeifyTheGod), [#9642](https://github.com/ClickHouse/ClickHouse/pull/9642) [#10359](https://github.com/ClickHouse/ClickHouse/pull/10359) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Use background thread pool (background_schedule_pool_size) for distributed sends [#10263](https://github.com/ClickHouse/ClickHouse/pull/10263) ([Azat Khuzhin](https://github.com/azat)) -* Use background thread pool for background buffer flushes. [#10315](https://github.com/ClickHouse/ClickHouse/pull/10315) ([Azat Khuzhin](https://github.com/azat)) -* Support for one special case of removing incompletely written parts. This fixes [#9940](https://github.com/ClickHouse/ClickHouse/issues/9940). [#10221](https://github.com/ClickHouse/ClickHouse/pull/10221) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Use isInjective() over manual list of such functions for GROUP BY optimization. [#10342](https://github.com/ClickHouse/ClickHouse/pull/10342) ([Azat Khuzhin](https://github.com/azat)) -* Avoid printing error message in log if client sends RST packet immediately on connect. It is typical behaviour of IPVS balancer with keepalived and VRRP. This fixes [#1851](https://github.com/ClickHouse/ClickHouse/issues/1851) [#10274](https://github.com/ClickHouse/ClickHouse/pull/10274) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Allow to parse `+inf` for floating point types. This closes [#1839](https://github.com/ClickHouse/ClickHouse/issues/1839) [#10272](https://github.com/ClickHouse/ClickHouse/pull/10272) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Implemented `generateRandom` table function for Nested types. This closes [#9903](https://github.com/ClickHouse/ClickHouse/issues/9903) [#10219](https://github.com/ClickHouse/ClickHouse/pull/10219) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Provide `max_allowed_packed` in MySQL compatibility interface that will help some clients to communicate with ClickHouse via MySQL protocol. [#10199](https://github.com/ClickHouse/ClickHouse/pull/10199) ([BohuTANG](https://github.com/BohuTANG)) -* Allow literals for GLOBAL IN (i.e. `SELECT * FROM remote('localhost', system.one) WHERE dummy global in (0)`) [#10196](https://github.com/ClickHouse/ClickHouse/pull/10196) ([Azat Khuzhin](https://github.com/azat)) -* Fix various small issues in interactive mode of clickhouse-client [#10194](https://github.com/ClickHouse/ClickHouse/pull/10194) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid superfluous dictionaries load (system.tables, DROP/SHOW CREATE TABLE) [#10164](https://github.com/ClickHouse/ClickHouse/pull/10164) ([Azat Khuzhin](https://github.com/azat)) -* Update to RWLock: timeout parameter for getLock() + implementation reworked to be phase fair [#10073](https://github.com/ClickHouse/ClickHouse/pull/10073) ([Alexander Kazakov](https://github.com/Akazz)) -* Enhanced compatibility with native mysql-connector-java(JDBC) [#10021](https://github.com/ClickHouse/ClickHouse/pull/10021) ([BohuTANG](https://github.com/BohuTANG)) -* The function `toString` is considered monotonic and can be used for index analysis even when applied in tautological cases with String or LowCardinality(String) argument. [#10110](https://github.com/ClickHouse/ClickHouse/pull/10110) ([Amos Bird](https://github.com/amosbird)) -* Add `ON CLUSTER` clause support to commands `{CREATE|DROP} USER/ROLE/ROW POLICY/SETTINGS PROFILE/QUOTA`, `GRANT`. [#9811](https://github.com/ClickHouse/ClickHouse/pull/9811) ([Vitaly Baranov](https://github.com/vitlibar)) -* Virtual hosted-style support for S3 URI [#9998](https://github.com/ClickHouse/ClickHouse/pull/9998) ([Pavel Kovalenko](https://github.com/Jokser)) -* Now layout type for dictionaries with no arguments can be specified without round brackets in dictionaries DDL-queries. Fixes [#10057](https://github.com/ClickHouse/ClickHouse/issues/10057). [#10064](https://github.com/ClickHouse/ClickHouse/pull/10064) ([alesapin](https://github.com/alesapin)) -* Add ability to use number ranges with leading zeros in filepath [#9989](https://github.com/ClickHouse/ClickHouse/pull/9989) ([Olga Khvostikova](https://github.com/stavrolia)) -* Better memory usage in CROSS JOIN. [#10029](https://github.com/ClickHouse/ClickHouse/pull/10029) ([Artem Zuikov](https://github.com/4ertus2)) -* Try to connect to all shards in cluster when getting structure of remote table and skip_unavailable_shards is set. [#7278](https://github.com/ClickHouse/ClickHouse/pull/7278) ([nvartolomei](https://github.com/nvartolomei)) -* Add `total_rows`/`total_bytes` into the `system.tables` table. [#9919](https://github.com/ClickHouse/ClickHouse/pull/9919) ([Azat Khuzhin](https://github.com/azat)) -* System log tables now use polymorpic parts by default. [#9905](https://github.com/ClickHouse/ClickHouse/pull/9905) ([Anton Popov](https://github.com/CurtizJ)) -* Add type column into system.settings/merge_tree_settings [#9909](https://github.com/ClickHouse/ClickHouse/pull/9909) ([Azat Khuzhin](https://github.com/azat)) -* Check for available CPU instructions at server startup as early as possible. [#9888](https://github.com/ClickHouse/ClickHouse/pull/9888) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove `ORDER BY` stage from mutations because we read from a single ordered part in a single thread. Also add check that the rows in mutation are ordered by sorting key and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)) -* Implement operator LIKE for FixedString at left hand side. This is needed to better support TPC-DS queries. [#9890](https://github.com/ClickHouse/ClickHouse/pull/9890) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add `force_optimize_skip_unused_shards_no_nested` that will disable `force_optimize_skip_unused_shards` for nested Distributed table [#9812](https://github.com/ClickHouse/ClickHouse/pull/9812) ([Azat Khuzhin](https://github.com/azat)) -* Now columns size is calculated only once for MergeTree data parts. [#9827](https://github.com/ClickHouse/ClickHouse/pull/9827) ([alesapin](https://github.com/alesapin)) -* Evaluate constant expressions for `optimize_skip_unused_shards` (i.e. `SELECT * FROM foo_dist WHERE key=xxHash32(0)`) [#8846](https://github.com/ClickHouse/ClickHouse/pull/8846) ([Azat Khuzhin](https://github.com/azat)) -* Check for using `Date` or `DateTime` column from TTL expressions was removed. [#9967](https://github.com/ClickHouse/ClickHouse/pull/9967) ([Vladimir Chebotarev](https://github.com/excitoon)) -* DiskS3 hard links optimal implementation. [#9760](https://github.com/ClickHouse/ClickHouse/pull/9760) ([Pavel Kovalenko](https://github.com/Jokser)) -* If `set multiple_joins_rewriter_version = 2` enables second version of multiple JOIN rewrites that keeps not clashed column names as is. It supports multiple JOINs with `USING` and allow `select *` for JOINs with subqueries. [#9739](https://github.com/ClickHouse/ClickHouse/pull/9739) ([Artem Zuikov](https://github.com/4ertus2)) -* Implementation of "non-blocking" alter for StorageMergeTree [#9606](https://github.com/ClickHouse/ClickHouse/pull/9606) ([alesapin](https://github.com/alesapin)) -* Add MergeTree full support for DiskS3 [#9646](https://github.com/ClickHouse/ClickHouse/pull/9646) ([Pavel Kovalenko](https://github.com/Jokser)) -* Extend `splitByString` to support empty strings as separators. [#9742](https://github.com/ClickHouse/ClickHouse/pull/9742) ([hcz](https://github.com/hczhcz)) -* Add a `timestamp_ns` column to `system.trace_log`. It contains a high-definition timestamp of the trace event, and allows to build timelines of thread profiles ("flame charts"). [#9696](https://github.com/ClickHouse/ClickHouse/pull/9696) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* When the setting `send_logs_level` is enabled, avoid intermixing of log messages and query progress. [#9634](https://github.com/ClickHouse/ClickHouse/pull/9634) ([Azat Khuzhin](https://github.com/azat)) -* Added support of `MATERIALIZE TTL IN PARTITION`. [#9581](https://github.com/ClickHouse/ClickHouse/pull/9581) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Support complex types inside Avro nested fields [#10502](https://github.com/ClickHouse/ClickHouse/pull/10502) ([Andrew Onyshchuk](https://github.com/oandrew)) - -#### Performance Improvement -* Better insert logic for right table for Partial MergeJoin. [#10467](https://github.com/ClickHouse/ClickHouse/pull/10467) ([Artem Zuikov](https://github.com/4ertus2)) -* Improved performance of row-oriented formats (more than 10% for CSV and more than 35% for Avro in case of narrow tables). [#10503](https://github.com/ClickHouse/ClickHouse/pull/10503) ([Andrew Onyshchuk](https://github.com/oandrew)) -* Improved performance of queries with explicitly defined sets at right side of IN operator and tuples on the left side. [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) -* Use less memory for hash table in HashJoin. [#10416](https://github.com/ClickHouse/ClickHouse/pull/10416) ([Artem Zuikov](https://github.com/4ertus2)) -* Special HashJoin over StorageDictionary. Allow rewrite `dictGet()` functions with JOINs. It's not backward incompatible itself but could uncover [#8400](https://github.com/ClickHouse/ClickHouse/issues/8400) on some installations. [#10133](https://github.com/ClickHouse/ClickHouse/pull/10133) ([Artem Zuikov](https://github.com/4ertus2)) -* Enable parallel insert of materialized view when its target table supports. [#10052](https://github.com/ClickHouse/ClickHouse/pull/10052) ([vxider](https://github.com/Vxider)) -* Improved performance of index analysis with monotonic functions. [#9607](https://github.com/ClickHouse/ClickHouse/pull/9607)[#10026](https://github.com/ClickHouse/ClickHouse/pull/10026) ([Anton Popov](https://github.com/CurtizJ)) -* Using SSE2 or SSE4.2 SIMD intrinsics to speed up tokenization in bloom filters. [#9968](https://github.com/ClickHouse/ClickHouse/pull/9968) ([Vasily Nemkov](https://github.com/Enmk)) -* Improved performance of queries with explicitly defined sets at right side of `IN` operator. This fixes performance regression in version 20.3. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740) ([Anton Popov](https://github.com/CurtizJ)) -* Now clickhouse-copier splits each partition in number of pieces and copies them independently. [#9075](https://github.com/ClickHouse/ClickHouse/pull/9075) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Adding more aggregation methods. For example TPC-H query 1 will now pick `FixedHashMap` and gets 25% performance gain [#9829](https://github.com/ClickHouse/ClickHouse/pull/9829) ([Amos Bird](https://github.com/amosbird)) -* Use single row counter for multiple streams in pre-limit transform. This helps to avoid uniting pipeline streams in queries with `limit` but without `order by` (like `select f(x) from (select x from t limit 1000000000)`) and use multiple threads for further processing. [#9602](https://github.com/ClickHouse/ClickHouse/pull/9602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### Build/Testing/Packaging Improvement -* Use a fork of AWS SDK libraries from ClickHouse-Extras [#10527](https://github.com/ClickHouse/ClickHouse/pull/10527) ([Pavel Kovalenko](https://github.com/Jokser)) -* Add integration tests for new ALTER RENAME COLUMN query. [#10654](https://github.com/ClickHouse/ClickHouse/pull/10654) ([vzakaznikov](https://github.com/vzakaznikov)) -* Fix possible signed integer overflow in invocation of function `now64` with wrong arguments. This fixes [#8973](https://github.com/ClickHouse/ClickHouse/issues/8973) [#10511](https://github.com/ClickHouse/ClickHouse/pull/10511) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Split fuzzer and sanitizer configurations to make build config compatible with Oss-fuzz. [#10494](https://github.com/ClickHouse/ClickHouse/pull/10494) ([kyprizel](https://github.com/kyprizel)) -* Fixes for clang-tidy on clang-10. [#10420](https://github.com/ClickHouse/ClickHouse/pull/10420) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. [#10434](https://github.com/ClickHouse/ClickHouse/pull/10434) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added `ASAN_OPTIONS` environment variable to investigate errors in CI stress tests with Address sanitizer. [#10440](https://github.com/ClickHouse/ClickHouse/pull/10440) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Enable ThinLTO for clang builds (experimental). [#10435](https://github.com/ClickHouse/ClickHouse/pull/10435) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove accidential dependency on Z3 that may be introduced if the system has Z3 solver installed. [#10426](https://github.com/ClickHouse/ClickHouse/pull/10426) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Move integration tests docker files to docker/ directory. [#10335](https://github.com/ClickHouse/ClickHouse/pull/10335) ([Ilya Yatsishin](https://github.com/qoega)) -* Allow to use `clang-10` in CI. It ensures that [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238) is fixed. [#10384](https://github.com/ClickHouse/ClickHouse/pull/10384) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update OpenSSL to upstream master. Fixed the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` and `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. The issue was present in version 20.1. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix clang-10 build. [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238) [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)) -* Add performance test for [Parallel INSERT for materialized view](https://github.com/ClickHouse/ClickHouse/pull/10052). [#10345](https://github.com/ClickHouse/ClickHouse/pull/10345) ([vxider](https://github.com/Vxider)) -* Fix flaky test `test_settings_constraints_distributed.test_insert_clamps_settings`. [#10346](https://github.com/ClickHouse/ClickHouse/pull/10346) ([Vitaly Baranov](https://github.com/vitlibar)) -* Add util to test results upload in CI ClickHouse [#10330](https://github.com/ClickHouse/ClickHouse/pull/10330) ([Ilya Yatsishin](https://github.com/qoega)) -* Convert test results to JSONEachRow format in junit_to_html tool [#10323](https://github.com/ClickHouse/ClickHouse/pull/10323) ([Ilya Yatsishin](https://github.com/qoega)) -* Update cctz. [#10215](https://github.com/ClickHouse/ClickHouse/pull/10215) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Allow to create HTML report from the purest JUnit XML report. [#10247](https://github.com/ClickHouse/ClickHouse/pull/10247) ([Ilya Yatsishin](https://github.com/qoega)) -* Update the check for minimal compiler version. Fix the root cause of the issue [#10250](https://github.com/ClickHouse/ClickHouse/issues/10250) [#10256](https://github.com/ClickHouse/ClickHouse/pull/10256) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Initial support for live view tables over distributed [#10179](https://github.com/ClickHouse/ClickHouse/pull/10179) ([vzakaznikov](https://github.com/vzakaznikov)) -* Fix (false) MSan report in MergeTreeIndexFullText. The issue first appeared in [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* clickhouse-docker-util [#10151](https://github.com/ClickHouse/ClickHouse/pull/10151) ([filimonov](https://github.com/filimonov)) -* Update pdqsort to recent version [#10171](https://github.com/ClickHouse/ClickHouse/pull/10171) ([Ivan](https://github.com/abyss7)) -* Update libdivide to v3.0 [#10169](https://github.com/ClickHouse/ClickHouse/pull/10169) ([Ivan](https://github.com/abyss7)) -* Add check with enabled polymorphic parts. [#10086](https://github.com/ClickHouse/ClickHouse/pull/10086) ([Anton Popov](https://github.com/CurtizJ)) -* Add cross-compile build for FreeBSD. This fixes [#9465](https://github.com/ClickHouse/ClickHouse/issues/9465) [#9643](https://github.com/ClickHouse/ClickHouse/pull/9643) ([Ivan](https://github.com/abyss7)) -* Add performance test for [#6924](https://github.com/ClickHouse/ClickHouse/issues/6924) [#6980](https://github.com/ClickHouse/ClickHouse/pull/6980) ([filimonov](https://github.com/filimonov)) -* Add support of `/dev/null` in the `File` engine for better performance testing [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) -* Move all folders inside /dbms one level up [#9974](https://github.com/ClickHouse/ClickHouse/pull/9974) ([Ivan](https://github.com/abyss7)) -* Add a test that checks that read from MergeTree with single thread is performed in order. Addition to [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670) [#9762](https://github.com/ClickHouse/ClickHouse/pull/9762) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix the `00964_live_view_watch_events_heartbeat.py` test to avoid race condition. [#9944](https://github.com/ClickHouse/ClickHouse/pull/9944) ([vzakaznikov](https://github.com/vzakaznikov)) -* Fix integration test `test_settings_constraints` [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)) -* Every function in its own file, part 12. [#9922](https://github.com/ClickHouse/ClickHouse/pull/9922) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added performance test for the case of extremely slow analysis of array of tuples. [#9872](https://github.com/ClickHouse/ClickHouse/pull/9872) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update zstd to 1.4.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. [#10663](https://github.com/ClickHouse/ClickHouse/pull/10663) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix TSan report in `system.stack_trace`. [#9832](https://github.com/ClickHouse/ClickHouse/pull/9832) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Removed dependency on `clock_getres`. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added identifier names check with clang-tidy. [#9799](https://github.com/ClickHouse/ClickHouse/pull/9799) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update "builder" docker image. This image is not used in CI but is useful for developers. [#9809](https://github.com/ClickHouse/ClickHouse/pull/9809) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove old `performance-test` tool that is no longer used in CI. `clickhouse-performance-test` is great but now we are using way superior tool that is doing comparison testing with sophisticated statistical formulas to achieve confident results regardless to various changes in environment. [#9796](https://github.com/ClickHouse/ClickHouse/pull/9796) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added most of clang-static-analyzer checks. [#9765](https://github.com/ClickHouse/ClickHouse/pull/9765) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update Poco to 1.9.3 in preparation for MongoDB URI support. [#6892](https://github.com/ClickHouse/ClickHouse/pull/6892) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Fix build with `-DUSE_STATIC_LIBRARIES=0 -DENABLE_JEMALLOC=0` [#9651](https://github.com/ClickHouse/ClickHouse/pull/9651) ([Artem Zuikov](https://github.com/4ertus2)) -* For change log script, if merge commit was cherry-picked to release branch, take PR name from commit description. [#9708](https://github.com/ClickHouse/ClickHouse/pull/9708) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Support `vX.X-conflicts` tag in backport script. [#9705](https://github.com/ClickHouse/ClickHouse/pull/9705) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix `auto-label` for backporting script. [#9685](https://github.com/ClickHouse/ClickHouse/pull/9685) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Use libc++ in Darwin cross-build to make it consistent with native build. [#9665](https://github.com/ClickHouse/ClickHouse/pull/9665) ([Hui Wang](https://github.com/huiwang)) -* Fix flacky test `01017_uniqCombined_memory_usage`. Continuation of [#7236](https://github.com/ClickHouse/ClickHouse/issues/7236). [#9667](https://github.com/ClickHouse/ClickHouse/pull/9667) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix build for native MacOS Clang compiler [#9649](https://github.com/ClickHouse/ClickHouse/pull/9649) ([Ivan](https://github.com/abyss7)) -* Allow to add various glitches around `pthread_mutex_lock`, `pthread_mutex_unlock` functions. [#9635](https://github.com/ClickHouse/ClickHouse/pull/9635) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add support for `clang-tidy` in `packager` script. [#9625](https://github.com/ClickHouse/ClickHouse/pull/9625) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add ability to use unbundled msgpack. [#10168](https://github.com/ClickHouse/ClickHouse/pull/10168) ([Azat Khuzhin](https://github.com/azat)) - - -## ClickHouse release v20.3 - - -### ClickHouse release v20.3.21.2-lts, 2020-11-02 - -#### Bug Fix - -* Fix dictGet in sharding_key (and similar places, i.e. when the function context is stored permanently). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). -* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix missing or excessive headers in `TSV/CSVWithNames` formats. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). - - -### ClickHouse release v20.3.20.6-lts, 2020-10-09 - -#### Bug Fix - -* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15724](https://github.com/ClickHouse/ClickHouse/pull/15724), [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). -* Fix hang of queries with a lot of subqueries to same table of `MySQL` engine. Previously, if there were more than 16 subqueries to same `MySQL` table in query, it hang forever. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). -* Fix 'Unknown identifier' in GROUP BY when query has JOIN over Merge table. [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). -* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). - - -### ClickHouse release v20.3.19.4-lts, 2020-09-18 - -#### Bug Fix - -* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). -* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). -* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). - -#### Improvement - -* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse release v20.3.18.10-lts, 2020-09-08 - -#### Bug Fix - -* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix CAST(Nullable(String), Enum()). [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). -* Fixed data race in `text_log`. It does not correspond to any real bug. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### Improvement - -* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* 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)). - -#### Performance Improvement - -* Slightly optimize very short queries with LowCardinality. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). - -#### Build/Testing/Packaging Improvement - -* Fix UBSan report (adding zero to nullptr) in HashTable that appeared after migration to clang-10. [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse release v20.3.17.173-lts, 2020-08-15 - -#### Bug Fix - -* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix invalid return type for comparison of tuples with `NULL` elements. Fixes [#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix queries with constant columns and `ORDER BY` prefix of primary key. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). -* Return passed number for numbers with MSB set in roundUpToPowerOfTwoOrZero(). [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). - - -### ClickHouse release v20.3.16.165-lts 2020-08-10 - -#### Bug Fix - -* Fixed error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with `NaN` values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed possible extra overflow row in data which could appear for queries `WITH TOTALS`. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). -* Fixed [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) allow push predicate when subquery contains with clause. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). -* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). -* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). -* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). -* Fixed `TOTALS/ROLLUP/CUBE` for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). -* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Show error after `TrieDictionary` failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). -* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Implement conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). -* Fixed `dictGet` arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). -* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid `There is no query` exception for materialized views with joins or with subqueries attached to system logs (system.query_log, metric_log, etc) or to engine=Buffer underlying table. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). -* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed error `Expected single dictionary argument for function` for function `defaultValueOfArgumentType` with `LowCardinality` type. Fixes [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Parse tables metadata in parallel when loading database. This fixes slow server startup when there are large number of tables. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). -* Make `topK` aggregate function return Enum for Enum types. This fixes [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed incorrect comparison of tuples with `Nullable` columns. Fixes [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* A query with function `neighbor` as the only returned expression may return empty result if the function is called with offset `-9223372036854775808`. This fixes [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential array size overflow in generateRandom that may lead to crash. This fixes [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential floating point exception. This closes [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed wrong setting name in log message at server startup. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `Query parameter was not set` in `Values` format. Fixes [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). -* Keep aliases for substitutions in query (parametrized queries). This fixes [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed potential floating point exception when parsing DateTime64. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed memory accounting via `HTTP` interface (can be significant with `wait_end_of_query=1`). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). -* Fixed wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). -* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). -* Fixed `LIMIT n WITH TIES` usage together with `ORDER BY` statement, which contains aliases. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). -* Fix potential read of uninitialized memory in cache dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### Performance Improvement - -* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). - - -### ClickHouse release v20.3.12.112-lts 2020-06-25 - -#### Bug Fix - -* Fix rare crash caused by using `Nullable` column in prewhere condition. Continuation of [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix using too many threads for queries. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* Now replicated fetches will be cancelled during metadata alter. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). -* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix error which leads to an incorrect state of `system.mutations`. It may show that whole mutation is already done but the server still has `MUTATE_PART` tasks in the replication queue and tries to execute them. This fixes [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). -* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Remove trivial count query optimization if row-level security is set. In previous versions the user get total count of records in a table instead filtered. This fixes [#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* Fix rare crash caused by using `Nullable` column in prewhere condition. (Probably it is connected with [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) somehow). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix error `Block structure mismatch` for queries with sampling reading from `Buffer` table. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix wrong exit code of the clickhouse-client, when exception.code() % 256 = 0. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* All queries in HTTP session have had the same query_id. It is fixed. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). -* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* Fix shard_num/replica_num for `` (breaks use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). -* Fix memory leak when exception is thrown in the middle of aggregation with -State functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix wrong results of distributed queries when alias could override qualified column name. Fixes [#9672](https://github.com/ClickHouse/ClickHouse/issues/9672) [#9714](https://github.com/ClickHouse/ClickHouse/issues/9714). [#9972](https://github.com/ClickHouse/ClickHouse/pull/9972) ([Artem Zuikov](https://github.com/4ertus2)). - - -### ClickHouse release v20.3.11.97-lts 2020-06-10 - -#### New Feature - -* Now ClickHouse controls timeouts of dictionary sources on its side. Two new settings added to cache dictionary configuration: `strict_max_lifetime_seconds`, which is `max_lifetime` by default and `query_wait_timeout_milliseconds`, which is one minute by default. The first settings is also useful with `allow_read_expired_keys` settings (to forbid reading very expired keys). [#10337](https://github.com/ClickHouse/ClickHouse/pull/10337) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). - -#### Bug Fix - -* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* Fix pointInPolygon with nan as point. Fixes [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* Fix crash in JOIN over LowCarinality(T) and Nullable(T). [#11380](https://github.com/ClickHouse/ClickHouse/issues/11380). [#11414](https://github.com/ClickHouse/ClickHouse/pull/11414) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix error code for wrong `USING` key. [#11373](https://github.com/ClickHouse/ClickHouse/issues/11373). [#11404](https://github.com/ClickHouse/ClickHouse/pull/11404) ([Artem Zuikov](https://github.com/4ertus2)). -* Fixed geohashesInBox with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* Better errors for `joinGet()` functions. [#11389](https://github.com/ClickHouse/ClickHouse/pull/11389) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Remove redundant lock during parts send in ReplicatedMergeTree. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). -* Fix support for `\G` (vertical output) in clickhouse-client in multiline mode. This closes [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash in direct selects from StorageJoin (without JOIN) and wrong nullability. [#11340](https://github.com/ClickHouse/ClickHouse/pull/11340) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Now merges stopped before change metadata in `ALTER` queries. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). -* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix visitParamExtractRaw when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash while reading malformed data in Protobuf format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed a bug when cache-dictionary could return default value instead of normal (when there are only expired keys). This affects only string fields. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fix error `Block structure mismatch in QueryPipeline` while reading from `VIEW` with constants in inner query. Fixes [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible exception `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed S3 globbing which could fail in case of more than 1000 keys and some backends. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). -* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* Fix excessive reserving of threads for simple queries (optimization for reducing the number of threads, which was partly broken after changes in pipeline). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). -* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). -* Introduce commit retry logic to decrease the possibility of getting duplicates from Kafka in rare cases when offset commit was failed. [#9884](https://github.com/ClickHouse/ClickHouse/pull/9884) ([filimonov](https://github.com/filimonov)). - -#### Performance Improvement - -* Get dictionary and check access rights only once per each call of any function reading external dictionaries. [#10928](https://github.com/ClickHouse/ClickHouse/pull/10928) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### Build/Testing/Packaging Improvement - -* Fix several flaky integration tests. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). - -### ClickHouse release v20.3.10.75-lts 2020-05-23 - -#### Bug Fix - -* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). -* Fixed incorrect raw data size in method `getRawData()`. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of `GROUP BY` result is large and aggregation is performed by a single `String` field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed backward compatibility with tuples in `Distributed` tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)) -* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed combinator -`OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* Fixed crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). -* Fixed order of parameters in `AggregateTransform` constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). -* Fixed the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed a bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). -* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). -* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). -* Fixed index corruption, which may occur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed the situation, when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). -* Fixed overflow at beginning of unix epoch for timezones with fractional offset from `UTC`. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed improper shutdown of `Distributed` storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). -* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). - - -#### Build/Testing/Packaging Improvement - -* Fix UBSan report in LZ4 library. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix clang-10 build. [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238). [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)). -* Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added some improvements in printing diagnostic info in input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204). [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)). -* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). - -#### Bug fix - -* Fix error `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). - - -### ClickHouse release v20.3.8.53, 2020-04-23 - -#### Bug Fix -* Fixed wrong behaviour of datetime functions for timezones that has altered between positive and negative offsets from UTC (e.g. Pacific/Kiritimati). This fixes [#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix possible segfault with `distributed_group_by_no_merge` enabled (introduced in 20.3.7.46 by [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131)). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix wrong flattening of `Array(Tuple(...))` data types. This fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259) [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Drop disks reservation in Aggregator. This fixes bug in disk space reservation, which may cause big external aggregation to fail even if it could be completed successfully [#10375](https://github.com/ClickHouse/ClickHouse/pull/10375) ([Azat Khuzhin](https://github.com/azat)) -* Fixed `DROP` vs `OPTIMIZE` race in `ReplicatedMergeTree`. `DROP` could left some garbage in replica path in ZooKeeper if there was concurrent `OPTIMIZE` query. [#10312](https://github.com/ClickHouse/ClickHouse/pull/10312) ([tavplubix](https://github.com/tavplubix)) -* Fix bug when server cannot attach table after column default was altered. [#10441](https://github.com/ClickHouse/ClickHouse/pull/10441) ([alesapin](https://github.com/alesapin)) -* Do not remove metadata directory when attach database fails before loading tables. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)) -* Fixed several bugs when some data was inserted with quorum, then deleted somehow (DROP PARTITION, TTL) and this leaded to the stuck of INSERTs or false-positive exceptions in SELECTs. This fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946) [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix possible `Pipeline stuck` error in `ConcatProcessor` which could have happened in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed wrong behavior in HashTable that caused compilation error when trying to read HashMap from buffer. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)) -* Allow to use `count(*)` with multiple JOINs. Fixes [#9853](https://github.com/ClickHouse/ClickHouse/issues/9853) [#10291](https://github.com/ClickHouse/ClickHouse/pull/10291) ([Artem Zuikov](https://github.com/4ertus2)) -* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`, otherwise when both settings specified and there are no up-to-date replicas the query will fail (patch from @alex-zaitsev). Fixes: [#2564](https://github.com/ClickHouse/ClickHouse/issues/2564). [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)) -* Fix the issue when a query with ARRAY JOIN, ORDER BY and LIMIT may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). Author: [Vadim Plakhtinskiy](https://github.com/VadimPlh). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Check the number and type of arguments when creating BloomFilter index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623) [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)) - -#### Performance Improvement -* Improved performance of queries with explicitly defined sets at right side of `IN` operator and tuples in the left side. This fixes performance regression in version 20.3. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740), [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) - -### ClickHouse release v20.3.7.46, 2020-04-17 - -#### Bug Fix - -* Fix `Logical error: CROSS JOIN has expressions` error for queries with comma and names joins mix. [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix queries with `max_bytes_before_external_group_by`. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix move-to-prewhere optimization in presense of arrayJoin functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). - -### ClickHouse release v20.3.6.40, 2020-04-16 - -#### New Feature - -* Added function `isConstant`. This function checks whether its argument is constant expression and returns 1 or 0. It is intended for development, debugging and demonstration purposes. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### Bug Fix - -* Fix error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed bug where ClickHouse would throw "Unknown function lambda." error message when user tries to run ALTER UPDATE/DELETE on tables with ENGINE = Replicated*. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). -* Fixed "generateRandom" function for Date type. This fixes [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). Fix an edge case when dates with year 2106 are inserted to MergeTree tables with old-style partitioning but partitions are named with year 1970. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022). [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix possible inifinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix using the current database for access checking when the database isn't specified. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)). -* Convert blocks if structure does not match on INSERT into Distributed(). [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). -* Fix possible incorrect result for extremes in processors pipeline. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix some kinds of alters with compact parts. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)). -* Fix incorrect `index_granularity_bytes` check while creating new replica. Fixes [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)). -* Fix SIGSEGV on INSERT into Distributed table when its structure differs from the underlying tables. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). -* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed replicated tables startup when updating from an old ClickHouse version where `/table/replicas/replica_name/metadata` node doesn't exist. Fixes [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)). -* Add some arguments check and support identifier arguments for MySQL Database Engine. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). -* Fix bug in clickhouse dictionary source from localhost clickhouse server. The bug may lead to memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). -* Fix bug in `CHECK TABLE` query when table contain skip indices. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)). -* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with different level from different shards (mixed single and two level aggregation). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix a segmentation fault that could occur in GROUP BY over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965). [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). -* Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). -* Fix parsing multiple hosts set in the CREATE USER command, e.g. `CREATE USER user6 HOST NAME REGEXP 'lo.?*host', NAME REGEXP 'lo*host'`. [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). -* Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). -* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). -* Fix error with qualified names in `distributed_product_mode='local'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756). [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix calculating grants for introspection functions from the setting 'allow_introspection_functions'. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)). - -#### Build/Testing/Packaging Improvement - -* Fix integration test `test_settings_constraints`. [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)). -* Removed dependency on `clock_getres`. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)). - - -### ClickHouse release v20.3.5.21, 2020-03-27 - -#### Bug Fix - -* Fix 'Different expressions with the same alias' error when query has PREWHERE and WHERE on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). -* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix 'COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query' error in case of subqueries with COMMA JOIN out of tables lists (i.e. in WHERE). Fixes [#9782](https://github.com/ClickHouse/ClickHouse/issues/9782). [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix SIGSEGV with optimize_skip_unused_shards when type cannot be converted. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). -* Fix broken `ALTER TABLE DELETE COLUMN` query for compact parts. [#9779](https://github.com/ClickHouse/ClickHouse/pull/9779) ([alesapin](https://github.com/alesapin)). -* Fix max_distributed_connections (w/ and w/o Processors). [#9673](https://github.com/ClickHouse/ClickHouse/pull/9673) ([Azat Khuzhin](https://github.com/azat)). -* Fixed a few cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)). - -#### Improvement - -* Remove order by stage from mutations because we read from a single ordered part in a single thread. Also add check that the order of rows in mutation is ordered in sorting key order and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse release v20.3.4.10, 2020-03-20 - -#### Bug Fix -* This release also contains all bug fixes from 20.1.8.41 -* Fix missing `rows_before_limit_at_least` for queries over http (with processors pipeline). This fixes [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730). [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - - -### ClickHouse release v20.3.3.6, 2020-03-17 - -#### Bug Fix -* This release also contains all bug fixes from 20.1.7.38 -* Fix bug in a replication that doesn't allow replication to work if the user has executed mutations on the previous version. This fixes [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again. -* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again. - -### ClickHouse release v20.3.2.1, 2020-03-12 - -#### Backward Incompatible Change - -* Fixed the issue `file name too long` when sending data for `Distributed` tables for a large number of replicas. Fixed the issue that replica credentials were exposed in the server log. The format of directory name on disk was changed to `[shard{shard_index}[_replica{replica_index}]]`. [#8911](https://github.com/ClickHouse/ClickHouse/pull/8911) ([Mikhail Korotov](https://github.com/millb)) After you upgrade to the new version, you will not be able to downgrade without manual intervention, because old server version does not recognize the new directory format. If you want to downgrade, you have to manually rename the corresponding directories to the old format. This change is relevant only if you have used asynchronous `INSERT`s to `Distributed` tables. In the version 20.3.3 we will introduce a setting that will allow you to enable the new format gradually. -* Changed the format of replication log entries for mutation commands. You have to wait for old mutations to process before installing the new version. -* Implement simple memory profiler that dumps stacktraces to `system.trace_log` every N bytes over soft allocation limit [#8765](https://github.com/ClickHouse/ClickHouse/pull/8765) ([Ivan](https://github.com/abyss7)) [#9472](https://github.com/ClickHouse/ClickHouse/pull/9472) ([alexey-milovidov](https://github.com/alexey-milovidov)) The column of `system.trace_log` was renamed from `timer_type` to `trace_type`. This will require changes in third-party performance analysis and flamegraph processing tools. -* Use OS thread id everywhere instead of internal thread number. This fixes [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) Old `clickhouse-client` cannot receive logs that are send from the server when the setting `send_logs_level` is enabled, because the names and types of the structured log messages were changed. On the other hand, different server versions can send logs with different types to each other. When you don't use the `send_logs_level` setting, you should not care. [#8954](https://github.com/ClickHouse/ClickHouse/pull/8954) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove `indexHint` function [#9542](https://github.com/ClickHouse/ClickHouse/pull/9542) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove `findClusterIndex`, `findClusterValue` functions. This fixes [#8641](https://github.com/ClickHouse/ClickHouse/issues/8641). If you were using these functions, send an email to `clickhouse-feedback@yandex-team.com` [#9543](https://github.com/ClickHouse/ClickHouse/pull/9543) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now it's not allowed to create columns or add columns with `SELECT` subquery as default expression. [#9481](https://github.com/ClickHouse/ClickHouse/pull/9481) ([alesapin](https://github.com/alesapin)) -* Require aliases for subqueries in JOIN. [#9274](https://github.com/ClickHouse/ClickHouse/pull/9274) ([Artem Zuikov](https://github.com/4ertus2)) -* Improved `ALTER MODIFY/ADD` queries logic. Now you cannot `ADD` column without type, `MODIFY` default expression doesn't change type of column and `MODIFY` type doesn't loose default expression value. Fixes [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). [#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) ([alesapin](https://github.com/alesapin)) -* Require server to be restarted to apply the changes in logging configuration. This is a temporary workaround to avoid the bug where the server logs to a deleted log file (see [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* The setting `experimental_use_processors` is enabled by default. This setting enables usage of the new query pipeline. This is internal refactoring and we expect no visible changes. If you will see any issues, set it to back zero. [#8768](https://github.com/ClickHouse/ClickHouse/pull/8768) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### New Feature -* Add `Avro` and `AvroConfluent` input/output formats [#8571](https://github.com/ClickHouse/ClickHouse/pull/8571) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8957](https://github.com/ClickHouse/ClickHouse/pull/8957) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8717](https://github.com/ClickHouse/ClickHouse/pull/8717) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Multi-threaded and non-blocking updates of expired keys in `cache` dictionaries (with optional permission to read old ones). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Add query `ALTER ... MATERIALIZE TTL`. It runs mutation that forces to remove expired data by TTL and recalculates meta-information about TTL in all parts. [#8775](https://github.com/ClickHouse/ClickHouse/pull/8775) ([Anton Popov](https://github.com/CurtizJ)) -* Switch from HashJoin to MergeJoin (on disk) if needed [#9082](https://github.com/ClickHouse/ClickHouse/pull/9082) ([Artem Zuikov](https://github.com/4ertus2)) -* Added `MOVE PARTITION` command for `ALTER TABLE` [#4729](https://github.com/ClickHouse/ClickHouse/issues/4729) [#6168](https://github.com/ClickHouse/ClickHouse/pull/6168) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Reloading storage configuration from configuration file on the fly. [#8594](https://github.com/ClickHouse/ClickHouse/pull/8594) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Allowed to change `storage_policy` to not less rich one. [#8107](https://github.com/ClickHouse/ClickHouse/pull/8107) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Added support for globs/wildcards for S3 storage and table function. [#8851](https://github.com/ClickHouse/ClickHouse/pull/8851) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Implement `bitAnd`, `bitOr`, `bitXor`, `bitNot` for `FixedString(N)` datatype. [#9091](https://github.com/ClickHouse/ClickHouse/pull/9091) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Added function `bitCount`. This fixes [#8702](https://github.com/ClickHouse/ClickHouse/issues/8702). [#8708](https://github.com/ClickHouse/ClickHouse/pull/8708) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#8749](https://github.com/ClickHouse/ClickHouse/pull/8749) ([ikopylov](https://github.com/ikopylov)) -* Add `generateRandom` table function to generate random rows with given schema. Allows to populate arbitrary test table with data. [#8994](https://github.com/ClickHouse/ClickHouse/pull/8994) ([Ilya Yatsishin](https://github.com/qoega)) -* `JSONEachRowFormat`: support special case when objects enclosed in top-level array. [#8860](https://github.com/ClickHouse/ClickHouse/pull/8860) ([Kruglov Pavel](https://github.com/Avogar)) -* Now it's possible to create a column with `DEFAULT` expression which depends on a column with default `ALIAS` expression. [#9489](https://github.com/ClickHouse/ClickHouse/pull/9489) ([alesapin](https://github.com/alesapin)) -* Allow to specify `--limit` more than the source data size in `clickhouse-obfuscator`. The data will repeat itself with different random seed. [#9155](https://github.com/ClickHouse/ClickHouse/pull/9155) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added `groupArraySample` function (similar to `groupArray`) with reservior sampling algorithm. [#8286](https://github.com/ClickHouse/ClickHouse/pull/8286) ([Amos Bird](https://github.com/amosbird)) -* Now you can monitor the size of update queue in `cache`/`complex_key_cache` dictionaries via system metrics. [#9413](https://github.com/ClickHouse/ClickHouse/pull/9413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Allow to use CRLF as a line separator in CSV output format with setting `output_format_csv_crlf_end_of_line` is set to 1 [#8934](https://github.com/ClickHouse/ClickHouse/pull/8934) [#8935](https://github.com/ClickHouse/ClickHouse/pull/8935) [#8963](https://github.com/ClickHouse/ClickHouse/pull/8963) ([Mikhail Korotov](https://github.com/millb)) -* Implement more functions of the [H3](https://github.com/uber/h3) API: `h3GetBaseCell`, `h3HexAreaM2`, `h3IndexesAreNeighbors`, `h3ToChildren`, `h3ToString` and `stringToH3` [#8938](https://github.com/ClickHouse/ClickHouse/pull/8938) ([Nico Mandery](https://github.com/nmandery)) -* New setting introduced: `max_parser_depth` to control maximum stack size and allow large complex queries. This fixes [#6681](https://github.com/ClickHouse/ClickHouse/issues/6681) and [#7668](https://github.com/ClickHouse/ClickHouse/issues/7668). [#8647](https://github.com/ClickHouse/ClickHouse/pull/8647) ([Maxim Smirnov](https://github.com/qMBQx8GH)) -* Add a setting `force_optimize_skip_unused_shards` setting to throw if skipping of unused shards is not possible [#8805](https://github.com/ClickHouse/ClickHouse/pull/8805) ([Azat Khuzhin](https://github.com/azat)) -* Allow to configure multiple disks/volumes for storing data for send in `Distributed` engine [#8756](https://github.com/ClickHouse/ClickHouse/pull/8756) ([Azat Khuzhin](https://github.com/azat)) -* Support storage policy (``) for storing temporary data. [#8750](https://github.com/ClickHouse/ClickHouse/pull/8750) ([Azat Khuzhin](https://github.com/azat)) -* Added `X-ClickHouse-Exception-Code` HTTP header that is set if exception was thrown before sending data. This implements [#4971](https://github.com/ClickHouse/ClickHouse/issues/4971). [#8786](https://github.com/ClickHouse/ClickHouse/pull/8786) ([Mikhail Korotov](https://github.com/millb)) -* Added function `ifNotFinite`. It is just a syntactic sugar: `ifNotFinite(x, y) = isFinite(x) ? x : y`. [#8710](https://github.com/ClickHouse/ClickHouse/pull/8710) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added `last_successful_update_time` column in `system.dictionaries` table [#9394](https://github.com/ClickHouse/ClickHouse/pull/9394) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Add `blockSerializedSize` function (size on disk without compression) [#8952](https://github.com/ClickHouse/ClickHouse/pull/8952) ([Azat Khuzhin](https://github.com/azat)) -* Add function `moduloOrZero` [#9358](https://github.com/ClickHouse/ClickHouse/pull/9358) ([hcz](https://github.com/hczhcz)) -* Added system tables `system.zeros` and `system.zeros_mt` as well as tale functions `zeros()` and `zeros_mt()`. Tables (and table functions) contain single column with name `zero` and type `UInt8`. This column contains zeros. It is needed for test purposes as the fastest method to generate many rows. This fixes [#6604](https://github.com/ClickHouse/ClickHouse/issues/6604) [#9593](https://github.com/ClickHouse/ClickHouse/pull/9593) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) - -#### Experimental Feature -* Add new compact format of parts in `MergeTree`-family tables in which all columns are stored in one file. It helps to increase performance of small and frequent inserts. The old format (one file per column) is now called wide. Data storing format is controlled by settings `min_bytes_for_wide_part` and `min_rows_for_wide_part`. [#8290](https://github.com/ClickHouse/ClickHouse/pull/8290) ([Anton Popov](https://github.com/CurtizJ)) -* Support for S3 storage for `Log`, `TinyLog` and `StripeLog` tables. [#8862](https://github.com/ClickHouse/ClickHouse/pull/8862) ([Pavel Kovalenko](https://github.com/Jokser)) - -#### Bug Fix -* Fixed inconsistent whitespaces in log messages. [#9322](https://github.com/ClickHouse/ClickHouse/pull/9322) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix bug in which arrays of unnamed tuples were flattened as Nested structures on table creation. [#8866](https://github.com/ClickHouse/ClickHouse/pull/8866) ([achulkov2](https://github.com/achulkov2)) -* Fixed the issue when "Too many open files" error may happen if there are too many files matching glob pattern in `File` table or `file` table function. Now files are opened lazily. This fixes [#8857](https://github.com/ClickHouse/ClickHouse/issues/8857) [#8861](https://github.com/ClickHouse/ClickHouse/pull/8861) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* DROP TEMPORARY TABLE now drops only temporary table. [#8907](https://github.com/ClickHouse/ClickHouse/pull/8907) ([Vitaly Baranov](https://github.com/vitlibar)) -* Remove outdated partition when we shutdown the server or DETACH/ATTACH a table. [#8602](https://github.com/ClickHouse/ClickHouse/pull/8602) ([Guillaume Tassery](https://github.com/YiuRULE)) -* For how the default disk calculates the free space from `data` subdirectory. Fixed the issue when the amount of free space is not calculated correctly if the `data` directory is mounted to a separate device (rare case). This fixes [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) [#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) ([Mikhail Korotov](https://github.com/millb)) -* Allow comma (cross) join with IN () inside. [#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) ([Artem Zuikov](https://github.com/4ertus2)) -* Allow to rewrite CROSS to INNER JOIN if there's [NOT] LIKE operator in WHERE section. [#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix possible incorrect result after `GROUP BY` with enabled setting `distributed_aggregation_memory_efficient`. Fixes [#9134](https://github.com/ClickHouse/ClickHouse/issues/9134). [#9289](https://github.com/ClickHouse/ClickHouse/pull/9289) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Found keys were counted as missed in metrics of cache dictionaries. [#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix replication protocol incompatibility introduced in [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) -* Fixed race condition on `queue_task_handle` at the startup of `ReplicatedMergeTree` tables. [#9552](https://github.com/ClickHouse/ClickHouse/pull/9552) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* The token `NOT` didn't work in `SHOW TABLES NOT LIKE` query [#8727](https://github.com/ClickHouse/ClickHouse/issues/8727) [#8940](https://github.com/ClickHouse/ClickHouse/pull/8940) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added range check to function `h3EdgeLengthM`. Without this check, buffer overflow is possible. [#8945](https://github.com/ClickHouse/ClickHouse/pull/8945) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed up a bug in batched calculations of ternary logical OPs on multiple arguments (more than 10). [#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alexander Kazakov](https://github.com/Akazz)) -* Fix error of PREWHERE optimization, which could lead to segfaults or `Inconsistent number of columns got from MergeTreeRangeReader` exception. [#9024](https://github.com/ClickHouse/ClickHouse/pull/9024) ([Anton Popov](https://github.com/CurtizJ)) -* Fix unexpected `Timeout exceeded while reading from socket` exception, which randomly happens on secure connection before timeout actually exceeded and when query profiler is enabled. Also add `connect_timeout_with_failover_secure_ms` settings (default 100ms), which is similar to `connect_timeout_with_failover_ms`, but is used for secure connections (because SSL handshake is slower, than ordinary TCP connection) [#9026](https://github.com/ClickHouse/ClickHouse/pull/9026) ([tavplubix](https://github.com/tavplubix)) -* Fix bug with mutations finalization, when mutation may hang in state with `parts_to_do=0` and `is_done=0`. [#9022](https://github.com/ClickHouse/ClickHouse/pull/9022) ([alesapin](https://github.com/alesapin)) -* Use new ANY JOIN logic with `partial_merge_join` setting. It's possible to make `ANY|ALL|SEMI LEFT` and `ALL INNER` joins with `partial_merge_join=1` now. [#8932](https://github.com/ClickHouse/ClickHouse/pull/8932) ([Artem Zuikov](https://github.com/4ertus2)) -* Shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send queries to a shard with another constraints. [#9447](https://github.com/ClickHouse/ClickHouse/pull/9447) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fixed memory management problem in `MergeTreeReadPool`. [#8791](https://github.com/ClickHouse/ClickHouse/pull/8791) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix `toDecimal*OrNull()` functions family when called with string `e`. Fixes [#8312](https://github.com/ClickHouse/ClickHouse/issues/8312) [#8764](https://github.com/ClickHouse/ClickHouse/pull/8764) ([Artem Zuikov](https://github.com/4ertus2)) -* Make sure that `FORMAT Null` sends no data to the client. [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Fix bug that timestamp in `LiveViewBlockInputStream` will not updated. `LIVE VIEW` is an experimental feature. [#8644](https://github.com/ClickHouse/ClickHouse/pull/8644) ([vxider](https://github.com/Vxider)) [#8625](https://github.com/ClickHouse/ClickHouse/pull/8625) ([vxider](https://github.com/Vxider)) -* Fixed `ALTER MODIFY TTL` wrong behavior which did not allow to delete old TTL expressions. [#8422](https://github.com/ClickHouse/ClickHouse/pull/8422) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fixed UBSan report in MergeTreeIndexSet. This fixes [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) [#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed the behaviour of `match` and `extract` functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. This fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid throwing from destructor in Apache Avro 3rd-party library. [#9066](https://github.com/ClickHouse/ClickHouse/pull/9066) ([Andrew Onyshchuk](https://github.com/oandrew)) -* Don't commit a batch polled from `Kafka` partially as it can lead to holes in data. [#8876](https://github.com/ClickHouse/ClickHouse/pull/8876) ([filimonov](https://github.com/filimonov)) -* Fix `joinGet` with nullable return types. [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) [#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) ([Amos Bird](https://github.com/amosbird)) -* Fix data incompatibility when compressed with `T64` codec. [#9016](https://github.com/ClickHouse/ClickHouse/pull/9016) ([Artem Zuikov](https://github.com/4ertus2)) Fix data type ids in `T64` compression codec that leads to wrong (de)compression in affected versions. [#9033](https://github.com/ClickHouse/ClickHouse/pull/9033) ([Artem Zuikov](https://github.com/4ertus2)) -* Add setting `enable_early_constant_folding` and disable it in some cases that leads to errors. [#9010](https://github.com/ClickHouse/ClickHouse/pull/9010) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix pushdown predicate optimizer with VIEW and enable the test [#9011](https://github.com/ClickHouse/ClickHouse/pull/9011) ([Winter Zhang](https://github.com/zhang2014)) -* Fix segfault in `Merge` tables, that can happen when reading from `File` storages [#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) ([tavplubix](https://github.com/tavplubix)) -* Added a check for storage policy in `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE`. Otherwise it could make data of part inaccessible after restart and prevent ClickHouse to start. [#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix alters if there is TTL set for table. [#8800](https://github.com/ClickHouse/ClickHouse/pull/8800) ([Anton Popov](https://github.com/CurtizJ)) -* Fix race condition that can happen when `SYSTEM RELOAD ALL DICTIONARIES` is executed while some dictionary is being modified/added/removed. [#8801](https://github.com/ClickHouse/ClickHouse/pull/8801) ([Vitaly Baranov](https://github.com/vitlibar)) -* In previous versions `Memory` database engine use empty data path, so tables are created in `path` directory (e.g. `/var/lib/clickhouse/`), not in data directory of database (e.g. `/var/lib/clickhouse/db_name`). [#8753](https://github.com/ClickHouse/ClickHouse/pull/8753) ([tavplubix](https://github.com/tavplubix)) -* Fixed wrong log messages about missing default disk or policy. [#9530](https://github.com/ClickHouse/ClickHouse/pull/9530) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix not(has()) for the bloom_filter index of array types. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -* Allow first column(s) in a table with `Log` engine be an alias [#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) ([Ivan](https://github.com/abyss7)) -* Fix order of ranges while reading from `MergeTree` table in one thread. It could lead to exceptions from `MergeTreeRangeReader` or wrong query results. [#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) ([Anton Popov](https://github.com/CurtizJ)) -* Make `reinterpretAsFixedString` to return `FixedString` instead of `String`. [#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) ([Andrew Onyshchuk](https://github.com/oandrew)) -* Avoid extremely rare cases when the user can get wrong error message (`Success` instead of detailed error description). [#9457](https://github.com/ClickHouse/ClickHouse/pull/9457) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Do not crash when using `Template` format with empty row template. [#8785](https://github.com/ClickHouse/ClickHouse/pull/8785) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Metadata files for system tables could be created in wrong place [#8653](https://github.com/ClickHouse/ClickHouse/pull/8653) ([tavplubix](https://github.com/tavplubix)) Fixes [#8581](https://github.com/ClickHouse/ClickHouse/issues/8581). -* Fix data race on exception_ptr in cache dictionary [#8303](https://github.com/ClickHouse/ClickHouse/issues/8303). [#9379](https://github.com/ClickHouse/ClickHouse/pull/9379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Do not throw an exception for query `ATTACH TABLE IF NOT EXISTS`. Previously it was thrown if table already exists, despite the `IF NOT EXISTS` clause. [#8967](https://github.com/ClickHouse/ClickHouse/pull/8967) ([Anton Popov](https://github.com/CurtizJ)) -* Fixed missing closing paren in exception message. [#8811](https://github.com/ClickHouse/ClickHouse/pull/8811) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid message `Possible deadlock avoided` at the startup of clickhouse-client in interactive mode. [#9455](https://github.com/ClickHouse/ClickHouse/pull/9455) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed the issue when padding at the end of base64 encoded value can be malformed. Update base64 library. This fixes [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378) [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) ([filimonov](https://github.com/filimonov)) -* Fixed exception in `DROP TABLE IF EXISTS` [#8663](https://github.com/ClickHouse/ClickHouse/pull/8663) ([Nikita Vasilev](https://github.com/nikvas0)) -* Fix crash when a user tries to `ALTER MODIFY SETTING` for old-formated `MergeTree` table engines family. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) -* Support for UInt64 numbers that don't fit in Int64 in JSON-related functions. Update SIMDJSON to master. This fixes [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) [#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed execution of inversed predicates when non-strictly monotinic functional index is used. [#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) ([Alexander Kazakov](https://github.com/Akazz)) -* Don't try to fold `IN` constant in `GROUP BY` [#8868](https://github.com/ClickHouse/ClickHouse/pull/8868) ([Amos Bird](https://github.com/amosbird)) -* Fix bug in `ALTER DELETE` mutations which leads to index corruption. This fixes [#9019](https://github.com/ClickHouse/ClickHouse/issues/9019) and [#8982](https://github.com/ClickHouse/ClickHouse/issues/8982). Additionally fix extremely rare race conditions in `ReplicatedMergeTree` `ALTER` queries. [#9048](https://github.com/ClickHouse/ClickHouse/pull/9048) ([alesapin](https://github.com/alesapin)) -* When the setting `compile_expressions` is enabled, you can get `unexpected column` in `LLVMExecutableFunction` when we use `Nullable` type [#8910](https://github.com/ClickHouse/ClickHouse/pull/8910) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Multiple fixes for `Kafka` engine: 1) fix duplicates that were appearing during consumer group rebalance. 2) Fix rare 'holes' appeared when data were polled from several partitions with one poll and committed partially (now we always process / commit the whole polled block of messages). 3) Fix flushes by block size (before that only flushing by timeout was working properly). 4) better subscription procedure (with assignment feedback). 5) Make tests work faster (with default intervals and timeouts). Due to the fact that data was not flushed by block size before (as it should according to documentation), that PR may lead to some performance degradation with default settings (due to more often & tinier flushes which are less optimal). If you encounter the performance issue after that change - please increase `kafka_max_block_size` in the table to the bigger value ( for example `CREATE TABLE ...Engine=Kafka ... SETTINGS ... kafka_max_block_size=524288`). Fixes [#7259](https://github.com/ClickHouse/ClickHouse/issues/7259) [#8917](https://github.com/ClickHouse/ClickHouse/pull/8917) ([filimonov](https://github.com/filimonov)) -* Fix `Parameter out of bound` exception in some queries after PREWHERE optimizations. [#8914](https://github.com/ClickHouse/ClickHouse/pull/8914) ([Baudouin Giard](https://github.com/bgiard)) -* Fixed the case of mixed-constness of arguments of function `arrayZip`. [#8705](https://github.com/ClickHouse/ClickHouse/pull/8705) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492) [#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) ([tavplubix](https://github.com/tavplubix)) -* Now it's not possible to create or add columns with simple cyclic aliases like `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) -* Fixed a bug with double move which may corrupt original part. This is relevant if you use `ALTER TABLE MOVE` [#8680](https://github.com/ClickHouse/ClickHouse/pull/8680) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Allow `interval` identifier to correctly parse without backticks. Fixed issue when a query cannot be executed even if the `interval` identifier is enclosed in backticks or double quotes. This fixes [#9124](https://github.com/ClickHouse/ClickHouse/issues/9124). [#9142](https://github.com/ClickHouse/ClickHouse/pull/9142) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed fuzz test and incorrect behaviour of `bitTestAll`/`bitTestAny` functions. [#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix possible crash/wrong number of rows in `LIMIT n WITH TIES` when there are a lot of rows equal to n'th row. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -* Fix mutations with parts written with enabled `insert_quorum`. [#9463](https://github.com/ClickHouse/ClickHouse/pull/9463) ([alesapin](https://github.com/alesapin)) -* Fix data race at destruction of `Poco::HTTPServer`. It could happen when server is started and immediately shut down. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) -* Fix bug in which a misleading error message was shown when running `SHOW CREATE TABLE a_table_that_does_not_exist`. [#8899](https://github.com/ClickHouse/ClickHouse/pull/8899) ([achulkov2](https://github.com/achulkov2)) -* Fixed `Parameters are out of bound` exception in some rare cases when we have a constant in the `SELECT` clause when we have an `ORDER BY` and a `LIMIT` clause. [#8892](https://github.com/ClickHouse/ClickHouse/pull/8892) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Fix mutations finalization, when already done mutation can have status `is_done=0`. [#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) ([alesapin](https://github.com/alesapin)) -* Prevent from executing `ALTER ADD INDEX` for MergeTree tables with old syntax, because it doesn't work. [#8822](https://github.com/ClickHouse/ClickHouse/pull/8822) ([Mikhail Korotov](https://github.com/millb)) -* During server startup do not access table, which `LIVE VIEW` depends on, so server will be able to start. Also remove `LIVE VIEW` dependencies when detaching `LIVE VIEW`. `LIVE VIEW` is an experimental feature. [#8824](https://github.com/ClickHouse/ClickHouse/pull/8824) ([tavplubix](https://github.com/tavplubix)) -* Fix possible segfault in `MergeTreeRangeReader`, while executing `PREWHERE`. [#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) ([Anton Popov](https://github.com/CurtizJ)) -* Fix possible mismatched checksums with column TTLs. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) -* Fixed a bug when parts were not being moved in background by TTL rules in case when there is only one volume. [#8672](https://github.com/ClickHouse/ClickHouse/pull/8672) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fixed the issue `Method createColumn() is not implemented for data type Set`. This fixes [#7799](https://github.com/ClickHouse/ClickHouse/issues/7799). [#8674](https://github.com/ClickHouse/ClickHouse/pull/8674) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now we will try finalize mutations more frequently. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) -* Fix `intDiv` by minus one constant [#9351](https://github.com/ClickHouse/ClickHouse/pull/9351) ([hcz](https://github.com/hczhcz)) -* Fix possible race condition in `BlockIO`. [#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) ([filimonov](https://github.com/filimonov)) -* Added workaround if OS returns wrong result for `timer_create` function. [#8837](https://github.com/ClickHouse/ClickHouse/pull/8837) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed error in usage of `min_marks_for_seek` parameter. Fixed the error message when there is no sharding key in Distributed table and we try to skip unused shards. [#8908](https://github.com/ClickHouse/ClickHouse/pull/8908) ([Azat Khuzhin](https://github.com/azat)) - -#### Improvement -* Implement `ALTER MODIFY/DROP` queries on top of mutations for `ReplicatedMergeTree*` engines family. Now `ALTERS` blocks only at the metadata update stage, and don't block after that. [#8701](https://github.com/ClickHouse/ClickHouse/pull/8701) ([alesapin](https://github.com/alesapin)) -* Add ability to rewrite CROSS to INNER JOINs with `WHERE` section containing unqialified names. [#9512](https://github.com/ClickHouse/ClickHouse/pull/9512) ([Artem Zuikov](https://github.com/4ertus2)) -* Make `SHOW TABLES` and `SHOW DATABASES` queries support the `WHERE` expressions and `FROM`/`IN` [#9076](https://github.com/ClickHouse/ClickHouse/pull/9076) ([sundyli](https://github.com/sundy-li)) -* Added a setting `deduplicate_blocks_in_dependent_materialized_views`. [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) ([urykhy](https://github.com/urykhy)) -* After recent changes MySQL client started to print binary strings in hex thereby making them not readable ([#9032](https://github.com/ClickHouse/ClickHouse/issues/9032)). The workaround in ClickHouse is to mark string columns as UTF-8, which is not always, but usually the case. [#9079](https://github.com/ClickHouse/ClickHouse/pull/9079) ([Yuriy Baranov](https://github.com/yurriy)) -* Add support of String and FixedString keys for `sumMap` [#8903](https://github.com/ClickHouse/ClickHouse/pull/8903) ([Baudouin Giard](https://github.com/bgiard)) -* Support string keys in SummingMergeTree maps [#8933](https://github.com/ClickHouse/ClickHouse/pull/8933) ([Baudouin Giard](https://github.com/bgiard)) -* Signal termination of thread to the thread pool even if the thread has thrown exception [#8736](https://github.com/ClickHouse/ClickHouse/pull/8736) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* Allow to set `query_id` in `clickhouse-benchmark` [#9416](https://github.com/ClickHouse/ClickHouse/pull/9416) ([Anton Popov](https://github.com/CurtizJ)) -* Don't allow strange expressions in `ALTER TABLE ... PARTITION partition` query. This addresses [#7192](https://github.com/ClickHouse/ClickHouse/issues/7192) [#8835](https://github.com/ClickHouse/ClickHouse/pull/8835) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* The table `system.table_engines` now provides information about feature support (like `supports_ttl` or `supports_sort_order`). [#8830](https://github.com/ClickHouse/ClickHouse/pull/8830) ([Max Akhmedov](https://github.com/zlobober)) -* Enable `system.metric_log` by default. It will contain rows with values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval (one second by default). The table is very small (usually in order of megabytes) and collecting this data by default is reasonable. [#9225](https://github.com/ClickHouse/ClickHouse/pull/9225) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries. Fixes [#6964](https://github.com/ClickHouse/ClickHouse/issues/6964) [#8874](https://github.com/ClickHouse/ClickHouse/pull/8874) ([Ivan](https://github.com/abyss7)) -* Now temporary `LIVE VIEW` is created by `CREATE LIVE VIEW name WITH TIMEOUT [42] ...` instead of `CREATE TEMPORARY LIVE VIEW ...`, because the previous syntax was not consistent with `CREATE TEMPORARY TABLE ...` [#9131](https://github.com/ClickHouse/ClickHouse/pull/9131) ([tavplubix](https://github.com/tavplubix)) -* Add text_log.level configuration parameter to limit entries that goes to `system.text_log` table [#8809](https://github.com/ClickHouse/ClickHouse/pull/8809) ([Azat Khuzhin](https://github.com/azat)) -* Allow to put downloaded part to a disks/volumes according to TTL rules [#8598](https://github.com/ClickHouse/ClickHouse/pull/8598) ([Vladimir Chebotarev](https://github.com/excitoon)) -* For external MySQL dictionaries, allow to mutualize MySQL connection pool to "share" them among dictionaries. This option significantly reduces the number of connections to MySQL servers. [#9409](https://github.com/ClickHouse/ClickHouse/pull/9409) ([Clément Rodriguez](https://github.com/clemrodriguez)) -* Show nearest query execution time for quantiles in `clickhouse-benchmark` output instead of interpolated values. It's better to show values that correspond to the execution time of some queries. [#8712](https://github.com/ClickHouse/ClickHouse/pull/8712) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Possibility to add key & timestamp for the message when inserting data to Kafka. Fixes [#7198](https://github.com/ClickHouse/ClickHouse/issues/7198) [#8969](https://github.com/ClickHouse/ClickHouse/pull/8969) ([filimonov](https://github.com/filimonov)) -* If server is run from terminal, highlight thread number, query id and log priority by colors. This is for improved readability of correlated log messages for developers. [#8961](https://github.com/ClickHouse/ClickHouse/pull/8961) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better exception message while loading tables for `Ordinary` database. [#9527](https://github.com/ClickHouse/ClickHouse/pull/9527) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Implement `arraySlice` for arrays with aggregate function states. This fixes [#9388](https://github.com/ClickHouse/ClickHouse/issues/9388) [#9391](https://github.com/ClickHouse/ClickHouse/pull/9391) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Allow constant functions and constant arrays to be used on the right side of IN operator. [#8813](https://github.com/ClickHouse/ClickHouse/pull/8813) ([Anton Popov](https://github.com/CurtizJ)) -* If zookeeper exception has happened while fetching data for system.replicas, display it in a separate column. This implements [#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Atomically remove MergeTree data parts on destroy. [#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Support row-level security for Distributed tables. [#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([Ivan](https://github.com/abyss7)) -* Now we recognize suffix (like KB, KiB...) in settings values. [#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([Mikhail Korotov](https://github.com/millb)) -* Prevent out of memory while constructing result of a large JOIN. [#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) -* Added names of clusters to suggestions in interactive mode in `clickhouse-client`. [#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries [#8820](https://github.com/ClickHouse/ClickHouse/pull/8820) ([Ivan](https://github.com/abyss7)) -* Added column `exception_code` in `system.query_log` table. [#8770](https://github.com/ClickHouse/ClickHouse/pull/8770) ([Mikhail Korotov](https://github.com/millb)) -* Enabled MySQL compatibility server on port `9004` in the default server configuration file. Fixed password generation command in the example in configuration. [#8771](https://github.com/ClickHouse/ClickHouse/pull/8771) ([Yuriy Baranov](https://github.com/yurriy)) -* Prevent abort on shutdown if the filesystem is readonly. This fixes [#9094](https://github.com/ClickHouse/ClickHouse/issues/9094) [#9100](https://github.com/ClickHouse/ClickHouse/pull/9100) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better exception message when length is required in HTTP POST query. [#9453](https://github.com/ClickHouse/ClickHouse/pull/9453) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add `_path` and `_file` virtual columns to `HDFS` and `File` engines and `hdfs` and `file` table functions [#8489](https://github.com/ClickHouse/ClickHouse/pull/8489) ([Olga Khvostikova](https://github.com/stavrolia)) -* Fix error `Cannot find column` while inserting into `MATERIALIZED VIEW` in case if new column was added to view's internal table. [#8766](https://github.com/ClickHouse/ClickHouse/pull/8766) [#8788](https://github.com/ClickHouse/ClickHouse/pull/8788) ([vzakaznikov](https://github.com/vzakaznikov)) [#8788](https://github.com/ClickHouse/ClickHouse/issues/8788) [#8806](https://github.com/ClickHouse/ClickHouse/pull/8806) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8803](https://github.com/ClickHouse/ClickHouse/pull/8803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix progress over native client-server protocol, by send progress after final update (like logs). This may be relevant only to some third-party tools that are using native protocol. [#9495](https://github.com/ClickHouse/ClickHouse/pull/9495) ([Azat Khuzhin](https://github.com/azat)) -* Add a system metric tracking the number of client connections using MySQL protocol ([#9013](https://github.com/ClickHouse/ClickHouse/issues/9013)). [#9015](https://github.com/ClickHouse/ClickHouse/pull/9015) ([Eugene Klimov](https://github.com/Slach)) -* From now on, HTTP responses will have `X-ClickHouse-Timezone` header set to the same timezone value that `SELECT timezone()` would report. [#9493](https://github.com/ClickHouse/ClickHouse/pull/9493) ([Denis Glazachev](https://github.com/traceon)) - -#### Performance Improvement -* Improve performance of analysing index with IN [#9261](https://github.com/ClickHouse/ClickHouse/pull/9261) ([Anton Popov](https://github.com/CurtizJ)) -* Simpler and more efficient code in Logical Functions + code cleanups. A followup to [#8718](https://github.com/ClickHouse/ClickHouse/issues/8718) [#8728](https://github.com/ClickHouse/ClickHouse/pull/8728) ([Alexander Kazakov](https://github.com/Akazz)) -* Overall performance improvement (in range of 5%..200% for affected queries) by ensuring even more strict aliasing with C++20 features. [#9304](https://github.com/ClickHouse/ClickHouse/pull/9304) ([Amos Bird](https://github.com/amosbird)) -* More strict aliasing for inner loops of comparison functions. [#9327](https://github.com/ClickHouse/ClickHouse/pull/9327) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* More strict aliasing for inner loops of arithmetic functions. [#9325](https://github.com/ClickHouse/ClickHouse/pull/9325) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* A ~3 times faster implementation for ColumnVector::replicate(), via which ColumnConst::convertToFullColumn() is implemented. Also will be useful in tests when materializing constants. [#9293](https://github.com/ClickHouse/ClickHouse/pull/9293) ([Alexander Kazakov](https://github.com/Akazz)) -* Another minor performance improvement to `ColumnVector::replicate()` (this speeds up the `materialize` function and higher order functions) an even further improvement to [#9293](https://github.com/ClickHouse/ClickHouse/issues/9293) [#9442](https://github.com/ClickHouse/ClickHouse/pull/9442) ([Alexander Kazakov](https://github.com/Akazz)) -* Improved performance of `stochasticLinearRegression` aggregate function. This patch is contributed by Intel. [#8652](https://github.com/ClickHouse/ClickHouse/pull/8652) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improve performance of `reinterpretAsFixedString` function. [#9342](https://github.com/ClickHouse/ClickHouse/pull/9342) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Do not send blocks to client for `Null` format in processors pipeline. [#8797](https://github.com/ClickHouse/ClickHouse/pull/8797) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) - -#### Build/Testing/Packaging Improvement -* Exception handling now works correctly on Windows Subsystem for Linux. See https://github.com/ClickHouse-Extras/libunwind/pull/3 This fixes [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) -* Replace `readline` with `replxx` for interactive line editing in `clickhouse-client` [#8416](https://github.com/ClickHouse/ClickHouse/pull/8416) ([Ivan](https://github.com/abyss7)) -* Better build time and less template instantiations in FunctionsComparison. [#9324](https://github.com/ClickHouse/ClickHouse/pull/9324) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added integration with `clang-tidy` in CI. See also [#6044](https://github.com/ClickHouse/ClickHouse/issues/6044) [#9566](https://github.com/ClickHouse/ClickHouse/pull/9566) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now we link ClickHouse in CI using `lld` even for `gcc`. [#9049](https://github.com/ClickHouse/ClickHouse/pull/9049) ([alesapin](https://github.com/alesapin)) -* Allow to randomize thread scheduling and insert glitches when `THREAD_FUZZER_*` environment variables are set. This helps testing. [#9459](https://github.com/ClickHouse/ClickHouse/pull/9459) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enable secure sockets in stateless tests [#9288](https://github.com/ClickHouse/ClickHouse/pull/9288) ([tavplubix](https://github.com/tavplubix)) -* Make SPLIT_SHARED_LIBRARIES=OFF more robust [#9156](https://github.com/ClickHouse/ClickHouse/pull/9156) ([Azat Khuzhin](https://github.com/azat)) -* Make "performance_introspection_and_logging" test reliable to random server stuck. This may happen in CI environment. See also [#9515](https://github.com/ClickHouse/ClickHouse/issues/9515) [#9528](https://github.com/ClickHouse/ClickHouse/pull/9528) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Validate XML in style check. [#9550](https://github.com/ClickHouse/ClickHouse/pull/9550) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed race condition in test `00738_lock_for_inner_table`. This test relied on sleep. [#9555](https://github.com/ClickHouse/ClickHouse/pull/9555) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove performance tests of type `once`. This is needed to run all performance tests in statistical comparison mode (more reliable). [#9557](https://github.com/ClickHouse/ClickHouse/pull/9557) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added performance test for arithmetic functions. [#9326](https://github.com/ClickHouse/ClickHouse/pull/9326) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added performance test for `sumMap` and `sumMapWithOverflow` aggregate functions. Follow-up for [#8933](https://github.com/ClickHouse/ClickHouse/issues/8933) [#8947](https://github.com/ClickHouse/ClickHouse/pull/8947) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Ensure style of ErrorCodes by style check. [#9370](https://github.com/ClickHouse/ClickHouse/pull/9370) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add script for tests history. [#8796](https://github.com/ClickHouse/ClickHouse/pull/8796) ([alesapin](https://github.com/alesapin)) -* Add GCC warning `-Wsuggest-override` to locate and fix all places where `override` keyword must be used. [#8760](https://github.com/ClickHouse/ClickHouse/pull/8760) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* Ignore weak symbol under Mac OS X because it must be defined [#9538](https://github.com/ClickHouse/ClickHouse/pull/9538) ([Deleted user](https://github.com/ghost)) -* Normalize running time of some queries in performance tests. This is done in preparation to run all the performance tests in comparison mode. [#9565](https://github.com/ClickHouse/ClickHouse/pull/9565) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix some tests to support pytest with query tests [#9062](https://github.com/ClickHouse/ClickHouse/pull/9062) ([Ivan](https://github.com/abyss7)) -* Enable SSL in build with MSan, so server will not fail at startup when running stateless tests [#9531](https://github.com/ClickHouse/ClickHouse/pull/9531) ([tavplubix](https://github.com/tavplubix)) -* Fix database substitution in test results [#9384](https://github.com/ClickHouse/ClickHouse/pull/9384) ([Ilya Yatsishin](https://github.com/qoega)) -* Build fixes for miscellaneous platforms [#9381](https://github.com/ClickHouse/ClickHouse/pull/9381) ([proller](https://github.com/proller)) [#8755](https://github.com/ClickHouse/ClickHouse/pull/8755) ([proller](https://github.com/proller)) [#8631](https://github.com/ClickHouse/ClickHouse/pull/8631) ([proller](https://github.com/proller)) -* Added disks section to stateless-with-coverage test docker image [#9213](https://github.com/ClickHouse/ClickHouse/pull/9213) ([Pavel Kovalenko](https://github.com/Jokser)) -* Get rid of in-source-tree files when building with GRPC [#9588](https://github.com/ClickHouse/ClickHouse/pull/9588) ([Amos Bird](https://github.com/amosbird)) -* Slightly faster build time by removing SessionCleaner from Context. Make the code of SessionCleaner more simple. [#9232](https://github.com/ClickHouse/ClickHouse/pull/9232) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Updated checking for hung queries in clickhouse-test script [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz)) -* Removed some useless files from repository. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Changed type of math perftests from `once` to `loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Add docker image which allows to build interactive code browser HTML report for our codebase. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/index.html) -* Suppress some test failures under MSan. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Speedup "exception while insert" test. This test often time out in debug-with-coverage build. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Updated `libcxx` and `libcxxabi` to master. In preparation to [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix flacky test `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Clean up duplicated linker flags. Make sure the linker won't look up an unexpected symbol. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)) -* Add `clickhouse-odbc` driver into test images. This allows to test interaction of ClickHouse with ClickHouse via its own ODBC driver. [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) -* Fix several bugs in unit tests. [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([alesapin](https://github.com/alesapin)) -* Enable `-Wmissing-include-dirs` GCC warning to eliminate all non-existing includes - mostly as a result of CMake scripting errors [#8704](https://github.com/ClickHouse/ClickHouse/pull/8704) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* Describe reasons if query profiler cannot work. This is intended for [#9049](https://github.com/ClickHouse/ClickHouse/issues/9049) [#9144](https://github.com/ClickHouse/ClickHouse/pull/9144) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update OpenSSL to upstream master. Fixed the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` and `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. The issue was present in version 20.1. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update Dockerfile for server [#8893](https://github.com/ClickHouse/ClickHouse/pull/8893) ([Ilya Mazaev](https://github.com/ne-ray)) -* Minor fixes in build-gcc-from-sources script [#8774](https://github.com/ClickHouse/ClickHouse/pull/8774) ([Michael Nacharov](https://github.com/mnach)) -* Replace `numbers` to `zeros` in perftests where `number` column is not used. This will lead to more clean test results. [#9600](https://github.com/ClickHouse/ClickHouse/pull/9600) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix stack overflow issue when using initializer_list in Column constructors. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)) -* Upgrade librdkafka to v1.3.0. Enable bundled `rdkafka` and `gsasl` libraries on Mac OS X. [#9000](https://github.com/ClickHouse/ClickHouse/pull/9000) ([Andrew Onyshchuk](https://github.com/oandrew)) -* build fix on GCC 9.2.0 [#9306](https://github.com/ClickHouse/ClickHouse/pull/9306) ([vxider](https://github.com/Vxider)) - - -## ClickHouse release v20.1 - -### ClickHouse release v20.1.16.120-stable 2020-60-26 - -#### Bug Fix - -* Fix rare crash caused by using `Nullable` column in prewhere condition. Continuation of [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). -* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). -* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). -* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). -* Fix rare crash caused by using `Nullable` column in prewhere condition. (Probably it is connected with [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) somehow). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix wrong exit code of the clickhouse-client, when exception.code() % 256 = 0. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). -* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). -* Fix memory leak when exception is thrown in the middle of aggregation with -State functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix usage of primary key wrapped into a function with 'FINAL' modifier and 'ORDER BY' optimization. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). - - -### ClickHouse release v20.1.15.109-stable 2020-06-19 - -#### Bug Fix - -* Fix excess lock for structure during alter. [#11790](https://github.com/ClickHouse/ClickHouse/pull/11790) ([alesapin](https://github.com/alesapin)). - - -### ClickHouse release v20.1.14.107-stable 2020-06-11 - -#### Bug Fix - -* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). - - -### ClickHouse release v20.1.13.105-stable 2020-06-10 - -#### Bug Fix - -* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). -* Fix pointInPolygon with nan as point. Fixes [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). -* Fixed geohashesInBox with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). -* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix visitParamExtractRaw when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). -* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix crash while reading malformed data in Protobuf format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). -* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). -* 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)). -* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). -* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). -* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). -* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). -* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). -* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* - Make use of `src_type` for correct type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). -* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). -* Fix disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). -* Fix order of parameters in AggregateTransform constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). -* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). -* Fix error `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). -* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). -* * Implemented comparison between DateTime64 and String values (just like for DateTime). [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). - - -### ClickHouse release v20.1.12.86, 2020-05-26 - -#### Bug Fix - -* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). -* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). -* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed the situation when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). -* Fixed overflow at beginning of unix epoch for timezones with fractional offset from UTC. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed improper shutdown of Distributed storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). -* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). -* Fixed removing metadata directory when attach database fails. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)). -* Added a check of number and type of arguments when creating `BloomFilter` index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623). [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)). -* Fixed the issue when a query with `ARRAY JOIN`, `ORDER BY` and `LIMIT` may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`. [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)). -* Fixed wrong flattening of `Array(Tuple(...))` data types. This fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259). [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed wrong behavior in `HashTable` that caused compilation error when trying to read HashMap from buffer. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)). -* Fixed possible `Pipeline stuck` error in `ConcatProcessor` which could have happened in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed several bugs when some data was inserted with quorum, then deleted somehow (DROP PARTITION, TTL) and this leaded to the stuck of INSERTs or false-positive exceptions in SELECTs. This fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946). [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed incompatibility when versions prior to 18.12.17 are used on remote servers and newer is used on initiating server, and GROUP BY both fixed and non-fixed keys, and when two-level group by method is activated. [#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) ([alexey-milovidov](https://github.com/alexey-milovidov)). - -#### Build/Testing/Packaging Improvement - -* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). - - -### ClickHouse release v20.1.10.70, 2020-04-17 - -#### Bug Fix - -* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed bug where ClickHouse would throw `'Unknown function lambda.'` error message when user tries to run `ALTER UPDATE/DELETE` on tables with `ENGINE = Replicated*`. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). -* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix column names of constants inside `JOIN` that may clash with names of constants outside of `JOIN`. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix possible inifinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix move-to-prewhere optimization in presense of `arrayJoin` functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). -* Convert blocks if structure does not match on `INSERT` into table with `Distributed` engine. [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). -* Fix `SIGSEGV` on `INSERT` into `Distributed` table when its structure differs from the underlying tables. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). -* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Add arguments check and support identifier arguments for MySQL Database Engine. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). -* Fix bug in clickhouse dictionary source from localhost clickhouse server. The bug may lead to memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). -* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with different level from different shards (mixed single and two level aggregation). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix a segmentation fault that could occur in `GROUP BY` over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). -* Fix `'Not found column in block'` error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). -* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). -* Fix `'scalar doesn't exist'` error in ALTER queries ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). -* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). -* Fixed `DeleteOnDestroy` logic in `ATTACH PART` which could lead to automatic removal of attached part and added few tests. [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)). - -#### Build/Testing/Packaging Improvement - -* Fix unit test `collapsing_sorted_stream`. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)). - -### ClickHouse release v20.1.9.54, 2020-03-28 - -#### Bug Fix - -* Fix `'Different expressions with the same alias'` error when query has `PREWHERE` and `WHERE` on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). -* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). -* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send `INSERT` queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix `SIGSEGV` with `optimize_skip_unused_shards` when type cannot be converted. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). -* Fixed a few cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)). - -#### Improvement - -* Remove `ORDER BY` stage from mutations because we read from a single ordered part in a single thread. Also add check that the order of rows in mutation is ordered in sorting key order and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)). - -#### Build/Testing/Packaging Improvement - -* Clean up duplicated linker flags. Make sure the linker won't look up an unexpected symbol. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)). - -### ClickHouse release v20.1.8.41, 2020-03-20 - -#### Bug Fix -* Fix possible permanent `Cannot schedule a task` error (due to unhandled exception in `ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread`). This fixes [#6833](https://github.com/ClickHouse/ClickHouse/issues/6833). [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) -* Fix excessive memory consumption in `ALTER` queries (mutations). This fixes [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) -* Fix bug in backquoting in external dictionaries DDL. This fixes [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) - -### ClickHouse release v20.1.7.38, 2020-03-18 - -#### Bug Fix -* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. I lead to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). This issue was in all ClickHouse releases. -* Allow `ALTER ON CLUSTER` of `Distributed` tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). This issue was in all ClickHouse releases. -* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. Fixes [#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) -* Fixed the issue: timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)). This issue was in all ClickHouse releases. -* Now it's not possible to create or add columns with simple cyclic aliases like `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) -* Fixed the issue when padding at the end of base64 encoded value can be malformed. Update base64 library. This fixes [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix data race at destruction of `Poco::HTTPServer`. It could happen when server is started and immediately shut down. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) -* Fix possible crash/wrong number of rows in `LIMIT n WITH TIES` when there are a lot of rows equal to n'th row. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) -* Fix possible mismatched checksums with column TTLs. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) -* Fix crash when a user tries to `ALTER MODIFY SETTING` for old-formated `MergeTree` table engines family. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) -* Now we will try finalize mutations more frequently. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) -* Fix replication protocol incompatibility introduced in [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) -* Fix not(has()) for the bloom_filter index of array types. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) -* Fixed the behaviour of `match` and `extract` functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. This fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Build/Testing/Packaging Improvement - -* Exception handling now works correctly on Windows Subsystem for Linux. See https://github.com/ClickHouse-Extras/libunwind/pull/3 This fixes [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) - - -### ClickHouse release v20.1.6.30, 2020-03-05 - -#### Bug Fix - -* Fix data incompatibility when compressed with `T64` codec. -[#9039](https://github.com/ClickHouse/ClickHouse/pull/9039) [(abyss7)](https://github.com/abyss7) -* Fix order of ranges while reading from MergeTree table in one thread. Fixes [#8964](https://github.com/ClickHouse/ClickHouse/issues/8964). -[#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) [(CurtizJ)](https://github.com/CurtizJ) -* Fix possible segfault in `MergeTreeRangeReader`, while executing `PREWHERE`. Fixes [#9064](https://github.com/ClickHouse/ClickHouse/issues/9064). -[#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) [(CurtizJ)](https://github.com/CurtizJ) -* Fix `reinterpretAsFixedString` to return `FixedString` instead of `String`. -[#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) [(oandrew)](https://github.com/oandrew) -* Fix `joinGet` with nullable return types. Fixes [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) -[#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) [(amosbird)](https://github.com/amosbird) -* Fix fuzz test and incorrect behaviour of bitTestAll/bitTestAny functions. -[#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fix the behaviour of match and extract functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. Fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) -[#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fixed execution of inversed predicates when non-strictly monotinic functional index is used. Fixes [#9034](https://github.com/ClickHouse/ClickHouse/issues/9034) -[#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) [(Akazz)](https://github.com/Akazz) -* Allow to rewrite `CROSS` to `INNER JOIN` if there's `[NOT] LIKE` operator in `WHERE` section. Fixes [#9191](https://github.com/ClickHouse/ClickHouse/issues/9191) -[#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) [(4ertus2)](https://github.com/4ertus2) -* Allow first column(s) in a table with Log engine be an alias. -[#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) [(abyss7)](https://github.com/abyss7) -* Allow comma join with `IN()` inside. Fixes [#7314](https://github.com/ClickHouse/ClickHouse/issues/7314). -[#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) [(4ertus2)](https://github.com/4ertus2) -* Improve `ALTER MODIFY/ADD` queries logic. Now you cannot `ADD` column without type, `MODIFY` default expression doesn't change type of column and `MODIFY` type doesn't loose default expression value. Fixes [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). -[#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) [(alesapin)](https://github.com/alesapin) -* Fix mutations finalization, when already done mutation can have status is_done=0. -[#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) [(alesapin)](https://github.com/alesapin) -* Support "Processors" pipeline for system.numbers and system.numbers_mt. This also fixes the bug when `max_execution_time` is not respected. -[#7796](https://github.com/ClickHouse/ClickHouse/pull/7796) [(KochetovNicolai)](https://github.com/KochetovNicolai) -* Fix wrong counting of `DictCacheKeysRequestedFound` metric. -[#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) [(nikitamikhaylov)](https://github.com/nikitamikhaylov) -* Added a check for storage policy in `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE` which otherwise could make data of part inaccessible after restart and prevent ClickHouse to start. -[#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) [(excitoon)](https://github.com/excitoon) -* Fixed UBSan report in `MergeTreeIndexSet`. This fixes [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) -[#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fix possible datarace in BlockIO. -[#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) [(KochetovNicolai)](https://github.com/KochetovNicolai) -* Support for `UInt64` numbers that don't fit in Int64 in JSON-related functions. Update `SIMDJSON` to master. This fixes [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) -[#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* Fix the issue when the amount of free space is not calculated correctly if the data directory is mounted to a separate device. For default disk calculate the free space from data subdirectory. This fixes [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) -[#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) [(millb)](https://github.com/millb) -* Fix the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error and SSL Exception: error:2400006E:random number generator::error retrieving entropy.` Update OpenSSL to upstream master. -[#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) [(alexey-milovidov)](https://github.com/alexey-milovidov) -* When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). Also fix check for local address in ClickHouseDictionarySource. -[#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) -* Fix segfault in `StorageMerge`, which can happen when reading from StorageFile. -[#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) [(tabplubix)](https://github.com/tavplubix) -* Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Related: [#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) -[#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) -* Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. Fixes [#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporates [#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). -[#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) - -#### New Feature -* Add `deduplicate_blocks_in_dependent_materialized_views` option to control the behaviour of idempotent inserts into tables with materialized views. This new feature was added to the bugfix release by a special request from Altinity. -[#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) - -### ClickHouse release v20.1.2.4, 2020-01-22 - -#### Backward Incompatible Change -* Make the setting `merge_tree_uniform_read_distribution` obsolete. The server still recognizes this setting but it has no effect. [#8308](https://github.com/ClickHouse/ClickHouse/pull/8308) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Changed return type of the function `greatCircleDistance` to `Float32` because now the result of calculation is `Float32`. [#7993](https://github.com/ClickHouse/ClickHouse/pull/7993) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now it's expected that query parameters are represented in "escaped" format. For example, to pass string `ab` you have to write `a\tb` or `a\b` and respectively, `a%5Ctb` or `a%5C%09b` in URL. This is needed to add the possibility to pass NULL as `\N`. This fixes [#7488](https://github.com/ClickHouse/ClickHouse/issues/7488). [#8517](https://github.com/ClickHouse/ClickHouse/pull/8517) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enable `use_minimalistic_part_header_in_zookeeper` setting for `ReplicatedMergeTree` by default. This will significantly reduce amount of data stored in ZooKeeper. This setting is supported since version 19.1 and we already use it in production in multiple services without any issues for more than half a year. Disable this setting if you have a chance to downgrade to versions older than 19.1. [#6850](https://github.com/ClickHouse/ClickHouse/pull/6850) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Data skipping indices are production ready and enabled by default. The settings `allow_experimental_data_skipping_indices`, `allow_experimental_cross_to_join_conversion` and `allow_experimental_multiple_joins_emulation` are now obsolete and do nothing. [#7974](https://github.com/ClickHouse/ClickHouse/pull/7974) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add new `ANY JOIN` logic for `StorageJoin` consistent with `JOIN` operation. To upgrade without changes in behaviour you need add `SETTINGS any_join_distinct_right_table_keys = 1` to Engine Join tables metadata or recreate these tables after upgrade. [#8400](https://github.com/ClickHouse/ClickHouse/pull/8400) ([Artem Zuikov](https://github.com/4ertus2)) -* Require server to be restarted to apply the changes in logging configuration. This is a temporary workaround to avoid the bug where the server logs to a deleted log file (see [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) - -#### New Feature -* Added information about part paths to `system.merges`. [#8043](https://github.com/ClickHouse/ClickHouse/pull/8043) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Add ability to execute `SYSTEM RELOAD DICTIONARY` query in `ON CLUSTER` mode. [#8288](https://github.com/ClickHouse/ClickHouse/pull/8288) ([Guillaume Tassery](https://github.com/YiuRULE)) -* Add ability to execute `CREATE DICTIONARY` queries in `ON CLUSTER` mode. [#8163](https://github.com/ClickHouse/ClickHouse/pull/8163) ([alesapin](https://github.com/alesapin)) -* Now user's profile in `users.xml` can inherit multiple profiles. [#8343](https://github.com/ClickHouse/ClickHouse/pull/8343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) -* Added `system.stack_trace` table that allows to look at stack traces of all server threads. This is useful for developers to introspect server state. This fixes [#7576](https://github.com/ClickHouse/ClickHouse/issues/7576). [#8344](https://github.com/ClickHouse/ClickHouse/pull/8344) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add `DateTime64` datatype with configurable sub-second precision. [#7170](https://github.com/ClickHouse/ClickHouse/pull/7170) ([Vasily Nemkov](https://github.com/Enmk)) -* Add table function `clusterAllReplicas` which allows to query all the nodes in the cluster. [#8493](https://github.com/ClickHouse/ClickHouse/pull/8493) ([kiran sunkari](https://github.com/kiransunkari)) -* Add aggregate function `categoricalInformationValue` which calculates the information value of a discrete feature. [#8117](https://github.com/ClickHouse/ClickHouse/pull/8117) ([hcz](https://github.com/hczhcz)) -* Speed up parsing of data files in `CSV`, `TSV` and `JSONEachRow` format by doing it in parallel. [#7780](https://github.com/ClickHouse/ClickHouse/pull/7780) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Add function `bankerRound` which performs banker's rounding. [#8112](https://github.com/ClickHouse/ClickHouse/pull/8112) ([hcz](https://github.com/hczhcz)) -* Support more languages in embedded dictionary for region names: 'ru', 'en', 'ua', 'uk', 'by', 'kz', 'tr', 'de', 'uz', 'lv', 'lt', 'et', 'pt', 'he', 'vi'. [#8189](https://github.com/ClickHouse/ClickHouse/pull/8189) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improvements in consistency of `ANY JOIN` logic. Now `t1 ANY LEFT JOIN t2` equals `t2 ANY RIGHT JOIN t1`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* Add setting `any_join_distinct_right_table_keys` which enables old behaviour for `ANY INNER JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* Add new `SEMI` and `ANTI JOIN`. Old `ANY INNER JOIN` behaviour now available as `SEMI LEFT JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) -* Added `Distributed` format for `File` engine and `file` table function which allows to read from `.bin` files generated by asynchronous inserts into `Distributed` table. [#8535](https://github.com/ClickHouse/ClickHouse/pull/8535) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Add optional reset column argument for `runningAccumulate` which allows to reset aggregation results for each new key value. [#8326](https://github.com/ClickHouse/ClickHouse/pull/8326) ([Sergey Kononenko](https://github.com/kononencheg)) -* Add ability to use ClickHouse as Prometheus endpoint. [#7900](https://github.com/ClickHouse/ClickHouse/pull/7900) ([vdimir](https://github.com/Vdimir)) -* Add section `` in `config.xml` which restricts allowed hosts for remote table engines and table functions `URL`, `S3`, `HDFS`. [#7154](https://github.com/ClickHouse/ClickHouse/pull/7154) ([Mikhail Korotov](https://github.com/millb)) -* Added function `greatCircleAngle` which calculates the distance on a sphere in degrees. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Changed Earth radius to be consistent with H3 library. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added `JSONCompactEachRow` and `JSONCompactEachRowWithNamesAndTypes` formats for input and output. [#7841](https://github.com/ClickHouse/ClickHouse/pull/7841) ([Mikhail Korotov](https://github.com/millb)) -* Added feature for file-related table engines and table functions (`File`, `S3`, `URL`, `HDFS`) which allows to read and write `gzip` files based on additional engine parameter or file extension. [#7840](https://github.com/ClickHouse/ClickHouse/pull/7840) ([Andrey Bodrov](https://github.com/apbodrov)) -* Added the `randomASCII(length)` function, generating a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. [#8401](https://github.com/ClickHouse/ClickHouse/pull/8401) ([BayoNet](https://github.com/BayoNet)) -* Added function `JSONExtractArrayRaw` which returns an array on unparsed json array elements from `JSON` string. [#8081](https://github.com/ClickHouse/ClickHouse/pull/8081) ([Oleg Matrokhin](https://github.com/errx)) -* Add `arrayZip` function which allows to combine multiple arrays of equal lengths into one array of tuples. [#8149](https://github.com/ClickHouse/ClickHouse/pull/8149) ([Winter Zhang](https://github.com/zhang2014)) -* Add ability to move data between disks according to configured `TTL`-expressions for `*MergeTree` table engines family. [#8140](https://github.com/ClickHouse/ClickHouse/pull/8140) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Added new aggregate function `avgWeighted` which allows to calculate weighted average. [#7898](https://github.com/ClickHouse/ClickHouse/pull/7898) ([Andrey Bodrov](https://github.com/apbodrov)) -* Now parallel parsing is enabled by default for `TSV`, `TSKV`, `CSV` and `JSONEachRow` formats. [#7894](https://github.com/ClickHouse/ClickHouse/pull/7894) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Add several geo functions from `H3` library: `h3GetResolution`, `h3EdgeAngle`, `h3EdgeLength`, `h3IsValid` and `h3kRing`. [#8034](https://github.com/ClickHouse/ClickHouse/pull/8034) ([Konstantin Malanchev](https://github.com/hombit)) -* Added support for brotli (`br`) compression in file-related storages and table functions. This fixes [#8156](https://github.com/ClickHouse/ClickHouse/issues/8156). [#8526](https://github.com/ClickHouse/ClickHouse/pull/8526) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add `groupBit*` functions for the `SimpleAggregationFunction` type. [#8485](https://github.com/ClickHouse/ClickHouse/pull/8485) ([Guillaume Tassery](https://github.com/YiuRULE)) - -#### Bug Fix -* Fix rename of tables with `Distributed` engine. Fixes issue [#7868](https://github.com/ClickHouse/ClickHouse/issues/7868). [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) -* Now dictionaries support `EXPRESSION` for attributes in arbitrary string in non-ClickHouse SQL dialect. [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) -* Fix broken `INSERT SELECT FROM mysql(...)` query. This fixes [#8070](https://github.com/ClickHouse/ClickHouse/issues/8070) and [#7960](https://github.com/ClickHouse/ClickHouse/issues/7960). [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) -* Fix error "Mismatch column sizes" when inserting default `Tuple` from `JSONEachRow`. This fixes [#5653](https://github.com/ClickHouse/ClickHouse/issues/5653). [#8606](https://github.com/ClickHouse/ClickHouse/pull/8606) ([tavplubix](https://github.com/tavplubix)) -* Now an exception will be thrown in case of using `WITH TIES` alongside `LIMIT BY`. Also add ability to use `TOP` with `LIMIT BY`. This fixes [#7472](https://github.com/ClickHouse/ClickHouse/issues/7472). [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix unintendent dependency from fresh glibc version in `clickhouse-odbc-bridge` binary. [#8046](https://github.com/ClickHouse/ClickHouse/pull/8046) ([Amos Bird](https://github.com/amosbird)) -* Fix bug in check function of `*MergeTree` engines family. Now it doesn't fail in case when we have equal amount of rows in last granule and last mark (non-final). [#8047](https://github.com/ClickHouse/ClickHouse/pull/8047) ([alesapin](https://github.com/alesapin)) -* Fix insert into `Enum*` columns after `ALTER` query, when underlying numeric type is equal to table specified type. This fixes [#7836](https://github.com/ClickHouse/ClickHouse/issues/7836). [#7908](https://github.com/ClickHouse/ClickHouse/pull/7908) ([Anton Popov](https://github.com/CurtizJ)) -* Allowed non-constant negative "size" argument for function `substring`. It was not allowed by mistake. This fixes [#4832](https://github.com/ClickHouse/ClickHouse/issues/4832). [#7703](https://github.com/ClickHouse/ClickHouse/pull/7703) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix parsing bug when wrong number of arguments passed to `(O|J)DBC` table engine. [#7709](https://github.com/ClickHouse/ClickHouse/pull/7709) ([alesapin](https://github.com/alesapin)) -* Using command name of the running clickhouse process when sending logs to syslog. In previous versions, empty string was used instead of command name. [#8460](https://github.com/ClickHouse/ClickHouse/pull/8460) ([Michael Nacharov](https://github.com/mnach)) -* Fix check of allowed hosts for `localhost`. This PR fixes the solution provided in [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241). [#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix rare crash in `argMin` and `argMax` functions for long string arguments, when result is used in `runningAccumulate` function. This fixes [#8325](https://github.com/ClickHouse/ClickHouse/issues/8325) [#8341](https://github.com/ClickHouse/ClickHouse/pull/8341) ([dinosaur](https://github.com/769344359)) -* Fix memory overcommit for tables with `Buffer` engine. [#8345](https://github.com/ClickHouse/ClickHouse/pull/8345) ([Azat Khuzhin](https://github.com/azat)) -* Fixed potential bug in functions that can take `NULL` as one of the arguments and return non-NULL. [#8196](https://github.com/ClickHouse/ClickHouse/pull/8196) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better metrics calculations in thread pool for background processes for `MergeTree` table engines. [#8194](https://github.com/ClickHouse/ClickHouse/pull/8194) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix function `IN` inside `WHERE` statement when row-level table filter is present. Fixes [#6687](https://github.com/ClickHouse/ClickHouse/issues/6687) [#8357](https://github.com/ClickHouse/ClickHouse/pull/8357) ([Ivan](https://github.com/abyss7)) -* Now an exception is thrown if the integral value is not parsed completely for settings values. [#7678](https://github.com/ClickHouse/ClickHouse/pull/7678) ([Mikhail Korotov](https://github.com/millb)) -* Fix exception when aggregate function is used in query to distributed table with more than two local shards. [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) -* Now bloom filter can handle zero length arrays and doesn't perform redundant calculations. [#8242](https://github.com/ClickHouse/ClickHouse/pull/8242) ([achimbab](https://github.com/achimbab)) -* Fixed checking if a client host is allowed by matching the client host to `host_regexp` specified in `users.xml`. [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) ([Vitaly Baranov](https://github.com/vitlibar)) -* Relax ambiguous column check that leads to false positives in multiple `JOIN ON` section. [#8385](https://github.com/ClickHouse/ClickHouse/pull/8385) ([Artem Zuikov](https://github.com/4ertus2)) -* Fixed possible server crash (`std::terminate`) when the server cannot send or write data in `JSON` or `XML` format with values of `String` data type (that require `UTF-8` validation) or when compressing result data with Brotli algorithm or in some other rare cases. This fixes [#7603](https://github.com/ClickHouse/ClickHouse/issues/7603) [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix race condition in `StorageDistributedDirectoryMonitor` found by CI. This fixes [#8364](https://github.com/ClickHouse/ClickHouse/issues/8364). [#8383](https://github.com/ClickHouse/ClickHouse/pull/8383) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now background merges in `*MergeTree` table engines family preserve storage policy volume order more accurately. [#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Now table engine `Kafka` works properly with `Native` format. This fixes [#6731](https://github.com/ClickHouse/ClickHouse/issues/6731) [#7337](https://github.com/ClickHouse/ClickHouse/issues/7337) [#8003](https://github.com/ClickHouse/ClickHouse/issues/8003). [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) -* Fixed formats with headers (like `CSVWithNames`) which were throwing exception about EOF for table engine `Kafka`. [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) -* Fixed a bug with making set from subquery in right part of `IN` section. This fixes [#5767](https://github.com/ClickHouse/ClickHouse/issues/5767) and [#2542](https://github.com/ClickHouse/ClickHouse/issues/2542). [#7755](https://github.com/ClickHouse/ClickHouse/pull/7755) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix possible crash while reading from storage `File`. [#7756](https://github.com/ClickHouse/ClickHouse/pull/7756) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed reading of the files in `Parquet` format containing columns of type `list`. [#8334](https://github.com/ClickHouse/ClickHouse/pull/8334) ([maxulan](https://github.com/maxulan)) -* Fix error `Not found column` for distributed queries with `PREWHERE` condition dependent on sampling key if `max_parallel_replicas > 1`. [#7913](https://github.com/ClickHouse/ClickHouse/pull/7913) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix error `Not found column` if query used `PREWHERE` dependent on table's alias and the result set was empty because of primary key condition. [#7911](https://github.com/ClickHouse/ClickHouse/pull/7911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed return type for functions `rand` and `randConstant` in case of `Nullable` argument. Now functions always return `UInt32` and never `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Disabled predicate push-down for `WITH FILL` expression. This fixes [#7784](https://github.com/ClickHouse/ClickHouse/issues/7784). [#7789](https://github.com/ClickHouse/ClickHouse/pull/7789) ([Winter Zhang](https://github.com/zhang2014)) -* Fixed incorrect `count()` result for `SummingMergeTree` when `FINAL` section is used. [#3280](https://github.com/ClickHouse/ClickHouse/issues/3280) [#7786](https://github.com/ClickHouse/ClickHouse/pull/7786) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fix possible incorrect result for constant functions from remote servers. It happened for queries with functions like `version()`, `uptime()`, etc. which returns different constant values for different servers. This fixes [#7666](https://github.com/ClickHouse/ClickHouse/issues/7666). [#7689](https://github.com/ClickHouse/ClickHouse/pull/7689) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix complicated bug in push-down predicate optimization which leads to wrong results. This fixes a lot of issues on push-down predicate optimization. [#8503](https://github.com/ClickHouse/ClickHouse/pull/8503) ([Winter Zhang](https://github.com/zhang2014)) -* Fix crash in `CREATE TABLE .. AS dictionary` query. [#8508](https://github.com/ClickHouse/ClickHouse/pull/8508) ([Azat Khuzhin](https://github.com/azat)) -* Several improvements ClickHouse grammar in `.g4` file. [#8294](https://github.com/ClickHouse/ClickHouse/pull/8294) ([taiyang-li](https://github.com/taiyang-li)) -* Fix bug that leads to crashes in `JOIN`s with tables with engine `Join`. This fixes [#7556](https://github.com/ClickHouse/ClickHouse/issues/7556) [#8254](https://github.com/ClickHouse/ClickHouse/issues/8254) [#7915](https://github.com/ClickHouse/ClickHouse/issues/7915) [#8100](https://github.com/ClickHouse/ClickHouse/issues/8100). [#8298](https://github.com/ClickHouse/ClickHouse/pull/8298) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix redundant dictionaries reload on `CREATE DATABASE`. [#7916](https://github.com/ClickHouse/ClickHouse/pull/7916) ([Azat Khuzhin](https://github.com/azat)) -* Limit maximum number of streams for read from `StorageFile` and `StorageHDFS`. Fixes [#7650](https://github.com/ClickHouse/ClickHouse/issues/7650). [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) -* Fix bug in `ALTER ... MODIFY ... CODEC` query, when user specify both default expression and codec. Fixes [8593](https://github.com/ClickHouse/ClickHouse/issues/8593). [#8614](https://github.com/ClickHouse/ClickHouse/pull/8614) ([alesapin](https://github.com/alesapin)) -* Fix error in background merge of columns with `SimpleAggregateFunction(LowCardinality)` type. [#8613](https://github.com/ClickHouse/ClickHouse/pull/8613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed type check in function `toDateTime64`. [#8375](https://github.com/ClickHouse/ClickHouse/pull/8375) ([Vasily Nemkov](https://github.com/Enmk)) -* Now server do not crash on `LEFT` or `FULL JOIN` with and Join engine and unsupported `join_use_nulls` settings. [#8479](https://github.com/ClickHouse/ClickHouse/pull/8479) ([Artem Zuikov](https://github.com/4ertus2)) -* Now `DROP DICTIONARY IF EXISTS db.dict` query doesn't throw exception if `db` doesn't exist. [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix possible crashes in table functions (`file`, `mysql`, `remote`) caused by usage of reference to removed `IStorage` object. Fix incorrect parsing of columns specified at insertion into table function. [#7762](https://github.com/ClickHouse/ClickHouse/pull/7762) ([tavplubix](https://github.com/tavplubix)) -* Ensure network be up before starting `clickhouse-server`. This fixes [#7507](https://github.com/ClickHouse/ClickHouse/issues/7507). [#8570](https://github.com/ClickHouse/ClickHouse/pull/8570) ([Zhichang Yu](https://github.com/yuzhichang)) -* Fix timeouts handling for secure connections, so queries doesn't hang indefenitely. This fixes [#8126](https://github.com/ClickHouse/ClickHouse/issues/8126). [#8128](https://github.com/ClickHouse/ClickHouse/pull/8128) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix `clickhouse-copier`'s redundant contention between concurrent workers. [#7816](https://github.com/ClickHouse/ClickHouse/pull/7816) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* Now mutations doesn't skip attached parts, even if their mutation version were larger than current mutation version. [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) -* Ignore redundant copies of `*MergeTree` data parts after move to another disk and server restart. [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix crash in `FULL JOIN` with `LowCardinality` in `JOIN` key. [#8252](https://github.com/ClickHouse/ClickHouse/pull/8252) ([Artem Zuikov](https://github.com/4ertus2)) -* Forbidden to use column name more than once in insert query like `INSERT INTO tbl (x, y, x)`. This fixes [#5465](https://github.com/ClickHouse/ClickHouse/issues/5465), [#7681](https://github.com/ClickHouse/ClickHouse/issues/7681). [#7685](https://github.com/ClickHouse/ClickHouse/pull/7685) ([alesapin](https://github.com/alesapin)) -* Added fallback for detection the number of physical CPU cores for unknown CPUs (using the number of logical CPU cores). This fixes [#5239](https://github.com/ClickHouse/ClickHouse/issues/5239). [#7726](https://github.com/ClickHouse/ClickHouse/pull/7726) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix `There's no column` error for materialized and alias columns. [#8210](https://github.com/ClickHouse/ClickHouse/pull/8210) ([Artem Zuikov](https://github.com/4ertus2)) -* Fixed sever crash when `EXISTS` query was used without `TABLE` or `DICTIONARY` qualifier. Just like `EXISTS t`. This fixes [#8172](https://github.com/ClickHouse/ClickHouse/issues/8172). This bug was introduced in version 19.17. [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix rare bug with error `"Sizes of columns doesn't match"` that might appear when using `SimpleAggregateFunction` column. [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) -* Fix bug where user with empty `allow_databases` got access to all databases (and same for `allow_dictionaries`). [#7793](https://github.com/ClickHouse/ClickHouse/pull/7793) ([DeifyTheGod](https://github.com/DeifyTheGod)) -* Fix client crash when server already disconnected from client. [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) -* Fix `ORDER BY` behaviour in case of sorting by primary key prefix and non primary key suffix. [#7759](https://github.com/ClickHouse/ClickHouse/pull/7759) ([Anton Popov](https://github.com/CurtizJ)) -* Check if qualified column present in the table. This fixes [#6836](https://github.com/ClickHouse/ClickHouse/issues/6836). [#7758](https://github.com/ClickHouse/ClickHouse/pull/7758) ([Artem Zuikov](https://github.com/4ertus2)) -* Fixed behavior with `ALTER MOVE` ran immediately after merge finish moves superpart of specified. Fixes [#8103](https://github.com/ClickHouse/ClickHouse/issues/8103). [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix possible server crash while using `UNION` with different number of columns. Fixes [#7279](https://github.com/ClickHouse/ClickHouse/issues/7279). [#7929](https://github.com/ClickHouse/ClickHouse/pull/7929) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix size of result substring for function `substr` with negative size. [#8589](https://github.com/ClickHouse/ClickHouse/pull/8589) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now server does not execute part mutation in `MergeTree` if there are not enough free threads in background pool. [#8588](https://github.com/ClickHouse/ClickHouse/pull/8588) ([tavplubix](https://github.com/tavplubix)) -* Fix a minor typo on formatting `UNION ALL` AST. [#7999](https://github.com/ClickHouse/ClickHouse/pull/7999) ([litao91](https://github.com/litao91)) -* Fixed incorrect bloom filter results for negative numbers. This fixes [#8317](https://github.com/ClickHouse/ClickHouse/issues/8317). [#8566](https://github.com/ClickHouse/ClickHouse/pull/8566) ([Winter Zhang](https://github.com/zhang2014)) -* Fixed potential buffer overflow in decompress. Malicious user can pass fabricated compressed data that will cause read after buffer. This issue was found by Eldar Zaitov from Yandex information security team. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix incorrect result because of integers overflow in `arrayIntersect`. [#7777](https://github.com/ClickHouse/ClickHouse/pull/7777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now `OPTIMIZE TABLE` query will not wait for offline replicas to perform the operation. [#8314](https://github.com/ClickHouse/ClickHouse/pull/8314) ([javi santana](https://github.com/javisantana)) -* Fixed `ALTER TTL` parser for `Replicated*MergeTree` tables. [#8318](https://github.com/ClickHouse/ClickHouse/pull/8318) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix communication between server and client, so server read temporary tables info after query failure. [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) -* Fix `bitmapAnd` function error when intersecting an aggregated bitmap and a scalar bitmap. [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) -* Refine the definition of `ZXid` according to the ZooKeeper Programmer's Guide which fixes bug in `clickhouse-cluster-copier`. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) -* `odbc` table function now respects `external_table_functions_use_nulls` setting. [#7506](https://github.com/ClickHouse/ClickHouse/pull/7506) ([Vasily Nemkov](https://github.com/Enmk)) -* Fixed bug that lead to a rare data race. [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz)) -* Now `SYSTEM RELOAD DICTIONARY` reloads a dictionary completely, ignoring `update_field`. This fixes [#7440](https://github.com/ClickHouse/ClickHouse/issues/7440). [#8037](https://github.com/ClickHouse/ClickHouse/pull/8037) ([Vitaly Baranov](https://github.com/vitlibar)) -* Add ability to check if dictionary exists in create query. [#8032](https://github.com/ClickHouse/ClickHouse/pull/8032) ([alesapin](https://github.com/alesapin)) -* Fix `Float*` parsing in `Values` format. This fixes [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817). [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) -* Fix crash when we cannot reserve space in some background operations of `*MergeTree` table engines family. [#7873](https://github.com/ClickHouse/ClickHouse/pull/7873) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix crash of merge operation when table contains `SimpleAggregateFunction(LowCardinality)` column. This fixes [#8515](https://github.com/ClickHouse/ClickHouse/issues/8515). [#8522](https://github.com/ClickHouse/ClickHouse/pull/8522) ([Azat Khuzhin](https://github.com/azat)) -* Restore support of all ICU locales and add the ability to apply collations for constant expressions. Also add language name to `system.collations` table. [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) -* Fix bug when external dictionaries with zero minimal lifetime (`LIFETIME(MIN 0 MAX N)`, `LIFETIME(N)`) don't update in background. [#7983](https://github.com/ClickHouse/ClickHouse/pull/7983) ([alesapin](https://github.com/alesapin)) -* Fix crash when external dictionary with ClickHouse source has subquery in query. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fix incorrect parsing of file extension in table with engine `URL`. This fixes [#8157](https://github.com/ClickHouse/ClickHouse/issues/8157). [#8419](https://github.com/ClickHouse/ClickHouse/pull/8419) ([Andrey Bodrov](https://github.com/apbodrov)) -* Fix `CHECK TABLE` query for `*MergeTree` tables without key. Fixes [#7543](https://github.com/ClickHouse/ClickHouse/issues/7543). [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) -* Fixed conversion of `Float64` to MySQL type. [#8079](https://github.com/ClickHouse/ClickHouse/pull/8079) ([Yuriy Baranov](https://github.com/yurriy)) -* Now if table was not completely dropped because of server crash, server will try to restore and load it. [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) -* Fixed crash in table function `file` while inserting into file that doesn't exist. Now in this case file would be created and then insert would be processed. [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) -* Fix rare deadlock which can happen when `trace_log` is in enabled. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) -* Add ability to work with different types besides `Date` in `RangeHashed` external dictionary created from DDL query. Fixes [7899](https://github.com/ClickHouse/ClickHouse/issues/7899). [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) -* Fixes crash when `now64()` is called with result of another function. [#8270](https://github.com/ClickHouse/ClickHouse/pull/8270) ([Vasily Nemkov](https://github.com/Enmk)) -* Fixed bug with detecting client IP for connections through mysql wire protocol. [#7743](https://github.com/ClickHouse/ClickHouse/pull/7743) ([Dmitry Muzyka](https://github.com/dmitriy-myz)) -* Fix empty array handling in `arraySplit` function. This fixes [#7708](https://github.com/ClickHouse/ClickHouse/issues/7708). [#7747](https://github.com/ClickHouse/ClickHouse/pull/7747) ([hcz](https://github.com/hczhcz)) -* Fixed the issue when `pid-file` of another running `clickhouse-server` may be deleted. [#8487](https://github.com/ClickHouse/ClickHouse/pull/8487) ([Weiqing Xu](https://github.com/weiqxu)) -* Fix dictionary reload if it has `invalidate_query`, which stopped updates and some exception on previous update tries. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) -* Fixed error in function `arrayReduce` that may lead to "double free" and error in aggregate function combinator `Resample` that may lead to memory leak. Added aggregate function `aggThrow`. This function can be used for testing purposes. [#8446](https://github.com/ClickHouse/ClickHouse/pull/8446) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Improvement -* Improved logging when working with `S3` table engine. [#8251](https://github.com/ClickHouse/ClickHouse/pull/8251) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -* Printed help message when no arguments are passed when calling `clickhouse-local`. This fixes [#5335](https://github.com/ClickHouse/ClickHouse/issues/5335). [#8230](https://github.com/ClickHouse/ClickHouse/pull/8230) ([Andrey Nagorny](https://github.com/Melancholic)) -* Add setting `mutations_sync` which allows to wait `ALTER UPDATE/DELETE` queries synchronously. [#8237](https://github.com/ClickHouse/ClickHouse/pull/8237) ([alesapin](https://github.com/alesapin)) -* Allow to set up relative `user_files_path` in `config.xml` (in the way similar to `format_schema_path`). [#7632](https://github.com/ClickHouse/ClickHouse/pull/7632) ([hcz](https://github.com/hczhcz)) -* Add exception for illegal types for conversion functions with `-OrZero` postfix. [#7880](https://github.com/ClickHouse/ClickHouse/pull/7880) ([Andrey Konyaev](https://github.com/akonyaev90)) -* Simplify format of the header of data sending to a shard in a distributed query. [#8044](https://github.com/ClickHouse/ClickHouse/pull/8044) ([Vitaly Baranov](https://github.com/vitlibar)) -* `Live View` table engine refactoring. [#8519](https://github.com/ClickHouse/ClickHouse/pull/8519) ([vzakaznikov](https://github.com/vzakaznikov)) -* Add additional checks for external dictionaries created from DDL-queries. [#8127](https://github.com/ClickHouse/ClickHouse/pull/8127) ([alesapin](https://github.com/alesapin)) -* Fix error `Column ... already exists` while using `FINAL` and `SAMPLE` together, e.g. `select count() from table final sample 1/2`. Fixes [#5186](https://github.com/ClickHouse/ClickHouse/issues/5186). [#7907](https://github.com/ClickHouse/ClickHouse/pull/7907) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now table the first argument of `joinGet` function can be table identifier. [#7707](https://github.com/ClickHouse/ClickHouse/pull/7707) ([Amos Bird](https://github.com/amosbird)) -* Allow using `MaterializedView` with subqueries above `Kafka` tables. [#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) -* Now background moves between disks run it the seprate thread pool. [#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) -* `SYSTEM RELOAD DICTIONARY` now executes synchronously. [#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([Vitaly Baranov](https://github.com/vitlibar)) -* Stack traces now display physical addresses (offsets in object file) instead of virtual memory addresses (where the object file was loaded). That allows the use of `addr2line` when binary is position independent and ASLR is active. This fixes [#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Support new syntax for row-level security filters: `…
`. Fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([Ivan](https://github.com/abyss7)) -* Now `cityHash` function can work with `Decimal` and `UUID` types. Fixes [#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([Mikhail Korotov](https://github.com/millb)) -* Removed fixed index granularity (it was 1024) from system logs because it's obsolete after implementation of adaptive granularity. [#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enabled MySQL compatibility server when ClickHouse is compiled without SSL. [#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([Yuriy Baranov](https://github.com/yurriy)) -* Now server checksums distributed batches, which gives more verbose errors in case of corrupted data in batch. [#7914](https://github.com/ClickHouse/ClickHouse/pull/7914) ([Azat Khuzhin](https://github.com/azat)) -* Support `DROP DATABASE`, `DETACH TABLE`, `DROP TABLE` and `ATTACH TABLE` for `MySQL` database engine. [#8202](https://github.com/ClickHouse/ClickHouse/pull/8202) ([Winter Zhang](https://github.com/zhang2014)) -* Add authentication in S3 table function and table engine. [#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Added check for extra parts of `MergeTree` at different disks, in order to not allow to miss data parts at undefined disks. [#8118](https://github.com/ClickHouse/ClickHouse/pull/8118) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Enable SSL support for Mac client and server. [#8297](https://github.com/ClickHouse/ClickHouse/pull/8297) ([Ivan](https://github.com/abyss7)) -* Now ClickHouse can work as MySQL federated server (see https://dev.mysql.com/doc/refman/5.7/en/federated-create-server.html). [#7717](https://github.com/ClickHouse/ClickHouse/pull/7717) ([Maxim Fedotov](https://github.com/MaxFedotov)) -* `clickhouse-client` now only enable `bracketed-paste` when multiquery is on and multiline is off. This fixes [#7757](https://github.com/ClickHouse/ClickHouse/issues/7757). [#7761](https://github.com/ClickHouse/ClickHouse/pull/7761) ([Amos Bird](https://github.com/amosbird)) -* Support `Array(Decimal)` in `if` function. [#7721](https://github.com/ClickHouse/ClickHouse/pull/7721) ([Artem Zuikov](https://github.com/4ertus2)) -* Support Decimals in `arrayDifference`, `arrayCumSum` and `arrayCumSumNegative` functions. [#7724](https://github.com/ClickHouse/ClickHouse/pull/7724) ([Artem Zuikov](https://github.com/4ertus2)) -* Added `lifetime` column to `system.dictionaries` table. [#6820](https://github.com/ClickHouse/ClickHouse/issues/6820) [#7727](https://github.com/ClickHouse/ClickHouse/pull/7727) ([kekekekule](https://github.com/kekekekule)) -* Improved check for existing parts on different disks for `*MergeTree` table engines. Addresses [#7660](https://github.com/ClickHouse/ClickHouse/issues/7660). [#8440](https://github.com/ClickHouse/ClickHouse/pull/8440) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Integration with `AWS SDK` for `S3` interactions which allows to use all S3 features out of the box. [#8011](https://github.com/ClickHouse/ClickHouse/pull/8011) ([Pavel Kovalenko](https://github.com/Jokser)) -* Added support for subqueries in `Live View` tables. [#7792](https://github.com/ClickHouse/ClickHouse/pull/7792) ([vzakaznikov](https://github.com/vzakaznikov)) -* Check for using `Date` or `DateTime` column from `TTL` expressions was removed. [#7920](https://github.com/ClickHouse/ClickHouse/pull/7920) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Information about disk was added to `system.detached_parts` table. [#7833](https://github.com/ClickHouse/ClickHouse/pull/7833) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Now settings `max_(table|partition)_size_to_drop` can be changed without a restart. [#7779](https://github.com/ClickHouse/ClickHouse/pull/7779) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) -* Slightly better usability of error messages. Ask user not to remove the lines below `Stack trace:`. [#7897](https://github.com/ClickHouse/ClickHouse/pull/7897) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better reading messages from `Kafka` engine in various formats after [#7935](https://github.com/ClickHouse/ClickHouse/issues/7935). [#8035](https://github.com/ClickHouse/ClickHouse/pull/8035) ([Ivan](https://github.com/abyss7)) -* Better compatibility with MySQL clients which don't support `sha2_password` auth plugin. [#8036](https://github.com/ClickHouse/ClickHouse/pull/8036) ([Yuriy Baranov](https://github.com/yurriy)) -* Support more column types in MySQL compatibility server. [#7975](https://github.com/ClickHouse/ClickHouse/pull/7975) ([Yuriy Baranov](https://github.com/yurriy)) -* Implement `ORDER BY` optimization for `Merge`, `Buffer` and `Materilized View` storages with underlying `MergeTree` tables. [#8130](https://github.com/ClickHouse/ClickHouse/pull/8130) ([Anton Popov](https://github.com/CurtizJ)) -* Now we always use POSIX implementation of `getrandom` to have better compatibility with old kernels (< 3.17). [#7940](https://github.com/ClickHouse/ClickHouse/pull/7940) ([Amos Bird](https://github.com/amosbird)) -* Better check for valid destination in a move TTL rule. [#8410](https://github.com/ClickHouse/ClickHouse/pull/8410) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Better checks for broken insert batches for `Distributed` table engine. [#7933](https://github.com/ClickHouse/ClickHouse/pull/7933) ([Azat Khuzhin](https://github.com/azat)) -* Add column with array of parts name which mutations must process in future to `system.mutations` table. [#8179](https://github.com/ClickHouse/ClickHouse/pull/8179) ([alesapin](https://github.com/alesapin)) -* Parallel merge sort optimization for processors. [#8552](https://github.com/ClickHouse/ClickHouse/pull/8552) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* The settings `mark_cache_min_lifetime` is now obsolete and does nothing. In previous versions, mark cache can grow in memory larger than `mark_cache_size` to accomodate data within `mark_cache_min_lifetime` seconds. That was leading to confusion and higher memory usage than expected, that is especially bad on memory constrained systems. If you will see performance degradation after installing this release, you should increase the `mark_cache_size`. [#8484](https://github.com/ClickHouse/ClickHouse/pull/8484) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Preparation to use `tid` everywhere. This is needed for [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477). [#8276](https://github.com/ClickHouse/ClickHouse/pull/8276) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -#### Performance Improvement -* Performance optimizations in processors pipeline. [#7988](https://github.com/ClickHouse/ClickHouse/pull/7988) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Non-blocking updates of expired keys in cache dictionaries (with permission to read old ones). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Compile ClickHouse without `-fno-omit-frame-pointer` globally to spare one more register. [#8097](https://github.com/ClickHouse/ClickHouse/pull/8097) ([Amos Bird](https://github.com/amosbird)) -* Speedup `greatCircleDistance` function and add performance tests for it. [#7307](https://github.com/ClickHouse/ClickHouse/pull/7307) ([Olga Khvostikova](https://github.com/stavrolia)) -* Improved performance of function `roundDown`. [#8465](https://github.com/ClickHouse/ClickHouse/pull/8465) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improved performance of `max`, `min`, `argMin`, `argMax` for `DateTime64` data type. [#8199](https://github.com/ClickHouse/ClickHouse/pull/8199) ([Vasily Nemkov](https://github.com/Enmk)) -* Improved performance of sorting without a limit or with big limit and external sorting. [#8545](https://github.com/ClickHouse/ClickHouse/pull/8545) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improved performance of formatting floating point numbers up to 6 times. [#8542](https://github.com/ClickHouse/ClickHouse/pull/8542) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improved performance of `modulo` function. [#7750](https://github.com/ClickHouse/ClickHouse/pull/7750) ([Amos Bird](https://github.com/amosbird)) -* Optimized `ORDER BY` and merging with single column key. [#8335](https://github.com/ClickHouse/ClickHouse/pull/8335) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Better implementation for `arrayReduce`, `-Array` and `-State` combinators. [#7710](https://github.com/ClickHouse/ClickHouse/pull/7710) ([Amos Bird](https://github.com/amosbird)) -* Now `PREWHERE` should be optimized to be at least as efficient as `WHERE`. [#7769](https://github.com/ClickHouse/ClickHouse/pull/7769) ([Amos Bird](https://github.com/amosbird)) -* Improve the way `round` and `roundBankers` handling negative numbers. [#8229](https://github.com/ClickHouse/ClickHouse/pull/8229) ([hcz](https://github.com/hczhcz)) -* Improved decoding performance of `DoubleDelta` and `Gorilla` codecs by roughly 30-40%. This fixes [#7082](https://github.com/ClickHouse/ClickHouse/issues/7082). [#8019](https://github.com/ClickHouse/ClickHouse/pull/8019) ([Vasily Nemkov](https://github.com/Enmk)) -* Improved performance of `base64` related functions. [#8444](https://github.com/ClickHouse/ClickHouse/pull/8444) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added a function `geoDistance`. It is similar to `greatCircleDistance` but uses approximation to WGS-84 ellipsoid model. The performance of both functions are near the same. [#8086](https://github.com/ClickHouse/ClickHouse/pull/8086) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Faster `min` and `max` aggregation functions for `Decimal` data type. [#8144](https://github.com/ClickHouse/ClickHouse/pull/8144) ([Artem Zuikov](https://github.com/4ertus2)) -* Vectorize processing `arrayReduce`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) -* `if` chains are now optimized as `multiIf`. [#8355](https://github.com/ClickHouse/ClickHouse/pull/8355) ([kamalov-ruslan](https://github.com/kamalov-ruslan)) -* Fix performance regression of `Kafka` table engine introduced in 19.15. This fixes [#7261](https://github.com/ClickHouse/ClickHouse/issues/7261). [#7935](https://github.com/ClickHouse/ClickHouse/pull/7935) ([filimonov](https://github.com/filimonov)) -* Removed "pie" code generation that `gcc` from Debian packages occasionally brings by default. [#8483](https://github.com/ClickHouse/ClickHouse/pull/8483) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Parallel parsing data formats [#6553](https://github.com/ClickHouse/ClickHouse/pull/6553) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Enable optimized parser of `Values` with expressions by default (`input_format_values_deduce_templates_of_expressions=1`). [#8231](https://github.com/ClickHouse/ClickHouse/pull/8231) ([tavplubix](https://github.com/tavplubix)) - -#### Build/Testing/Packaging Improvement -* Build fixes for `ARM` and in minimal mode. [#8304](https://github.com/ClickHouse/ClickHouse/pull/8304) ([proller](https://github.com/proller)) -* Add coverage file flush for `clickhouse-server` when std::atexit is not called. Also slightly improved logging in stateless tests with coverage. [#8267](https://github.com/ClickHouse/ClickHouse/pull/8267) ([alesapin](https://github.com/alesapin)) -* Update LLVM library in contrib. Avoid using LLVM from OS packages. [#8258](https://github.com/ClickHouse/ClickHouse/pull/8258) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Make bundled `curl` build fully quiet. [#8232](https://github.com/ClickHouse/ClickHouse/pull/8232) [#8203](https://github.com/ClickHouse/ClickHouse/pull/8203) ([Pavel Kovalenko](https://github.com/Jokser)) -* Fix some `MemorySanitizer` warnings. [#8235](https://github.com/ClickHouse/ClickHouse/pull/8235) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Use `add_warning` and `no_warning` macros in `CMakeLists.txt`. [#8604](https://github.com/ClickHouse/ClickHouse/pull/8604) ([Ivan](https://github.com/abyss7)) -* Add support of Minio S3 Compatible object (https://min.io/) for better integration tests. [#7863](https://github.com/ClickHouse/ClickHouse/pull/7863) [#7875](https://github.com/ClickHouse/ClickHouse/pull/7875) ([Pavel Kovalenko](https://github.com/Jokser)) -* Imported `libc` headers to contrib. It allows to make builds more consistent across various systems (only for `x86_64-linux-gnu`). [#5773](https://github.com/ClickHouse/ClickHouse/pull/5773) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Remove `-fPIC` from some libraries. [#8464](https://github.com/ClickHouse/ClickHouse/pull/8464) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Clean `CMakeLists.txt` for curl. See https://github.com/ClickHouse/ClickHouse/pull/8011#issuecomment-569478910 [#8459](https://github.com/ClickHouse/ClickHouse/pull/8459) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Silent warnings in `CapNProto` library. [#8220](https://github.com/ClickHouse/ClickHouse/pull/8220) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Add performance tests for short string optimized hash tables. [#7679](https://github.com/ClickHouse/ClickHouse/pull/7679) ([Amos Bird](https://github.com/amosbird)) -* Now ClickHouse will build on `AArch64` even if `MADV_FREE` is not available. This fixes [#8027](https://github.com/ClickHouse/ClickHouse/issues/8027). [#8243](https://github.com/ClickHouse/ClickHouse/pull/8243) ([Amos Bird](https://github.com/amosbird)) -* Update `zlib-ng` to fix memory sanitizer problems. [#7182](https://github.com/ClickHouse/ClickHouse/pull/7182) [#8206](https://github.com/ClickHouse/ClickHouse/pull/8206) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Enable internal MySQL library on non-Linux system, because usage of OS packages is very fragile and usually doesn't work at all. This fixes [#5765](https://github.com/ClickHouse/ClickHouse/issues/5765). [#8426](https://github.com/ClickHouse/ClickHouse/pull/8426) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed build on some systems after enabling `libc++`. This supersedes [#8374](https://github.com/ClickHouse/ClickHouse/issues/8374). [#8380](https://github.com/ClickHouse/ClickHouse/pull/8380) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Make `Field` methods more type-safe to find more errors. [#7386](https://github.com/ClickHouse/ClickHouse/pull/7386) [#8209](https://github.com/ClickHouse/ClickHouse/pull/8209) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Added missing files to the `libc-headers` submodule. [#8507](https://github.com/ClickHouse/ClickHouse/pull/8507) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix wrong `JSON` quoting in performance test output. [#8497](https://github.com/ClickHouse/ClickHouse/pull/8497) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Now stack trace is displayed for `std::exception` and `Poco::Exception`. In previous versions it was available only for `DB::Exception`. This improves diagnostics. [#8501](https://github.com/ClickHouse/ClickHouse/pull/8501) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Porting `clock_gettime` and `clock_nanosleep` for fresh glibc versions. [#8054](https://github.com/ClickHouse/ClickHouse/pull/8054) ([Amos Bird](https://github.com/amosbird)) -* Enable `part_log` in example config for developers. [#8609](https://github.com/ClickHouse/ClickHouse/pull/8609) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix async nature of reload in `01036_no_superfluous_dict_reload_on_create_database*`. [#8111](https://github.com/ClickHouse/ClickHouse/pull/8111) ([Azat Khuzhin](https://github.com/azat)) -* Fixed codec performance tests. [#8615](https://github.com/ClickHouse/ClickHouse/pull/8615) ([Vasily Nemkov](https://github.com/Enmk)) -* Add install scripts for `.tgz` build and documentation for them. [#8612](https://github.com/ClickHouse/ClickHouse/pull/8612) [#8591](https://github.com/ClickHouse/ClickHouse/pull/8591) ([alesapin](https://github.com/alesapin)) -* Removed old `ZSTD` test (it was created in year 2016 to reproduce the bug that pre 1.0 version of ZSTD has had). This fixes [#8618](https://github.com/ClickHouse/ClickHouse/issues/8618). [#8619](https://github.com/ClickHouse/ClickHouse/pull/8619) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed build on Mac OS Catalina. [#8600](https://github.com/ClickHouse/ClickHouse/pull/8600) ([meo](https://github.com/meob)) -* Increased number of rows in codec performance tests to make results noticeable. [#8574](https://github.com/ClickHouse/ClickHouse/pull/8574) ([Vasily Nemkov](https://github.com/Enmk)) -* In debug builds, treat `LOGICAL_ERROR` exceptions as assertion failures, so that they are easier to notice. [#8475](https://github.com/ClickHouse/ClickHouse/pull/8475) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Make formats-related performance test more deterministic. [#8477](https://github.com/ClickHouse/ClickHouse/pull/8477) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Update `lz4` to fix a MemorySanitizer failure. [#8181](https://github.com/ClickHouse/ClickHouse/pull/8181) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Suppress a known MemorySanitizer false positive in exception handling. [#8182](https://github.com/ClickHouse/ClickHouse/pull/8182) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Update `gcc` and `g++` to version 9 in `build/docker/build.sh` [#7766](https://github.com/ClickHouse/ClickHouse/pull/7766) ([TLightSky](https://github.com/tlightsky)) -* Add performance test case to test that `PREWHERE` is worse than `WHERE`. [#7768](https://github.com/ClickHouse/ClickHouse/pull/7768) ([Amos Bird](https://github.com/amosbird)) -* Progress towards fixing one flacky test. [#8621](https://github.com/ClickHouse/ClickHouse/pull/8621) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Avoid MemorySanitizer report for data from `libunwind`. [#8539](https://github.com/ClickHouse/ClickHouse/pull/8539) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Updated `libc++` to the latest version. [#8324](https://github.com/ClickHouse/ClickHouse/pull/8324) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Build ICU library from sources. This fixes [#6460](https://github.com/ClickHouse/ClickHouse/issues/6460). [#8219](https://github.com/ClickHouse/ClickHouse/pull/8219) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Switched from `libressl` to `openssl`. ClickHouse should support TLS 1.3 and SNI after this change. This fixes [#8171](https://github.com/ClickHouse/ClickHouse/issues/8171). [#8218](https://github.com/ClickHouse/ClickHouse/pull/8218) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed UBSan report when using `chacha20_poly1305` from SSL (happens on connect to https://yandex.ru/). [#8214](https://github.com/ClickHouse/ClickHouse/pull/8214) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix mode of default password file for `.deb` linux distros. [#8075](https://github.com/ClickHouse/ClickHouse/pull/8075) ([proller](https://github.com/proller)) -* Improved expression for getting `clickhouse-server` PID in `clickhouse-test`. [#8063](https://github.com/ClickHouse/ClickHouse/pull/8063) ([Alexander Kazakov](https://github.com/Akazz)) -* Updated contrib/googletest to v1.10.0. [#8587](https://github.com/ClickHouse/ClickHouse/pull/8587) ([Alexander Burmak](https://github.com/Alex-Burmak)) -* Fixed ThreadSaninitizer report in `base64` library. Also updated this library to the latest version, but it doesn't matter. This fixes [#8397](https://github.com/ClickHouse/ClickHouse/issues/8397). [#8403](https://github.com/ClickHouse/ClickHouse/pull/8403) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix `00600_replace_running_query` for processors. [#8272](https://github.com/ClickHouse/ClickHouse/pull/8272) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Remove support for `tcmalloc` to make `CMakeLists.txt` simpler. [#8310](https://github.com/ClickHouse/ClickHouse/pull/8310) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Release gcc builds now use `libc++` instead of `libstdc++`. Recently `libc++` was used only with clang. This will improve consistency of build configurations and portability. [#8311](https://github.com/ClickHouse/ClickHouse/pull/8311) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Enable ICU library for build with MemorySanitizer. [#8222](https://github.com/ClickHouse/ClickHouse/pull/8222) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Suppress warnings from `CapNProto` library. [#8224](https://github.com/ClickHouse/ClickHouse/pull/8224) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Removed special cases of code for `tcmalloc`, because it's no longer supported. [#8225](https://github.com/ClickHouse/ClickHouse/pull/8225) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* In CI coverage task, kill the server gracefully to allow it to save the coverage report. This fixes incomplete coverage reports we've been seeing lately. [#8142](https://github.com/ClickHouse/ClickHouse/pull/8142) ([alesapin](https://github.com/alesapin)) -* Performance tests for all codecs against `Float64` and `UInt64` values. [#8349](https://github.com/ClickHouse/ClickHouse/pull/8349) ([Vasily Nemkov](https://github.com/Enmk)) -* `termcap` is very much deprecated and lead to various problems (f.g. missing "up" cap and echoing `^J` instead of multi line) . Favor `terminfo` or bundled `ncurses`. [#7737](https://github.com/ClickHouse/ClickHouse/pull/7737) ([Amos Bird](https://github.com/amosbird)) -* Fix `test_storage_s3` integration test. [#7734](https://github.com/ClickHouse/ClickHouse/pull/7734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Support `StorageFile(, null) ` to insert block into given format file without actually write to disk. This is required for performance tests. [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) -* Added argument `--print-time` to functional tests which prints execution time per test. [#8001](https://github.com/ClickHouse/ClickHouse/pull/8001) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Added asserts to `KeyCondition` while evaluating RPN. This will fix warning from gcc-9. [#8279](https://github.com/ClickHouse/ClickHouse/pull/8279) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Dump cmake options in CI builds. [#8273](https://github.com/ClickHouse/ClickHouse/pull/8273) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Don't generate debug info for some fat libraries. [#8271](https://github.com/ClickHouse/ClickHouse/pull/8271) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Make `log_to_console.xml` always log to stderr, regardless of is it interactive or not. [#8395](https://github.com/ClickHouse/ClickHouse/pull/8395) ([Alexander Kuzmenkov](https://github.com/akuzm)) -* Removed some unused features from `clickhouse-performance-test` tool. [#8555](https://github.com/ClickHouse/ClickHouse/pull/8555) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Now we will also search for `lld-X` with corresponding `clang-X` version. [#8092](https://github.com/ClickHouse/ClickHouse/pull/8092) ([alesapin](https://github.com/alesapin)) -* Parquet build improvement. [#8421](https://github.com/ClickHouse/ClickHouse/pull/8421) ([maxulan](https://github.com/maxulan)) -* More GCC warnings [#8221](https://github.com/ClickHouse/ClickHouse/pull/8221) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) -* Package for Arch Linux now allows to run ClickHouse server, and not only client. [#8534](https://github.com/ClickHouse/ClickHouse/pull/8534) ([Vladimir Chebotarev](https://github.com/excitoon)) -* Fix test with processors. Tiny performance fixes. [#7672](https://github.com/ClickHouse/ClickHouse/pull/7672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Update contrib/protobuf. [#8256](https://github.com/ClickHouse/ClickHouse/pull/8256) ([Matwey V. Kornilov](https://github.com/matwey)) -* In preparation of switching to c++20 as a new year celebration. "May the C++ force be with ClickHouse." [#8447](https://github.com/ClickHouse/ClickHouse/pull/8447) ([Amos Bird](https://github.com/amosbird)) - -#### Experimental Feature -* Added experimental setting `min_bytes_to_use_mmap_io`. It allows to read big files without copying data from kernel to userspace. The setting is disabled by default. Recommended threshold is about 64 MB, because mmap/munmap is slow. [#8520](https://github.com/ClickHouse/ClickHouse/pull/8520) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Reworked quotas as a part of access control system. Added new table `system.quotas`, new functions `currentQuota`, `currentQuotaKey`, new SQL syntax `CREATE QUOTA`, `ALTER QUOTA`, `DROP QUOTA`, `SHOW QUOTA`. [#7257](https://github.com/ClickHouse/ClickHouse/pull/7257) ([Vitaly Baranov](https://github.com/vitlibar)) -* Allow skipping unknown settings with warnings instead of throwing exceptions. [#7653](https://github.com/ClickHouse/ClickHouse/pull/7653) ([Vitaly Baranov](https://github.com/vitlibar)) -* Reworked row policies as a part of access control system. Added new table `system.row_policies`, new function `currentRowPolicies()`, new SQL syntax `CREATE POLICY`, `ALTER POLICY`, `DROP POLICY`, `SHOW CREATE POLICY`, `SHOW POLICIES`. [#7808](https://github.com/ClickHouse/ClickHouse/pull/7808) ([Vitaly Baranov](https://github.com/vitlibar)) - -#### Security Fix -* Fixed the possibility of reading directories structure in tables with `File` table engine. This fixes [#8536](https://github.com/ClickHouse/ClickHouse/issues/8536). [#8537](https://github.com/ClickHouse/ClickHouse/pull/8537) ([alexey-milovidov](https://github.com/alexey-milovidov)) - -## [Changelog for 2019](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/whats-new/changelog/2019.md) +* Add `SYSTEM SUSPEND` command for fault injection. It can be used to faciliate failover tests. This closes [#15979](https://github.com/ClickHouse/ClickHouse/issues/15979). [#18850](https://github.com/ClickHouse/ClickHouse/pull/18850) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Removed the -finline-hint-functions flag not present in GCC. [#18846](https://github.com/ClickHouse/ClickHouse/pull/18846) ([Mike](https://github.com/myrrc)). +* Add simple integrity check for ClickHouse binary. It allows to detect corruption due to faulty hardware (bit rot on storage media or bit flips in RAM). [#18811](https://github.com/ClickHouse/ClickHouse/pull/18811) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Generate build id when ClickHouse is linked with `lld`. It's appeared that `lld` does not generate it by default on my machine. Build id is used for crash reports and introspection. [#18808](https://github.com/ClickHouse/ClickHouse/pull/18808) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* TestFlows: fixes to LDAP tests that fail due to slow test execution. [#18790](https://github.com/ClickHouse/ClickHouse/pull/18790) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fix shellcheck errors in style check. [#18566](https://github.com/ClickHouse/ClickHouse/pull/18566) ([Ilya Yatsishin](https://github.com/qoega)). +* Update timezones info to 2020e. [#18531](https://github.com/ClickHouse/ClickHouse/pull/18531) ([alesapin](https://github.com/alesapin)). +* Fix codespell warnings. Split style checks into separate parts. Update style checks docker image. [#18463](https://github.com/ClickHouse/ClickHouse/pull/18463) ([Ilya Yatsishin](https://github.com/qoega)). +* Check for leftovers of conflict markers in docs. [#18332](https://github.com/ClickHouse/ClickHouse/pull/18332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Enable Thread Fuzzer for stateless tests flaky check. [#18299](https://github.com/ClickHouse/ClickHouse/pull/18299) ([alesapin](https://github.com/alesapin)). +* Merging requirements for AES encryption functions. Updating aes_encryption tests to use new requirements. Updating TestFlows version to 1.6.72. [#18221](https://github.com/ClickHouse/ClickHouse/pull/18221) ([vzakaznikov](https://github.com/vzakaznikov)). +* - Updating TestFlows version to the latest 1.6.72 - Re-generating requirements.py. [#18208](https://github.com/ClickHouse/ClickHouse/pull/18208) ([vzakaznikov](https://github.com/vzakaznikov)). +* Do not use non thread-safe function `strerror`. [#18204](https://github.com/ClickHouse/ClickHouse/pull/18204) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Update `anchore/scan-action@main` workflow action (was moved from master). [#18192](https://github.com/ClickHouse/ClickHouse/pull/18192) ([Stig Bakken](https://github.com/stigsb)). +* Fix usage of uninitialized value in function toModifiedJulianDayOrNull, reported by MSan. Was discovered [here](https://github.com/ClickHouse/ClickHouse/pull/17726#issuecomment-744050500). [#18172](https://github.com/ClickHouse/ClickHouse/pull/18172) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Change OpenSSL to BoringSSL. It allows to avoid issues with sanitizers. This fixes [#12490](https://github.com/ClickHouse/ClickHouse/issues/12490). This fixes [#17502](https://github.com/ClickHouse/ClickHouse/issues/17502). This fixes [#12952](https://github.com/ClickHouse/ClickHouse/issues/12952). [#18129](https://github.com/ClickHouse/ClickHouse/pull/18129) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now, `clickhouse-test` DROP/CREATE databases with a timeout. [#18098](https://github.com/ClickHouse/ClickHouse/pull/18098) ([alesapin](https://github.com/alesapin)). +* Adjusting timeouts a bit, in the good hope that it will prevent flakiness of the test. [#18000](https://github.com/ClickHouse/ClickHouse/pull/18000) ([filimonov](https://github.com/filimonov)). +* Enable Pytest framework for stateless tests. [#17902](https://github.com/ClickHouse/ClickHouse/pull/17902) ([Ivan](https://github.com/abyss7)). +* Add our own CMakeList for dragonbox which was added in [#17831](https://github.com/ClickHouse/ClickHouse/issues/17831). [#17869](https://github.com/ClickHouse/ClickHouse/pull/17869) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Updating TestFlows README.md to include "How To Debug Why Test Failed" section. [#17808](https://github.com/ClickHouse/ClickHouse/pull/17808) ([vzakaznikov](https://github.com/vzakaznikov)). +* - Testflows tests for RBAC [ACCESS MANAGEMENT](https://clickhouse.tech/docs/en/sql-reference/statements/grant/#grant-access-management) privileges. [#17804](https://github.com/ClickHouse/ClickHouse/pull/17804) ([MyroTk](https://github.com/MyroTk)). +* Now we use the fresh docker daemon version in integration tests. [#17671](https://github.com/ClickHouse/ClickHouse/pull/17671) ([alesapin](https://github.com/alesapin)). +* - RBAC testflows tests for SHOW, TRUNCATE, KILL, and OPTIMIZE. - Updates to old tests. - Resolved comments from #https://github.com/ClickHouse/ClickHouse/pull/16977. [#17657](https://github.com/ClickHouse/ClickHouse/pull/17657) ([MyroTk](https://github.com/MyroTk)). +* Add an integration test: ClickHouse killed while insert for MaterializeMySQL ENGINE. [#17622](https://github.com/ClickHouse/ClickHouse/pull/17622) ([TCeason](https://github.com/TCeason)). +* Add an integration test: MySQL server killed while insert for MaterializeMySQL Engine. [#17614](https://github.com/ClickHouse/ClickHouse/pull/17614) ([TCeason](https://github.com/TCeason)). +* Send info about official build, memory, cpu and free disk space to Sentry if it is enabled. Sentry is opt-in feature to help ClickHouse developers. This closes [#17279](https://github.com/ClickHouse/ClickHouse/issues/17279). [#17543](https://github.com/ClickHouse/ClickHouse/pull/17543) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* There was an uninitialized variable in the code of Copier. [#17363](https://github.com/ClickHouse/ClickHouse/pull/17363) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* `PODArray` does not initialize "new" elements when resizing, unlike `std::vector`. This probably fixes [this failure](https://clickhouse-test-reports.s3.yandex.net/17309/065cd002578f2e8228f12a2744bd40c970065e0c/stress_test_(memory)/stderr.log) from [#17309](https://github.com/ClickHouse/ClickHouse/issues/17309). [#17344](https://github.com/ClickHouse/ClickHouse/pull/17344) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* * Added RBAC tests for `ATTACH`, `CREATE`, `DROP`, and `DETACH`. [#16977](https://github.com/ClickHouse/ClickHouse/pull/16977) ([MyroTk](https://github.com/MyroTk)). +* Now ClickHouse can pretend to be a fake ZooKeeper. Currently, storage implementation is just stored in-memory hash-table, and server partially support ZooKeeper protocol. [#16877](https://github.com/ClickHouse/ClickHouse/pull/16877) ([alesapin](https://github.com/alesapin)). +* Add some test for MaterializeMySQL. e.g. network partition, MySQL kill sync thread... [#16806](https://github.com/ClickHouse/ClickHouse/pull/16806) ([TCeason](https://github.com/TCeason)). +* ... Detailed description / Documentation draft: ClickHouse-Extras repo contains fix for the issue with ipv6 in Arrow Flight library. See https://github.com/ClickHouse/ClickHouse/pull/16243#issuecomment-720830294 for details. [#16664](https://github.com/ClickHouse/ClickHouse/pull/16664) ([Zhanna](https://github.com/FawnD2)). +* Add a library that replaces some `libc` functions to traps that will terminate the process. [#16366](https://github.com/ClickHouse/ClickHouse/pull/16366) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Provide diagnostics in server logs in case of stack overflow, send error message to clickhouse-client. This closes [#14840](https://github.com/ClickHouse/ClickHouse/issues/14840). [#16346](https://github.com/ClickHouse/ClickHouse/pull/16346) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now we can run almost all stateless functional tests in parallel. [#15236](https://github.com/ClickHouse/ClickHouse/pull/15236) ([alesapin](https://github.com/alesapin)). +* If server was terminated by OOM killer, print message in log. [#13516](https://github.com/ClickHouse/ClickHouse/pull/13516) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Revert "Add metrics for part number in MergeTree in ClickHouse"'. [#18834](https://github.com/ClickHouse/ClickHouse/pull/18834) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Fix typo in array functions' documentation'. [#18792](https://github.com/ClickHouse/ClickHouse/pull/18792) ([Bertrand Junqua](https://github.com/Bertrand31)). +* NO CL ENTRY: 'Revert "Add some extra tests to copier"'. [#18636](https://github.com/ClickHouse/ClickHouse/pull/18636) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* NO CL ENTRY: 'Revert "Fix access rights required for the merge() table function."'. [#18103](https://github.com/ClickHouse/ClickHouse/pull/18103) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Исправил опечатку в названии ОС RedHad->RedHat'. [#18028](https://github.com/ClickHouse/ClickHouse/pull/18028) ([Erixonich](https://github.com/Erixonich)). +* NO CL ENTRY: 'Revert "Date vs DateTime64 comparison"'. [#17985](https://github.com/ClickHouse/ClickHouse/pull/17985) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Fix index granularity calculation on block borders"'. [#17918](https://github.com/ClickHouse/ClickHouse/pull/17918) ([alesapin](https://github.com/alesapin)). +* NO CL ENTRY: 'Update README.md'. [#17596](https://github.com/ClickHouse/ClickHouse/pull/17596) ([Robert Hodges](https://github.com/hodgesrm)). +* NO CL ENTRY: 'Revert "Bump mkdocs-macros-plugin from 0.4.20 to 0.5.0 in /docs/tools"'. [#17405](https://github.com/ClickHouse/ClickHouse/pull/17405) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* NO CL ENTRY: 'Revert "Attempt to fix Stress test (MSan)"'. [#17372](https://github.com/ClickHouse/ClickHouse/pull/17372) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). diff --git a/docs/en/whats-new/changelog/2020.md b/docs/en/whats-new/changelog/2020.md new file mode 100644 index 00000000000..2ec48bcd584 --- /dev/null +++ b/docs/en/whats-new/changelog/2020.md @@ -0,0 +1,3376 @@ +### ClickHouse release 20.12 + +### ClickHouse release v20.12.4.5-stable, 2020-12-24 + +#### Bug Fix + +* Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual IPv4/IPv6 stack; - Fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes; Possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). +* Fixed key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). +* Fixed unique key convert crash in `MaterializeMySQL` database engine. This fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) and fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed the issue when some tables not synchronized to ClickHouse from MySQL caused by the fact that convertion MySQL prefix index wasn't supported for MaterializeMySQL. This fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) and fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed the issue when query optimization was producing wrong result if query contains `ARRAY JOIN`. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). +* Fixed possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). + + +### ClickHouse release v20.12.3.3-stable, 2020-12-13 + +#### Backward Incompatible Change + +* Enable `use_compact_format_in_distributed_parts_names` by default (see the documentation for the reference). [#16728](https://github.com/ClickHouse/ClickHouse/pull/16728) ([Azat Khuzhin](https://github.com/azat)). +* Accept user settings related to file formats (e.g. `format_csv_delimiter`) in the `SETTINGS` clause when creating a table that uses `File` engine, and use these settings in all `INSERT`s and `SELECT`s. The file format settings changed in the current user session, or in the `SETTINGS` clause of a DML query itself, no longer affect the query. [#16591](https://github.com/ClickHouse/ClickHouse/pull/16591) ([Alexander Kuzmenkov](https://github.com/akuzm)). + +#### New Feature + +* add `*.xz` compression/decompression support.It enables using `*.xz` in `file()` function. This closes [#8828](https://github.com/ClickHouse/ClickHouse/issues/8828). [#16578](https://github.com/ClickHouse/ClickHouse/pull/16578) ([Abi Palagashvili](https://github.com/fibersel)). +* Introduce the query `ALTER TABLE ... DROP|DETACH PART 'part_name'`. [#15511](https://github.com/ClickHouse/ClickHouse/pull/15511) ([nvartolomei](https://github.com/nvartolomei)). +* Added new ALTER UPDATE/DELETE IN PARTITION syntax. [#13403](https://github.com/ClickHouse/ClickHouse/pull/13403) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Allow formatting named tuples as JSON objects when using JSON input/output formats, controlled by the `output_format_json_named_tuples_as_objects` setting, disabled by default. [#17175](https://github.com/ClickHouse/ClickHouse/pull/17175) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Add a possibility to input enum value as it's id in TSV and CSV formats by default. [#16834](https://github.com/ClickHouse/ClickHouse/pull/16834) ([Kruglov Pavel](https://github.com/Avogar)). +* Add COLLATE support for Nullable, LowCardinality, Array and Tuple, where nested type is String. Also refactor the code associated with collations in ColumnString.cpp. [#16273](https://github.com/ClickHouse/ClickHouse/pull/16273) ([Kruglov Pavel](https://github.com/Avogar)). +* New `tcpPort` function returns TCP port listened by this server. [#17134](https://github.com/ClickHouse/ClickHouse/pull/17134) ([Ivan](https://github.com/abyss7)). +* Add new math functions: `acosh`, `asinh`, `atan2`, `atanh`, `cosh`, `hypot`, `log1p`, `sinh`. [#16636](https://github.com/ClickHouse/ClickHouse/pull/16636) ([Konstantin Malanchev](https://github.com/hombit)). +* Possibility to distribute the merges between different replicas. Introduces the `execute_merges_on_single_replica_time_threshold` mergetree setting. [#16424](https://github.com/ClickHouse/ClickHouse/pull/16424) ([filimonov](https://github.com/filimonov)). +* Add setting `aggregate_functions_null_for_empty` for SQL standard compatibility. This option will rewrite all aggregate functions in a query, adding -OrNull suffix to them. Implements [10273](https://github.com/ClickHouse/ClickHouse/issues/10273). [#16123](https://github.com/ClickHouse/ClickHouse/pull/16123) ([flynn](https://github.com/ucasFL)). +* Updated DateTime, DateTime64 parsing to accept string Date literal format. [#16040](https://github.com/ClickHouse/ClickHouse/pull/16040) ([Maksim Kita](https://github.com/kitaisreal)). +* Make it possible to change the path to history file in `clickhouse-client` using the `--history_file` parameter. [#15960](https://github.com/ClickHouse/ClickHouse/pull/15960) ([Maksim Kita](https://github.com/kitaisreal)). + +#### Bug Fix + +* Fix the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([Amos Bird](https://github.com/amosbird)). +* Fixed `Function not implemented` error when executing `RENAME` query in `Atomic` database with ClickHouse running on Windows Subsystem for Linux. Fixes [#17661](https://github.com/ClickHouse/ClickHouse/issues/17661). [#17664](https://github.com/ClickHouse/ClickHouse/pull/17664) ([tavplubix](https://github.com/tavplubix)). +* Do not restore parts from WAL if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). +* fix incorrect initialization of `max_compress_block_size` of MergeTreeWriterSettings with `min_compress_block_size`. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). +* Exception message about max table size to drop was displayed incorrectly. [#17764](https://github.com/ClickHouse/ClickHouse/pull/17764) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault when there is not enough space when inserting into `Distributed` table. [#17737](https://github.com/ClickHouse/ClickHouse/pull/17737) ([tavplubix](https://github.com/tavplubix)). +* Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). +* In might be determined incorrectly if cluster is circular- (cross-) replicated or not when executing `ON CLUSTER` query due to race condition when `pool_size` > 1. It's fixed. [#17640](https://github.com/ClickHouse/ClickHouse/pull/17640) ([tavplubix](https://github.com/tavplubix)). +* Exception `fmt::v7::format_error` can be logged in background for MergeTree tables. This fixes [#17613](https://github.com/ClickHouse/ClickHouse/issues/17613). [#17615](https://github.com/ClickHouse/ClickHouse/pull/17615) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* When clickhouse-client is used in interactive mode with multiline queries, single line comment was erronously extended till the end of query. This fixes [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix alter query hang when the corresponding mutation was killed on the different replica. Fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). +* Fix issue when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). +* Fix `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). +* Fix duplicates after `DISTINCT` which were possible because of incorrect optimization. Fixes [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash while reading from `JOIN` table with `LowCardinality` types. Fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* fix `toInt256(inf)` stack overflow. Int256 is an experimental feature. Closed [#17235](https://github.com/ClickHouse/ClickHouse/issues/17235). [#17257](https://github.com/ClickHouse/ClickHouse/pull/17257) ([flynn](https://github.com/ucasFL)). +* Fix possible `Unexpected packet Data received from client` error logged for Distributed queries with `LIMIT`. [#17254](https://github.com/ClickHouse/ClickHouse/pull/17254) ([Azat Khuzhin](https://github.com/azat)). +* Fix set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246). [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). +* Fix possible wrong index analysis when the types of the index comparison are different. This fixes [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). +* Fix ColumnConst comparison which leads to crash. This fixed [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). +* Multiple fixed for MaterializeMySQL (experimental feature). Fixes [#16923](https://github.com/ClickHouse/ClickHouse/issues/16923) Fixes [#15883](https://github.com/ClickHouse/ClickHouse/issues/15883) Fix MaterializeMySQL SYNC failure when the modify MySQL binlog_checksum. [#17091](https://github.com/ClickHouse/ClickHouse/pull/17091) ([Winter Zhang](https://github.com/zhang2014)). +* Fix bug when `ON CLUSTER` queries may hang forever for non-leader ReplicatedMergeTreeTables. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). +* Fixed crash on `CREATE TABLE ... AS some_table` query when `some_table` was created `AS table_function()` Fixes [#16944](https://github.com/ClickHouse/ClickHouse/issues/16944). [#17072](https://github.com/ClickHouse/ClickHouse/pull/17072) ([tavplubix](https://github.com/tavplubix)). +* Bug unfinished implementation for funciton fuzzBits, related issue: [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). +* Fix LLVM's libunwind in the case when CFA register is RAX. This is the [bug](https://bugs.llvm.org/show_bug.cgi?id=48186) in [LLVM's libunwind](https://github.com/llvm/llvm-project/tree/master/libunwind). We already have workarounds for this bug. [#17046](https://github.com/ClickHouse/ClickHouse/pull/17046) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). +* Fix `optimize_distributed_group_by_sharding_key` setting (that is disabled by default) for query with OFFSET only. [#16996](https://github.com/ClickHouse/ClickHouse/pull/16996) ([Azat Khuzhin](https://github.com/azat)). +* Fix for Merge tables over Distributed tables with JOIN. [#16993](https://github.com/ClickHouse/ClickHouse/pull/16993) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong result in big integers (128, 256 bit) when casting from double. Big integers support is experimental. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). +* Fix possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). +* Blame info was not calculated correctly in `clickhouse-git-import`. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix order by optimization with monotonous functions. Fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). +* Fix optimization of group by with enabled setting `optimize_aggregators_of_group_by_keys` and joins. Fixes [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible error `Illegal type of argument` for queries with `ORDER BY`. Fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix strange code in InterpreterShowAccessQuery. [#16866](https://github.com/ClickHouse/ClickHouse/pull/16866) ([tavplubix](https://github.com/tavplubix)). +* Prevent clickhouse server crashes when using the function `timeSeriesGroupSum`. The function is removed from newer ClickHouse releases. [#16865](https://github.com/ClickHouse/ClickHouse/pull/16865) ([filimonov](https://github.com/filimonov)). +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash when using `any` without any arguments. This is for [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803) . cc @azat. [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). +* If no memory can be allocated while writing table metadata on disk, broken metadata file can be written. [#16772](https://github.com/ClickHouse/ClickHouse/pull/16772) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix trivial query optimization with partition predicate. [#16767](https://github.com/ClickHouse/ClickHouse/pull/16767) ([Azat Khuzhin](https://github.com/azat)). +* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* Return number of affected rows for INSERT queries via MySQL protocol. Previously ClickHouse used to always return 0, it's fixed. Fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). +* Fix remote query failure when using 'if' suffix aggregate function. Fixes [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) Fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). +* Fix inconsistent behavior caused by `select_sequential_consistency` for optimized trivial count query and system.tables. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). + +#### Improvement + +* Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm. [#16895](https://github.com/ClickHouse/ClickHouse/pull/16895) ([Anton Popov](https://github.com/CurtizJ)). +* Enable compact format of directories for asynchronous sends in Distributed tables: `use_compact_format_in_distributed_parts_names` is set to 1 by default. [#16788](https://github.com/ClickHouse/ClickHouse/pull/16788) ([Azat Khuzhin](https://github.com/azat)). +* Abort multipart upload if no data was written to S3. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). +* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). +* Mask password in data_path in the system.distribution_queue. [#16727](https://github.com/ClickHouse/ClickHouse/pull/16727) ([Azat Khuzhin](https://github.com/azat)). +* Throw error when use column transformer replaces non existing column. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). +* Turn off parallel parsing when there is no enough memory for all threads to work simultaneously. Also there could be exceptions like "Memory limit exceeded" when somebody will try to insert extremely huge rows (> min_chunk_bytes_for_parallel_parsing), because each piece to parse has to be independent set of strings (one or more). [#16721](https://github.com/ClickHouse/ClickHouse/pull/16721) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). +* Correct grammar in error message in JSONEachRow, JSONCompactEachRow, and RegexpRow input formats. [#17205](https://github.com/ClickHouse/ClickHouse/pull/17205) ([nico piderman](https://github.com/sneako)). +* Set default `host` and `port` parameters for `SOURCE(CLICKHOUSE(...))` to current instance and set default `user` value to `'default'`. [#16997](https://github.com/ClickHouse/ClickHouse/pull/16997) ([vdimir](https://github.com/vdimir)). +* Throw an informative error message when doing `ATTACH/DETACH TABLE `. Before this PR, `detach table ` works but leads to an ill-formed in-memory metadata. [#16885](https://github.com/ClickHouse/ClickHouse/pull/16885) ([Amos Bird](https://github.com/amosbird)). +* Add cutToFirstSignificantSubdomainWithWWW(). [#16845](https://github.com/ClickHouse/ClickHouse/pull/16845) ([Azat Khuzhin](https://github.com/azat)). +* Server refused to startup with exception message if wrong config is given (`metric_log`.`collect_interval_milliseconds` is missing). [#16815](https://github.com/ClickHouse/ClickHouse/pull/16815) ([Ivan](https://github.com/abyss7)). +* Better exception message when configuration for distributed DDL is absent. This fixes [#5075](https://github.com/ClickHouse/ClickHouse/issues/5075). [#16769](https://github.com/ClickHouse/ClickHouse/pull/16769) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Usability improvement: better suggestions in syntax error message when `CODEC` expression is misplaced in `CREATE TABLE` query. This fixes [#12493](https://github.com/ClickHouse/ClickHouse/issues/12493). [#16768](https://github.com/ClickHouse/ClickHouse/pull/16768) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove empty directories for async INSERT at start of Distributed engine. [#16729](https://github.com/ClickHouse/ClickHouse/pull/16729) ([Azat Khuzhin](https://github.com/azat)). +* Workaround for use S3 with nginx server as proxy. Nginx currenty does not accept urls with empty path like `http://domain.com?delete`, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like `http://domain.com/?delete`. [#16709](https://github.com/ClickHouse/ClickHouse/pull/16709) ([ianton-ru](https://github.com/ianton-ru)). +* Allow `reinterpretAs*` functions to work for integers and floats of the same size. Implements [16640](https://github.com/ClickHouse/ClickHouse/issues/16640). [#16657](https://github.com/ClickHouse/ClickHouse/pull/16657) ([flynn](https://github.com/ucasFL)). +* Now, `` configuration can be changed in `config.xml` and reloaded without server startup. [#16627](https://github.com/ClickHouse/ClickHouse/pull/16627) ([Amos Bird](https://github.com/amosbird)). +* Support SNI in https connections to remote resources. This will allow to connect to Cloudflare servers that require SNI. This fixes [#10055](https://github.com/ClickHouse/ClickHouse/issues/10055). [#16252](https://github.com/ClickHouse/ClickHouse/pull/16252) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make it possible to connect to `clickhouse-server` secure endpoint which requires SNI. This is possible when `clickhouse-server` is hosted behind TLS proxy. [#16938](https://github.com/ClickHouse/ClickHouse/pull/16938) ([filimonov](https://github.com/filimonov)). +* Fix possible stack overflow if a loop of materialized views is created. This closes [#15732](https://github.com/ClickHouse/ClickHouse/issues/15732). [#16048](https://github.com/ClickHouse/ClickHouse/pull/16048) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Simplify the implementation of background tasks processing for the MergeTree table engines family. There should be no visible changes for user. [#15983](https://github.com/ClickHouse/ClickHouse/pull/15983) ([alesapin](https://github.com/alesapin)). +* Improvement for MaterializeMySQL (experimental feature). Throw exception about right sync privileges when MySQL sync user has error privileges. [#15977](https://github.com/ClickHouse/ClickHouse/pull/15977) ([TCeason](https://github.com/TCeason)). +* Made `indexOf()` use BloomFilter. [#14977](https://github.com/ClickHouse/ClickHouse/pull/14977) ([achimbab](https://github.com/achimbab)). + +#### Performance Improvement + +* Use Floyd-Rivest algorithm, it is the best for the ClickHouse use case of partial sorting. Bechmarks are in https://github.com/danlark1/miniselect and [here](https://drive.google.com/drive/folders/1DHEaeXgZuX6AJ9eByeZ8iQVQv0ueP8XM). [#16825](https://github.com/ClickHouse/ClickHouse/pull/16825) ([Danila Kutenin](https://github.com/danlark1)). +* Now `ReplicatedMergeTree` tree engines family uses a separate thread pool for replicated fetches. Size of the pool limited by setting `background_fetches_pool_size` which can be tuned with a server restart. The default value of the setting is 3 and it means that the maximum amount of parallel fetches is equal to 3 (and it allows to utilize 10G network). Fixes #520. [#16390](https://github.com/ClickHouse/ClickHouse/pull/16390) ([alesapin](https://github.com/alesapin)). +* Fixed uncontrolled growth of the state of `quantileTDigest`. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). +* Add `VIEW` subquery description to `EXPLAIN`. Limit push down optimisation for `VIEW`. Add local replicas of `Distributed` to query plan. [#14936](https://github.com/ClickHouse/ClickHouse/pull/14936) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix optimize_read_in_order/optimize_aggregation_in_order with max_threads > 0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). +* Fix performance of reading from `Merge` tables over huge number of `MergeTree` tables. Fixes [#7748](https://github.com/ClickHouse/ClickHouse/issues/7748). [#16988](https://github.com/ClickHouse/ClickHouse/pull/16988) ([Anton Popov](https://github.com/CurtizJ)). +* Now we can safely prune partitions with exact match. Useful case: Suppose table is partitioned by `intHash64(x) % 100` and the query has condition on `intHash64(x) % 100` verbatim, not on x. [#16253](https://github.com/ClickHouse/ClickHouse/pull/16253) ([Amos Bird](https://github.com/amosbird)). + +#### Experimental Feature + +* Add `EmbeddedRocksDB` table engine (can be used for dictionaries). [#15073](https://github.com/ClickHouse/ClickHouse/pull/15073) ([sundyli](https://github.com/sundy-li)). + +#### Build/Testing/Packaging Improvement + +* Improvements in test coverage building images. [#17233](https://github.com/ClickHouse/ClickHouse/pull/17233) ([alesapin](https://github.com/alesapin)). +* Update embedded timezone data to version 2020d (also update cctz to the latest master). [#17204](https://github.com/ClickHouse/ClickHouse/pull/17204) ([filimonov](https://github.com/filimonov)). +* Fix UBSan report in Poco. This closes [#12719](https://github.com/ClickHouse/ClickHouse/issues/12719). [#16765](https://github.com/ClickHouse/ClickHouse/pull/16765) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Do not instrument 3rd-party libraries with UBSan. [#16764](https://github.com/ClickHouse/ClickHouse/pull/16764) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan report in cache dictionaries. This closes [#12641](https://github.com/ClickHouse/ClickHouse/issues/12641). [#16763](https://github.com/ClickHouse/ClickHouse/pull/16763) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan report when trying to convert infinite floating point number to integer. This closes [#14190](https://github.com/ClickHouse/ClickHouse/issues/14190). [#16677](https://github.com/ClickHouse/ClickHouse/pull/16677) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + +## ClickHouse release 20.11 + +### ClickHouse release v20.11.6.6-stable, 2020-12-24 + +#### Bug Fix + +* Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual `IPv4/IPv6 stack` and fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes. This possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). +* Fixed key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). +* Fixed unique key convert crash in `MaterializeMySQL` database engine. This fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) and fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed the issue when some tables not synchronized to ClickHouse from MySQL caused by the fact that convertion MySQL prefix index wasn't supported for MaterializeMySQL. This fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) and fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed the issue when query optimization was producing wrong result if query contains `ARRAY JOIN`. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). +* Fix possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). +* Do not restore parts from WAL if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). +* Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). +* Fixed inconsistent behaviour of `optimize_trivial_count_query` with partition predicate. [#17644](https://github.com/ClickHouse/ClickHouse/pull/17644) ([Azat Khuzhin](https://github.com/azat)). +* Fixed empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). +* Fixed the behaviour when xxception `fmt::v7::format_error` can be logged in background for MergeTree tables. This fixes [#17613](https://github.com/ClickHouse/ClickHouse/issues/17613). [#17615](https://github.com/ClickHouse/ClickHouse/pull/17615) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behaviour when clickhouse-client is used in interactive mode with multiline queries and single line comment was erronously extended till the end of query. This fixes [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed alter query hang when the corresponding mutation was killed on the different replica. This fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). +* Fixed bug when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). +* Fixed `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed duplicates after `DISTINCT` which were possible because of incorrect optimization. This fixes [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed crash while reading from `JOIN` table with `LowCardinality` types. This fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246) . [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). +* Fixed possible wrong index analysis when the types of the index comparison are different. This fixes [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). +* Fixed `ColumnConst` comparison which leads to crash. This fixes [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). +* Fixed bug when `ON CLUSTER` queries may hang forever for non-leader `ReplicatedMergeTreeTables`. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). +* Fixed fuzzer-found bug in funciton `fuzzBits`. This fixes [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). +* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong result in big integers (128, 256 bit) when casting from double. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). +* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). +* Fixed possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). +* Blame info was not calculated correctly in `clickhouse-git-import`. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed order by optimization with monotonous functions. Fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed optimization of group by with enabled setting `optimize_aggregators_of_group_by_keys` and joins. This fixes [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). +* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). +* Fixed possible error `Illegal type of argument` for queries with `ORDER BY`. This fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Abort multipart upload if no data was written to WriteBufferFromS3. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixed crash when using `any` without any arguments. This fixes [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). +* Fixed the behaviour when ClickHouse used to always return 0 insted of a number of affected rows for `INSERT` queries via MySQL protocol. This fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed uncontrolled growth of TDigest. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). +* Fixed remote query failure when using suffix `if` in Aggregate function. This fixes [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed inconsistent behavior caused by `select_sequential_consistency` for optimized trivial count query and system.tables. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). +* Throw error when use ColumnTransformer replace non exist column. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). + + +### ClickHouse release v20.11.3.3-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + +### ClickHouse release v20.11.2.1, 2020-11-11 + +#### Backward Incompatible Change + +* If some `profile` was specified in `distributed_ddl` config section, then this profile could overwrite settings of `default` profile on server startup. It's fixed, now settings of distributed DDL queries should not affect global server settings. [#16635](https://github.com/ClickHouse/ClickHouse/pull/16635) ([tavplubix](https://github.com/tavplubix)). +* Restrict to use of non-comparable data types (like `AggregateFunction`) in keys (Sorting key, Primary key, Partition key, and so on). [#16601](https://github.com/ClickHouse/ClickHouse/pull/16601) ([alesapin](https://github.com/alesapin)). +* Remove `ANALYZE` and `AST` queries, and make the setting `enable_debug_queries` obsolete since now it is the part of full featured `EXPLAIN` query. [#16536](https://github.com/ClickHouse/ClickHouse/pull/16536) ([Ivan](https://github.com/abyss7)). +* Aggregate functions `boundingRatio`, `rankCorr`, `retention`, `timeSeriesGroupSum`, `timeSeriesGroupRateSum`, `windowFunnel` were erroneously made case-insensitive. Now their names are made case sensitive as designed. Only functions that are specified in SQL standard or made for compatibility with other DBMS or functions similar to those should be case-insensitive. [#16407](https://github.com/ClickHouse/ClickHouse/pull/16407) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make `rankCorr` function return nan on insufficient data [#16124](https://github.com/ClickHouse/ClickHouse/issues/16124). [#16135](https://github.com/ClickHouse/ClickHouse/pull/16135) ([hexiaoting](https://github.com/hexiaoting)). +* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). + +#### New Feature + +* Added support of LDAP as a user directory for locally non-existent users. [#12736](https://github.com/ClickHouse/ClickHouse/pull/12736) ([Denis Glazachev](https://github.com/traceon)). +* Add `system.replicated_fetches` table which shows currently running background fetches. [#16428](https://github.com/ClickHouse/ClickHouse/pull/16428) ([alesapin](https://github.com/alesapin)). +* Added setting `date_time_output_format`. [#15845](https://github.com/ClickHouse/ClickHouse/pull/15845) ([Maksim Kita](https://github.com/kitaisreal)). +* Added minimal web UI to ClickHouse. [#16158](https://github.com/ClickHouse/ClickHouse/pull/16158) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allows to read/write Single protobuf message at once (w/o length-delimiters). [#15199](https://github.com/ClickHouse/ClickHouse/pull/15199) ([filimonov](https://github.com/filimonov)). +* Added initial OpenTelemetry support. ClickHouse now accepts OpenTelemetry traceparent headers over Native and HTTP protocols, and passes them downstream in some cases. The trace spans for executed queries are saved into the `system.opentelemetry_span_log` table. [#14195](https://github.com/ClickHouse/ClickHouse/pull/14195) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Allow specify primary key in column list of `CREATE TABLE` query. This is needed for compatibility with other SQL dialects. [#15823](https://github.com/ClickHouse/ClickHouse/pull/15823) ([Maksim Kita](https://github.com/kitaisreal)). +* Implement `OFFSET offset_row_count {ROW | ROWS} FETCH {FIRST | NEXT} fetch_row_count {ROW | ROWS} {ONLY | WITH TIES}` in SELECT query with ORDER BY. This is the SQL-standard way to specify `LIMIT`. [#15855](https://github.com/ClickHouse/ClickHouse/pull/15855) ([hexiaoting](https://github.com/hexiaoting)). +* `errorCodeToName` function - return variable name of the error (useful for analyzing query_log and similar). `system.errors` table - shows how many times errors has been happened (respects `system_events_show_zero_values`). [#16438](https://github.com/ClickHouse/ClickHouse/pull/16438) ([Azat Khuzhin](https://github.com/azat)). +* Added function `untuple` which is a special function which can introduce new columns to the SELECT list by expanding a named tuple. [#16242](https://github.com/ClickHouse/ClickHouse/pull/16242) ([Nikolai Kochetov](https://github.com/KochetovNicolai), [Amos Bird](https://github.com/amosbird)). +* Now we can provide identifiers via query parameters. And these parameters can be used as table objects or columns. [#16594](https://github.com/ClickHouse/ClickHouse/pull/16594) ([Amos Bird](https://github.com/amosbird)). +* Added big integers (UInt256, Int128, Int256) and UUID data types support for MergeTree BloomFilter index. Big integers is an experimental feature. [#16642](https://github.com/ClickHouse/ClickHouse/pull/16642) ([Maksim Kita](https://github.com/kitaisreal)). +* Add `farmFingerprint64` function (non-cryptographic string hashing). [#16570](https://github.com/ClickHouse/ClickHouse/pull/16570) ([Jacob Hayes](https://github.com/JacobHayes)). +* Add `log_queries_min_query_duration_ms`, only queries slower than the value of this setting will go to `query_log`/`query_thread_log` (i.e. something like `slow_query_log` in mysql). [#16529](https://github.com/ClickHouse/ClickHouse/pull/16529) ([Azat Khuzhin](https://github.com/azat)). +* Ability to create a docker image on the top of `Alpine`. Uses precompiled binary and glibc components from ubuntu 20.04. [#16479](https://github.com/ClickHouse/ClickHouse/pull/16479) ([filimonov](https://github.com/filimonov)). +* Added `toUUIDOrNull`, `toUUIDOrZero` cast functions. [#16337](https://github.com/ClickHouse/ClickHouse/pull/16337) ([Maksim Kita](https://github.com/kitaisreal)). +* Add `max_concurrent_queries_for_all_users` setting, see [#6636](https://github.com/ClickHouse/ClickHouse/issues/6636) for use cases. [#16154](https://github.com/ClickHouse/ClickHouse/pull/16154) ([nvartolomei](https://github.com/nvartolomei)). +* Add a new option `print_query_id` to clickhouse-client. It helps generate arbitrary strings with the current query id generated by the client. Also print query id in clickhouse-client by default. [#15809](https://github.com/ClickHouse/ClickHouse/pull/15809) ([Amos Bird](https://github.com/amosbird)). +* Add `tid` and `logTrace` functions. This closes [#9434](https://github.com/ClickHouse/ClickHouse/issues/9434). [#15803](https://github.com/ClickHouse/ClickHouse/pull/15803) ([flynn](https://github.com/ucasFL)). +* Add function `formatReadableTimeDelta` that format time delta to human readable string ... [#15497](https://github.com/ClickHouse/ClickHouse/pull/15497) ([Filipe Caixeta](https://github.com/filipecaixeta)). +* Added `disable_merges` option for volumes in multi-disk configuration. [#13956](https://github.com/ClickHouse/ClickHouse/pull/13956) ([Vladimir Chebotarev](https://github.com/excitoon)). + +#### Experimental Feature + +* New functions `encrypt`, `aes_encrypt_mysql`, `decrypt`, `aes_decrypt_mysql`. These functions are working slowly, so we consider it as an experimental feature. [#11844](https://github.com/ClickHouse/ClickHouse/pull/11844) ([Vasily Nemkov](https://github.com/Enmk)). + +#### Bug Fix + +* Mask password in data_path in the `system.distribution_queue`. [#16727](https://github.com/ClickHouse/ClickHouse/pull/16727) ([Azat Khuzhin](https://github.com/azat)). +* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* The setting `max_parallel_replicas` worked incorrectly if the queried table has no sampling. This fixes [#5733](https://github.com/ClickHouse/ClickHouse/issues/5733). [#16675](https://github.com/ClickHouse/ClickHouse/pull/16675) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix optimize_read_in_order/optimize_aggregation_in_order with max_threads > 0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). +* Calculation of `DEFAULT` expressions was involving possible name collisions (that was very unlikely to encounter). This fixes [#9359](https://github.com/ClickHouse/ClickHouse/issues/9359). [#16612](https://github.com/ClickHouse/ClickHouse/pull/16612) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `query_thread_log.query_duration_ms` unit. [#16563](https://github.com/ClickHouse/ClickHouse/pull/16563) ([Azat Khuzhin](https://github.com/azat)). +* Fix a bug when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine. `MaterializeMySQL` is an experimental feature. [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Specifically crafted argument of `round` function with `Decimal` was leading to integer division by zero. This fixes [#13338](https://github.com/ClickHouse/ClickHouse/issues/13338). [#16451](https://github.com/ClickHouse/ClickHouse/pull/16451) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix dictGet in sharding_key (and similar places, i.e. when the function context is stored permanently). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). +* Fix the exception thrown in `clickhouse-local` when trying to execute `OPTIMIZE` command. Fixes [#16076](https://github.com/ClickHouse/ClickHouse/issues/16076). [#16192](https://github.com/ClickHouse/ClickHouse/pull/16192) ([filimonov](https://github.com/filimonov)). +* Fixes [#15780](https://github.com/ClickHouse/ClickHouse/issues/15780) regression, e.g. `indexOf([1, 2, 3], toLowCardinality(1))` now is prohibited but it should not be. [#16038](https://github.com/ClickHouse/ClickHouse/pull/16038) ([Mike](https://github.com/myrrc)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). +* Now exception will be thrown when `ALTER MODIFY COLUMN ... DEFAULT ...` has incompatible default with column type. Fixes [#15854](https://github.com/ClickHouse/ClickHouse/issues/15854). [#15858](https://github.com/ClickHouse/ClickHouse/pull/15858) ([alesapin](https://github.com/alesapin)). +* Fixed IPv4CIDRToRange/IPv6CIDRToRange functions to accept const IP-column values. [#15856](https://github.com/ClickHouse/ClickHouse/pull/15856) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). + +#### Improvement + +* Treat `INTERVAL '1 hour'` as equivalent to `INTERVAL 1 HOUR`, to be compatible with Postgres and similar. This fixes [#15637](https://github.com/ClickHouse/ClickHouse/issues/15637). [#15978](https://github.com/ClickHouse/ClickHouse/pull/15978) ([flynn](https://github.com/ucasFL)). +* Enable parsing enum values by their numeric ids for CSV, TSV and JSON input formats. [#15685](https://github.com/ClickHouse/ClickHouse/pull/15685) ([vivarum](https://github.com/vivarum)). +* Better read task scheduling for JBOD architecture and `MergeTree` storage. New setting `read_backoff_min_concurrency` which serves as the lower limit to the number of reading threads. [#16423](https://github.com/ClickHouse/ClickHouse/pull/16423) ([Amos Bird](https://github.com/amosbird)). +* Add missing support for `LowCardinality` in `Avro` format. [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Workaround for use `S3` with nginx server as proxy. Nginx currenty does not accept urls with empty path like `http://domain.com?delete`, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like `http://domain.com/?delete`. [#16814](https://github.com/ClickHouse/ClickHouse/pull/16814) ([ianton-ru](https://github.com/ianton-ru)). +* Better diagnostics on parse errors in input data. Provide row number on `Cannot read all data` errors. [#16644](https://github.com/ClickHouse/ClickHouse/pull/16644) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make the behaviour of `minMap` and `maxMap` more desireable. It will not skip zero values in the result. Fixes [#16087](https://github.com/ClickHouse/ClickHouse/issues/16087). [#16631](https://github.com/ClickHouse/ClickHouse/pull/16631) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Better update of ZooKeeper configuration in runtime. [#16630](https://github.com/ClickHouse/ClickHouse/pull/16630) ([sundyli](https://github.com/sundy-li)). +* Apply SETTINGS clause as early as possible. It allows to modify more settings in the query. This closes [#3178](https://github.com/ClickHouse/ClickHouse/issues/3178). [#16619](https://github.com/ClickHouse/ClickHouse/pull/16619) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now `event_time_microseconds` field stores in Decimal64, not UInt64. [#16617](https://github.com/ClickHouse/ClickHouse/pull/16617) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Now paratmeterized functions can be used in `APPLY` column transformer. [#16589](https://github.com/ClickHouse/ClickHouse/pull/16589) ([Amos Bird](https://github.com/amosbird)). +* Improve scheduling of background task which removes data of dropped tables in `Atomic` databases. `Atomic` databases do not create broken symlink to table data directory if table actually has no data directory. [#16584](https://github.com/ClickHouse/ClickHouse/pull/16584) ([tavplubix](https://github.com/tavplubix)). +* Subqueries in `WITH` section (CTE) can reference previous subqueries in `WITH` section by their name. [#16575](https://github.com/ClickHouse/ClickHouse/pull/16575) ([Amos Bird](https://github.com/amosbird)). +* Add current_database into `system.query_thread_log`. [#16558](https://github.com/ClickHouse/ClickHouse/pull/16558) ([Azat Khuzhin](https://github.com/azat)). +* Allow to fetch parts that are already committed or outdated in the current instance into the detached directory. It's useful when migrating tables from another cluster and having N to 1 shards mapping. It's also consistent with the current fetchPartition implementation. [#16538](https://github.com/ClickHouse/ClickHouse/pull/16538) ([Amos Bird](https://github.com/amosbird)). +* Multiple improvements for `RabbitMQ`: Fixed bug for [#16263](https://github.com/ClickHouse/ClickHouse/issues/16263). Also minimized event loop lifetime. Added more efficient queues setup. [#16426](https://github.com/ClickHouse/ClickHouse/pull/16426) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix debug assertion in `quantileDeterministic` function. In previous version it may also transfer up to two times more data over the network. Although no bug existed. This fixes [#15683](https://github.com/ClickHouse/ClickHouse/issues/15683). [#16410](https://github.com/ClickHouse/ClickHouse/pull/16410) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add `TablesToDropQueueSize` metric. It's equal to number of dropped tables, that are waiting for background data removal. [#16364](https://github.com/ClickHouse/ClickHouse/pull/16364) ([tavplubix](https://github.com/tavplubix)). +* Better diagnostics when client has dropped connection. In previous versions, `Attempt to read after EOF` and `Broken pipe` exceptions were logged in server. In new version, it's information message `Client has dropped the connection, cancel the query.`. [#16329](https://github.com/ClickHouse/ClickHouse/pull/16329) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add total_rows/total_bytes (from system.tables) support for Set/Join table engines. [#16306](https://github.com/ClickHouse/ClickHouse/pull/16306) ([Azat Khuzhin](https://github.com/azat)). +* Now it's possible to specify `PRIMARY KEY` without `ORDER BY` for MergeTree table engines family. Closes [#15591](https://github.com/ClickHouse/ClickHouse/issues/15591). [#16284](https://github.com/ClickHouse/ClickHouse/pull/16284) ([alesapin](https://github.com/alesapin)). +* If there is no tmp folder in the system (chroot, misconfigutation etc) `clickhouse-local` will create temporary subfolder in the current directory. [#16280](https://github.com/ClickHouse/ClickHouse/pull/16280) ([filimonov](https://github.com/filimonov)). +* Add support for nested data types (like named tuple) as sub-types. Fixes [#15587](https://github.com/ClickHouse/ClickHouse/issues/15587). [#16262](https://github.com/ClickHouse/ClickHouse/pull/16262) ([Ivan](https://github.com/abyss7)). +* Support for `database_atomic_wait_for_drop_and_detach_synchronously`/`NO DELAY`/`SYNC` for `DROP DATABASE`. [#16127](https://github.com/ClickHouse/ClickHouse/pull/16127) ([Azat Khuzhin](https://github.com/azat)). +* Add `allow_nondeterministic_optimize_skip_unused_shards` (to allow non deterministic like `rand()` or `dictGet()` in sharding key). [#16105](https://github.com/ClickHouse/ClickHouse/pull/16105) ([Azat Khuzhin](https://github.com/azat)). +* Fix `memory_profiler_step`/`max_untracked_memory` for queries via HTTP (test included). Fix the issue that adjusting this value globally in xml config does not help either, since those settings are not applied anyway, only default (4MB) value is [used](https://github.com/ClickHouse/ClickHouse/blob/17731245336d8c84f75e4c0894c5797ed7732190/src/Common/ThreadStatus.h#L104). Fix `query_id` for the most root ThreadStatus of the http query (by initializing QueryScope after reading query_id). [#16101](https://github.com/ClickHouse/ClickHouse/pull/16101) ([Azat Khuzhin](https://github.com/azat)). +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Fix rare issue when `clickhouse-client` may abort on exit due to loading of suggestions. This fixes [#16035](https://github.com/ClickHouse/ClickHouse/issues/16035). [#16047](https://github.com/ClickHouse/ClickHouse/pull/16047) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support of `cache` layout for `Redis` dictionaries with complex key. [#15985](https://github.com/ClickHouse/ClickHouse/pull/15985) ([Anton Popov](https://github.com/CurtizJ)). +* Fix query hang (endless loop) in case of misconfiguration (`connections_with_failover_max_tries` set to 0). [#15876](https://github.com/ClickHouse/ClickHouse/pull/15876) ([Azat Khuzhin](https://github.com/azat)). +* Change level of some log messages from information to debug, so information messages will not appear for every query. This closes [#5293](https://github.com/ClickHouse/ClickHouse/issues/5293). [#15816](https://github.com/ClickHouse/ClickHouse/pull/15816) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove `MemoryTrackingInBackground*` metrics to avoid potentially misleading results. This fixes [#15684](https://github.com/ClickHouse/ClickHouse/issues/15684). [#15813](https://github.com/ClickHouse/ClickHouse/pull/15813) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add reconnects to `zookeeper-dump-tree` tool. [#15711](https://github.com/ClickHouse/ClickHouse/pull/15711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow explicitly specify columns list in `CREATE TABLE table AS table_function(...)` query. Fixes [#9249](https://github.com/ClickHouse/ClickHouse/issues/9249) Fixes [#14214](https://github.com/ClickHouse/ClickHouse/issues/14214). [#14295](https://github.com/ClickHouse/ClickHouse/pull/14295) ([tavplubix](https://github.com/tavplubix)). + +#### Performance Improvement + +* Do not merge parts across partitions in SELECT FINAL. [#15938](https://github.com/ClickHouse/ClickHouse/pull/15938) ([Kruglov Pavel](https://github.com/Avogar)). +* Improve performance of `-OrNull` and `-OrDefault` aggregate functions. [#16661](https://github.com/ClickHouse/ClickHouse/pull/16661) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improve performance of `quantileMerge`. In previous versions it was obnoxiously slow. This closes [#1463](https://github.com/ClickHouse/ClickHouse/issues/1463). [#16643](https://github.com/ClickHouse/ClickHouse/pull/16643) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improve performance of logical functions a little. [#16347](https://github.com/ClickHouse/ClickHouse/pull/16347) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improved performance of merges assignment in MergeTree table engines. Shouldn't be visible for the user. [#16191](https://github.com/ClickHouse/ClickHouse/pull/16191) ([alesapin](https://github.com/alesapin)). +* Speedup hashed/sparse_hashed dictionary loading by preallocating the hash table. [#15454](https://github.com/ClickHouse/ClickHouse/pull/15454) ([Azat Khuzhin](https://github.com/azat)). +* Now trivial count optimization becomes slightly non-trivial. Predicates that contain exact partition expr can be optimized too. This also fixes [#11092](https://github.com/ClickHouse/ClickHouse/issues/11092) which returns wrong count when `max_parallel_replicas > 1`. [#15074](https://github.com/ClickHouse/ClickHouse/pull/15074) ([Amos Bird](https://github.com/amosbird)). + +#### Build/Testing/Packaging Improvement + +* Add flaky check for stateless tests. It will detect potentially flaky functional tests in advance, before they are merged. [#16238](https://github.com/ClickHouse/ClickHouse/pull/16238) ([alesapin](https://github.com/alesapin)). +* Use proper version for `croaring` instead of amalgamation. [#16285](https://github.com/ClickHouse/ClickHouse/pull/16285) ([sundyli](https://github.com/sundy-li)). +* Improve generation of build files for `ya.make` build system (Arcadia). [#16700](https://github.com/ClickHouse/ClickHouse/pull/16700) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add MySQL BinLog file check tool for `MaterializeMySQL` database engine. `MaterializeMySQL` is an experimental feature. [#16223](https://github.com/ClickHouse/ClickHouse/pull/16223) ([Winter Zhang](https://github.com/zhang2014)). +* Check for executable bit on non-executable files. People often accidentially commit executable files from Windows. [#15843](https://github.com/ClickHouse/ClickHouse/pull/15843) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Check for `#pragma once` in headers. [#15818](https://github.com/ClickHouse/ClickHouse/pull/15818) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix illegal code style `&vector[idx]` in libhdfs3. This fixes libcxx debug build. See also https://github.com/ClickHouse-Extras/libhdfs3/pull/8 . [#15815](https://github.com/ClickHouse/ClickHouse/pull/15815) ([Amos Bird](https://github.com/amosbird)). +* Fix build of one miscellaneous example tool on Mac OS. Note that we don't build examples on Mac OS in our CI (we build only ClickHouse binary), so there is zero chance it will not break again. This fixes [#15804](https://github.com/ClickHouse/ClickHouse/issues/15804). [#15808](https://github.com/ClickHouse/ClickHouse/pull/15808) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Simplify Sys/V init script. [#14135](https://github.com/ClickHouse/ClickHouse/pull/14135) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added `boost::program_options` to `db_generator` in order to increase its usability. This closes [#15940](https://github.com/ClickHouse/ClickHouse/issues/15940). [#15973](https://github.com/ClickHouse/ClickHouse/pull/15973) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + + +## ClickHouse release 20.10 + +### ClickHouse release v20.10.7.4-stable, 2020-12-24 + +#### Bug Fix + +* Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual `IPv4/IPv6` stack and fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes. This possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). +* Fix key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). +* Fixed unique key convert crash in `MaterializeMySQL` database engine. This fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) and fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed the issue when some tables not synchronized to ClickHouse from MySQL caused by the fact that convertion MySQL prefix index wasn't supported for MaterializeMySQL. This fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) and fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). +* Fix possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). +* Do not restore parts from `WAL` if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). +* Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). +* Fixed empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). +* Fixed the behaviour when `clickhouse-client` is used in interactive mode with multiline queries and single line comment was erronously extended till the end of query. This fixes [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `ALTER` query hang when the corresponding mutation was killed on the different replica. This fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). +* Fixed bug when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). +* Fixed `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed duplicates after `DISTINCT` which were possible because of incorrect optimization. Fixes [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed crash while reading from `JOIN` table with `LowCardinality` types. This fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246) . [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). +* Fixed `ColumnConst` comparison which leads to crash. This fixed [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). +* Fixed bug when `ON CLUSTER` queries may hang forever for non-leader `ReplicatedMergeTreeTables`. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). +* Fixed fuzzer-found bug in function `fuzzBits`. This fixes [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). +* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong result in big integers (128, 256 bit) when casting from double. [#16986](https://github.com/ClickHouse/ClickHouse/pull/16986) ([Mike](https://github.com/myrrc)). +* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). +* Fixed possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). +* Blame info was not calculated correctly in `clickhouse-git-import`. [#16959](https://github.com/ClickHouse/ClickHouse/pull/16959) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed order by optimization with monotonous functions. This fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). +* Fixrf optimization of group by with enabled setting `optimize_aggregators_of_group_by_keys` and joins. This fixes [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). +* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). +* Fixrf possible error `Illegal type of argument` for queries with `ORDER BY`. This fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Abort multipart upload if no data was written to `WriteBufferFromS3`. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixed crash when using `any` without any arguments. This fixes [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). +* Fixed the behaviour when ClickHouse used to always return 0 insted of a number of affected rows for `INSERT` queries via MySQL protocol. This fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed uncontrolled growth of `TDigest`. [#16680](https://github.com/ClickHouse/ClickHouse/pull/16680) ([hrissan](https://github.com/hrissan)). +* Fixed remote query failure when using suffix `if` in Aggregate function. This fixes [#16574](https://github.com/ClickHouse/ClickHouse/issues/16574) fixes [#16231](https://github.com/ClickHouse/ClickHouse/issues/16231) [#16610](https://github.com/ClickHouse/ClickHouse/pull/16610) ([Winter Zhang](https://github.com/zhang2014)). + + +### ClickHouse release v20.10.4.1-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* This will fix optimize_read_in_order/optimize_aggregation_in_order with max_threads>0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Improvement + +* Workaround for use S3 with nginx server as proxy. Nginx currenty does not accept urls with empty path like http://domain.com?delete, but vanilla aws-sdk-cpp produces this kind of urls. This commit uses patched aws-sdk-cpp version, which makes urls with "/" as path in this cases, like http://domain.com/?delete. [#16813](https://github.com/ClickHouse/ClickHouse/pull/16813) ([ianton-ru](https://github.com/ianton-ru)). + + +### ClickHouse release v20.10.3.30, 2020-10-28 + +#### Backward Incompatible Change + +* Make `multiple_joins_rewriter_version` obsolete. Remove first version of joins rewriter. [#15472](https://github.com/ClickHouse/ClickHouse/pull/15472) ([Artem Zuikov](https://github.com/4ertus2)). +* Change default value of `format_regexp_escaping_rule` setting (it's related to `Regexp` format) to `Raw` (it means - read whole subpattern as a value) to make the behaviour more like to what users expect. [#15426](https://github.com/ClickHouse/ClickHouse/pull/15426) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for nested multiline comments `/* comment /* comment */ */` in SQL. This conforms to the SQL standard. [#14655](https://github.com/ClickHouse/ClickHouse/pull/14655) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added MergeTree settings (`max_replicated_merges_with_ttl_in_queue` and `max_number_of_merges_with_ttl_in_pool`) to control the number of merges with TTL in the background pool and replicated queue. This change breaks compatibility with older versions only if you use delete TTL. Otherwise, replication will stay compatible. You can avoid incompatibility issues if you update all shard replicas at once or execute `SYSTEM STOP TTL MERGES` until you finish the update of all replicas. If you'll get an incompatible entry in the replication queue, first of all, execute `SYSTEM STOP TTL MERGES` and after `ALTER TABLE ... DETACH PARTITION ...` the partition where incompatible TTL merge was assigned. Attach it back on a single replica. [#14490](https://github.com/ClickHouse/ClickHouse/pull/14490) ([alesapin](https://github.com/alesapin)). +* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). + +#### New Feature + +* Background data recompression. Add the ability to specify `TTL ... RECOMPRESS codec_name` for MergeTree table engines family. [#14494](https://github.com/ClickHouse/ClickHouse/pull/14494) ([alesapin](https://github.com/alesapin)). +* Add parallel quorum inserts. This closes [#15601](https://github.com/ClickHouse/ClickHouse/issues/15601). [#15601](https://github.com/ClickHouse/ClickHouse/pull/15601) ([Latysheva Alexandra](https://github.com/alexelex)). +* Settings for additional enforcement of data durability. Useful for non-replicated setups. [#11948](https://github.com/ClickHouse/ClickHouse/pull/11948) ([Anton Popov](https://github.com/CurtizJ)). +* When duplicate block is written to replica where it does not exist locally (has not been fetched from replicas), don't ignore it and write locally to achieve the same effect as if it was successfully replicated. [#11684](https://github.com/ClickHouse/ClickHouse/pull/11684) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now we support `WITH AS (subquery) ... ` to introduce named subqueries in the query context. This closes [#2416](https://github.com/ClickHouse/ClickHouse/issues/2416). This closes [#4967](https://github.com/ClickHouse/ClickHouse/issues/4967). [#14771](https://github.com/ClickHouse/ClickHouse/pull/14771) ([Amos Bird](https://github.com/amosbird)). +* Introduce `enable_global_with_statement` setting which propagates the first select's `WITH` statements to other select queries at the same level, and makes aliases in `WITH` statements visible to subqueries. [#15451](https://github.com/ClickHouse/ClickHouse/pull/15451) ([Amos Bird](https://github.com/amosbird)). +* Secure inter-cluster query execution (with initial_user as current query user). [#13156](https://github.com/ClickHouse/ClickHouse/pull/13156) ([Azat Khuzhin](https://github.com/azat)). [#15551](https://github.com/ClickHouse/ClickHouse/pull/15551) ([Azat Khuzhin](https://github.com/azat)). +* Add the ability to remove column properties and table TTLs. Introduced queries `ALTER TABLE MODIFY COLUMN col_name REMOVE what_to_remove` and `ALTER TABLE REMOVE TTL`. Both operations are lightweight and executed at the metadata level. [#14742](https://github.com/ClickHouse/ClickHouse/pull/14742) ([alesapin](https://github.com/alesapin)). +* Added format `RawBLOB`. It is intended for input or output a single value without any escaping and delimiters. This closes [#15349](https://github.com/ClickHouse/ClickHouse/issues/15349). [#15364](https://github.com/ClickHouse/ClickHouse/pull/15364) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the `reinterpretAsUUID` function that allows to convert a big-endian byte string to UUID. [#15480](https://github.com/ClickHouse/ClickHouse/pull/15480) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Implement `force_data_skipping_indices` setting. [#15642](https://github.com/ClickHouse/ClickHouse/pull/15642) ([Azat Khuzhin](https://github.com/azat)). +* Add a setting `output_format_pretty_row_numbers` to numerate the result in Pretty formats. This closes [#15350](https://github.com/ClickHouse/ClickHouse/issues/15350). [#15443](https://github.com/ClickHouse/ClickHouse/pull/15443) ([flynn](https://github.com/ucasFL)). +* Added query obfuscation tool. It allows to share more queries for better testing. This closes [#15268](https://github.com/ClickHouse/ClickHouse/issues/15268). [#15321](https://github.com/ClickHouse/ClickHouse/pull/15321) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add table function `null('structure')`. [#14797](https://github.com/ClickHouse/ClickHouse/pull/14797) ([vxider](https://github.com/Vxider)). +* Added `formatReadableQuantity` function. It is useful for reading big numbers by human. [#14725](https://github.com/ClickHouse/ClickHouse/pull/14725) ([Artem Hnilov](https://github.com/BooBSD)). +* Add format `LineAsString` that accepts a sequence of lines separated by newlines, every line is parsed as a whole as a single String field. [#14703](https://github.com/ClickHouse/ClickHouse/pull/14703) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)), [#13846](https://github.com/ClickHouse/ClickHouse/pull/13846) ([hexiaoting](https://github.com/hexiaoting)). +* Add `JSONStrings` format which output data in arrays of strings. [#14333](https://github.com/ClickHouse/ClickHouse/pull/14333) ([hcz](https://github.com/hczhcz)). +* Add support for "Raw" column format for `Regexp` format. It allows to simply extract subpatterns as a whole without any escaping rules. [#15363](https://github.com/ClickHouse/ClickHouse/pull/15363) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow configurable `NULL` representation for `TSV` output format. It is controlled by the setting `output_format_tsv_null_representation` which is `\N` by default. This closes [#9375](https://github.com/ClickHouse/ClickHouse/issues/9375). Note that the setting only controls output format and `\N` is the only supported `NULL` representation for `TSV` input format. [#14586](https://github.com/ClickHouse/ClickHouse/pull/14586) ([Kruglov Pavel](https://github.com/Avogar)). +* Support Decimal data type for `MaterializeMySQL`. `MaterializeMySQL` is an experimental feature. [#14535](https://github.com/ClickHouse/ClickHouse/pull/14535) ([Winter Zhang](https://github.com/zhang2014)). +* Add new feature: `SHOW DATABASES LIKE 'xxx'`. [#14521](https://github.com/ClickHouse/ClickHouse/pull/14521) ([hexiaoting](https://github.com/hexiaoting)). +* Added a script to import (arbitrary) git repository to ClickHouse as a sample dataset. [#14471](https://github.com/ClickHouse/ClickHouse/pull/14471) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now insert statements can have asterisk (or variants) with column transformers in the column list. [#14453](https://github.com/ClickHouse/ClickHouse/pull/14453) ([Amos Bird](https://github.com/amosbird)). +* New query complexity limit settings `max_rows_to_read_leaf`, `max_bytes_to_read_leaf` for distributed queries to limit max rows/bytes read on the leaf nodes. Limit is applied for local reads only, *excluding* the final merge stage on the root node. [#14221](https://github.com/ClickHouse/ClickHouse/pull/14221) ([Roman Khavronenko](https://github.com/hagen1778)). +* Allow user to specify settings for `ReplicatedMergeTree*` storage in `` section of config file. It works similarly to `` section. For `ReplicatedMergeTree*` storages settings from `` and `` are applied together, but settings from `` has higher priority. Added `system.replicated_merge_tree_settings` table. [#13573](https://github.com/ClickHouse/ClickHouse/pull/13573) ([Amos Bird](https://github.com/amosbird)). +* Add `mapPopulateSeries` function. [#13166](https://github.com/ClickHouse/ClickHouse/pull/13166) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Supporting MySQL types: `decimal` (as ClickHouse `Decimal`) and `datetime` with sub-second precision (as `DateTime64`). [#11512](https://github.com/ClickHouse/ClickHouse/pull/11512) ([Vasily Nemkov](https://github.com/Enmk)). +* Introduce `event_time_microseconds` field to `system.text_log`, `system.trace_log`, `system.query_log` and `system.query_thread_log` tables. [#14760](https://github.com/ClickHouse/ClickHouse/pull/14760) ([Bharat Nallan](https://github.com/bharatnc)). +* Add `event_time_microseconds` to `system.asynchronous_metric_log` & `system.metric_log` tables. [#14514](https://github.com/ClickHouse/ClickHouse/pull/14514) ([Bharat Nallan](https://github.com/bharatnc)). +* Add `query_start_time_microseconds` field to `system.query_log` & `system.query_thread_log` tables. [#14252](https://github.com/ClickHouse/ClickHouse/pull/14252) ([Bharat Nallan](https://github.com/bharatnc)). + +#### Bug Fix + +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `executable` dictionary source hang. In previous versions, when using some formats (e.g. `JSONEachRow`) data was not feed to a child process before it outputs at least something. This closes [#1697](https://github.com/ClickHouse/ClickHouse/issues/1697). This closes [#2455](https://github.com/ClickHouse/ClickHouse/issues/2455). [#14525](https://github.com/ClickHouse/ClickHouse/pull/14525) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT with prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix segfault in some cases of wrong aggregation in lambdas. [#16082](https://github.com/ClickHouse/ClickHouse/pull/16082) ([Anton Popov](https://github.com/CurtizJ)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* `MaterializeMySQL` (experimental feature): Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use `direct` layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* `MaterializeMySQL` (experimental feature): Fix crash on create database failure. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) - Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixes [#12513](https://github.com/ClickHouse/ClickHouse/issues/12513): difference expressions with same alias when query is reanalyzed. [#15886](https://github.com/ClickHouse/ClickHouse/pull/15886) ([Winter Zhang](https://github.com/zhang2014)). +* Fix possible very rare deadlocks in RBAC implementation. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* `MaterializeMySQL` (experimental feature): Fix `select count()` inaccuracy. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fix drop of materialized view with inner table in Atomic database (hangs all subsequent DROP TABLE due to hang of the worker thread, due to recursive DROP TABLE for inner table of MV). [#15743](https://github.com/ClickHouse/ClickHouse/pull/15743) ([Azat Khuzhin](https://github.com/azat)). +* Possibility to move part to another disk/volume if the first attempt was failed. [#15723](https://github.com/ClickHouse/ClickHouse/pull/15723) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix rare race condition in dictionaries and tables from MySQL. [#15686](https://github.com/ClickHouse/ClickHouse/pull/15686) ([alesapin](https://github.com/alesapin)). +* Fix (benign) race condition in AMQP-CPP. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Proper error handling during insert into MergeTree with S3. MergeTree over S3 is an experimental feature. [#15657](https://github.com/ClickHouse/ClickHouse/pull/15657) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixed bug with S3 table function: region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Subtract `ReadonlyReplica` metric when detach readonly tables. [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Query will finish faster in case of exception. Cancel execution on remote replicas if exception happens. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `Database doesn't exist.` in queries with IN and Distributed table when there's no database on initiator. [#15538](https://github.com/ClickHouse/ClickHouse/pull/15538) ([Artem Zuikov](https://github.com/4ertus2)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fixes [#15365](https://github.com/ClickHouse/ClickHouse/issues/15365): attach a database with MySQL engine throws exception (no query context). [#15384](https://github.com/ClickHouse/ClickHouse/pull/15384) ([Winter Zhang](https://github.com/zhang2014)). +* Fix the case of multiple occurrences of column transformers in a select query. [#15378](https://github.com/ClickHouse/ClickHouse/pull/15378) ([Amos Bird](https://github.com/amosbird)). +* Fixed compression in `S3` storage. [#15376](https://github.com/ClickHouse/ClickHouse/pull/15376) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fix bug where queries like `SELECT toStartOfDay(today())` fail complaining about empty time_zone argument. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix hang of queries with a lot of subqueries to same table of `MySQL` engine. Previously, if there were more than 16 subqueries to same `MySQL` table in query, it hang forever. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix 'Unknown identifier' in GROUP BY when query has JOIN over Merge table. [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix instance crash when using `joinGet` with `LowCardinality` types. This fixes [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust Decimal field size in MySQL column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* Fixes `Data compressed with different methods` in `join_algorithm='auto'`. Keep LowCardinality as type for left table join key in `join_algorithm='partial_merge'`. [#15088](https://github.com/ClickHouse/ClickHouse/pull/15088) ([Artem Zuikov](https://github.com/4ertus2)). +* Update `jemalloc` to fix `percpu_arena` with affinity mask. [#15035](https://github.com/ClickHouse/ClickHouse/pull/15035) ([Azat Khuzhin](https://github.com/azat)). [#14957](https://github.com/ClickHouse/ClickHouse/pull/14957) ([Azat Khuzhin](https://github.com/azat)). +* We already use padded comparison between String and FixedString (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333). This PR applies the same logic to field comparison which corrects the usage of FixedString as primary keys. This fixes [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). +* If function `bar` was called with specifically crafted arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in Docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Fix crash in RIGHT or FULL JOIN with join_algorith='auto' when memory limit exceeded and we should change HashJoin with MergeJoin. [#15002](https://github.com/ClickHouse/ClickHouse/pull/15002) ([Artem Zuikov](https://github.com/4ertus2)). +* Now settings `number_of_free_entries_in_pool_to_execute_mutation` and `number_of_free_entries_in_pool_to_lower_max_size_of_merge` can be equal to `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). +* Fix to make predicate push down work when subquery contains `finalizeAggregation` function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* `MaterializeMySQL` (experimental feature): Fixed `.metadata.tmp File exists` error. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). +* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix SIGSEGV for an attempt to INSERT into StorageFile with file descriptor. [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). +* Fixed segfault in `cache` dictionary [#14837](https://github.com/ClickHouse/ClickHouse/issues/14837). [#14879](https://github.com/ClickHouse/ClickHouse/pull/14879) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* `MaterializeMySQL` (experimental feature): Fixed bug in parsing MySQL binlog events, which causes `Attempt to read after eof` and `Packet payload is not fully read` in `MaterializeMySQL` database engine. [#14852](https://github.com/ClickHouse/ClickHouse/pull/14852) ([Winter Zhang](https://github.com/zhang2014)). +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). +* `Replace` column transformer should replace identifiers with cloned ASTs. This fixes [#14695](https://github.com/ClickHouse/ClickHouse/issues/14695) . [#14734](https://github.com/ClickHouse/ClickHouse/pull/14734) ([Amos Bird](https://github.com/amosbird)). +* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ALTER UPDATE` mutation with `Nullable` column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix function `has` with `LowCardinality` of `Nullable`. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([Mike](https://github.com/myrrc)). +* Cleanup data directory after Zookeeper exceptions during CreateQuery for StorageReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix rare segfaults in functions with combinator `-Resample`, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). +* Fix a bug when converting `Nullable(String)` to Enum. Introduced by [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745). This fixes [#14435](https://github.com/ClickHouse/ClickHouse/issues/14435). [#14530](https://github.com/ClickHouse/ClickHouse/pull/14530) ([Amos Bird](https://github.com/amosbird)). +* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix `currentDatabase()` function cannot be used in `ON CLUSTER` ddl query. [#14211](https://github.com/ClickHouse/ClickHouse/pull/14211) ([Winter Zhang](https://github.com/zhang2014)). +* `MaterializeMySQL` (experimental feature): Fixed `Packet payload is not fully read` error in `MaterializeMySQL` database engine. [#14696](https://github.com/ClickHouse/ClickHouse/pull/14696) ([BohuTANG](https://github.com/BohuTANG)). + +#### Improvement + +* Enable `Atomic` database engine by default for newly created databases. [#15003](https://github.com/ClickHouse/ClickHouse/pull/15003) ([tavplubix](https://github.com/tavplubix)). +* Add the ability to specify specialized codecs like `Delta`, `T64`, etc. for columns with subtypes. Implements [#12551](https://github.com/ClickHouse/ClickHouse/issues/12551), fixes [#11397](https://github.com/ClickHouse/ClickHouse/issues/11397), fixes [#4609](https://github.com/ClickHouse/ClickHouse/issues/4609). [#15089](https://github.com/ClickHouse/ClickHouse/pull/15089) ([alesapin](https://github.com/alesapin)). +* Dynamic reload of zookeeper config. [#14678](https://github.com/ClickHouse/ClickHouse/pull/14678) ([sundyli](https://github.com/sundy-li)). +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Now `joinGet` supports multi-key lookup. Continuation of [#12418](https://github.com/ClickHouse/ClickHouse/issues/12418). [#13015](https://github.com/ClickHouse/ClickHouse/pull/13015) ([Amos Bird](https://github.com/amosbird)). +* Wait for `DROP/DETACH TABLE` to actually finish if `NO DELAY` or `SYNC` is specified for `Atomic` database. [#15448](https://github.com/ClickHouse/ClickHouse/pull/15448) ([tavplubix](https://github.com/tavplubix)). +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `zookeeper_path` on replicated table creation. Do not allow `RENAME TABLE` if it may break `zookeeper_path` after server restart. Fixes [#6917](https://github.com/ClickHouse/ClickHouse/issues/6917). [#15348](https://github.com/ClickHouse/ClickHouse/pull/15348) ([tavplubix](https://github.com/tavplubix)). +* The function `now` allows an argument with timezone. This closes [15264](https://github.com/ClickHouse/ClickHouse/issues/15264). [#15285](https://github.com/ClickHouse/ClickHouse/pull/15285) ([flynn](https://github.com/ucasFL)). +* Do not allow connections to ClickHouse server until all scripts in `/docker-entrypoint-initdb.d/` are executed. [#15244](https://github.com/ClickHouse/ClickHouse/pull/15244) ([Aleksei Kozharin](https://github.com/alekseik1)). +* Added `optimize` setting to `EXPLAIN PLAN` query. If enabled, query plan level optimisations are applied. Enabled by default. [#15201](https://github.com/ClickHouse/ClickHouse/pull/15201) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Proper exception message for wrong number of arguments of CAST. This closes [#13992](https://github.com/ClickHouse/ClickHouse/issues/13992). [#15029](https://github.com/ClickHouse/ClickHouse/pull/15029) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add option to disable TTL move on data part insert. [#15000](https://github.com/ClickHouse/ClickHouse/pull/15000) ([Pavel Kovalenko](https://github.com/Jokser)). +* Ignore key constraints when doing mutations. Without this pull request, it's not possible to do mutations when `force_index_by_date = 1` or `force_primary_key = 1`. [#14973](https://github.com/ClickHouse/ClickHouse/pull/14973) ([Amos Bird](https://github.com/amosbird)). +* Allow to drop Replicated table if previous drop attempt was failed due to ZooKeeper session expiration. This fixes [#11891](https://github.com/ClickHouse/ClickHouse/issues/11891). [#14926](https://github.com/ClickHouse/ClickHouse/pull/14926) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed excessive settings constraint violation when running SELECT with SETTINGS from a distributed table. [#14876](https://github.com/ClickHouse/ClickHouse/pull/14876) ([Amos Bird](https://github.com/amosbird)). +* Provide a `load_balancing_first_offset` query setting to explicitly state what the first replica is. It's used together with `FIRST_OR_RANDOM` load balancing strategy, which allows to control replicas workload. [#14867](https://github.com/ClickHouse/ClickHouse/pull/14867) ([Amos Bird](https://github.com/amosbird)). +* Show subqueries for `SET` and `JOIN` in `EXPLAIN` result. [#14856](https://github.com/ClickHouse/ClickHouse/pull/14856) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Allow using multi-volume storage configuration in storage `Distributed`. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). +* Construct `query_start_time` and `query_start_time_microseconds` from the same timespec. [#14831](https://github.com/ClickHouse/ClickHouse/pull/14831) ([Bharat Nallan](https://github.com/bharatnc)). +* Support for disabling persistency for `StorageJoin` and `StorageSet`, this feature is controlled by setting `disable_set_and_join_persistency`. And this PR solved issue [#6318](https://github.com/ClickHouse/ClickHouse/issues/6318). [#14776](https://github.com/ClickHouse/ClickHouse/pull/14776) ([vxider](https://github.com/Vxider)). +* Now `COLUMNS` can be used to wrap over a list of columns and apply column transformers afterwards. [#14775](https://github.com/ClickHouse/ClickHouse/pull/14775) ([Amos Bird](https://github.com/amosbird)). +* Add `merge_algorithm` to `system.merges` table to improve merging inspections. [#14705](https://github.com/ClickHouse/ClickHouse/pull/14705) ([Amos Bird](https://github.com/amosbird)). +* Fix potential memory leak caused by zookeeper exists watch. [#14693](https://github.com/ClickHouse/ClickHouse/pull/14693) ([hustnn](https://github.com/hustnn)). +* Allow parallel execution of distributed DDL. [#14684](https://github.com/ClickHouse/ClickHouse/pull/14684) ([Azat Khuzhin](https://github.com/azat)). +* Add `QueryMemoryLimitExceeded` event counter. This closes [#14589](https://github.com/ClickHouse/ClickHouse/issues/14589). [#14647](https://github.com/ClickHouse/ClickHouse/pull/14647) ([fastio](https://github.com/fastio)). +* Fix some trailing whitespaces in query formatting. [#14595](https://github.com/ClickHouse/ClickHouse/pull/14595) ([Azat Khuzhin](https://github.com/azat)). +* ClickHouse treats partition expr and key expr differently. Partition expr is used to construct an minmax index containing related columns, while primary key expr is stored as an expr. Sometimes user might partition a table at coarser levels, such as `partition by i / 1000`. However, binary operators are not monotonic and this PR tries to fix that. It might also benifit other use cases. [#14513](https://github.com/ClickHouse/ClickHouse/pull/14513) ([Amos Bird](https://github.com/amosbird)). +* Add an option to skip access checks for `DiskS3`. `s3` disk is an experimental feature. [#14497](https://github.com/ClickHouse/ClickHouse/pull/14497) ([Pavel Kovalenko](https://github.com/Jokser)). +* Speed up server shutdown process if there are ongoing S3 requests. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). +* `SYSTEM RELOAD CONFIG` now throws an exception if failed to reload and continues using the previous users.xml. The background periodic reloading also continues using the previous users.xml if failed to reload. [#14492](https://github.com/ClickHouse/ClickHouse/pull/14492) ([Vitaly Baranov](https://github.com/vitlibar)). +* For INSERTs with inline data in VALUES format in the script mode of `clickhouse-client`, support semicolon as the data terminator, in addition to the new line. Closes [#12288](https://github.com/ClickHouse/ClickHouse/issues/12288). [#13192](https://github.com/ClickHouse/ClickHouse/pull/13192) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + +#### Performance Improvement + +* Enable compact parts by default for small parts. This will allow to process frequent inserts slightly more efficiently (4..100 times). [#11913](https://github.com/ClickHouse/ClickHouse/pull/11913) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improve `quantileTDigest` performance. This fixes [#2668](https://github.com/ClickHouse/ClickHouse/issues/2668). [#15542](https://github.com/ClickHouse/ClickHouse/pull/15542) ([Kruglov Pavel](https://github.com/Avogar)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Faster 256-bit multiplication. [#15418](https://github.com/ClickHouse/ClickHouse/pull/15418) ([Artem Zuikov](https://github.com/4ertus2)). +* Improve performance of 256-bit types using (u)int64_t as base type for wide integers. Original wide integers use 8-bit types as base. [#14859](https://github.com/ClickHouse/ClickHouse/pull/14859) ([Artem Zuikov](https://github.com/4ertus2)). +* Explicitly use a temporary disk to store vertical merge temporary data. [#15639](https://github.com/ClickHouse/ClickHouse/pull/15639) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). +* Use one S3 DeleteObjects request instead of multiple DeleteObject in a loop. No any functionality changes, so covered by existing tests like integration/test_log_family_s3. [#15238](https://github.com/ClickHouse/ClickHouse/pull/15238) ([ianton-ru](https://github.com/ianton-ru)). +* Fix `DateTime DateTime` mistakenly choosing the slow generic implementation. This fixes [#15153](https://github.com/ClickHouse/ClickHouse/issues/15153). [#15178](https://github.com/ClickHouse/ClickHouse/pull/15178) ([Amos Bird](https://github.com/amosbird)). +* Improve performance of GROUP BY key of type `FixedString`. [#15034](https://github.com/ClickHouse/ClickHouse/pull/15034) ([Amos Bird](https://github.com/amosbird)). +* Only `mlock` code segment when starting clickhouse-server. In previous versions, all mapped regions were locked in memory, including debug info. Debug info is usually splitted to a separate file but if it isn't, it led to +2..3 GiB memory usage. [#14929](https://github.com/ClickHouse/ClickHouse/pull/14929) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* ClickHouse binary become smaller due to link time optimization. + +#### Build/Testing/Packaging Improvement + +* Now we use clang-11 for production ClickHouse build. [#15239](https://github.com/ClickHouse/ClickHouse/pull/15239) ([alesapin](https://github.com/alesapin)). +* Now we use clang-11 to build ClickHouse in CI. [#14846](https://github.com/ClickHouse/ClickHouse/pull/14846) ([alesapin](https://github.com/alesapin)). +* Switch binary builds (Linux, Darwin, AArch64, FreeDSD) to clang-11. [#15622](https://github.com/ClickHouse/ClickHouse/pull/15622) ([Ilya Yatsishin](https://github.com/qoega)). +* Now all test images use `llvm-symbolizer-11`. [#15069](https://github.com/ClickHouse/ClickHouse/pull/15069) ([alesapin](https://github.com/alesapin)). +* Allow to build with llvm-11. [#15366](https://github.com/ClickHouse/ClickHouse/pull/15366) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Switch from `clang-tidy-10` to `clang-tidy-11`. [#14922](https://github.com/ClickHouse/ClickHouse/pull/14922) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Use LLVM's experimental pass manager by default. [#15608](https://github.com/ClickHouse/ClickHouse/pull/15608) ([Danila Kutenin](https://github.com/danlark1)). +* Don't allow any C++ translation unit to build more than 10 minutes or to use more than 10 GB or memory. This fixes [#14925](https://github.com/ClickHouse/ClickHouse/issues/14925). [#15060](https://github.com/ClickHouse/ClickHouse/pull/15060) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make performance test more stable and representative by splitting test runs and profile runs. [#15027](https://github.com/ClickHouse/ClickHouse/pull/15027) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Attempt to make performance test more reliable. It is done by remapping the executable memory of the process on the fly with `madvise` to use transparent huge pages - it can lower the number of iTLB misses which is the main source of instabilities in performance tests. [#14685](https://github.com/ClickHouse/ClickHouse/pull/14685) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Convert to python3. This closes [#14886](https://github.com/ClickHouse/ClickHouse/issues/14886). [#15007](https://github.com/ClickHouse/ClickHouse/pull/15007) ([Azat Khuzhin](https://github.com/azat)). +* Fail early in functional tests if server failed to respond. This closes [#15262](https://github.com/ClickHouse/ClickHouse/issues/15262). [#15267](https://github.com/ClickHouse/ClickHouse/pull/15267) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to run AArch64 version of clickhouse-server without configs. This facilitates [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174). [#15266](https://github.com/ClickHouse/ClickHouse/pull/15266) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improvements in CI docker images: get rid of ZooKeeper and single script for test configs installation. [#15215](https://github.com/ClickHouse/ClickHouse/pull/15215) ([alesapin](https://github.com/alesapin)). +* Fix CMake options forwarding in fast test script. Fixes error in [#14711](https://github.com/ClickHouse/ClickHouse/issues/14711). [#15155](https://github.com/ClickHouse/ClickHouse/pull/15155) ([alesapin](https://github.com/alesapin)). +* Added a script to perform hardware benchmark in a single command. [#15115](https://github.com/ClickHouse/ClickHouse/pull/15115) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Splitted huge test `test_dictionaries_all_layouts_and_sources` into smaller ones. [#15110](https://github.com/ClickHouse/ClickHouse/pull/15110) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Maybe fix MSan report in base64 (on servers with AVX-512). This fixes [#14006](https://github.com/ClickHouse/ClickHouse/issues/14006). [#15030](https://github.com/ClickHouse/ClickHouse/pull/15030) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Reformat and cleanup code in all integration test *.py files. [#14864](https://github.com/ClickHouse/ClickHouse/pull/14864) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix MaterializeMySQL empty transaction unstable test case found in CI. [#14854](https://github.com/ClickHouse/ClickHouse/pull/14854) ([Winter Zhang](https://github.com/zhang2014)). +* Attempt to speed up build a little. [#14808](https://github.com/ClickHouse/ClickHouse/pull/14808) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Speed up build a little by removing unused headers. [#14714](https://github.com/ClickHouse/ClickHouse/pull/14714) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix build failure in OSX. [#14761](https://github.com/ClickHouse/ClickHouse/pull/14761) ([Winter Zhang](https://github.com/zhang2014)). +* Enable ccache by default in cmake if it's found in OS. [#14575](https://github.com/ClickHouse/ClickHouse/pull/14575) ([alesapin](https://github.com/alesapin)). +* Control CI builds configuration from the ClickHouse repository. [#14547](https://github.com/ClickHouse/ClickHouse/pull/14547) ([alesapin](https://github.com/alesapin)). +* In CMake files: - Moved some options' descriptions' parts to comments above. - Replace 0 -> `OFF`, 1 -> `ON` in `option`s default values. - Added some descriptions and links to docs to the options. - Replaced `FUZZER` option (there is another option `ENABLE_FUZZING` which also enables same functionality). - Removed `ENABLE_GTEST_LIBRARY` option as there is `ENABLE_TESTS`. See the full description in PR: [#14711](https://github.com/ClickHouse/ClickHouse/pull/14711) ([Mike](https://github.com/myrrc)). +* Make binary a bit smaller (~50 Mb for debug version). [#14555](https://github.com/ClickHouse/ClickHouse/pull/14555) ([Artem Zuikov](https://github.com/4ertus2)). +* Use std::filesystem::path in ConfigProcessor for concatenating file paths. [#14558](https://github.com/ClickHouse/ClickHouse/pull/14558) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix debug assertion in `bitShiftLeft()` when called with negative big integer. [#14697](https://github.com/ClickHouse/ClickHouse/pull/14697) ([Artem Zuikov](https://github.com/4ertus2)). + + +## ClickHouse release 20.9 + +### ClickHouse release v20.9.5.5-stable, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + + +### ClickHouse release v20.9.4.76-stable (2020-10-29) + +#### Bug Fix + +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a very wrong code in TwoLevelStringHashTable implementation, which might lead to memory leak. I'm suprised how this bug can lurk for so long.... [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* Fix `select count()` inaccuracy for MaterializeMySQL. [#15767](https://github.com/ClickHouse/ClickHouse/pull/15767) ([tavplubix](https://github.com/tavplubix)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). + +#### Improvement + +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16160](https://github.com/ClickHouse/ClickHouse/pull/16160) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.9.3.45-stable (2020-10-09) + +#### Bug Fix + +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix race condition in AMQP-CPP. [#15667](https://github.com/ClickHouse/ClickHouse/pull/15667) ([alesapin](https://github.com/alesapin)). +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fix bug where queries like `SELECT toStartOfDay(today())` fail complaining about empty time_zone argument. [#15319](https://github.com/ClickHouse/ClickHouse/pull/15319) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix instance crash when using joinGet with LowCardinality types. This fixes [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). + +#### Improvement + +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). + + +### ClickHouse release v20.9.2.20, 2020-09-22 + +#### Backward Incompatible Change + +* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). + +#### New Feature + +* Added column transformers `EXCEPT`, `REPLACE`, `APPLY`, which can be applied to the list of selected columns (after `*` or `COLUMNS(...)`). For example, you can write `SELECT * EXCEPT(URL) REPLACE(number + 1 AS number)`. Another example: `select * apply(length) apply(max) from wide_string_table` to find out the maxium length of all string columns. [#14233](https://github.com/ClickHouse/ClickHouse/pull/14233) ([Amos Bird](https://github.com/amosbird)). +* Added an aggregate function `rankCorr` which computes a rank correlation coefficient. [#11769](https://github.com/ClickHouse/ClickHouse/pull/11769) ([antikvist](https://github.com/antikvist)) [#14411](https://github.com/ClickHouse/ClickHouse/pull/14411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Added table function `view` which turns a subquery into a table object. This helps passing queries around. For instance, it can be used in remote/cluster table functions. [#12567](https://github.com/ClickHouse/ClickHouse/pull/12567) ([Amos Bird](https://github.com/amosbird)). + +#### Bug Fix + +* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed inconsistent comparison with primary key of type `FixedString` on index analysis if they're compered with a string of less size. This fixes [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). +* Fix bug which leads to wrong merges assignment if table has partitions with a single part. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). +* If function `bar` was called with specifically crafted arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fixed `.metadata.tmp File exists` error when using `MaterializeMySQL` database engine. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). +* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix SIGSEGV for an attempt to INSERT into StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). +* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). +* Fix possibly incorrect result of function `has` when LowCardinality and Nullable types are involved. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([Mike](https://github.com/myrrc)). +* Cleanup data directory after Zookeeper exceptions during CREATE query for tables with ReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix rare segfaults in functions with combinator `-Resample`, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). +* Check for array size overflow in `topK` aggregate function. Without this check the user may send a query with carefully crafted parameters that will lead to server crash. This closes [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Proxy restart/start/stop/reload of SysVinit to systemd (if it is used). [#14460](https://github.com/ClickHouse/ClickHouse/pull/14460) ([Azat Khuzhin](https://github.com/azat)). +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash during `ALTER` query for table which was created `AS table_function`. Fixes [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). +* Fix exception during ALTER LIVE VIEW query with REFRESH command. LIVE VIEW is an experimental feature. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter. [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). +* Better check for tuple size in SSD cache complex key external dictionaries. This fixes [#13981](https://github.com/ClickHouse/ClickHouse/issues/13981). [#14313](https://github.com/ClickHouse/ClickHouse/pull/14313) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Disallows `CODEC` on `ALIAS` column type. Fixes [#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix GRANT ALL statement when executed on a non-global level. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix arrayJoin() capturing in lambda (exception with logical error message was thrown). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). + +#### Experimental Feature + +* Added `db-generator` tool for random database generation by given SELECT queries. It may faciliate reproducing issues when there is only incomplete bug report from the user. [#14442](https://github.com/ClickHouse/ClickHouse/pull/14442) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) [#10973](https://github.com/ClickHouse/ClickHouse/issues/10973) ([ZeDRoman](https://github.com/ZeDRoman)). + +#### Improvement + +* Allow using multi-volume storage configuration in storage Distributed. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). +* Disallow empty time_zone argument in `toStartOf*` type of functions. [#14509](https://github.com/ClickHouse/ClickHouse/pull/14509) ([Bharat Nallan](https://github.com/bharatnc)). +* MySQL handler returns `OK` for queries like `SET @@var = value`. Such statement is ignored. It is needed because some MySQL drivers send `SET @@` query for setup after handshake https://github.com/ClickHouse/ClickHouse/issues/9336#issuecomment-686222422 . [#14469](https://github.com/ClickHouse/ClickHouse/pull/14469) ([BohuTANG](https://github.com/BohuTANG)). +* Now TTLs will be applied during merge if they were not previously materialized. [#14438](https://github.com/ClickHouse/ClickHouse/pull/14438) ([alesapin](https://github.com/alesapin)). +* Now `clickhouse-obfuscator` supports UUID type as proposed in [#13163](https://github.com/ClickHouse/ClickHouse/issues/13163). [#14409](https://github.com/ClickHouse/ClickHouse/pull/14409) ([dimarub2000](https://github.com/dimarub2000)). +* Added new setting `system_events_show_zero_values` as proposed in [#11384](https://github.com/ClickHouse/ClickHouse/issues/11384). [#14404](https://github.com/ClickHouse/ClickHouse/pull/14404) ([dimarub2000](https://github.com/dimarub2000)). +* Implicitly convert primary key to not null in `MaterializeMySQL` (Same as `MySQL`). Fixes [#14114](https://github.com/ClickHouse/ClickHouse/issues/14114). [#14397](https://github.com/ClickHouse/ClickHouse/pull/14397) ([Winter Zhang](https://github.com/zhang2014)). +* Replace wide integers (256 bit) from boost multiprecision with implementation from https://github.com/cerevra/int. 256bit integers are experimental. [#14229](https://github.com/ClickHouse/ClickHouse/pull/14229) ([Artem Zuikov](https://github.com/4ertus2)). +* Add default compression codec for parts in `system.part_log` with the name `default_compression_codec`. [#14116](https://github.com/ClickHouse/ClickHouse/pull/14116) ([alesapin](https://github.com/alesapin)). +* Add precision argument for `DateTime` type. It allows to use `DateTime` name instead of `DateTime64`. [#13761](https://github.com/ClickHouse/ClickHouse/pull/13761) ([Winter Zhang](https://github.com/zhang2014)). +* Added requirepass authorization for `Redis` external dictionary. [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). +* Improvements in `RabbitMQ` engine: added connection and channels failure handling, proper commits, insert failures handling, better exchanges, queue durability and queue resume opportunity, new queue settings. Fixed tests. [#12761](https://github.com/ClickHouse/ClickHouse/pull/12761) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + +#### Performance Improvement + +* Optimize queries with LIMIT/LIMIT BY/ORDER BY for distributed with GROUP BY sharding_key (under `optimize_skip_unused_shards` and `optimize_distributed_group_by_sharding_key`). [#10373](https://github.com/ClickHouse/ClickHouse/pull/10373) ([Azat Khuzhin](https://github.com/azat)). +* Creating sets for multiple `JOIN` and `IN` in parallel. It may slightly improve performance for queries with several different `IN subquery` expressions. [#14412](https://github.com/ClickHouse/ClickHouse/pull/14412) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Improve Kafka engine performance by providing independent thread for each consumer. Separate thread pool for streaming engines (like Kafka). [#13939](https://github.com/ClickHouse/ClickHouse/pull/13939) ([fastio](https://github.com/fastio)). + +#### Build/Testing/Packaging Improvement + +* Lower binary size in debug build by removing debug info from `Functions`. This is needed only for one internal project in Yandex who is using very old linker. [#14549](https://github.com/ClickHouse/ClickHouse/pull/14549) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Prepare for build with clang 11. [#14455](https://github.com/ClickHouse/ClickHouse/pull/14455) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the logic in backport script. In previous versions it was triggered for any labels of 100% red color. It was strange. [#14433](https://github.com/ClickHouse/ClickHouse/pull/14433) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Integration tests use default base config. All config changes are explicit with main_configs, user_configs and dictionaries parameters for instance. [#13647](https://github.com/ClickHouse/ClickHouse/pull/13647) ([Ilya Yatsishin](https://github.com/qoega)). + + + +## ClickHouse release 20.8 + +### ClickHouse release v20.8.10.13-lts, 2020-12-24 + +#### Bug Fix + +* When server log rotation was configured using `logger.size` parameter with numeric value larger than 2^32, the logs were not rotated properly. [#17905](https://github.com/ClickHouse/ClickHouse/pull/17905) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fixed incorrect initialization of `max_compress_block_size` in MergeTreeWriterSettings with `min_compress_block_size`. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). +* Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). +* Fixed `ALTER` query hang when the corresponding mutation was killed on the different replica. This fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). +* Fixed a bug when mark cache size was underestimated by ClickHouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). +* Fixed `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `ColumnConst` comparison which leads to crash. This fixed [#17088](https://github.com/ClickHouse/ClickHouse/issues/17088) . [#17135](https://github.com/ClickHouse/ClickHouse/pull/17135) ([Amos Bird](https://github.com/amosbird)). +* Fixed bug when `ON CLUSTER` queries may hang forever for non-leader ReplicatedMergeTreeTables. [#17089](https://github.com/ClickHouse/ClickHouse/pull/17089) ([alesapin](https://github.com/alesapin)). +* Avoid unnecessary network errors for remote queries which may be cancelled while execution, like queries with `LIMIT`. [#17006](https://github.com/ClickHouse/ClickHouse/pull/17006) ([Azat Khuzhin](https://github.com/azat)). +* Reresolve the IP of the `format_avro_schema_registry_url` in case of errors. [#16985](https://github.com/ClickHouse/ClickHouse/pull/16985) ([filimonov](https://github.com/filimonov)). +* Fixed possible server crash after `ALTER TABLE ... MODIFY COLUMN ... NewType` when `SELECT` have `WHERE` expression on altering column and alter doesn't finished yet. [#16968](https://github.com/ClickHouse/ClickHouse/pull/16968) ([Amos Bird](https://github.com/amosbird)). +* Install script should always create subdirs in config folders. This is only relevant for Docker build with custom config. [#16936](https://github.com/ClickHouse/ClickHouse/pull/16936) ([filimonov](https://github.com/filimonov)). +* Fixed possible error `Illegal type of argument` for queries with `ORDER BY`. Fixes [#16580](https://github.com/ClickHouse/ClickHouse/issues/16580). [#16928](https://github.com/ClickHouse/ClickHouse/pull/16928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Abort multipart upload if no data was written to WriteBufferFromS3. [#16840](https://github.com/ClickHouse/ClickHouse/pull/16840) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixed crash when using `any` without any arguments. This fixes [#16803](https://github.com/ClickHouse/ClickHouse/issues/16803). [#16826](https://github.com/ClickHouse/ClickHouse/pull/16826) ([Amos Bird](https://github.com/amosbird)). +* Fixed `IN` operator over several columns and tuples with enabled `transform_null_in` setting. Fixes [#15310](https://github.com/ClickHouse/ClickHouse/issues/15310). [#16722](https://github.com/ClickHouse/ClickHouse/pull/16722) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed inconsistent behaviour of `optimize_read_in_order/optimize_aggregation_in_order` with max_threads > 0 and expression in ORDER BY. [#16637](https://github.com/ClickHouse/ClickHouse/pull/16637) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the issue when query optimization was producing wrong result if query contains `ARRAY JOIN`. [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). +* Query is finished faster in case of exception. Cancel execution on remote replicas if exception happens. [#15578](https://github.com/ClickHouse/ClickHouse/pull/15578) ([Azat Khuzhin](https://github.com/azat)). + + +### ClickHouse release v20.8.6.6-lts, 2020-11-13 + +#### Bug Fix + +* Fix rare silent crashes when query profiler is on and ClickHouse is installed on OS with glibc version that has (supposedly) broken asynchronous unwind tables for some functions. This fixes [#15301](https://github.com/ClickHouse/ClickHouse/issues/15301). This fixes [#13098](https://github.com/ClickHouse/ClickHouse/issues/13098). [#16846](https://github.com/ClickHouse/ClickHouse/pull/16846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now when parsing AVRO from input the LowCardinality is removed from type. Fixes [#16188](https://github.com/ClickHouse/ClickHouse/issues/16188). [#16521](https://github.com/ClickHouse/ClickHouse/pull/16521) ([Mike](https://github.com/myrrc)). +* Fix rapid growth of metadata when using MySQL Master -> MySQL Slave -> ClickHouse MaterializeMySQL Engine, and `slave_parallel_worker` enabled on MySQL Slave, by properly shrinking GTID sets. This fixes [#15951](https://github.com/ClickHouse/ClickHouse/issues/15951). [#16504](https://github.com/ClickHouse/ClickHouse/pull/16504) ([TCeason](https://github.com/TCeason)). +* Fix DROP TABLE for Distributed (racy with INSERT). [#16409](https://github.com/ClickHouse/ClickHouse/pull/16409) ([Azat Khuzhin](https://github.com/azat)). +* Fix processing of very large entries in replication queue. Very large entries may appear in ALTER queries if table structure is extremely large (near 1 MB). This fixes [#16307](https://github.com/ClickHouse/ClickHouse/issues/16307). [#16332](https://github.com/ClickHouse/ClickHouse/pull/16332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the inconsistent behaviour when a part of return data could be dropped because the set for its filtration wasn't created. [#16308](https://github.com/ClickHouse/ClickHouse/pull/16308) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix bug with MySQL database. When MySQL server used as database engine is down some queries raise Exception, because they try to get tables from disabled server, while it's unnecessary. For example, query `SELECT ... FROM system.parts` should work only with MergeTree tables and don't touch MySQL database at all. [#16032](https://github.com/ClickHouse/ClickHouse/pull/16032) ([Kruglov Pavel](https://github.com/Avogar)). + + +### ClickHouse release v20.8.5.45-lts, 2020-10-29 + +#### Bug Fix + +* Fix double free in case of exception in function `dictGet`. It could have happened if dictionary was loaded with error. [#16429](https://github.com/ClickHouse/ClickHouse/pull/16429) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix group by with totals/rollup/cube modifers and min/max functions over group by keys. Fixes [#16393](https://github.com/ClickHouse/ClickHouse/issues/16393). [#16397](https://github.com/ClickHouse/ClickHouse/pull/16397) ([Anton Popov](https://github.com/CurtizJ)). +* Fix async Distributed INSERT w/ prefer_localhost_replica=0 and internal_replication. [#16358](https://github.com/ClickHouse/ClickHouse/pull/16358) ([Azat Khuzhin](https://github.com/azat)). +* Fix a possible memory leak during `GROUP BY` with string keys, caused by an error in `TwoLevelStringHashTable` implementation. [#16264](https://github.com/ClickHouse/ClickHouse/pull/16264) ([Amos Bird](https://github.com/amosbird)). +* Fix the case when memory can be overallocated regardless to the limit. This closes [#14560](https://github.com/ClickHouse/ClickHouse/issues/14560). [#16206](https://github.com/ClickHouse/ClickHouse/pull/16206) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `ALTER MODIFY ... ORDER BY` query hang for `ReplicatedVersionedCollapsingMergeTree`. This fixes [#15980](https://github.com/ClickHouse/ClickHouse/issues/15980). [#16011](https://github.com/ClickHouse/ClickHouse/pull/16011) ([alesapin](https://github.com/alesapin)). +* Fix collate name & charset name parser and support `length = 0` for string type. [#16008](https://github.com/ClickHouse/ClickHouse/pull/16008) ([Winter Zhang](https://github.com/zhang2014)). +* Allow to use direct layout for dictionaries with complex keys. [#16007](https://github.com/ClickHouse/ClickHouse/pull/16007) ([Anton Popov](https://github.com/CurtizJ)). +* Prevent replica hang for 5-10 mins when replication error happens after a period of inactivity. [#15987](https://github.com/ClickHouse/ClickHouse/pull/15987) ([filimonov](https://github.com/filimonov)). +* Fix rare segfaults when inserting into or selecting from MaterializedView and concurrently dropping target table (for Atomic database engine). [#15984](https://github.com/ClickHouse/ClickHouse/pull/15984) ([tavplubix](https://github.com/tavplubix)). +* Fix ambiguity in parsing of settings profiles: `CREATE USER ... SETTINGS profile readonly` is now considered as using a profile named `readonly`, not a setting named `profile` with the readonly constraint. This fixes [#15628](https://github.com/ClickHouse/ClickHouse/issues/15628). [#15982](https://github.com/ClickHouse/ClickHouse/pull/15982) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix a crash when database creation fails. [#15954](https://github.com/ClickHouse/ClickHouse/pull/15954) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `DROP TABLE IF EXISTS` failure with `Table ... doesn't exist` error when table is concurrently renamed (for Atomic database engine). Fixed rare deadlock when concurrently executing some DDL queries with multiple tables (like `DROP DATABASE` and `RENAME TABLE`) Fixed `DROP/DETACH DATABASE` failure with `Table ... doesn't exist` when concurrently executing `DROP/DETACH TABLE`. [#15934](https://github.com/ClickHouse/ClickHouse/pull/15934) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible deadlocks in RBAC. [#15875](https://github.com/ClickHouse/ClickHouse/pull/15875) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix exception `Block structure mismatch` in `SELECT ... ORDER BY DESC` queries which were executed after `ALTER MODIFY COLUMN` query. Fixes [#15800](https://github.com/ClickHouse/ClickHouse/issues/15800). [#15852](https://github.com/ClickHouse/ClickHouse/pull/15852) ([alesapin](https://github.com/alesapin)). +* Fix some cases of queries, in which only virtual columns are selected. Previously `Not found column _nothing in block` exception may be thrown. Fixes [#12298](https://github.com/ClickHouse/ClickHouse/issues/12298). [#15756](https://github.com/ClickHouse/ClickHouse/pull/15756) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Cannot find column` which may happen at insertion into `MATERIALIZED VIEW` in case if query for `MV` containes `ARRAY JOIN`. [#15717](https://github.com/ClickHouse/ClickHouse/pull/15717) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed too low default value of `max_replicated_logs_to_keep` setting, which might cause replicas to become lost too often. Improve lost replica recovery process by choosing the most up-to-date replica to clone. Also do not remove old parts from lost replica, detach them instead. [#15701](https://github.com/ClickHouse/ClickHouse/pull/15701) ([tavplubix](https://github.com/tavplubix)). +* Fix error `Cannot add simple transform to empty Pipe` which happened while reading from `Buffer` table which has different structure than destination table. It was possible if destination table returned empty result for query. Fixes [#15529](https://github.com/ClickHouse/ClickHouse/issues/15529). [#15662](https://github.com/ClickHouse/ClickHouse/pull/15662) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug with globs in S3 table function, region from URL was not applied to S3 client configuration. [#15646](https://github.com/ClickHouse/ClickHouse/pull/15646) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Decrement the `ReadonlyReplica` metric when detaching read-only tables. This fixes [#15598](https://github.com/ClickHouse/ClickHouse/issues/15598). [#15592](https://github.com/ClickHouse/ClickHouse/pull/15592) ([sundyli](https://github.com/sundy-li)). +* Throw an error when a single parameter is passed to ReplicatedMergeTree instead of ignoring it. [#15516](https://github.com/ClickHouse/ClickHouse/pull/15516) ([nvartolomei](https://github.com/nvartolomei)). + +#### Improvement + +* Now it's allowed to execute `ALTER ... ON CLUSTER` queries regardless of the `` setting in cluster config. [#16075](https://github.com/ClickHouse/ClickHouse/pull/16075) ([alesapin](https://github.com/alesapin)). +* Unfold `{database}`, `{table}` and `{uuid}` macros in `ReplicatedMergeTree` arguments on table creation. [#16159](https://github.com/ClickHouse/ClickHouse/pull/16159) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.8.4.11-lts, 2020-10-09 + +#### Bug Fix + +* Fix the order of destruction for resources in `ReadFromStorage` step of query plan. It might cause crashes in rare cases. Possibly connected with [#15610](https://github.com/ClickHouse/ClickHouse/issues/15610). [#15645](https://github.com/ClickHouse/ClickHouse/pull/15645) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `Element ... is not a constant expression` error when using `JSON*` function result in `VALUES`, `LIMIT` or right side of `IN` operator. [#15589](https://github.com/ClickHouse/ClickHouse/pull/15589) ([tavplubix](https://github.com/tavplubix)). +* Prevent the possibility of error message `Could not calculate available disk space (statvfs), errno: 4, strerror: Interrupted system call`. This fixes [#15541](https://github.com/ClickHouse/ClickHouse/issues/15541). [#15557](https://github.com/ClickHouse/ClickHouse/pull/15557) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Significantly reduce memory usage in AggregatingInOrderTransform/optimize_aggregation_in_order. [#15543](https://github.com/ClickHouse/ClickHouse/pull/15543) ([Azat Khuzhin](https://github.com/azat)). +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ILIKE` operator stops being case insensitive if `LIKE` with the same pattern was executed. [#15536](https://github.com/ClickHouse/ClickHouse/pull/15536) ([alesapin](https://github.com/alesapin)). +* Fix `Missing columns` errors when selecting columns which absent in data, but depend on other columns which also absent in data. Fixes [#15530](https://github.com/ClickHouse/ClickHouse/issues/15530). [#15532](https://github.com/ClickHouse/ClickHouse/pull/15532) ([alesapin](https://github.com/alesapin)). +* Fix bug with event subscription in DDLWorker which rarely may lead to query hangs in `ON CLUSTER`. Introduced in [#13450](https://github.com/ClickHouse/ClickHouse/issues/13450). [#15477](https://github.com/ClickHouse/ClickHouse/pull/15477) ([alesapin](https://github.com/alesapin)). +* Report proper error when the second argument of `boundingRatio` aggregate function has a wrong type. [#15407](https://github.com/ClickHouse/ClickHouse/pull/15407) ([detailyang](https://github.com/detailyang)). +* Fix race condition during MergeTree table rename and background cleanup. [#15304](https://github.com/ClickHouse/ClickHouse/pull/15304) ([alesapin](https://github.com/alesapin)). +* Fix rare race condition on server startup when system.logs are enabled. [#15300](https://github.com/ClickHouse/ClickHouse/pull/15300) ([alesapin](https://github.com/alesapin)). +* Fix MSan report in QueryLog. Uninitialized memory can be used for the field `memory_usage`. [#15258](https://github.com/ClickHouse/ClickHouse/pull/15258) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix instance crash when using joinGet with LowCardinality types. This fixes [#15214](https://github.com/ClickHouse/ClickHouse/issues/15214). [#15220](https://github.com/ClickHouse/ClickHouse/pull/15220) ([Amos Bird](https://github.com/amosbird)). +* Fix bug in table engine `Buffer` which doesn't allow to insert data of new structure into `Buffer` after `ALTER` query. Fixes [#15117](https://github.com/ClickHouse/ClickHouse/issues/15117). [#15192](https://github.com/ClickHouse/ClickHouse/pull/15192) ([alesapin](https://github.com/alesapin)). +* Adjust decimals field size in mysql column definition packet. [#15152](https://github.com/ClickHouse/ClickHouse/pull/15152) ([maqroll](https://github.com/maqroll)). +* We already use padded comparison between String and FixedString (https://github.com/ClickHouse/ClickHouse/blob/master/src/Functions/FunctionsComparison.h#L333). This PR applies the same logic to field comparison which corrects the usage of FixedString as primary keys. This fixes [#14908](https://github.com/ClickHouse/ClickHouse/issues/14908). [#15033](https://github.com/ClickHouse/ClickHouse/pull/15033) ([Amos Bird](https://github.com/amosbird)). +* If function `bar` was called with specifically crafted arguments, buffer overflow was possible. This closes [#13926](https://github.com/ClickHouse/ClickHouse/issues/13926). [#15028](https://github.com/ClickHouse/ClickHouse/pull/15028) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Cannot rename ... errno: 22, strerror: Invalid argument` error on DDL query execution in Atomic database when running clickhouse-server in docker on Mac OS. [#15024](https://github.com/ClickHouse/ClickHouse/pull/15024) ([tavplubix](https://github.com/tavplubix)). +* Now settings `number_of_free_entries_in_pool_to_execute_mutation` and `number_of_free_entries_in_pool_to_lower_max_size_of_merge` can be equal to `background_pool_size`. [#14975](https://github.com/ClickHouse/ClickHouse/pull/14975) ([alesapin](https://github.com/alesapin)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Publish CPU frequencies per logical core in `system.asynchronous_metrics`. This fixes [#14923](https://github.com/ClickHouse/ClickHouse/issues/14923). [#14924](https://github.com/ClickHouse/ClickHouse/pull/14924) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fixed `.metadata.tmp File exists` error when using `MaterializeMySQL` database engine. [#14898](https://github.com/ClickHouse/ClickHouse/pull/14898) ([Winter Zhang](https://github.com/zhang2014)). +* Fix a problem where the server may get stuck on startup while talking to ZooKeeper, if the configuration files have to be fetched from ZK (using the `from_zk` include option). This fixes [#14814](https://github.com/ClickHouse/ClickHouse/issues/14814). [#14843](https://github.com/ClickHouse/ClickHouse/pull/14843) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix wrong monotonicity detection for shrunk `Int -> Int` cast of signed types. It might lead to incorrect query result. This bug is unveiled in [#14513](https://github.com/ClickHouse/ClickHouse/issues/14513). [#14783](https://github.com/ClickHouse/ClickHouse/pull/14783) ([Amos Bird](https://github.com/amosbird)). +* Fixed the incorrect sorting order of `Nullable` column. This fixes [#14344](https://github.com/ClickHouse/ClickHouse/issues/14344). [#14495](https://github.com/ClickHouse/ClickHouse/pull/14495) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Improvement + +* Now it's possible to change the type of version column for `VersionedCollapsingMergeTree` with `ALTER` query. [#15442](https://github.com/ClickHouse/ClickHouse/pull/15442) ([alesapin](https://github.com/alesapin)). + + +### ClickHouse release v20.8.3.18-stable, 2020-09-18 + +#### Bug Fix + +* Fix the issue when some invocations of `extractAllGroups` function may trigger "Memory limit exceeded" error. This fixes [#13383](https://github.com/ClickHouse/ClickHouse/issues/13383). [#14889](https://github.com/ClickHouse/ClickHouse/pull/14889) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix SIGSEGV for an attempt to INSERT into StorageFile(fd). [#14887](https://github.com/ClickHouse/ClickHouse/pull/14887) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fixed missed default database name in metadata of materialized view when executing `ALTER ... MODIFY QUERY`. [#14664](https://github.com/ClickHouse/ClickHouse/pull/14664) ([tavplubix](https://github.com/tavplubix)). +* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). +* Added the checker as neither calling `lc->isNullable()` nor calling `ls->getDictionaryPtr()->isNullable()` would return the correct result. [#14591](https://github.com/ClickHouse/ClickHouse/pull/14591) ([myrrc](https://github.com/myrrc)). +* Cleanup data directory after Zookeeper exceptions during CreateQuery for StorageReplicatedMergeTree Engine. [#14563](https://github.com/ClickHouse/ClickHouse/pull/14563) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix rare segfaults in functions with combinator -Resample, which could appear in result of overflow with very large parameters. [#14562](https://github.com/ClickHouse/ClickHouse/pull/14562) ([Anton Popov](https://github.com/CurtizJ)). + +#### Improvement + +* Speed up server shutdown process if there are ongoing S3 requests. [#14858](https://github.com/ClickHouse/ClickHouse/pull/14858) ([Pavel Kovalenko](https://github.com/Jokser)). +* Allow using multi-volume storage configuration in storage Distributed. [#14839](https://github.com/ClickHouse/ClickHouse/pull/14839) ([Pavel Kovalenko](https://github.com/Jokser)). +* Speed up server shutdown process if there are ongoing S3 requests. [#14496](https://github.com/ClickHouse/ClickHouse/pull/14496) ([Pavel Kovalenko](https://github.com/Jokser)). +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + + +### ClickHouse release v20.8.2.3-stable, 2020-09-08 + +#### Backward Incompatible Change + +* Now `OPTIMIZE FINAL` query doesn't recalculate TTL for parts that were added before TTL was created. Use `ALTER TABLE ... MATERIALIZE TTL` once to calculate them, after that `OPTIMIZE FINAL` will evaluate TTL's properly. This behavior never worked for replicated tables. [#14220](https://github.com/ClickHouse/ClickHouse/pull/14220) ([alesapin](https://github.com/alesapin)). +* Extend `parallel_distributed_insert_select` setting, adding an option to run `INSERT` into local table. The setting changes type from `Bool` to `UInt64`, so the values `false` and `true` are no longer supported. If you have these values in server configuration, the server will not start. Please replace them with `0` and `1`, respectively. [#14060](https://github.com/ClickHouse/ClickHouse/pull/14060) ([Azat Khuzhin](https://github.com/azat)). +* Remove support for the `ODBCDriver` input/output format. This was a deprecated format once used for communication with the ClickHouse ODBC driver, now long superseded by the `ODBCDriver2` format. Resolves [#13629](https://github.com/ClickHouse/ClickHouse/issues/13629). [#13847](https://github.com/ClickHouse/ClickHouse/pull/13847) ([hexiaoting](https://github.com/hexiaoting)). +* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). + +#### New Feature + +* Add the ability to specify `Default` compression codec for columns that correspond to settings specified in `config.xml`. Implements: [#9074](https://github.com/ClickHouse/ClickHouse/issues/9074). [#14049](https://github.com/ClickHouse/ClickHouse/pull/14049) ([alesapin](https://github.com/alesapin)). +* Support Kerberos authentication in Kafka, using `krb5` and `cyrus-sasl` libraries. [#12771](https://github.com/ClickHouse/ClickHouse/pull/12771) ([Ilya Golshtein](https://github.com/ilejn)). +* Add function `normalizeQuery` that replaces literals, sequences of literals and complex aliases with placeholders. Add function `normalizedQueryHash` that returns identical 64bit hash values for similar queries. It helps to analyze query log. This closes [#11271](https://github.com/ClickHouse/ClickHouse/issues/11271). [#13816](https://github.com/ClickHouse/ClickHouse/pull/13816) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add `time_zones` table. [#13880](https://github.com/ClickHouse/ClickHouse/pull/13880) ([Bharat Nallan](https://github.com/bharatnc)). +* Add function `defaultValueOfTypeName` that returns the default value for a given type. [#13877](https://github.com/ClickHouse/ClickHouse/pull/13877) ([hcz](https://github.com/hczhcz)). +* Add `countDigits(x)` function that count number of decimal digits in integer or decimal column. Add `isDecimalOverflow(d, [p])` function that checks if the value in Decimal column is out of its (or specified) precision. [#14151](https://github.com/ClickHouse/ClickHouse/pull/14151) ([Artem Zuikov](https://github.com/4ertus2)). +* Add `quantileExactLow` and `quantileExactHigh` implementations with respective aliases for `medianExactLow` and `medianExactHigh`. [#13818](https://github.com/ClickHouse/ClickHouse/pull/13818) ([Bharat Nallan](https://github.com/bharatnc)). +* Added `date_trunc` function that truncates a date/time value to a specified date/time part. [#13888](https://github.com/ClickHouse/ClickHouse/pull/13888) ([Vladimir Golovchenko](https://github.com/vladimir-golovchenko)). +* Add new optional section `` to the main config. [#13425](https://github.com/ClickHouse/ClickHouse/pull/13425) ([Vitaly Baranov](https://github.com/vitlibar)). +* Add `ALTER SAMPLE BY` statement that allows to change table sample clause. [#13280](https://github.com/ClickHouse/ClickHouse/pull/13280) ([Amos Bird](https://github.com/amosbird)). +* Function `position` now supports optional `start_pos` argument. [#13237](https://github.com/ClickHouse/ClickHouse/pull/13237) ([vdimir](https://github.com/vdimir)). + +#### Bug Fix + +* Fix visible data clobbering by progress bar in client in interactive mode. This fixes [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) and [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) and [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) and fixes [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect sorting order if `LowCardinality` column when sorting by multiple columns. This fixes [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Check for array size overflow in `topK` aggregate function. Without this check the user may send a query with carefully crafted parameters that will lead to server crash. This closes [#14452](https://github.com/ClickHouse/ClickHouse/issues/14452). [#14467](https://github.com/ClickHouse/ClickHouse/pull/14467) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug which can lead to wrong merges assignment if table has partitions with a single part. [#14444](https://github.com/ClickHouse/ClickHouse/pull/14444) ([alesapin](https://github.com/alesapin)). +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) [#14334](https://github.com/ClickHouse/ClickHouse/pull/14334) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash during `ALTER` query for table which was created `AS table_function`. Fixes [#14212](https://github.com/ClickHouse/ClickHouse/issues/14212). [#14326](https://github.com/ClickHouse/ClickHouse/pull/14326) ([alesapin](https://github.com/alesapin)). +* Fix exception during ALTER LIVE VIEW query with REFRESH command. Live view is an experimental feature. [#14320](https://github.com/ClickHouse/ClickHouse/pull/14320) ([Bharat Nallan](https://github.com/bharatnc)). +* Fix QueryPlan lifetime (for EXPLAIN PIPELINE graph=1) for queries with nested interpreter. [#14315](https://github.com/ClickHouse/ClickHouse/pull/14315) ([Azat Khuzhin](https://github.com/azat)). +* Fix segfault in `clickhouse-odbc-bridge` during schema fetch from some external sources. This PR fixes [#13861](https://github.com/ClickHouse/ClickHouse/issues/13861). [#14267](https://github.com/ClickHouse/ClickHouse/pull/14267) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix crash in mark inclusion search introduced in [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277). [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). +* Fix creation of tables with named tuples. This fixes [#13027](https://github.com/ClickHouse/ClickHouse/issues/13027). [#14143](https://github.com/ClickHouse/ClickHouse/pull/14143) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix formatting of minimal negative decimal numbers. This fixes [#14111](https://github.com/ClickHouse/ClickHouse/issues/14111). [#14119](https://github.com/ClickHouse/ClickHouse/pull/14119) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix `DistributedFilesToInsert` metric (zeroed when it should not). [#14095](https://github.com/ClickHouse/ClickHouse/pull/14095) ([Azat Khuzhin](https://github.com/azat)). +* Fix `pointInPolygon` with const 2d array as polygon. [#14079](https://github.com/ClickHouse/ClickHouse/pull/14079) ([Alexey Ilyukhov](https://github.com/livace)). +* Fixed wrong mount point in extra info for `Poco::Exception: no space left on device`. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). +* Fix GRANT ALL statement when executed on a non-global level. [#13987](https://github.com/ClickHouse/ClickHouse/pull/13987) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix parser to reject create table as table function with engine. [#13940](https://github.com/ClickHouse/ClickHouse/pull/13940) ([hcz](https://github.com/hczhcz)). +* Fix wrong results in select queries with `DISTINCT` keyword and subqueries with UNION ALL in case `optimize_duplicate_order_by_and_distinct` setting is enabled. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed potential deadlock when renaming `Distributed` table. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect sorting for `FixedString` columns when sorting by multiple columns. Fixes [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix potentially imprecise result of `topK`/`topKWeighted` merge (with non-default parameters). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). +* Fix reading from MergeTree table with INDEX of type SET fails when comparing against NULL. This fixes [#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). +* Fix `arrayJoin` capturing in lambda (LOGICAL_ERROR). [#13792](https://github.com/ClickHouse/ClickHouse/pull/13792) ([Azat Khuzhin](https://github.com/azat)). +* Add step overflow check in function `range`. [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `Directory not empty` error when concurrently executing `DROP DATABASE` and `CREATE TABLE`. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add range check for `h3KRing` function. This fixes [#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix race condition between DETACH and background merges. Parts may revive after detach. This is continuation of [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) that did not fix the issue but introduced a test that started to fail in very rare cases, demonstrating the issue. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix logging Settings.Names/Values when log_queries_min_type > QUERY_START. [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). +* Fixes `/replicas_status` endpoint response status code when verbose=1. [#13722](https://github.com/ClickHouse/ClickHouse/pull/13722) ([javi santana](https://github.com/javisantana)). +* Fix incorrect message in `clickhouse-server.init` while checking user and group. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). +* Do not optimize any(arrayJoin()) -> arrayJoin() under `optimize_move_functions_out_of_any` setting. [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix typo in error message about `The value of 'number_of_free_entries_in_pool_to_lower_max_size_of_merge' setting`. [#13678](https://github.com/ClickHouse/ClickHouse/pull/13678) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). +* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix secondary indices corruption in compact parts. Compact parts are experimental feature. [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). +* Fix premature `ON CLUSTER` timeouts for queries that must be executed on a single replica. Fixes [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). +* Fix wrong code in function `netloc`. This fixes [#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible race in `StorageMemory`. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix missing or excessive headers in `TSV/CSVWithNames` formats in HTTP protocol. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). +* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix access to `redis` dictionary after connection was dropped once. It may happen with `cache` and `direct` dictionary layouts. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). +* Removed wrong auth access check when using ClickHouseDictionarySource to query remote tables. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). +* Properly distinguish subqueries in some cases for common subexpression elimination. [#8333](https://github.com/ClickHouse/ClickHouse/issues/8333). [#8367](https://github.com/ClickHouse/ClickHouse/pull/8367) ([Amos Bird](https://github.com/amosbird)). + +#### Improvement + +* Disallows `CODEC` on `ALIAS` column type. Fixes [#13911](https://github.com/ClickHouse/ClickHouse/issues/13911). [#14263](https://github.com/ClickHouse/ClickHouse/pull/14263) ([Bharat Nallan](https://github.com/bharatnc)). +* When waiting for a dictionary update to complete, use the timeout specified by `query_wait_timeout_milliseconds` setting instead of a hard-coded value. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add setting `min_index_granularity_bytes` that protects against accidentally creating a table with very low `index_granularity_bytes` setting. [#14139](https://github.com/ClickHouse/ClickHouse/pull/14139) ([Bharat Nallan](https://github.com/bharatnc)). +* Now it's possible to fetch partitions from clusters that use different ZooKeeper: `ALTER TABLE table_name FETCH PARTITION partition_expr FROM 'zk-name:/path-in-zookeeper'`. It's useful for shipping data to new clusters. [#14155](https://github.com/ClickHouse/ClickHouse/pull/14155) ([Amos Bird](https://github.com/amosbird)). +* Slightly better performance of Memory table if it was constructed from a huge number of very small blocks (that's unlikely). Author of the idea: [Mark Papadakis](https://github.com/markpapadakis). Closes [#14043](https://github.com/ClickHouse/ClickHouse/issues/14043). [#14056](https://github.com/ClickHouse/ClickHouse/pull/14056) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Conditional aggregate functions (for example: `avgIf`, `sumIf`, `maxIf`) should return `NULL` when miss rows and use nullable arguments. [#13964](https://github.com/ClickHouse/ClickHouse/pull/13964) ([Winter Zhang](https://github.com/zhang2014)). +* Increase limit in -Resample combinator to 1M. [#13947](https://github.com/ClickHouse/ClickHouse/pull/13947) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Corrected an error in AvroConfluent format that caused the Kafka table engine to stop processing messages when an abnormally small, malformed, message was received. [#13941](https://github.com/ClickHouse/ClickHouse/pull/13941) ([Gervasio Varela](https://github.com/gervarela)). +* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Better error message for null value of `TabSeparated` format. [#13906](https://github.com/ClickHouse/ClickHouse/pull/13906) ([jiang tao](https://github.com/tomjiang1987)). +* Function `arrayCompact` will compare NaNs bitwise if the type of array elements is Float32/Float64. In previous versions NaNs were always not equal if the type of array elements is Float32/Float64 and were always equal if the type is more complex, like Nullable(Float64). This closes [#13857](https://github.com/ClickHouse/ClickHouse/issues/13857). [#13868](https://github.com/ClickHouse/ClickHouse/pull/13868) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix data race in `lgamma` function. This race was caught only in `tsan`, no side effects a really happened. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Avoid too slow queries when arrays are manipulated as fields. Throw exception instead. [#13753](https://github.com/ClickHouse/ClickHouse/pull/13753) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added Redis requirepass authorization (for redis dictionary source). [#13688](https://github.com/ClickHouse/ClickHouse/pull/13688) ([Ivan Torgashov](https://github.com/it1804)). +* Add MergeTree Write-Ahead-Log (WAL) dump tool. WAL is an experimental feature. [#13640](https://github.com/ClickHouse/ClickHouse/pull/13640) ([BohuTANG](https://github.com/BohuTANG)). +* In previous versions `lcm` function may produce assertion violation in debug build if called with specifically crafted arguments. This fixes [#13368](https://github.com/ClickHouse/ClickHouse/issues/13368). [#13510](https://github.com/ClickHouse/ClickHouse/pull/13510) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Provide monotonicity for `toDate/toDateTime` functions in more cases. Monotonicity information is used for index analysis (more complex queries will be able to use index). Now the input arguments are saturated more naturally and provides better monotonicity. [#13497](https://github.com/ClickHouse/ClickHouse/pull/13497) ([Amos Bird](https://github.com/amosbird)). +* Support compound identifiers for custom settings. Custom settings is an integration point of ClickHouse codebase with other codebases (no benefits for ClickHouse itself) [#13496](https://github.com/ClickHouse/ClickHouse/pull/13496) ([Vitaly Baranov](https://github.com/vitlibar)). +* Move parts from DiskLocal to DiskS3 in parallel. `DiskS3` is an experimental feature. [#13459](https://github.com/ClickHouse/ClickHouse/pull/13459) ([Pavel Kovalenko](https://github.com/Jokser)). +* Enable mixed granularity parts by default. [#13449](https://github.com/ClickHouse/ClickHouse/pull/13449) ([alesapin](https://github.com/alesapin)). +* Proper remote host checking in S3 redirects (security-related thing). [#13404](https://github.com/ClickHouse/ClickHouse/pull/13404) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Add `QueryTimeMicroseconds`, `SelectQueryTimeMicroseconds` and `InsertQueryTimeMicroseconds` to system.events. [#13336](https://github.com/ClickHouse/ClickHouse/pull/13336) ([ianton-ru](https://github.com/ianton-ru)). +* Fix debug assertion when Decimal has too large negative exponent. Fixes [#13188](https://github.com/ClickHouse/ClickHouse/issues/13188). [#13228](https://github.com/ClickHouse/ClickHouse/pull/13228) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added cache layer for DiskS3 (cache to local disk mark and index files). `DiskS3` is an experimental feature. [#13076](https://github.com/ClickHouse/ClickHouse/pull/13076) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fix readline so it dumps history to file now. [#13600](https://github.com/ClickHouse/ClickHouse/pull/13600) ([Amos Bird](https://github.com/amosbird)). +* Create `system` database with `Atomic` engine by default (a preparation to enable `Atomic` database engine by default everywhere). [#13680](https://github.com/ClickHouse/ClickHouse/pull/13680) ([tavplubix](https://github.com/tavplubix)). + +#### Performance Improvement + +* Slightly optimize very short queries with `LowCardinality`. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). +* Enable parallel INSERTs for table engines `Null`, `Memory`, `Distributed` and `Buffer` when the setting `max_insert_threads` is set. [#14120](https://github.com/ClickHouse/ClickHouse/pull/14120) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fail fast if `max_rows_to_read` limit is exceeded on parts scan. The motivation behind this change is to skip ranges scan for all selected parts if it is clear that `max_rows_to_read` is already exceeded. The change is quite noticeable for queries over big number of parts. [#13677](https://github.com/ClickHouse/ClickHouse/pull/13677) ([Roman Khavronenko](https://github.com/hagen1778)). +* Slightly improve performance of aggregation by UInt8/UInt16 keys. [#13099](https://github.com/ClickHouse/ClickHouse/pull/13099) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimize `has()`, `indexOf()` and `countEqual()` functions for `Array(LowCardinality(T))` and constant right arguments. [#12550](https://github.com/ClickHouse/ClickHouse/pull/12550) ([myrrc](https://github.com/myrrc)). +* When performing trivial `INSERT SELECT` queries, automatically set `max_threads` to 1 or `max_insert_threads`, and set `max_block_size` to `min_insert_block_size_rows`. Related to [#5907](https://github.com/ClickHouse/ClickHouse/issues/5907). [#12195](https://github.com/ClickHouse/ClickHouse/pull/12195) ([flynn](https://github.com/ucasFL)). + +#### Experimental Feature + +* ClickHouse can work as MySQL replica - it is implemented by `MaterializeMySQL` database engine. Implements [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#10851](https://github.com/ClickHouse/ClickHouse/pull/10851) ([Winter Zhang](https://github.com/zhang2014)). +* Add types `Int128`, `Int256`, `UInt256` and related functions for them. Extend Decimals with Decimal256 (precision up to 76 digits). New types are under the setting `allow_experimental_bigint_types`. It is working extremely slow and bad. The implementation is incomplete. Please don't use this feature. [#13097](https://github.com/ClickHouse/ClickHouse/pull/13097) ([Artem Zuikov](https://github.com/4ertus2)). + +#### Build/Testing/Packaging Improvement + +* Added `clickhouse install` script, that is useful if you only have a single binary. [#13528](https://github.com/ClickHouse/ClickHouse/pull/13528) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to run `clickhouse` binary without configuration. [#13515](https://github.com/ClickHouse/ClickHouse/pull/13515) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Enable check for typos in code with `codespell`. [#13513](https://github.com/ClickHouse/ClickHouse/pull/13513) [#13511](https://github.com/ClickHouse/ClickHouse/pull/13511) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Enable Shellcheck in CI as a linter of .sh tests. This closes [#13168](https://github.com/ClickHouse/ClickHouse/issues/13168). [#13530](https://github.com/ClickHouse/ClickHouse/pull/13530) [#13529](https://github.com/ClickHouse/ClickHouse/pull/13529) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add a CMake option to fail configuration instead of auto-reconfiguration, enabled by default. [#13687](https://github.com/ClickHouse/ClickHouse/pull/13687) ([Konstantin](https://github.com/podshumok)). +* Expose version of embedded tzdata via TZDATA_VERSION in system.build_options. [#13648](https://github.com/ClickHouse/ClickHouse/pull/13648) ([filimonov](https://github.com/filimonov)). +* Improve generation of system.time_zones table during build. Closes [#14209](https://github.com/ClickHouse/ClickHouse/issues/14209). [#14215](https://github.com/ClickHouse/ClickHouse/pull/14215) ([filimonov](https://github.com/filimonov)). +* Build ClickHouse with the most fresh tzdata from package repository. [#13623](https://github.com/ClickHouse/ClickHouse/pull/13623) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the ability to write js-style comments in skip_list.json. [#14159](https://github.com/ClickHouse/ClickHouse/pull/14159) ([alesapin](https://github.com/alesapin)). +* Ensure that there is no copy-pasted GPL code. [#13514](https://github.com/ClickHouse/ClickHouse/pull/13514) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Switch tests docker images to use test-base parent. [#14167](https://github.com/ClickHouse/ClickHouse/pull/14167) ([Ilya Yatsishin](https://github.com/qoega)). +* Adding retry logic when bringing up docker-compose cluster; Increasing COMPOSE_HTTP_TIMEOUT. [#14112](https://github.com/ClickHouse/ClickHouse/pull/14112) ([vzakaznikov](https://github.com/vzakaznikov)). +* Enabled `system.text_log` in stress test to find more bugs. [#13855](https://github.com/ClickHouse/ClickHouse/pull/13855) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Testflows LDAP module: adding missing certificates and dhparam.pem for openldap4. [#13780](https://github.com/ClickHouse/ClickHouse/pull/13780) ([vzakaznikov](https://github.com/vzakaznikov)). +* ZooKeeper cannot work reliably in unit tests in CI infrastructure. Using unit tests for ZooKeeper interaction with real ZooKeeper is bad idea from the start (unit tests are not supposed to verify complex distributed systems). We already using integration tests for this purpose and they are better suited. [#13745](https://github.com/ClickHouse/ClickHouse/pull/13745) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added docker image for style check. Added style check that all docker and docker compose files are located in docker directory. [#13724](https://github.com/ClickHouse/ClickHouse/pull/13724) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix cassandra build on Mac OS. [#13708](https://github.com/ClickHouse/ClickHouse/pull/13708) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix link error in shared build. [#13700](https://github.com/ClickHouse/ClickHouse/pull/13700) ([Amos Bird](https://github.com/amosbird)). +* Updating LDAP user authentication suite to check that it works with RBAC. [#13656](https://github.com/ClickHouse/ClickHouse/pull/13656) ([vzakaznikov](https://github.com/vzakaznikov)). +* Removed `-DENABLE_CURL_CLIENT` for `contrib/aws`. [#13628](https://github.com/ClickHouse/ClickHouse/pull/13628) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Increasing health-check timeouts for ClickHouse nodes and adding support to dump docker-compose logs if unhealthy containers found. [#13612](https://github.com/ClickHouse/ClickHouse/pull/13612) ([vzakaznikov](https://github.com/vzakaznikov)). +* Make sure [#10977](https://github.com/ClickHouse/ClickHouse/issues/10977) is invalid. [#13539](https://github.com/ClickHouse/ClickHouse/pull/13539) ([Amos Bird](https://github.com/amosbird)). +* Skip PR's from robot-clickhouse. [#13489](https://github.com/ClickHouse/ClickHouse/pull/13489) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Move Dockerfiles from integration tests to `docker/test` directory. docker_compose files are available in `runner` docker container. Docker images are built in CI and not in integration tests. [#13448](https://github.com/ClickHouse/ClickHouse/pull/13448) ([Ilya Yatsishin](https://github.com/qoega)). + + +## ClickHouse release 20.7 + +### ClickHouse release v20.7.2.30-stable, 2020-08-31 + +#### Backward Incompatible Change + +* Function `modulo` (operator `%`) with at least one floating point number as argument will calculate remainder of division directly on floating point numbers without converting both arguments to integers. It makes behaviour compatible with most of DBMS. This also applicable for Date and DateTime data types. Added alias `mod`. This closes [#7323](https://github.com/ClickHouse/ClickHouse/issues/7323). [#12585](https://github.com/ClickHouse/ClickHouse/pull/12585) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Deprecate special printing of zero Date/DateTime values as `0000-00-00` and `0000-00-00 00:00:00`. [#12442](https://github.com/ClickHouse/ClickHouse/pull/12442) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* The function `groupArrayMoving*` was not working for distributed queries. It's result was calculated within incorrect data type (without promotion to the largest type). The function `groupArrayMovingAvg` was returning integer number that was inconsistent with the `avg` function. This fixes [#12568](https://github.com/ClickHouse/ClickHouse/issues/12568). [#12622](https://github.com/ClickHouse/ClickHouse/pull/12622) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add sanity check for MergeTree settings. If the settings are incorrect, the server will refuse to start or to create a table, printing detailed explanation to the user. [#13153](https://github.com/ClickHouse/ClickHouse/pull/13153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Protect from the cases when user may set `background_pool_size` to value lower than `number_of_free_entries_in_pool_to_execute_mutation` or `number_of_free_entries_in_pool_to_lower_max_size_of_merge`. In these cases ALTERs won't work or the maximum size of merge will be too limited. It will throw exception explaining what to do. This closes [#10897](https://github.com/ClickHouse/ClickHouse/issues/10897). [#12728](https://github.com/ClickHouse/ClickHouse/pull/12728) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). + +#### New Feature + +* Polygon dictionary type that provides efficient "reverse geocoding" lookups - to find the region by coordinates in a dictionary of many polygons (world map). It is using carefully optimized algorithm with recursive grids to maintain low CPU and memory usage. [#9278](https://github.com/ClickHouse/ClickHouse/pull/9278) ([achulkov2](https://github.com/achulkov2)). +* Added support of LDAP authentication for preconfigured users ("Simple Bind" method). [#11234](https://github.com/ClickHouse/ClickHouse/pull/11234) ([Denis Glazachev](https://github.com/traceon)). +* Introduce setting `alter_partition_verbose_result` which outputs information about touched parts for some types of `ALTER TABLE ... PARTITION ...` queries (currently `ATTACH` and `FREEZE`). Closes [#8076](https://github.com/ClickHouse/ClickHouse/issues/8076). [#13017](https://github.com/ClickHouse/ClickHouse/pull/13017) ([alesapin](https://github.com/alesapin)). +* Add `bayesAB` function for bayesian-ab-testing. [#12327](https://github.com/ClickHouse/ClickHouse/pull/12327) ([achimbab](https://github.com/achimbab)). +* Added `system.crash_log` table into which stack traces for fatal errors are collected. This table should be empty. [#12316](https://github.com/ClickHouse/ClickHouse/pull/12316) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added http headers `X-ClickHouse-Database` and `X-ClickHouse-Format` which may be used to set default database and output format. [#12981](https://github.com/ClickHouse/ClickHouse/pull/12981) ([hcz](https://github.com/hczhcz)). +* Add `minMap` and `maxMap` functions support to `SimpleAggregateFunction`. [#12662](https://github.com/ClickHouse/ClickHouse/pull/12662) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Add setting `allow_non_metadata_alters` which restricts to execute `ALTER` queries which modify data on disk. Disabled be default. Closes [#11547](https://github.com/ClickHouse/ClickHouse/issues/11547). [#12635](https://github.com/ClickHouse/ClickHouse/pull/12635) ([alesapin](https://github.com/alesapin)). +* A function `formatRow` is added to support turning arbitrary expressions into a string via given format. It's useful for manipulating SQL outputs and is quite versatile combined with the `columns` function. [#12574](https://github.com/ClickHouse/ClickHouse/pull/12574) ([Amos Bird](https://github.com/amosbird)). +* Add `FROM_UNIXTIME` function for compatibility with MySQL, related to [12149](https://github.com/ClickHouse/ClickHouse/issues/12149). [#12484](https://github.com/ClickHouse/ClickHouse/pull/12484) ([flynn](https://github.com/ucasFL)). +* Allow Nullable types as keys in MergeTree tables if `allow_nullable_key` table setting is enabled. Closes [#5319](https://github.com/ClickHouse/ClickHouse/issues/5319). [#12433](https://github.com/ClickHouse/ClickHouse/pull/12433) ([Amos Bird](https://github.com/amosbird)). +* Integration with [COS](https://intl.cloud.tencent.com/product/cos). [#12386](https://github.com/ClickHouse/ClickHouse/pull/12386) ([fastio](https://github.com/fastio)). +* Add `mapAdd` and `mapSubtract` functions for adding/subtracting key-mapped values. [#11735](https://github.com/ClickHouse/ClickHouse/pull/11735) ([Ildus Kurbangaliev](https://github.com/ildus)). + +#### Bug Fix + +* Fix premature `ON CLUSTER` timeouts for queries that must be executed on a single replica. Fixes [#6704](https://github.com/ClickHouse/ClickHouse/issues/6704), [#7228](https://github.com/ClickHouse/ClickHouse/issues/7228), [#13361](https://github.com/ClickHouse/ClickHouse/issues/13361), [#11884](https://github.com/ClickHouse/ClickHouse/issues/11884). [#13450](https://github.com/ClickHouse/ClickHouse/pull/13450) ([alesapin](https://github.com/alesapin)). +* Fix crash in mark inclusion search introduced in [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277). [#14225](https://github.com/ClickHouse/ClickHouse/pull/14225) ([Amos Bird](https://github.com/amosbird)). +* Fix race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Fix visible data clobbering by progress bar in client in interactive mode. This fixes [#12562](https://github.com/ClickHouse/ClickHouse/issues/12562) and [#13369](https://github.com/ClickHouse/ClickHouse/issues/13369) and [#13584](https://github.com/ClickHouse/ClickHouse/issues/13584) and fixes [#12964](https://github.com/ClickHouse/ClickHouse/issues/12964). [#13691](https://github.com/ClickHouse/ClickHouse/pull/13691) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect sorting order for `LowCardinality` columns when ORDER BY multiple columns is used. This fixes [#13958](https://github.com/ClickHouse/ClickHouse/issues/13958). [#14223](https://github.com/ClickHouse/ClickHouse/pull/14223) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Removed hardcoded timeout, which wrongly overruled `query_wait_timeout_milliseconds` setting for cache-dictionary. [#14105](https://github.com/ClickHouse/ClickHouse/pull/14105) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed wrong mount point in extra info for `Poco::Exception: no space left on device`. [#14050](https://github.com/ClickHouse/ClickHouse/pull/14050) ([tavplubix](https://github.com/tavplubix)). +* Fix wrong query optimization of select queries with `DISTINCT` keyword when subqueries also have `DISTINCT` in case `optimize_duplicate_order_by_and_distinct` setting is enabled. [#13925](https://github.com/ClickHouse/ClickHouse/pull/13925) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed potential deadlock when renaming `Distributed` table. [#13922](https://github.com/ClickHouse/ClickHouse/pull/13922) ([tavplubix](https://github.com/tavplubix)). +* Fix incorrect sorting for `FixedString` columns when ORDER BY multiple columns is used. Fixes [#13182](https://github.com/ClickHouse/ClickHouse/issues/13182). [#13887](https://github.com/ClickHouse/ClickHouse/pull/13887) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix potentially lower precision of `topK`/`topKWeighted` aggregations (with non-default parameters). [#13817](https://github.com/ClickHouse/ClickHouse/pull/13817) ([Azat Khuzhin](https://github.com/azat)). +* Fix reading from MergeTree table with INDEX of type SET fails when compared against NULL. This fixes [#13686](https://github.com/ClickHouse/ClickHouse/issues/13686). [#13793](https://github.com/ClickHouse/ClickHouse/pull/13793) ([Amos Bird](https://github.com/amosbird)). +* Fix step overflow in function `range()`. [#13790](https://github.com/ClickHouse/ClickHouse/pull/13790) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `Directory not empty` error when concurrently executing `DROP DATABASE` and `CREATE TABLE`. [#13756](https://github.com/ClickHouse/ClickHouse/pull/13756) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add range check for `h3KRing` function. This fixes [#13633](https://github.com/ClickHouse/ClickHouse/issues/13633). [#13752](https://github.com/ClickHouse/ClickHouse/pull/13752) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix race condition between DETACH and background merges. Parts may revive after detach. This is continuation of [#8602](https://github.com/ClickHouse/ClickHouse/issues/8602) that did not fix the issue but introduced a test that started to fail in very rare cases, demonstrating the issue. [#13746](https://github.com/ClickHouse/ClickHouse/pull/13746) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix logging Settings.Names/Values when `log_queries_min_type` greater than `QUERY_START`. [#13737](https://github.com/ClickHouse/ClickHouse/pull/13737) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect message in `clickhouse-server.init` while checking user and group. [#13711](https://github.com/ClickHouse/ClickHouse/pull/13711) ([ylchou](https://github.com/ylchou)). +* Do not optimize `any(arrayJoin())` to `arrayJoin()` under `optimize_move_functions_out_of_any`. [#13681](https://github.com/ClickHouse/ClickHouse/pull/13681) ([Azat Khuzhin](https://github.com/azat)). +* Fixed possible deadlock in concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). +* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix secondary indices corruption in compact parts (compact parts is an experimental feature). [#13538](https://github.com/ClickHouse/ClickHouse/pull/13538) ([Anton Popov](https://github.com/CurtizJ)). +* Fix wrong code in function `netloc`. This fixes [#13335](https://github.com/ClickHouse/ClickHouse/issues/13335). [#13446](https://github.com/ClickHouse/ClickHouse/pull/13446) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix invalid return type for comparison of tuples with `NULL` elements. Fixes [#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix wrong optimization caused `aggregate function any(x) is found inside another aggregate function in query` error with `SET optimize_move_functions_out_of_any = 1` and aliases inside `any()`. [#13419](https://github.com/ClickHouse/ClickHouse/pull/13419) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix possible race in `StorageMemory`. [#13416](https://github.com/ClickHouse/ClickHouse/pull/13416) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix empty output for `Arrow` and `Parquet` formats in case if query return zero rows. It was done because empty output is not valid for this formats. [#13399](https://github.com/ClickHouse/ClickHouse/pull/13399) ([hcz](https://github.com/hczhcz)). +* Fix select queries with constant columns and prefix of primary key in `ORDER BY` clause. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). +* Fix `PrettyCompactMonoBlock` for clickhouse-local. Fix extremes/totals with `PrettyCompactMonoBlock`. Fixes [#7746](https://github.com/ClickHouse/ClickHouse/issues/7746). [#13394](https://github.com/ClickHouse/ClickHouse/pull/13394) ([Azat Khuzhin](https://github.com/azat)). +* Fixed deadlock in system.text_log. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). It is a part of [#12339](https://github.com/ClickHouse/ClickHouse/issues/12339). This fixes [#12325](https://github.com/ClickHouse/ClickHouse/issues/12325). [#13386](https://github.com/ClickHouse/ClickHouse/pull/13386) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed `File(TSVWithNames*)` (header was written multiple times), fixed `clickhouse-local --format CSVWithNames*` (lacks header, broken after [#12197](https://github.com/ClickHouse/ClickHouse/issues/12197)), fixed `clickhouse-local --format CSVWithNames*` with zero rows (lacks header). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). +* Fix segfault when function `groupArrayMovingSum` deserializes empty state. Fixes [#13339](https://github.com/ClickHouse/ClickHouse/issues/13339). [#13341](https://github.com/ClickHouse/ClickHouse/pull/13341) ([alesapin](https://github.com/alesapin)). +* Throw error on `arrayJoin()` function in `JOIN ON` section. [#13330](https://github.com/ClickHouse/ClickHouse/pull/13330) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix crash in `LEFT ASOF JOIN` with `join_use_nulls=1`. [#13291](https://github.com/ClickHouse/ClickHouse/pull/13291) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix possible error `Totals having transform was already added to pipeline` in case of a query from delayed replica. [#13290](https://github.com/ClickHouse/ClickHouse/pull/13290) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The server may crash if user passed specifically crafted arguments to the function `h3ToChildren`. This fixes [#13275](https://github.com/ClickHouse/ClickHouse/issues/13275). [#13277](https://github.com/ClickHouse/ClickHouse/pull/13277) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with `NaN` values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix assertion in KeyCondition when primary key contains expression with monotonic function and query contains comparison with constant whose type is different. This fixes [#12465](https://github.com/ClickHouse/ClickHouse/issues/12465). [#13251](https://github.com/ClickHouse/ClickHouse/pull/13251) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Return passed number for numbers with MSB set in function roundUpToPowerOfTwoOrZero(). It prevents potential errors in case of overflow of array sizes. [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). +* Fix function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix DateTime64 conversion functions with constant argument. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)). +* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix access to `redis` dictionary after connection was dropped once. It may happen with `cache` and `direct` dictionary layouts. [#13082](https://github.com/ClickHouse/ClickHouse/pull/13082) ([Anton Popov](https://github.com/CurtizJ)). +* Fix wrong index analysis with functions. It could lead to some data parts being skipped when reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash which was possible for queries with `ORDER BY` tuple and small `LIMIT`. Fixes [#12623](https://github.com/ClickHouse/ClickHouse/issues/12623). [#13009](https://github.com/ClickHouse/ClickHouse/pull/13009) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `Block structure mismatch` error for queries with `UNION` and `JOIN`. Fixes [#12602](https://github.com/ClickHouse/ClickHouse/issues/12602). [#12989](https://github.com/ClickHouse/ClickHouse/pull/12989) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Corrected `merge_with_ttl_timeout` logic which did not work well when expiration affected more than one partition over one time interval. (Authored by @excitoon). [#12982](https://github.com/ClickHouse/ClickHouse/pull/12982) ([Alexander Kazakov](https://github.com/Akazz)). +* Fix columns duplication for range hashed dictionary created from DDL query. This fixes [#10605](https://github.com/ClickHouse/ClickHouse/issues/10605). [#12857](https://github.com/ClickHouse/ClickHouse/pull/12857) ([alesapin](https://github.com/alesapin)). +* Fix unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix rare bug when `ALTER DELETE` and `ALTER MODIFY COLUMN` queries executed simultaneously as a single mutation. Bug leads to an incorrect amount of rows in `count.txt` and as a consequence incorrect data in part. Also, fix a small bug with simultaneous `ALTER RENAME COLUMN` and `ALTER ADD COLUMN`. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). +* Wrong credentials being used when using `clickhouse` dictionary source to query remote tables. [#12756](https://github.com/ClickHouse/ClickHouse/pull/12756) ([sundyli](https://github.com/sundy-li)). +* Fix `CAST(Nullable(String), Enum())`. [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). +* Fix performance with large tuples, which are interpreted as functions in `IN` section. The case when user writes `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fix memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fix wrong optimization `optimize_move_functions_out_of_any=1` in case of `any(func())`. [#12664](https://github.com/ClickHouse/ClickHouse/pull/12664) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fix SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Add support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* CREATE USER IF NOT EXISTS now doesn't throw exception if the user exists. This fixes [#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)). +* Exception `There is no supertype...` can be thrown during `ALTER ... UPDATE` in unexpected cases (e.g. when subtracting from UInt64 column). This fixes [#7306](https://github.com/ClickHouse/ClickHouse/issues/7306). This fixes [#4165](https://github.com/ClickHouse/ClickHouse/issues/4165). [#12633](https://github.com/ClickHouse/ClickHouse/pull/12633) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible `Pipeline stuck` error for queries with external sorting. Fixes [#12617](https://github.com/ClickHouse/ClickHouse/issues/12617). [#12618](https://github.com/ClickHouse/ClickHouse/pull/12618) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix error `Output of TreeExecutor is not sorted` for `OPTIMIZE DEDUPLICATE`. Fixes [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572). [#12613](https://github.com/ClickHouse/ClickHouse/pull/12613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix the issue when alias on result of function `any` can be lost during query optimization. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). +* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). +* Now ClickHouse will recalculate checksums for parts when file `checksums.txt` is absent. Broken since [#9827](https://github.com/ClickHouse/ClickHouse/issues/9827). [#12545](https://github.com/ClickHouse/ClickHouse/pull/12545) ([alesapin](https://github.com/alesapin)). +* Fix bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Fixing race condition in live view tables which could cause data duplication. LIVE VIEW is an experimental feature. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fix backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash in JOIN with dictionary when we are joining over expression of dictionary key: `t JOIN dict ON expr(dict.id) = t.id`. Disable dictionary join optimisation for this case. [#12458](https://github.com/ClickHouse/ClickHouse/pull/12458) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix overflow when very large LIMIT or OFFSET is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* kafka: fix SIGSEGV if there is a message with error in the middle of the batch. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). + +#### Improvement + +* Keep smaller amount of logs in ZooKeeper. Avoid excessive growing of ZooKeeper nodes in case of offline replicas when having many servers/tables/inserts. [#13100](https://github.com/ClickHouse/ClickHouse/pull/13100) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now exceptions forwarded to the client if an error happened during ALTER or mutation. Closes [#11329](https://github.com/ClickHouse/ClickHouse/issues/11329). [#12666](https://github.com/ClickHouse/ClickHouse/pull/12666) ([alesapin](https://github.com/alesapin)). +* Add `QueryTimeMicroseconds`, `SelectQueryTimeMicroseconds` and `InsertQueryTimeMicroseconds` to `system.events`, along with system.metrics, processes, query_log, etc. [#13028](https://github.com/ClickHouse/ClickHouse/pull/13028) ([ianton-ru](https://github.com/ianton-ru)). +* Added `SelectedRows` and `SelectedBytes` to `system.events`, along with system.metrics, processes, query_log, etc. [#12638](https://github.com/ClickHouse/ClickHouse/pull/12638) ([ianton-ru](https://github.com/ianton-ru)). +* Added `current_database` information to `system.query_log`. [#12652](https://github.com/ClickHouse/ClickHouse/pull/12652) ([Amos Bird](https://github.com/amosbird)). +* Allow `TabSeparatedRaw` as input format. [#12009](https://github.com/ClickHouse/ClickHouse/pull/12009) ([hcz](https://github.com/hczhcz)). +* Now `joinGet` supports multi-key lookup. [#12418](https://github.com/ClickHouse/ClickHouse/pull/12418) ([Amos Bird](https://github.com/amosbird)). +* Allow `*Map` aggregate functions to work on Arrays with NULLs. Fixes [#13157](https://github.com/ClickHouse/ClickHouse/issues/13157). [#13225](https://github.com/ClickHouse/ClickHouse/pull/13225) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid overflow in parsing of DateTime values that will lead to negative unix timestamp in their timezone (for example, `1970-01-01 00:00:00` in Moscow). Saturate to zero instead. This fixes [#3470](https://github.com/ClickHouse/ClickHouse/issues/3470). This fixes [#4172](https://github.com/ClickHouse/ClickHouse/issues/4172). [#12443](https://github.com/ClickHouse/ClickHouse/pull/12443) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* AvroConfluent: Skip Kafka tombstone records - Support skipping broken records [#13203](https://github.com/ClickHouse/ClickHouse/pull/13203) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix data race in `lgamma` function. This race was caught only in `tsan`, no side effects really happened. [#13842](https://github.com/ClickHouse/ClickHouse/pull/13842) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a 'Week'-interval formatting for ATTACH/ALTER/CREATE QUOTA-statements. [#13417](https://github.com/ClickHouse/ClickHouse/pull/13417) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). +* Now broken parts are also reported when encountered in compact part processing. Compact parts is an experimental feature. [#13282](https://github.com/ClickHouse/ClickHouse/pull/13282) ([Amos Bird](https://github.com/amosbird)). +* Fix assert in `geohashesInBox`. This fixes [#12554](https://github.com/ClickHouse/ClickHouse/issues/12554). [#13229](https://github.com/ClickHouse/ClickHouse/pull/13229) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix assert in `parseDateTimeBestEffort`. This fixes [#12649](https://github.com/ClickHouse/ClickHouse/issues/12649). [#13227](https://github.com/ClickHouse/ClickHouse/pull/13227) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Minor optimization in Processors/PipelineExecutor: breaking out of a loop because it makes sense to do so. [#13058](https://github.com/ClickHouse/ClickHouse/pull/13058) ([Mark Papadakis](https://github.com/markpapadakis)). +* Support TRUNCATE table without TABLE keyword. [#12653](https://github.com/ClickHouse/ClickHouse/pull/12653) ([Winter Zhang](https://github.com/zhang2014)). +* Fix explain query format overwrite by default. This fixes [#12541](https://github.com/ClickHouse/ClickHouse/issues/12432). [#12541](https://github.com/ClickHouse/ClickHouse/pull/12541) ([BohuTANG](https://github.com/BohuTANG)). +* Allow to set JOIN kind and type in more standad way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). +* Changes default value for `multiple_joins_rewriter_version` to 2. It enables new multiple joins rewriter that knows about column names. [#12469](https://github.com/ClickHouse/ClickHouse/pull/12469) ([Artem Zuikov](https://github.com/4ertus2)). +* Add several metrics for requests to S3 storages. [#12464](https://github.com/ClickHouse/ClickHouse/pull/12464) ([ianton-ru](https://github.com/ianton-ru)). +* Use correct default secure port for clickhouse-benchmark with `--secure` argument. This fixes [#11044](https://github.com/ClickHouse/ClickHouse/issues/11044). [#12440](https://github.com/ClickHouse/ClickHouse/pull/12440) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Rollback insertion errors in `Log`, `TinyLog`, `StripeLog` engines. In previous versions insertion error lead to inconsisent table state (this works as documented and it is normal for these table engines). This fixes [#12402](https://github.com/ClickHouse/ClickHouse/issues/12402). [#12426](https://github.com/ClickHouse/ClickHouse/pull/12426) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Implement `RENAME DATABASE` and `RENAME DICTIONARY` for `Atomic` database engine - Add implicit `{uuid}` macro, which can be used in ZooKeeper path for `ReplicatedMergeTree`. It works with `CREATE ... ON CLUSTER ...` queries. Set `show_table_uuid_in_table_create_query_if_not_nil` to `true` to use it. - Make `ReplicatedMergeTree` engine arguments optional, `/clickhouse/tables/{uuid}/{shard}/` and `{replica}` are used by default. Closes [#12135](https://github.com/ClickHouse/ClickHouse/issues/12135). - Minor fixes. - These changes break backward compatibility of `Atomic` database engine. Previously created `Atomic` databases must be manually converted to new format. Atomic database is an experimental feature. [#12343](https://github.com/ClickHouse/ClickHouse/pull/12343) ([tavplubix](https://github.com/tavplubix)). +* Separated `AWSAuthV4Signer` into different logger, removed excessive `AWSClient: AWSClient` from log messages. [#12320](https://github.com/ClickHouse/ClickHouse/pull/12320) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Better exception message in disk access storage. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error message about adaptive granularity. [#12624](https://github.com/ClickHouse/ClickHouse/pull/12624) ([alesapin](https://github.com/alesapin)). +* Fix SETTINGS parse after FORMAT. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). +* If MergeTree table does not contain ORDER BY or PARTITION BY, it was possible to request ALTER to CLEAR all the columns and ALTER will stuck. Fixed [#7941](https://github.com/ClickHouse/ClickHouse/issues/7941). [#12382](https://github.com/ClickHouse/ClickHouse/pull/12382) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid re-loading completion from the history file after each query (to avoid history overlaps with other client sessions). [#13086](https://github.com/ClickHouse/ClickHouse/pull/13086) ([Azat Khuzhin](https://github.com/azat)). + +#### Performance Improvement + +* Lower memory usage for some operations up to 2 times. [#12424](https://github.com/ClickHouse/ClickHouse/pull/12424) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimize PK lookup for queries that match exact PK range. [#12277](https://github.com/ClickHouse/ClickHouse/pull/12277) ([Ivan Babrou](https://github.com/bobrik)). +* Slightly optimize very short queries with `LowCardinality`. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). +* Slightly improve performance of aggregation by UInt8/UInt16 keys. [#13091](https://github.com/ClickHouse/ClickHouse/pull/13091) and [#13055](https://github.com/ClickHouse/ClickHouse/pull/13055) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Push down `LIMIT` step for query plan (inside subqueries). [#13016](https://github.com/ClickHouse/ClickHouse/pull/13016) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Parallel primary key lookup and skipping index stages on parts, as described in [#11564](https://github.com/ClickHouse/ClickHouse/issues/11564). [#12589](https://github.com/ClickHouse/ClickHouse/pull/12589) ([Ivan Babrou](https://github.com/bobrik)). +* Converting String-type arguments of function "if" and "transform" into enum if `set optimize_if_transform_strings_to_enum = 1`. [#12515](https://github.com/ClickHouse/ClickHouse/pull/12515) ([Artem Zuikov](https://github.com/4ertus2)). +* Replaces monotonic functions with its argument in `ORDER BY` if `set optimize_monotonous_functions_in_order_by=1`. [#12467](https://github.com/ClickHouse/ClickHouse/pull/12467) ([Artem Zuikov](https://github.com/4ertus2)). +* Add order by optimization that rewrites `ORDER BY x, f(x)` with `ORDER by x` if `set optimize_redundant_functions_in_order_by = 1`. [#12404](https://github.com/ClickHouse/ClickHouse/pull/12404) ([Artem Zuikov](https://github.com/4ertus2)). +* Allow pushdown predicate when subquery contains `WITH` clause. This fixes [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). +* Improve performance of reading from compact parts. Compact parts is an experimental feature. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Attempt to implement streaming optimization in `DiskS3`. DiskS3 is an experimental feature. [#12434](https://github.com/ClickHouse/ClickHouse/pull/12434) ([Vladimir Chebotarev](https://github.com/excitoon)). + +#### Build/Testing/Packaging Improvement + +* Use `shellcheck` for sh tests linting. [#13200](https://github.com/ClickHouse/ClickHouse/pull/13200) [#13207](https://github.com/ClickHouse/ClickHouse/pull/13207) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add script which set labels for pull requests in GitHub hook. [#13183](https://github.com/ClickHouse/ClickHouse/pull/13183) ([alesapin](https://github.com/alesapin)). +* Remove some of recursive submodules. See [#13378](https://github.com/ClickHouse/ClickHouse/issues/13378). [#13379](https://github.com/ClickHouse/ClickHouse/pull/13379) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Ensure that all the submodules are from proper URLs. Continuation of [#13379](https://github.com/ClickHouse/ClickHouse/issues/13379). This fixes [#13378](https://github.com/ClickHouse/ClickHouse/issues/13378). [#13397](https://github.com/ClickHouse/ClickHouse/pull/13397) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added support for user-declared settings, which can be accessed from inside queries. This is needed when ClickHouse engine is used as a component of another system. [#13013](https://github.com/ClickHouse/ClickHouse/pull/13013) ([Vitaly Baranov](https://github.com/vitlibar)). +* Added testing for RBAC functionality of INSERT privilege in TestFlows. Expanded tables on which SELECT is being tested. Added Requirements to match new table engine tests. [#13340](https://github.com/ClickHouse/ClickHouse/pull/13340) ([MyroTk](https://github.com/MyroTk)). +* Fix timeout error during server restart in the stress test. [#13321](https://github.com/ClickHouse/ClickHouse/pull/13321) ([alesapin](https://github.com/alesapin)). +* Now fast test will wait server with retries. [#13284](https://github.com/ClickHouse/ClickHouse/pull/13284) ([alesapin](https://github.com/alesapin)). +* Function `materialize()` (the function for ClickHouse testing) will work for NULL as expected - by transforming it to non-constant column. [#13212](https://github.com/ClickHouse/ClickHouse/pull/13212) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix libunwind build in AArch64. This fixes [#13204](https://github.com/ClickHouse/ClickHouse/issues/13204). [#13208](https://github.com/ClickHouse/ClickHouse/pull/13208) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Even more retries in zkutil gtest to prevent test flakiness. [#13165](https://github.com/ClickHouse/ClickHouse/pull/13165) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Small fixes to the RBAC TestFlows. [#13152](https://github.com/ClickHouse/ClickHouse/pull/13152) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fixing `00960_live_view_watch_events_live.py` test. [#13108](https://github.com/ClickHouse/ClickHouse/pull/13108) ([vzakaznikov](https://github.com/vzakaznikov)). +* Improve cache purge in documentation deploy script. [#13107](https://github.com/ClickHouse/ClickHouse/pull/13107) ([alesapin](https://github.com/alesapin)). +* Rewrote some orphan tests to gtest. Removed useless includes from tests. [#13073](https://github.com/ClickHouse/ClickHouse/pull/13073) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Added tests for RBAC functionality of `SELECT` privilege in TestFlows. [#13061](https://github.com/ClickHouse/ClickHouse/pull/13061) ([Ritaank Tiwari](https://github.com/ritaank)). +* Rerun some tests in fast test check. [#12992](https://github.com/ClickHouse/ClickHouse/pull/12992) ([alesapin](https://github.com/alesapin)). +* Fix MSan error in "rdkafka" library. This closes [#12990](https://github.com/ClickHouse/ClickHouse/issues/12990). Updated `rdkafka` to version 1.5 (master). [#12991](https://github.com/ClickHouse/ClickHouse/pull/12991) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan report in base64 if tests were run on server with AVX-512. This fixes [#12318](https://github.com/ClickHouse/ClickHouse/issues/12318). Author: @qoega. [#12441](https://github.com/ClickHouse/ClickHouse/pull/12441) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan report in HDFS library. This closes [#12330](https://github.com/ClickHouse/ClickHouse/issues/12330). [#12453](https://github.com/ClickHouse/ClickHouse/pull/12453) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Check an ability that we able to restore the backup from an old version to the new version. This closes [#8979](https://github.com/ClickHouse/ClickHouse/issues/8979). [#12959](https://github.com/ClickHouse/ClickHouse/pull/12959) ([alesapin](https://github.com/alesapin)). +* Do not build helper_container image inside integrational tests. Build docker container in CI and use pre-built helper_container in integration tests. [#12953](https://github.com/ClickHouse/ClickHouse/pull/12953) ([Ilya Yatsishin](https://github.com/qoega)). +* Add a test for `ALTER TABLE CLEAR COLUMN` query for primary key columns. [#12951](https://github.com/ClickHouse/ClickHouse/pull/12951) ([alesapin](https://github.com/alesapin)). +* Increased timeouts in testflows tests. [#12949](https://github.com/ClickHouse/ClickHouse/pull/12949) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fix build of test under Mac OS X. This closes [#12767](https://github.com/ClickHouse/ClickHouse/issues/12767). [#12772](https://github.com/ClickHouse/ClickHouse/pull/12772) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Connector-ODBC updated to mysql-connector-odbc-8.0.21. [#12739](https://github.com/ClickHouse/ClickHouse/pull/12739) ([Ilya Yatsishin](https://github.com/qoega)). +* Adding RBAC syntax tests in TestFlows. [#12642](https://github.com/ClickHouse/ClickHouse/pull/12642) ([vzakaznikov](https://github.com/vzakaznikov)). +* Improve performance of TestKeeper. This will speedup tests with heavy usage of Replicated tables. [#12505](https://github.com/ClickHouse/ClickHouse/pull/12505) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now we check that server is able to start after stress tests run. This fixes [#12473](https://github.com/ClickHouse/ClickHouse/issues/12473). [#12496](https://github.com/ClickHouse/ClickHouse/pull/12496) ([alesapin](https://github.com/alesapin)). +* Update fmtlib to master (7.0.1). [#12446](https://github.com/ClickHouse/ClickHouse/pull/12446) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add docker image for fast tests. [#12294](https://github.com/ClickHouse/ClickHouse/pull/12294) ([alesapin](https://github.com/alesapin)). +* Rework configuration paths for integration tests. [#12285](https://github.com/ClickHouse/ClickHouse/pull/12285) ([Ilya Yatsishin](https://github.com/qoega)). +* Add compiler option to control that stack frames are not too large. This will help to run the code in fibers with small stack size. [#11524](https://github.com/ClickHouse/ClickHouse/pull/11524) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Update gitignore-files. [#13447](https://github.com/ClickHouse/ClickHouse/pull/13447) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). + + +## ClickHouse release 20.6 + +### ClickHouse release v20.6.3.28-stable + +#### Backward Incompatible Change + +* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). + +#### New Feature + +* Added an initial implementation of `EXPLAIN` query. Syntax: `EXPLAIN SELECT ...`. This fixes [#1118](https://github.com/ClickHouse/ClickHouse/issues/1118). [#11873](https://github.com/ClickHouse/ClickHouse/pull/11873) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Added storage `RabbitMQ`. [#11069](https://github.com/ClickHouse/ClickHouse/pull/11069) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Implemented PostgreSQL-like `ILIKE` operator for [#11710](https://github.com/ClickHouse/ClickHouse/issues/11710). [#12125](https://github.com/ClickHouse/ClickHouse/pull/12125) ([Mike](https://github.com/myrrc)). +* Supported RIGHT and FULL JOIN with `SET join_algorithm = 'partial_merge'`. Only ALL strictness is allowed (ANY, SEMI, ANTI, ASOF are not). [#12118](https://github.com/ClickHouse/ClickHouse/pull/12118) ([Artem Zuikov](https://github.com/4ertus2)). +* Added a function `initializeAggregation` to initialize an aggregation based on a single value. [#12109](https://github.com/ClickHouse/ClickHouse/pull/12109) ([Guillaume Tassery](https://github.com/YiuRULE)). +* Supported `ALTER TABLE ... [ADD|MODIFY] COLUMN ... FIRST` [#4006](https://github.com/ClickHouse/ClickHouse/issues/4006). [#12073](https://github.com/ClickHouse/ClickHouse/pull/12073) ([Winter Zhang](https://github.com/zhang2014)). +* Added function `parseDateTimeBestEffortUS`. [#12028](https://github.com/ClickHouse/ClickHouse/pull/12028) ([flynn](https://github.com/ucasFL)). +* Support format `ORC` for output (was supported only for input). [#11662](https://github.com/ClickHouse/ClickHouse/pull/11662) ([Kruglov Pavel](https://github.com/Avogar)). + +#### Bug Fix + +* Fixed `aggregate function any(x) is found inside another aggregate function in query` error with `SET optimize_move_functions_out_of_any = 1` and aliases inside `any()`. [#13419](https://github.com/ClickHouse/ClickHouse/pull/13419) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed `PrettyCompactMonoBlock` for clickhouse-local. Fixed extremes/totals with `PrettyCompactMonoBlock`. This fixes [#7746](https://github.com/ClickHouse/ClickHouse/issues/7746). [#13394](https://github.com/ClickHouse/ClickHouse/pull/13394) ([Azat Khuzhin](https://github.com/azat)). +* Fixed possible error `Totals having transform was already added to pipeline` in case of a query from delayed replica. [#13290](https://github.com/ClickHouse/ClickHouse/pull/13290) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* The server may crash if user passed specifically crafted arguments to the function `h3ToChildren`. This fixes [#13275](https://github.com/ClickHouse/ClickHouse/issues/13275). [#13277](https://github.com/ClickHouse/ClickHouse/pull/13277) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with NaN values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `DateTime64` conversion functions with constant argument. [#13205](https://github.com/ClickHouse/ClickHouse/pull/13205) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed error `Cannot convert column because it is constant but values of constants are different in source and result` for remote queries which use deterministic functions in scope of query, but not deterministic between queries, like `now()`, `now64()`, `randConstant()`. Fixes [#11327](https://github.com/ClickHouse/ClickHouse/issues/11327). [#13075](https://github.com/ClickHouse/ClickHouse/pull/13075) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed rare bug when `ALTER DELETE` and `ALTER MODIFY COLUMN` queries executed simultaneously as a single mutation. Bug leads to an incorrect amount of rows in `count.txt` and as a consequence incorrect data in part. Also, fix a small bug with simultaneous `ALTER RENAME COLUMN` and `ALTER ADD COLUMN`. [#12760](https://github.com/ClickHouse/ClickHouse/pull/12760) ([alesapin](https://github.com/alesapin)). +* Fixed `CAST(Nullable(String), Enum())`. [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). +* Fixed a performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for `input_format_parallel_parsing` (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bloom filter index with const expression. This fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* `CREATE USER IF NOT EXISTS` now doesn't throw exception if the user exists. This fixes [#12507](https://github.com/ClickHouse/ClickHouse/issues/12507). [#12646](https://github.com/ClickHouse/ClickHouse/pull/12646) ([Vitaly Baranov](https://github.com/vitlibar)). +* Better exception message in disk access storage. [#12625](https://github.com/ClickHouse/ClickHouse/pull/12625) ([alesapin](https://github.com/alesapin)). +* The function `groupArrayMoving*` was not working for distributed queries. It's result was calculated within incorrect data type (without promotion to the largest type). The function `groupArrayMovingAvg` was returning integer number that was inconsistent with the `avg` function. This fixes [#12568](https://github.com/ClickHouse/ClickHouse/issues/12568). [#12622](https://github.com/ClickHouse/ClickHouse/pull/12622) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixing race condition in live view tables which could cause data duplication. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed SETTINGS parse after FORMAT. [#12480](https://github.com/ClickHouse/ClickHouse/pull/12480) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed overflow when very large `LIMIT` or `OFFSET` is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault if `StorageMerge`. This fixes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Reverted change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). +* Additional check for arguments of bloom filter index. This fixes [#11408](https://github.com/ClickHouse/ClickHouse/issues/11408). [#12388](https://github.com/ClickHouse/ClickHouse/pull/12388) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid exception when negative or floating point constant is used in WHERE condition for indexed tables. This fixes [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allowed to `CLEAR` column even if there are depending `DEFAULT` expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `TOTALS/ROLLUP/CUBE` for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed error message and exit codes for `ALTER RENAME COLUMN` queries, when `RENAME` is not allowed. Fixes [#12301](https://github.com/ClickHouse/ClickHouse/issues/12301) and [#12303](https://github.com/ClickHouse/ClickHouse/issues/12303). [#12335](https://github.com/ClickHouse/ClickHouse/pull/12335) ([alesapin](https://github.com/alesapin)). +* Fixed very rare race condition in `ReplicatedMergeTreeQueue`. [#12315](https://github.com/ClickHouse/ClickHouse/pull/12315) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* When using codec `Delta` or `DoubleDelta` with non fixed width types, exception with code `LOGICAL_ERROR` was returned instead of exception with code `BAD_ARGUMENTS` (we ensure that exceptions with code logical error never happen). This fixes [#12110](https://github.com/ClickHouse/ClickHouse/issues/12110). [#12308](https://github.com/ClickHouse/ClickHouse/pull/12308) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). +* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `TTL` after renaming column, on which depends TTL expression. [#12304](https://github.com/ClickHouse/ClickHouse/pull/12304) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed SIGSEGV if there is an message with error in the middle of the batch in `Kafka` Engine. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the situation when some threads might randomly hang for a few seconds during `DNS` cache updating. [#12296](https://github.com/ClickHouse/ClickHouse/pull/12296) ([tavplubix](https://github.com/tavplubix)). +* Fixed typo in setting name. [#12292](https://github.com/ClickHouse/ClickHouse/pull/12292) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Show error after `TrieDictionary` failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). +* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Implement conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behaviour on reaching redirect limit in request to `S3` storage. [#12256](https://github.com/ClickHouse/ClickHouse/pull/12256) ([ianton-ru](https://github.com/ianton-ru)). +* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). +* Cap max_memory_usage* limits to the process resident memory. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). +* Fix dictGet arguments check during `GROUP BY` injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the behaviour when `SummingMergeTree` engine sums up columns from partition key. Added an exception in case of explicit definition of columns to sum which intersects with partition key columns. This fixes [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Don't split the dictionary source's table name into schema and table name itself if ODBC connection doesn't support schema. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bad code in redundant ORDER BY optimization. The bug was introduced in [#10067](https://github.com/ClickHouse/ClickHouse/issues/10067). [#12148](https://github.com/ClickHouse/ClickHouse/pull/12148) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Normalize "pid" file handling. In previous versions the server may refuse to start if it was killed without proper shutdown and if there is another process that has the same pid as previously runned server. Also pid file may be removed in unsuccessful server startup even if there is another server running. This fixes [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). +* Avoid "There is no query" exception for materialized views with joins or with subqueries attached to system logs (system.query_log, metric_log, etc) or to engine=Buffer underlying table. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). +* Fixed handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). +* Format `Parquet` now properly works with `LowCardinality` and `LowCardinality(Nullable)` types. Fixes [#12086](https://github.com/ClickHouse/ClickHouse/issues/12086), [#8406](https://github.com/ClickHouse/ClickHouse/issues/8406). [#12108](https://github.com/ClickHouse/ClickHouse/pull/12108) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed SIGSEGV in StorageKafka on DROP TABLE. [#12075](https://github.com/ClickHouse/ClickHouse/pull/12075) ([Azat Khuzhin](https://github.com/azat)). +* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Improvement + +* Allowed to set `JOIN` kind and type in more standard way: `LEFT SEMI JOIN` instead of `SEMI LEFT JOIN`. For now both are correct. [#12520](https://github.com/ClickHouse/ClickHouse/pull/12520) ([Artem Zuikov](https://github.com/4ertus2)). +* lifetime_rows/lifetime_bytes for Buffer engine. [#12421](https://github.com/ClickHouse/ClickHouse/pull/12421) ([Azat Khuzhin](https://github.com/azat)). +* Write the detail exception message to the client instead of 'MySQL server has gone away'. [#12383](https://github.com/ClickHouse/ClickHouse/pull/12383) ([BohuTANG](https://github.com/BohuTANG)). +* Allows to change a charset which is used for printing grids borders. Available charsets are following: UTF-8, ASCII. Setting `output_format_pretty_grid_charset` enables this feature. [#12372](https://github.com/ClickHouse/ClickHouse/pull/12372) ([Sabyanin Maxim](https://github.com/s-mx)). +* Supported MySQL 'SELECT DATABASE()' [#9336](https://github.com/ClickHouse/ClickHouse/issues/9336) 2. Add MySQL replacement query integration test. [#12314](https://github.com/ClickHouse/ClickHouse/pull/12314) ([BohuTANG](https://github.com/BohuTANG)). +* Added `KILL QUERY [connection_id]` for the MySQL client/driver to cancel the long query, issue [#12038](https://github.com/ClickHouse/ClickHouse/issues/12038). [#12152](https://github.com/ClickHouse/ClickHouse/pull/12152) ([BohuTANG](https://github.com/BohuTANG)). +* Added support for `%g` (two digit ISO year) and `%G` (four digit ISO year) substitutions in `formatDateTime` function. [#12136](https://github.com/ClickHouse/ClickHouse/pull/12136) ([vivarum](https://github.com/vivarum)). +* Added 'type' column in system.disks. [#12115](https://github.com/ClickHouse/ClickHouse/pull/12115) ([ianton-ru](https://github.com/ianton-ru)). +* Improved `REVOKE` command: now it requires grant/admin option for only access which will be revoked. For example, to execute `REVOKE ALL ON *.* FROM user1` now it doesn't require to have full access rights granted with grant option. Added command `REVOKE ALL FROM user1` - it revokes all granted roles from `user1`. [#12083](https://github.com/ClickHouse/ClickHouse/pull/12083) ([Vitaly Baranov](https://github.com/vitlibar)). +* Added replica priority for load_balancing (for manual prioritization of the load balancing). [#11995](https://github.com/ClickHouse/ClickHouse/pull/11995) ([Azat Khuzhin](https://github.com/azat)). +* Switched paths in S3 metadata to relative which allows to handle S3 blobs more easily. [#11892](https://github.com/ClickHouse/ClickHouse/pull/11892) ([Vladimir Chebotarev](https://github.com/excitoon)). + +#### Performance Improvement + +* Improved performace of 'ORDER BY' and 'GROUP BY' by prefix of sorting key (enabled with `optimize_aggregation_in_order` setting, disabled by default). [#11696](https://github.com/ClickHouse/ClickHouse/pull/11696) ([Anton Popov](https://github.com/CurtizJ)). +* Removed injective functions inside `uniq*()` if `set optimize_injective_functions_inside_uniq=1`. [#12337](https://github.com/ClickHouse/ClickHouse/pull/12337) ([Ruslan Kamalov](https://github.com/kamalov-ruslan)). +* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). +* Implemented single part uploads for DiskS3 (experimental feature). [#12026](https://github.com/ClickHouse/ClickHouse/pull/12026) ([Vladimir Chebotarev](https://github.com/excitoon)). + +#### Experimental Feature +* Added new in-memory format of parts in `MergeTree`-family tables, which stores data in memory. Parts are written on disk at first merge. Part will be created in in-memory format if its size in rows or bytes is below thresholds `min_rows_for_compact_part` and `min_bytes_for_compact_part`. Also optional support of Write-Ahead-Log is available, which is enabled by default and is controlled by setting `in_memory_parts_enable_wal`. [#10697](https://github.com/ClickHouse/ClickHouse/pull/10697) ([Anton Popov](https://github.com/CurtizJ)). + +#### Build/Testing/Packaging Improvement + +* Implement AST-based query fuzzing mode for clickhouse-client. See [this label](https://github.com/ClickHouse/ClickHouse/issues?q=label%3Afuzz+is%3Aissue) for the list of issues we recently found by fuzzing. Most of them were found by this tool, and a couple by SQLancer and `00746_sql_fuzzy.pl`. [#12111](https://github.com/ClickHouse/ClickHouse/pull/12111) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Add new type of tests based on Testflows framework. [#12090](https://github.com/ClickHouse/ClickHouse/pull/12090) ([vzakaznikov](https://github.com/vzakaznikov)). +* Added S3 HTTPS integration test. [#12412](https://github.com/ClickHouse/ClickHouse/pull/12412) ([Pavel Kovalenko](https://github.com/Jokser)). +* Log sanitizer trap messages from separate thread. This will prevent possible deadlock under thread sanitizer. [#12313](https://github.com/ClickHouse/ClickHouse/pull/12313) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now functional and stress tests will be able to run with old version of `clickhouse-test` script. [#12287](https://github.com/ClickHouse/ClickHouse/pull/12287) ([alesapin](https://github.com/alesapin)). +* Remove strange file creation during build in `orc`. [#12258](https://github.com/ClickHouse/ClickHouse/pull/12258) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Place common docker compose files to integration docker container. [#12168](https://github.com/ClickHouse/ClickHouse/pull/12168) ([Ilya Yatsishin](https://github.com/qoega)). +* Fix warnings from CodeQL. `CodeQL` is another static analyzer that we will use along with `clang-tidy` and `PVS-Studio` that we use already. [#12138](https://github.com/ClickHouse/ClickHouse/pull/12138) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Minor CMake fixes for UNBUNDLED build. [#12131](https://github.com/ClickHouse/ClickHouse/pull/12131) ([Matwey V. Kornilov](https://github.com/matwey)). +* Added a showcase of the minimal Docker image without using any Linux distribution. [#12126](https://github.com/ClickHouse/ClickHouse/pull/12126) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Perform an upgrade of system packages in the `clickhouse-server` docker image. [#12124](https://github.com/ClickHouse/ClickHouse/pull/12124) ([Ivan Blinkov](https://github.com/blinkov)). +* Add `UNBUNDLED` flag to `system.build_options` table. Move skip lists for `clickhouse-test` to clickhouse repo. [#12107](https://github.com/ClickHouse/ClickHouse/pull/12107) ([alesapin](https://github.com/alesapin)). +* Regular check by [Anchore Container Analysis](https://docs.anchore.com) security analysis tool that looks for [CVE](https://cve.mitre.org/) in `clickhouse-server` Docker image. Also confirms that `Dockerfile` is buildable. Runs daily on `master` and on pull-requests to `Dockerfile`. [#12102](https://github.com/ClickHouse/ClickHouse/pull/12102) ([Ivan Blinkov](https://github.com/blinkov)). +* Daily check by [GitHub CodeQL](https://securitylab.github.com/tools/codeql) security analysis tool that looks for [CWE](https://cwe.mitre.org/). [#12101](https://github.com/ClickHouse/ClickHouse/pull/12101) ([Ivan Blinkov](https://github.com/blinkov)). +* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). + +## ClickHouse release 20.5 + +### ClickHouse release v20.5.4.40-stable 2020-08-10 + +#### Bug Fix + +* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bloom filter index with const expression. This fixes [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572). [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed lack of aliases with function `any`. [#12593](https://github.com/ClickHouse/ClickHouse/pull/12593) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Remove data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed race condition in live view tables which could cause data duplication. [#12519](https://github.com/ClickHouse/ClickHouse/pull/12519) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed backwards compatibility in binary format of `AggregateFunction(avg, ...)` values. This fixes [#12342](https://github.com/ClickHouse/ClickHouse/issues/12342). [#12486](https://github.com/ClickHouse/ClickHouse/pull/12486) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed overflow when very large LIMIT or OFFSET is specified. This fixes [#10470](https://github.com/ClickHouse/ClickHouse/issues/10470). This fixes [#11372](https://github.com/ClickHouse/ClickHouse/issues/11372). [#12427](https://github.com/ClickHouse/ClickHouse/pull/12427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Reverts change introduced in [#11079](https://github.com/ClickHouse/ClickHouse/issues/11079) to resolve [#12098](https://github.com/ClickHouse/ClickHouse/issues/12098). [#12397](https://github.com/ClickHouse/ClickHouse/pull/12397) ([Mike](https://github.com/myrrc)). +* Avoid exception when negative or floating point constant is used in WHERE condition for indexed tables. This fixes [#11905](https://github.com/ClickHouse/ClickHouse/issues/11905). [#12384](https://github.com/ClickHouse/ClickHouse/pull/12384) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to CLEAR column even if there are depending DEFAULT expressions. This fixes [#12333](https://github.com/ClickHouse/ClickHouse/issues/12333). [#12378](https://github.com/ClickHouse/ClickHouse/pull/12378) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed TOTALS/ROLLUP/CUBE for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed SIGSEGV if there is an message with error in the middle of the batch in `Kafka` Engine. [#12302](https://github.com/ClickHouse/ClickHouse/pull/12302) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the behaviour when `SummingMergeTree` engine sums up columns from partition key. Added an exception in case of explicit definition of columns to sum which intersects with partition key columns. This fixes [#7867](https://github.com/ClickHouse/ClickHouse/issues/7867). [#12173](https://github.com/ClickHouse/ClickHouse/pull/12173) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug which leads to incorrect table metadata in ZooKeepeer for ReplicatedVersionedCollapsingMergeTree tables. Fixes [#12093](https://github.com/ClickHouse/ClickHouse/issues/12093). [#12121](https://github.com/ClickHouse/ClickHouse/pull/12121) ([alesapin](https://github.com/alesapin)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed crash in JOIN with LowCardinality type with `join_algorithm=partial_merge`. [#12035](https://github.com/ClickHouse/ClickHouse/pull/12035) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). + +#### Performance Improvement + +* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). + +#### Build/Testing/Packaging Improvement + +* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). + + +### ClickHouse release v20.5.2.7-stable 2020-07-02 + +#### Backward Incompatible Change + +* Return non-Nullable result from COUNT(DISTINCT), and `uniq` aggregate functions family. If all passed values are NULL, return zero instead. This improves SQL compatibility. [#11661](https://github.com/ClickHouse/ClickHouse/pull/11661) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added a check for the case when user-level setting is specified in a wrong place. User-level settings should be specified in `users.xml` inside `` section for specific user profile (or in `` for default settings). The server won't start with exception message in log. This fixes [#9051](https://github.com/ClickHouse/ClickHouse/issues/9051). If you want to skip the check, you can either move settings to the appropriate place or add `1` to config.xml. [#11449](https://github.com/ClickHouse/ClickHouse/pull/11449) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* The setting `input_format_with_names_use_header` is enabled by default. It will affect parsing of input formats `-WithNames` and `-WithNamesAndTypes`. [#10937](https://github.com/ClickHouse/ClickHouse/pull/10937) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove `experimental_use_processors` setting. It is enabled by default. [#10924](https://github.com/ClickHouse/ClickHouse/pull/10924) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Update `zstd` to 1.4.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. This change is backward compatible but we list it here in changelog in case you will wonder about these messages. [#10663](https://github.com/ClickHouse/ClickHouse/pull/10663) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added a check for meaningless codecs and a setting `allow_suspicious_codecs` to control this check. This closes [#4966](https://github.com/ClickHouse/ClickHouse/issues/4966). [#10645](https://github.com/ClickHouse/ClickHouse/pull/10645) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Several Kafka setting changes their defaults. See [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388). +* When upgrading from versions older than 20.5, if rolling update is performed and cluster contains both versions 20.5 or greater and less than 20.5, if ClickHouse nodes with old versions are restarted and old version has been started up in presence of newer versions, it may lead to `Part ... intersects previous part` errors. To prevent this error, first install newer clickhouse-server packages on all cluster nodes and then do restarts (so, when clickhouse-server is restarted, it will start up with the new version). + +#### New Feature + +* `TTL DELETE WHERE` and `TTL GROUP BY` for automatic data coarsening and rollup in tables. [#10537](https://github.com/ClickHouse/ClickHouse/pull/10537) ([expl0si0nn](https://github.com/expl0si0nn)). +* Implementation of PostgreSQL wire protocol. [#10242](https://github.com/ClickHouse/ClickHouse/pull/10242) ([Movses](https://github.com/MovElb)). +* Added system tables for users, roles, grants, settings profiles, quotas, row policies; added commands SHOW USER, SHOW [CURRENT|ENABLED] ROLES, SHOW SETTINGS PROFILES. [#10387](https://github.com/ClickHouse/ClickHouse/pull/10387) ([Vitaly Baranov](https://github.com/vitlibar)). +* 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)). +* 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)). +* 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)). +* Add SHOW CLUSTER(S) queries. [#11467](https://github.com/ClickHouse/ClickHouse/pull/11467) ([hexiaoting](https://github.com/hexiaoting)). +* 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)). +* Add `cast_keep_nullable` setting. If set `CAST(something_nullable AS Type)` return `Nullable(Type)`. [#11733](https://github.com/ClickHouse/ClickHouse/pull/11733) ([Artem Zuikov](https://github.com/4ertus2)). +* Added column `position` to `system.columns` table and `column_position` to `system.parts_columns` table. It contains ordinal position of a column in a table starting with 1. This closes [#7744](https://github.com/ClickHouse/ClickHouse/issues/7744). [#11655](https://github.com/ClickHouse/ClickHouse/pull/11655) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* ON CLUSTER support for SYSTEM {FLUSH DISTRIBUTED,STOP/START DISTRIBUTED SEND}. [#11415](https://github.com/ClickHouse/ClickHouse/pull/11415) ([Azat Khuzhin](https://github.com/azat)). +* Add system.distribution_queue table. [#11394](https://github.com/ClickHouse/ClickHouse/pull/11394) ([Azat Khuzhin](https://github.com/azat)). +* Support for all format settings in Kafka, expose some setting on table level, adjust the defaults for better performance. [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388) ([filimonov](https://github.com/filimonov)). +* Add `port` function (to extract port from URL). [#11120](https://github.com/ClickHouse/ClickHouse/pull/11120) ([Azat Khuzhin](https://github.com/azat)). +* Now `dictGet*` functions accept table names. [#11050](https://github.com/ClickHouse/ClickHouse/pull/11050) ([Vitaly Baranov](https://github.com/vitlibar)). +* The `clickhouse-format` tool is now able to format multiple queries when the `-n` argument is used. [#10852](https://github.com/ClickHouse/ClickHouse/pull/10852) ([Darío](https://github.com/dgrr)). +* Possibility to configure proxy-resolver for DiskS3. [#10744](https://github.com/ClickHouse/ClickHouse/pull/10744) ([Pavel Kovalenko](https://github.com/Jokser)). +* Make `pointInPolygon` work with non-constant polygon. PointInPolygon now can take Array(Array(Tuple(..., ...))) as second argument, array of polygon and holes. [#10623](https://github.com/ClickHouse/ClickHouse/pull/10623) ([Alexey Ilyukhov](https://github.com/livace)) [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). +* Added `move_ttl_info` to `system.parts` in order to provide introspection of move TTL functionality. [#10591](https://github.com/ClickHouse/ClickHouse/pull/10591) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Possibility to work with S3 through proxies. [#10576](https://github.com/ClickHouse/ClickHouse/pull/10576) ([Pavel Kovalenko](https://github.com/Jokser)). +* Add `NCHAR` and `NVARCHAR` synonims for data types. [#11025](https://github.com/ClickHouse/ClickHouse/pull/11025) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Resolved [#7224](https://github.com/ClickHouse/ClickHouse/issues/7224): added `FailedQuery`, `FailedSelectQuery` and `FailedInsertQuery` metrics to `system.events` table. [#11151](https://github.com/ClickHouse/ClickHouse/pull/11151) ([Nikita Orlov](https://github.com/naorlov)). +* Add more `jemalloc` statistics to `system.asynchronous_metrics`, and ensure that we see up-to-date values for them. [#11748](https://github.com/ClickHouse/ClickHouse/pull/11748) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Allow to specify default S3 credentials and custom auth headers. [#11134](https://github.com/ClickHouse/ClickHouse/pull/11134) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). +* Added new functions to import/export DateTime64 as Int64 with various precision: `to-/fromUnixTimestamp64Milli/-Micro/-Nano`. [#10923](https://github.com/ClickHouse/ClickHouse/pull/10923) ([Vasily Nemkov](https://github.com/Enmk)). +* Allow specifying `mongodb://` URI for MongoDB dictionaries. [#10915](https://github.com/ClickHouse/ClickHouse/pull/10915) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* OFFSET keyword can now be used without an affiliated LIMIT clause. [#10802](https://github.com/ClickHouse/ClickHouse/pull/10802) ([Guillaume Tassery](https://github.com/YiuRULE)). +* Added `system.licenses` table. This table contains licenses of third-party libraries that are located in `contrib` directory. This closes [#2890](https://github.com/ClickHouse/ClickHouse/issues/2890). [#10795](https://github.com/ClickHouse/ClickHouse/pull/10795) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* New function function toStartOfSecond(DateTime64) -> DateTime64 that nullifies sub-second part of DateTime64 value. [#10722](https://github.com/ClickHouse/ClickHouse/pull/10722) ([Vasily Nemkov](https://github.com/Enmk)). +* Add new input format `JSONAsString` that accepts a sequence of JSON objects separated by newlines, spaces and/or commas. [#10607](https://github.com/ClickHouse/ClickHouse/pull/10607) ([Kruglov Pavel](https://github.com/Avogar)). +* Allowed to profile memory with finer granularity steps than 4 MiB. Added sampling memory profiler to capture random allocations/deallocations. [#10598](https://github.com/ClickHouse/ClickHouse/pull/10598) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* `SimpleAggregateFunction` now also supports `sumMap`. [#10000](https://github.com/ClickHouse/ClickHouse/pull/10000) ([Ildus Kurbangaliev](https://github.com/ildus)). +* Support `ALTER RENAME COLUMN` for the distributed table engine. Continuation of [#10727](https://github.com/ClickHouse/ClickHouse/issues/10727). Fixes [#10747](https://github.com/ClickHouse/ClickHouse/issues/10747). [#10887](https://github.com/ClickHouse/ClickHouse/pull/10887) ([alesapin](https://github.com/alesapin)). + +#### Bug Fix + +* Fix UBSan report in Decimal parse. This fixes [#7540](https://github.com/ClickHouse/ClickHouse/issues/7540). [#10512](https://github.com/ClickHouse/ClickHouse/pull/10512) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential floating point exception when parsing DateTime64. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix rare crash caused by using `Nullable` column in prewhere condition. [#11895](https://github.com/ClickHouse/ClickHouse/pull/11895) [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608) [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix wrong result of comparison of FixedString with constant String. This fixes [#11393](https://github.com/ClickHouse/ClickHouse/issues/11393). This bug appeared in version 20.4. [#11828](https://github.com/ClickHouse/ClickHouse/pull/11828) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix wrong result for `if` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix using too many threads for queries. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `Scalar doesn't exist` exception when using `WITH ...` in `SELECT ... FROM merge_tree_table ...` [#11621](https://github.com/ClickHouse/ClickHouse/issues/11621). [#11767](https://github.com/ClickHouse/ClickHouse/pull/11767) ([Amos Bird](https://github.com/amosbird)). +* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). +* Now replicated fetches will be cancelled during metadata alter. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). +* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). +* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). +* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). +* Fix very rare race condition in SYSTEM SYNC REPLICA. If the replicated table is created and at the same time from the separate connection another client is issuing `SYSTEM SYNC REPLICA` command on that table (this is unlikely, because another client should be aware that the table is created), it's possible to get nullptr dereference. [#11691](https://github.com/ClickHouse/ClickHouse/pull/11691) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `LIMIT n WITH TIES` usage together with `ORDER BY` statement, which contains aliases. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible `Pipeline stuck` for selects with parallel `FINAL`. Fixes [#11636](https://github.com/ClickHouse/ClickHouse/issues/11636). [#11682](https://github.com/ClickHouse/ClickHouse/pull/11682) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix error which leads to an incorrect state of `system.mutations`. It may show that whole mutation is already done but the server still has `MUTATE_PART` tasks in the replication queue and tries to execute them. This fixes [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). +* Fix syntax hilite in CREATE USER query. [#11664](https://github.com/ClickHouse/ClickHouse/pull/11664) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove trivial count query optimization if row-level security is set. In previous versions the user get total count of records in a table instead filtered. This fixes [#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). +* Without `-q` option the database does not get created at startup. [#11604](https://github.com/ClickHouse/ClickHouse/pull/11604) ([giordyb](https://github.com/giordyb)). +* Fix error `Block structure mismatch` for queries with sampling reading from `Buffer` table. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix wrong exit code of the clickhouse-client, when `exception.code() % 256 == 0`. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). +* Fix race conditions in CREATE/DROP of different replicas of ReplicatedMergeTree. Continue to work if the table was not removed completely from ZooKeeper or not created successfully. This fixes [#11432](https://github.com/ClickHouse/ClickHouse/issues/11432). [#11592](https://github.com/ClickHouse/ClickHouse/pull/11592) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed rare segfault in `SHOW CREATE TABLE` Fixes [#11490](https://github.com/ClickHouse/ClickHouse/issues/11490). [#11579](https://github.com/ClickHouse/ClickHouse/pull/11579) ([tavplubix](https://github.com/tavplubix)). +* All queries in HTTP session have had the same query_id. It is fixed. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). +* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). +* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix shard_num/replica_num for `` (breaks use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). +* Fix async INSERT into Distributed for prefer_localhost_replica=0 and w/o internal_replication. [#11527](https://github.com/ClickHouse/ClickHouse/pull/11527) ([Azat Khuzhin](https://github.com/azat)). +* Fix memory leak when exception is thrown in the middle of aggregation with `-State` functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `Pipeline stuck` exception for `INSERT SELECT FINAL` where `SELECT` (`max_threads`>1) has multiple streams but `INSERT` has only one (`max_insert_threads`==0). [#11455](https://github.com/ClickHouse/ClickHouse/pull/11455) ([Azat Khuzhin](https://github.com/azat)). +* Fix wrong result in queries like `select count() from t, u`. [#11454](https://github.com/ClickHouse/ClickHouse/pull/11454) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). +* Fix potential uninitialized memory read in MergeTree shutdown if table was not created successfully. [#11420](https://github.com/ClickHouse/ClickHouse/pull/11420) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash in JOIN over `LowCarinality(T)` and `Nullable(T)`. [#11380](https://github.com/ClickHouse/ClickHouse/issues/11380). [#11414](https://github.com/ClickHouse/ClickHouse/pull/11414) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix error code for wrong `USING` key. [#11373](https://github.com/ClickHouse/ClickHouse/issues/11373). [#11404](https://github.com/ClickHouse/ClickHouse/pull/11404) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed `geohashesInBox` with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). +* Better errors for `joinGet()` functions. [#11389](https://github.com/ClickHouse/ClickHouse/pull/11389) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Remove redundant lock during parts send in ReplicatedMergeTree. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). +* Fix support for `\G` (vertical output) in clickhouse-client in multiline mode. This closes [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential segfault when using `Lazy` database. [#11348](https://github.com/ClickHouse/ClickHouse/pull/11348) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash in direct selects from `Join` table engine (without JOIN) and wrong nullability. [#11340](https://github.com/ClickHouse/ClickHouse/pull/11340) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Now merges stopped before change metadata in `ALTER` queries. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). +* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `visitParamExtractRaw` when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). +* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix insignificant data race in `clickhouse-copier`. Found by integration tests. [#11313](https://github.com/ClickHouse/ClickHouse/pull/11313) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash when `SET DEFAULT ROLE` is called with wrong arguments. This fixes [#10586](https://github.com/ClickHouse/ClickHouse/issues/10586). [#11278](https://github.com/ClickHouse/ClickHouse/pull/11278) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix crash while reading malformed data in `Protobuf` format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed a bug when `cache` dictionary could return default value instead of normal (when there are only expired keys). This affects only string fields. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix error `Block structure mismatch in QueryPipeline` while reading from `VIEW` with constants in inner query. Fixes [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible exception `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Now `primary.idx` will be checked if it's defined in `CREATE` query. [#11199](https://github.com/ClickHouse/ClickHouse/pull/11199) ([alesapin](https://github.com/alesapin)). +* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `S3` globbing which could fail in case of more than 1000 keys and some backends. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). +* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). +* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#11145](https://github.com/ClickHouse/ClickHouse/pull/11145) ([filimonov](https://github.com/filimonov)). +* Fix excessive reserving of threads for simple queries (optimization for reducing the number of threads, which was partly broken after changes in pipeline). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). +* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed deadlock during server startup after update with changes in structure of system log tables. [#11106](https://github.com/ClickHouse/ClickHouse/pull/11106) ([alesapin](https://github.com/alesapin)). +* Fixed memory leak in registerDiskS3. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fix error `No such name in Block::erase()` when JOIN appears with PREWHERE or `optimize_move_to_prewhere` makes PREWHERE from WHERE. [#11051](https://github.com/ClickHouse/ClickHouse/pull/11051) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixes the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). +* 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)). +* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fix incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid sending partially written files by the DistributedBlockOutputStream. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). +* Fix crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). +* Now it's possible to execute multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). +* Fix possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way (which I found) it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quanite*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Now constraints are updated if the column participating in `CONSTRAINT` expression was renamed. Fixes [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). +* Fix potential read of uninitialized memory in cache dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make use of `src_type` for correct type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Get rid of old libunwind patches. https://github.com/ClickHouse-Extras/libunwind/commit/500aa227911bd185a94bfc071d68f4d3b03cb3b1#r39048012 This allows to disable `-fno-omit-frame-pointer` in `clang` builds that improves performance at least by 1% in average. [#10761](https://github.com/ClickHouse/ClickHouse/pull/10761) ([Amos Bird](https://github.com/amosbird)). +* Fix avgWeighted when using floating-point weight over multiple shards. [#10758](https://github.com/ClickHouse/ClickHouse/pull/10758) ([Baudouin Giard](https://github.com/bgiard)). +* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fix crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix usage of primary key wrapped into a function with 'FINAL' modifier and 'ORDER BY' optimization. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix atomicity of HTTP insert. This fixes [#9666](https://github.com/ClickHouse/ClickHouse/issues/9666). [#10687](https://github.com/ClickHouse/ClickHouse/pull/10687) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fix multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). +* Fix order of parameters in AggregateTransform constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). +* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fix nullptr dereference in StorageBuffer if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). +* Fix optimize_skip_unused_shards with LowCardinality. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). +* Fix segfault in StorageBuffer when exception on server startup. Fixes [#10550](https://github.com/ClickHouse/ClickHouse/issues/10550). [#10609](https://github.com/ClickHouse/ClickHouse/pull/10609) ([tavplubix](https://github.com/tavplubix)). +* On `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixes possible crash `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix error `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). +* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* Implemented comparison between DateTime64 and String values (just like for DateTime). [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Fix index corruption, which may occur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). +* Disable GROUP BY sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for WITH ROLLUP/CUBE/TOTALS. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). +* Fixes: [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263) (after that PR dist send via INSERT had been postponing on each INSERT) Fixes: [#8756](https://github.com/ClickHouse/ClickHouse/issues/8756) (that PR breaks distributed sends with all of the following conditions met (unlikely setup for now I guess): `internal_replication == false`, multiple local shards (activates the hardlinking code) and `distributed_storage_policy` (makes `link(2)` fails on `EXDEV`)). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). +* Fixed error with "max_rows_to_sort" limit. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Get dictionary and check access rights only once per each call of any function reading external dictionaries. [#10928](https://github.com/ClickHouse/ClickHouse/pull/10928) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Improvement + +* Apply `TTL` for old data, after `ALTER MODIFY TTL` query. This behaviour is controlled by setting `materialize_ttl_after_modify`, which is enabled by default. [#11042](https://github.com/ClickHouse/ClickHouse/pull/11042) ([Anton Popov](https://github.com/CurtizJ)). +* When parsing C-style backslash escapes in string literals, VALUES and various text formats (this is an extension to SQL standard that is endemic for ClickHouse and MySQL), keep backslash if unknown escape sequence is found (e.g. `\%` or `\w`) that will make usage of `LIKE` and `match` regular expressions more convenient (it's enough to write `name LIKE 'used\_cars'` instead of `name LIKE 'used\\_cars'`) and more compatible at the same time. This fixes [#10922](https://github.com/ClickHouse/ClickHouse/issues/10922). [#11208](https://github.com/ClickHouse/ClickHouse/pull/11208) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* When reading Decimal value, cut extra digits after point. This behaviour is more compatible with MySQL and PostgreSQL. This fixes [#10202](https://github.com/ClickHouse/ClickHouse/issues/10202). [#11831](https://github.com/ClickHouse/ClickHouse/pull/11831) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow to DROP replicated table if the metadata in ZooKeeper was already removed and does not exist (this is also the case when using TestKeeper for testing and the server was restarted). Allow to RENAME replicated table even if there is an error communicating with ZooKeeper. This fixes [#10720](https://github.com/ClickHouse/ClickHouse/issues/10720). [#11652](https://github.com/ClickHouse/ClickHouse/pull/11652) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Slightly improve diagnostic of reading decimal from string. This closes [#10202](https://github.com/ClickHouse/ClickHouse/issues/10202). [#11829](https://github.com/ClickHouse/ClickHouse/pull/11829) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix sleep invocation in signal handler. It was sleeping for less amount of time than expected. [#11825](https://github.com/ClickHouse/ClickHouse/pull/11825) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* (Only Linux) OS related performance metrics (for CPU and I/O) will work even without `CAP_NET_ADMIN` capability. [#10544](https://github.com/ClickHouse/ClickHouse/pull/10544) ([Alexander Kazakov](https://github.com/Akazz)). +* Added `hostname` as an alias to function `hostName`. This feature was suggested by Victor Tarnavskiy from Yandex.Metrica. [#11821](https://github.com/ClickHouse/ClickHouse/pull/11821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added support for distributed `DDL` (update/delete/drop partition) on cross replication clusters. [#11703](https://github.com/ClickHouse/ClickHouse/pull/11703) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Emit warning instead of error in server log at startup if we cannot listen one of the listen addresses (e.g. IPv6 is unavailable inside Docker). Note that if server fails to listen all listed addresses, it will refuse to startup as before. This fixes [#4406](https://github.com/ClickHouse/ClickHouse/issues/4406). [#11687](https://github.com/ClickHouse/ClickHouse/pull/11687) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* 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)). +* Skip empty parameters in requested URL. They may appear when you write `http://localhost:8123/?&a=b` or `http://localhost:8123/?a=b&&c=d`. This closes [#10749](https://github.com/ClickHouse/ClickHouse/issues/10749). [#11651](https://github.com/ClickHouse/ClickHouse/pull/11651) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow using `groupArrayArray` and `groupUniqArrayArray` as `SimpleAggregateFunction`. [#11650](https://github.com/ClickHouse/ClickHouse/pull/11650) ([Volodymyr Kuznetsov](https://github.com/ksvladimir)). +* Allow comparison with constant strings by implicit conversions when analysing index conditions on other types. This may close [#11630](https://github.com/ClickHouse/ClickHouse/issues/11630). [#11648](https://github.com/ClickHouse/ClickHouse/pull/11648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* https://github.com/ClickHouse/ClickHouse/pull/7572#issuecomment-642815377 Support config default HTTPHandlers. [#11628](https://github.com/ClickHouse/ClickHouse/pull/11628) ([Winter Zhang](https://github.com/zhang2014)). +* Make more input formats to work with Kafka engine. Fix the issue with premature flushes. Fix the performance issue when `kafka_num_consumers` is greater than number of partitions in topic. [#11599](https://github.com/ClickHouse/ClickHouse/pull/11599) ([filimonov](https://github.com/filimonov)). +* Improve `multiple_joins_rewriter_version=2` logic. Fix unknown columns error for lambda aliases. [#11587](https://github.com/ClickHouse/ClickHouse/pull/11587) ([Artem Zuikov](https://github.com/4ertus2)). +* Better exception message when cannot parse columns declaration list. This closes [#10403](https://github.com/ClickHouse/ClickHouse/issues/10403). [#11537](https://github.com/ClickHouse/ClickHouse/pull/11537) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improve `enable_optimize_predicate_expression=1` logic for VIEW. [#11513](https://github.com/ClickHouse/ClickHouse/pull/11513) ([Artem Zuikov](https://github.com/4ertus2)). +* Adding support for PREWHERE in live view tables. [#11495](https://github.com/ClickHouse/ClickHouse/pull/11495) ([vzakaznikov](https://github.com/vzakaznikov)). +* 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)). +* Better exception message in case when there is shortage of memory mappings. This closes [#11027](https://github.com/ClickHouse/ClickHouse/issues/11027). [#11316](https://github.com/ClickHouse/ClickHouse/pull/11316) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Support (U)Int8, (U)Int16, Date in ASOF JOIN. [#11301](https://github.com/ClickHouse/ClickHouse/pull/11301) ([Artem Zuikov](https://github.com/4ertus2)). +* 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)). +* Set thread names for internal threads of rdkafka library. Make logs from rdkafka available in server logs. [#10983](https://github.com/ClickHouse/ClickHouse/pull/10983) ([Azat Khuzhin](https://github.com/azat)). +* Support for unicode whitespaces in queries. This helps when queries are copy-pasted from Word or from web page. This fixes [#10896](https://github.com/ClickHouse/ClickHouse/issues/10896). [#10903](https://github.com/ClickHouse/ClickHouse/pull/10903) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow large UInt types as the index in function `tupleElement`. [#10874](https://github.com/ClickHouse/ClickHouse/pull/10874) ([hcz](https://github.com/hczhcz)). +* Respect prefer_localhost_replica/load_balancing on INSERT into Distributed. [#10867](https://github.com/ClickHouse/ClickHouse/pull/10867) ([Azat Khuzhin](https://github.com/azat)). +* Introduce `min_insert_block_size_rows_for_materialized_views`, `min_insert_block_size_bytes_for_materialized_views` settings. This settings are similar to `min_insert_block_size_rows` and `min_insert_block_size_bytes`, but applied only for blocks inserted into `MATERIALIZED VIEW`. It helps to control blocks squashing while pushing to MVs and avoid excessive memory usage. [#10858](https://github.com/ClickHouse/ClickHouse/pull/10858) ([Azat Khuzhin](https://github.com/azat)). +* Get rid of exception from replicated queue during server shutdown. Fixes [#10819](https://github.com/ClickHouse/ClickHouse/issues/10819). [#10841](https://github.com/ClickHouse/ClickHouse/pull/10841) ([alesapin](https://github.com/alesapin)). +* Ensure that `varSamp`, `varPop` cannot return negative results due to numerical errors and that `stddevSamp`, `stddevPop` cannot be calculated from negative variance. This fixes [#10532](https://github.com/ClickHouse/ClickHouse/issues/10532). [#10829](https://github.com/ClickHouse/ClickHouse/pull/10829) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Better DNS exception message. This fixes [#10813](https://github.com/ClickHouse/ClickHouse/issues/10813). [#10828](https://github.com/ClickHouse/ClickHouse/pull/10828) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Change HTTP response code in case of some parse errors to 400 Bad Request. This fix [#10636](https://github.com/ClickHouse/ClickHouse/issues/10636). [#10640](https://github.com/ClickHouse/ClickHouse/pull/10640) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Print a message if clickhouse-client is newer than clickhouse-server. [#10627](https://github.com/ClickHouse/ClickHouse/pull/10627) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Adding support for `INSERT INTO [db.]table WATCH` query. [#10498](https://github.com/ClickHouse/ClickHouse/pull/10498) ([vzakaznikov](https://github.com/vzakaznikov)). +* Allow to pass quota_key in clickhouse-client. This closes [#10227](https://github.com/ClickHouse/ClickHouse/issues/10227). [#10270](https://github.com/ClickHouse/ClickHouse/pull/10270) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement + +* Allow multiple replicas to assign merges, mutations, partition drop, move and replace concurrently. This closes [#10367](https://github.com/ClickHouse/ClickHouse/issues/10367). [#11639](https://github.com/ClickHouse/ClickHouse/pull/11639) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#11795](https://github.com/ClickHouse/ClickHouse/pull/11795) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimization of GROUP BY with respect to table sorting key, enabled with `optimize_aggregation_in_order` setting. [#9113](https://github.com/ClickHouse/ClickHouse/pull/9113) ([dimarub2000](https://github.com/dimarub2000)). +* Selects with final are executed in parallel. Added setting `max_final_threads` to limit the number of threads used. [#10463](https://github.com/ClickHouse/ClickHouse/pull/10463) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Improve performance for INSERT queries via `INSERT SELECT` or INSERT with clickhouse-client when small blocks are generated (typical case with parallel parsing). This fixes [#11275](https://github.com/ClickHouse/ClickHouse/issues/11275). Fix the issue that CONSTRAINTs were not working for DEFAULT fields. This fixes [#11273](https://github.com/ClickHouse/ClickHouse/issues/11273). Fix the issue that CONSTRAINTS were ignored for TEMPORARY tables. This fixes [#11274](https://github.com/ClickHouse/ClickHouse/issues/11274). [#11276](https://github.com/ClickHouse/ClickHouse/pull/11276) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimization that eliminates min/max/any aggregators of GROUP BY keys in SELECT section, enabled with `optimize_aggregators_of_group_by_keys` setting. [#11667](https://github.com/ClickHouse/ClickHouse/pull/11667) ([xPoSx](https://github.com/xPoSx)). [#11806](https://github.com/ClickHouse/ClickHouse/pull/11806) ([Azat Khuzhin](https://github.com/azat)). +* 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)). +* Sort bigger parts of the left table in MergeJoin. Buffer left blocks in memory. Add `partial_merge_join_left_table_buffer_bytes` setting to manage the left blocks buffers sizes. [#10601](https://github.com/ClickHouse/ClickHouse/pull/10601) ([Artem Zuikov](https://github.com/4ertus2)). +* Remove duplicate ORDER BY and DISTINCT from subqueries, this optimization is enabled with `optimize_duplicate_order_by_and_distinct` [#10067](https://github.com/ClickHouse/ClickHouse/pull/10067) ([Mikhail Malafeev](https://github.com/demo-99)). +* This feature eliminates functions of other keys in GROUP BY section, enabled with `optimize_group_by_function_keys` [#10051](https://github.com/ClickHouse/ClickHouse/pull/10051) ([xPoSx](https://github.com/xPoSx)). +* New optimization that takes arithmetic operations out of aggregate functions, enabled with `optimize_arithmetic_operations_in_aggregate_functions` [#10047](https://github.com/ClickHouse/ClickHouse/pull/10047) ([Ruslan](https://github.com/kamalov-ruslan)). +* Use HTTP client for S3 based on Poco instead of curl. This will improve performance and lower memory usage of s3 storage and table functions. [#11230](https://github.com/ClickHouse/ClickHouse/pull/11230) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fix Kafka performance issue related to reschedules based on limits, which were always applied. [#11149](https://github.com/ClickHouse/ClickHouse/pull/11149) ([filimonov](https://github.com/filimonov)). +* Enable percpu_arena:percpu for jemalloc (This will reduce memory fragmentation due to thread pool). [#11084](https://github.com/ClickHouse/ClickHouse/pull/11084) ([Azat Khuzhin](https://github.com/azat)). +* Optimize memory usage when reading a response from an S3 HTTP client. [#11561](https://github.com/ClickHouse/ClickHouse/pull/11561) ([Pavel Kovalenko](https://github.com/Jokser)). +* Adjust the default Kafka settings for better performance. [#11388](https://github.com/ClickHouse/ClickHouse/pull/11388) ([filimonov](https://github.com/filimonov)). + +#### Experimental Feature + +* Add data type `Point` (Tuple(Float64, Float64)) and `Polygon` (Array(Array(Tuple(Float64, Float64))). [#10678](https://github.com/ClickHouse/ClickHouse/pull/10678) ([Alexey Ilyukhov](https://github.com/livace)). +* Add's a `hasSubstr` function that allows for look for subsequences in arrays. Note: this function is likely to be renamed without further notice. [#11071](https://github.com/ClickHouse/ClickHouse/pull/11071) ([Ryad Zenine](https://github.com/r-zenine)). +* Added OpenCL support and bitonic sort algorithm, which can be used for sorting integer types of data in single column. Needs to be build with flag `-DENABLE_OPENCL=1`. For using bitonic sort algorithm instead of others you need to set `bitonic_sort` for Setting's option `special_sort` and make sure that OpenCL is available. This feature does not improve performance or anything else, it is only provided as an example and for demonstration purposes. It is likely to be removed in near future if there will be no further development in this direction. [#10232](https://github.com/ClickHouse/ClickHouse/pull/10232) ([Ri](https://github.com/margaritiko)). + +#### Build/Testing/Packaging Improvement + +* Enable clang-tidy for programs and utils. [#10991](https://github.com/ClickHouse/ClickHouse/pull/10991) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove dependency on `tzdata`: do not fail if `/usr/share/zoneinfo` directory does not exist. Note that all timezones work in ClickHouse even without tzdata installed in system. [#11827](https://github.com/ClickHouse/ClickHouse/pull/11827) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added MSan and UBSan stress tests. Note that we already have MSan, UBSan for functional tests and "stress" test is another kind of tests. [#10871](https://github.com/ClickHouse/ClickHouse/pull/10871) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Print compiler build id in crash messages. It will make us slightly more certain about what binary has crashed. Added new function `buildId`. [#11824](https://github.com/ClickHouse/ClickHouse/pull/11824) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added a test to ensure that mutations continue to work after FREEZE query. [#11820](https://github.com/ClickHouse/ClickHouse/pull/11820) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Don't allow tests with "fail" substring in their names because it makes looking at the tests results in browser less convenient when you type Ctrl+F and search for "fail". [#11817](https://github.com/ClickHouse/ClickHouse/pull/11817) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Removes unused imports from HTTPHandlerFactory. [#11660](https://github.com/ClickHouse/ClickHouse/pull/11660) ([Bharat Nallan](https://github.com/bharatnc)). +* Added a random sampling of instances where copier is executed. It is needed to avoid `Too many simultaneous queries` error. Also increased timeout and decreased fault probability. [#11573](https://github.com/ClickHouse/ClickHouse/pull/11573) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix missed include. [#11525](https://github.com/ClickHouse/ClickHouse/pull/11525) ([Matwey V. Kornilov](https://github.com/matwey)). +* Speed up build by removing old example programs. Also found some orphan functional test. [#11486](https://github.com/ClickHouse/ClickHouse/pull/11486) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Increase ccache size for builds in CI. [#11450](https://github.com/ClickHouse/ClickHouse/pull/11450) ([alesapin](https://github.com/alesapin)). +* Leave only unit_tests_dbms in deb build. [#11429](https://github.com/ClickHouse/ClickHouse/pull/11429) ([Ilya Yatsishin](https://github.com/qoega)). +* Update librdkafka to version [1.4.2](https://github.com/edenhill/librdkafka/releases/tag/v1.4.2). [#11256](https://github.com/ClickHouse/ClickHouse/pull/11256) ([filimonov](https://github.com/filimonov)). +* Refactor CMake build files. [#11390](https://github.com/ClickHouse/ClickHouse/pull/11390) ([Ivan](https://github.com/abyss7)). +* Fix several flaky integration tests. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). +* Add support for unit tests run with UBSan. [#11345](https://github.com/ClickHouse/ClickHouse/pull/11345) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove redundant timeout from integration test `test_insertion_sync_fails_with_timeout`. [#11343](https://github.com/ClickHouse/ClickHouse/pull/11343) ([alesapin](https://github.com/alesapin)). +* Better check for hung queries in clickhouse-test. [#11321](https://github.com/ClickHouse/ClickHouse/pull/11321) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Emit a warning if server was build in debug or with sanitizers. [#11304](https://github.com/ClickHouse/ClickHouse/pull/11304) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now clickhouse-test check the server aliveness before tests run. [#11285](https://github.com/ClickHouse/ClickHouse/pull/11285) ([alesapin](https://github.com/alesapin)). +* Fix potentially flacky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Repeat test in CI if `curl` invocation was timed out. It is possible due to system hangups for 10+ seconds that are typical in our CI infrastructure. This fixes [#11267](https://github.com/ClickHouse/ClickHouse/issues/11267). [#11268](https://github.com/ClickHouse/ClickHouse/pull/11268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add a test for Join table engine from @donmikel. This closes [#9158](https://github.com/ClickHouse/ClickHouse/issues/9158). [#11265](https://github.com/ClickHouse/ClickHouse/pull/11265) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix several non significant errors in unit tests. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). +* Now parts of linker command for `cctz` library will not be shuffled with other libraries. [#11213](https://github.com/ClickHouse/ClickHouse/pull/11213) ([alesapin](https://github.com/alesapin)). +* Split /programs/server into actual program and library. [#11186](https://github.com/ClickHouse/ClickHouse/pull/11186) ([Ivan](https://github.com/abyss7)). +* Improve build scripts for protobuf & gRPC. [#11172](https://github.com/ClickHouse/ClickHouse/pull/11172) ([Vitaly Baranov](https://github.com/vitlibar)). +* Enable performance test that was not working. [#11158](https://github.com/ClickHouse/ClickHouse/pull/11158) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Create root S3 bucket for tests before any CH instance is started. [#11142](https://github.com/ClickHouse/ClickHouse/pull/11142) ([Pavel Kovalenko](https://github.com/Jokser)). +* Add performance test for non-constant polygons. [#11141](https://github.com/ClickHouse/ClickHouse/pull/11141) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixing `00979_live_view_watch_continuous_aggregates` test. [#11024](https://github.com/ClickHouse/ClickHouse/pull/11024) ([vzakaznikov](https://github.com/vzakaznikov)). +* Add ability to run zookeeper in integration tests over tmpfs. [#11002](https://github.com/ClickHouse/ClickHouse/pull/11002) ([alesapin](https://github.com/alesapin)). +* Wait for odbc-bridge with exponential backoff. Previous wait time of 200 ms was not enough in our CI environment. [#10990](https://github.com/ClickHouse/ClickHouse/pull/10990) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix non-deterministic test. [#10989](https://github.com/ClickHouse/ClickHouse/pull/10989) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added a test for empty external data. [#10926](https://github.com/ClickHouse/ClickHouse/pull/10926) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Database is recreated for every test. This improves separation of tests. [#10902](https://github.com/ClickHouse/ClickHouse/pull/10902) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added more asserts in columns code. [#10833](https://github.com/ClickHouse/ClickHouse/pull/10833) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Better cooperation with sanitizers. Print information about query_id in the message of sanitizer failure. [#10832](https://github.com/ClickHouse/ClickHouse/pull/10832) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix obvious race condition in "Split build smoke test" check. [#10820](https://github.com/ClickHouse/ClickHouse/pull/10820) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix (false) MSan report in MergeTreeIndexFullText. The issue first appeared in [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add MSan suppression for MariaDB Client library. [#10800](https://github.com/ClickHouse/ClickHouse/pull/10800) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* GRPC make couldn't find protobuf files, changed make file by adding the right link. [#10794](https://github.com/ClickHouse/ClickHouse/pull/10794) ([mnkonkova](https://github.com/mnkonkova)). +* Enable extra warnings (`-Weverything`) for base, utils, programs. Note that we already have it for the most of the code. [#10779](https://github.com/ClickHouse/ClickHouse/pull/10779) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Suppressions of warnings from libraries was mistakenly declared as public in [#10396](https://github.com/ClickHouse/ClickHouse/issues/10396). [#10776](https://github.com/ClickHouse/ClickHouse/pull/10776) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Restore a patch that was accidentially deleted in [#10396](https://github.com/ClickHouse/ClickHouse/issues/10396). [#10774](https://github.com/ClickHouse/ClickHouse/pull/10774) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix performance tests errors, part 2. [#10773](https://github.com/ClickHouse/ClickHouse/pull/10773) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix performance test errors. [#10766](https://github.com/ClickHouse/ClickHouse/pull/10766) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Update cross-builds to use clang-10 compiler. [#10724](https://github.com/ClickHouse/ClickHouse/pull/10724) ([Ivan](https://github.com/abyss7)). +* Update instruction to install RPM packages. This was suggested by Denis (TG login @ldviolet) and implemented by Arkady Shejn. [#10707](https://github.com/ClickHouse/ClickHouse/pull/10707) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Trying to fix `tests/queries/0_stateless/01246_insert_into_watch_live_view.py` test. [#10670](https://github.com/ClickHouse/ClickHouse/pull/10670) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fixing and re-enabling 00979_live_view_watch_continuous_aggregates.py test. [#10658](https://github.com/ClickHouse/ClickHouse/pull/10658) ([vzakaznikov](https://github.com/vzakaznikov)). +* Fix OOM in ASan stress test. [#10646](https://github.com/ClickHouse/ClickHouse/pull/10646) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan report (adding zero to nullptr) in HashTable that appeared after migration to clang-10. [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove external call to `ld` (bfd) linker during tzdata processing in compile time. [#10634](https://github.com/ClickHouse/ClickHouse/pull/10634) ([alesapin](https://github.com/alesapin)). +* Allow to use `lld` to link blobs (resources). [#10632](https://github.com/ClickHouse/ClickHouse/pull/10632) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan report in `LZ4` library. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). See also [https://github.com/lz4/lz4/issues/857](https://github.com/lz4/lz4/issues/857) +* Update LZ4 to the latest dev branch. [#10630](https://github.com/ClickHouse/ClickHouse/pull/10630) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added auto-generated machine-readable file with the list of stable versions. [#10628](https://github.com/ClickHouse/ClickHouse/pull/10628) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `capnproto` version check for `capnp::UnalignedFlatArrayMessageReader`. [#10618](https://github.com/ClickHouse/ClickHouse/pull/10618) ([Matwey V. Kornilov](https://github.com/matwey)). +* Lower memory usage in tests. [#10617](https://github.com/ClickHouse/ClickHouse/pull/10617) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixing hard coded timeouts in new live view tests. [#10604](https://github.com/ClickHouse/ClickHouse/pull/10604) ([vzakaznikov](https://github.com/vzakaznikov)). +* Increasing timeout when opening a client in tests/queries/0_stateless/helpers/client.py. [#10599](https://github.com/ClickHouse/ClickHouse/pull/10599) ([vzakaznikov](https://github.com/vzakaznikov)). +* Enable ThinLTO for clang builds, continuation of [#10435](https://github.com/ClickHouse/ClickHouse/pull/10435). [#10585](https://github.com/ClickHouse/ClickHouse/pull/10585) ([Amos Bird](https://github.com/amosbird)). +* Adding fuzzers and preparing for oss-fuzz integration. [#10546](https://github.com/ClickHouse/ClickHouse/pull/10546) ([kyprizel](https://github.com/kyprizel)). +* Fix FreeBSD build. [#10150](https://github.com/ClickHouse/ClickHouse/pull/10150) ([Ivan](https://github.com/abyss7)). +* Add new build for query tests using pytest framework. [#10039](https://github.com/ClickHouse/ClickHouse/pull/10039) ([Ivan](https://github.com/abyss7)). + + +## ClickHouse release v20.4 + +### ClickHouse release v20.4.8.99-stable 2020-08-10 + +#### Bug Fix + +* Fixed error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with NaN values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong index analysis with functions. It could lead to pruning wrong parts, while reading from `MergeTree` tables. Fixes [#13060](https://github.com/ClickHouse/ClickHouse/issues/13060). Fixes [#12406](https://github.com/ClickHouse/ClickHouse/issues/12406). [#13081](https://github.com/ClickHouse/ClickHouse/pull/13081) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible extra overflow row in data which could appear for queries `WITH TOTALS`. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for `input_format_parallel_parsing` (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) allow push predicate when subquery contains with clause. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed `SIGSEGV` in `StorageKafka` when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Added support for function `if` with `Array(UUID)` arguments. This fixes [#11066](https://github.com/ClickHouse/ClickHouse/issues/11066). [#12648](https://github.com/ClickHouse/ClickHouse/pull/12648) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Removed data for Distributed tables (blocks from async INSERTs) on DROP TABLE. [#12556](https://github.com/ClickHouse/ClickHouse/pull/12556) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed crash in JOIN with dictionary when we are joining over expression of dictionary key: `t JOIN dict ON expr(dict.id) = t.id`. Disable dictionary join optimisation for this case. [#12458](https://github.com/ClickHouse/ClickHouse/pull/12458) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). +* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Show error after TrieDictionary failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). +* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Implemented conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). +* Cap max_memory_usage* limits to the process resident memory. [#12182](https://github.com/ClickHouse/ClickHouse/pull/12182) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `dictGet` arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). +* Don't split the dictionary source's table name into schema and table name itself if ODBC connection doesn't support schema. [#12165](https://github.com/ClickHouse/ClickHouse/pull/12165) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Normalize "pid" file handling. In previous versions the server may refuse to start if it was killed without proper shutdown and if there is another process that has the same pid as previously runned server. Also pid file may be removed in unsuccessful server startup even if there is another server running. This fixes [#3501](https://github.com/ClickHouse/ClickHouse/issues/3501). [#12133](https://github.com/ClickHouse/ClickHouse/pull/12133) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed handling dependency of table with ENGINE=Dictionary on dictionary. This fixes [#10994](https://github.com/ClickHouse/ClickHouse/issues/10994). This fixes [#10397](https://github.com/ClickHouse/ClickHouse/issues/10397). [#12116](https://github.com/ClickHouse/ClickHouse/pull/12116) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed empty `result_rows` and `result_bytes` metrics in `system.quey_log` for selects. Fixes [#11595](https://github.com/ClickHouse/ClickHouse/issues/11595). [#12089](https://github.com/ClickHouse/ClickHouse/pull/12089) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Expected single dictionary argument for function` for function `defaultValueOfArgumentType` with `LowCardinality` type. Fixes [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Parse tables metadata in parallel when loading database. This fixes slow server startup when there are large number of tables. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). +* Make `topK` aggregate function return Enum for Enum types. This fixes [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect comparison of tuples with `Nullable` columns. Fixes [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed calculation of access rights when allow_introspection_functions=0. [#12031](https://github.com/ClickHouse/ClickHouse/pull/12031) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* A query with function `neighbor` as the only returned expression may return empty result if the function is called with offset `-9223372036854775808`. This fixes [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed calculation of access rights when allow_ddl=0. [#12015](https://github.com/ClickHouse/ClickHouse/pull/12015) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed potential array size overflow in generateRandom that may lead to crash. This fixes [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential floating point exception. This closes [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong setting name in log message at server startup. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Query parameter was not set` in `Values` format. Fixes [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). +* Keep aliases for substitutions in query (parametrized queries). This fixes [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug with no moves when changing storage policy from default one. [#11893](https://github.com/ClickHouse/ClickHouse/pull/11893) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed potential floating point exception when parsing `DateTime64`. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed memory accounting via HTTP interface (can be significant with `wait_end_of_query=1`). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). +* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). + +#### Performance Improvement + +* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). + +#### Build/Testing/Packaging Improvement + +* Install `ca-certificates` before the first `apt-get update` in Dockerfile. [#12095](https://github.com/ClickHouse/ClickHouse/pull/12095) ([Ivan Blinkov](https://github.com/blinkov)). + + +### ClickHouse release v20.4.6.53-stable 2020-06-25 + +#### Bug Fix + +* Fix rare crash caused by using `Nullable` column in prewhere condition. Continuation of [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix wrong result of comparison of FixedString with constant String. This fixes [#11393](https://github.com/ClickHouse/ClickHouse/issues/11393). This bug appeared in version 20.4. [#11828](https://github.com/ClickHouse/ClickHouse/pull/11828) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix using too many threads for queries. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). +* Now replicated fetches will be cancelled during metadata alter. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). +* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). +* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). +* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `LIMIT n WITH TIES` usage together with `ORDER BY` statement, which contains aliases. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). +* Fix error which leads to an incorrect state of `system.mutations`. It may show that whole mutation is already done but the server still has `MUTATE_PART` tasks in the replication queue and tries to execute them. This fixes [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). +* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove trivial count query optimization if row-level security is set. In previous versions the user get total count of records in a table instead filtered. This fixes [#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare crash caused by using `Nullable` column in prewhere condition. (Probably it is connected with [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) somehow). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix error `Block structure mismatch` for queries with sampling reading from `Buffer` table. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix wrong exit code of the clickhouse-client, when exception.code() % 256 = 0. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). +* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed rare segfault in `SHOW CREATE TABLE` Fixes [#11490](https://github.com/ClickHouse/ClickHouse/issues/11490). [#11579](https://github.com/ClickHouse/ClickHouse/pull/11579) ([tavplubix](https://github.com/tavplubix)). +* All queries in HTTP session have had the same query_id. It is fixed. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). +* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). +* Fix shard_num/replica_num for `` (breaks use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). +* Fix race condition which may lead to an exception during table drop. It's a bit tricky and not dangerous at all. If you want an explanation, just notice me in telegram. [#11523](https://github.com/ClickHouse/ClickHouse/pull/11523) ([alesapin](https://github.com/alesapin)). +* Fix memory leak when exception is thrown in the middle of aggregation with -State functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). +* Get rid of old libunwind patches. https://github.com/ClickHouse-Extras/libunwind/commit/500aa227911bd185a94bfc071d68f4d3b03cb3b1#r39048012 This allows to disable `-fno-omit-frame-pointer` in `clang` builds that improves performance at least by 1% in average. [#10761](https://github.com/ClickHouse/ClickHouse/pull/10761) ([Amos Bird](https://github.com/amosbird)). +* Fix usage of primary key wrapped into a function with 'FINAL' modifier and 'ORDER BY' optimization. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). + +#### Build/Testing/Packaging Improvement + +* Fix several non significant errors in unit tests. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). +* Fix (false) MSan report in MergeTreeIndexFullText. The issue first appeared in [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + +### ClickHouse release v20.4.5.36-stable 2020-06-10 + +#### Bug Fix + +* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). +* Fix pointInPolygon with nan as point. Fixes [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). +* Fix potential uninitialized memory read in MergeTree shutdown if table was not created successfully. [#11420](https://github.com/ClickHouse/ClickHouse/pull/11420) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed geohashesInBox with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). +* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Remove redundant lock during parts send in ReplicatedMergeTree. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). +* Fix support for `\G` (vertical output) in clickhouse-client in multiline mode. This closes [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential segfault when using `Lazy` database. [#11348](https://github.com/ClickHouse/ClickHouse/pull/11348) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Now merges stopped before change metadata in `ALTER` queries. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). +* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix visitParamExtractRaw when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). +* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix insignificant data race in clickhouse-copier. Found by integration tests. [#11313](https://github.com/ClickHouse/ClickHouse/pull/11313) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash when SET DEFAULT ROLE is called with wrong arguments. This fixes [#10586](https://github.com/ClickHouse/ClickHouse/issues/10586). [#11278](https://github.com/ClickHouse/ClickHouse/pull/11278) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix crash while reading malformed data in Protobuf format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed a bug when cache-dictionary could return default value instead of normal (when there are only expired keys). This affects only string fields. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix error `Block structure mismatch in QueryPipeline` while reading from `VIEW` with constants in inner query. Fixes [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible exception `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed S3 globbing which could fail in case of more than 1000 keys and some backends. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). +* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). +* Fix Kafka performance issue related to reschedules based on limits, which were always applied. [#11149](https://github.com/ClickHouse/ClickHouse/pull/11149) ([filimonov](https://github.com/filimonov)). +* Fix for the hang which was happening sometimes during DROP of table engine=Kafka (or during server restarts). [#11145](https://github.com/ClickHouse/ClickHouse/pull/11145) ([filimonov](https://github.com/filimonov)). +* Fix excessive reserving of threads for simple queries (optimization for reducing the number of threads, which was partly broken after changes in pipeline). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). +* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). + +#### Build/Testing/Packaging Improvement + +* Fix several flaky integration tests. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). + +### ClickHouse release v20.4.4.18-stable 2020-05-26 + +No changes compared to v20.4.3.16-stable. + +### ClickHouse release v20.4.3.16-stable 2020-05-23 + +#### Bug Fix + +* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed memory leak in registerDiskS3. [#11074](https://github.com/ClickHouse/ClickHouse/pull/11074) ([Pavel Kovalenko](https://github.com/Jokser)). +* Fixed the potential missed data during termination of Kafka engine table. [#11048](https://github.com/ClickHouse/ClickHouse/pull/11048) ([filimonov](https://github.com/filimonov)). +* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed very rare potential use-after-free error in `MergeTree` if table was not created successfully. [#10986](https://github.com/ClickHouse/ClickHouse/pull/10986), [#10970](https://github.com/ClickHouse/ClickHouse/pull/10970) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed 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)). +* Fixed 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)). +* Fixed incorrect raw data size in `getRawData()` method. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed sending partially written files by the `DistributedBlockOutputStream`. [#10940](https://github.com/ClickHouse/ClickHouse/pull/10940) ([Azat Khuzhin](https://github.com/azat)). +* Fixed crash in `SELECT count(notNullIn(NULL, []))`. [#10920](https://github.com/ClickHouse/ClickHouse/pull/10920) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the hang which was happening sometimes during `DROP` of `Kafka` table engine. (or during server restarts). [#10910](https://github.com/ClickHouse/ClickHouse/pull/10910) ([filimonov](https://github.com/filimonov)). +* Fixed the impossibility of executing multiple `ALTER RENAME` like `a TO b, c TO a`. [#10895](https://github.com/ClickHouse/ClickHouse/pull/10895) ([alesapin](https://github.com/alesapin)). +* Fixed possible race which could happen when you get result from aggregate function state from multiple thread for the same column. The only way it can happen is when you use `finalizeAggregation` function while reading from table with `Memory` engine which stores `AggregateFunction` state for `quantile*` function. [#10890](https://github.com/ClickHouse/ClickHouse/pull/10890) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `WATCH` hangs after `LiveView` table was dropped from database with `Atomic` engine. [#10859](https://github.com/ClickHouse/ClickHouse/pull/10859) ([tavplubix](https://github.com/tavplubix)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Now constraints are updated if the column participating in `CONSTRAINT` expression was renamed. Fixes [#10844](https://github.com/ClickHouse/ClickHouse/issues/10844). [#10847](https://github.com/ClickHouse/ClickHouse/pull/10847) ([alesapin](https://github.com/alesapin)). +* Fixed potential read of uninitialized memory in cache-dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed combinator `-OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fixed `nullptr` dereference in `StorageBuffer` if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `optimize_skip_unused_shards` with `LowCardinality`. [#10611](https://github.com/ClickHouse/ClickHouse/pull/10611) ([Azat Khuzhin](https://github.com/azat)). +* Fixed handling condition variable for synchronous mutations. In some cases signals to that condition variable could be lost. [#10588](https://github.com/ClickHouse/ClickHouse/pull/10588) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed possible crash when `createDictionary()` is called before `loadStoredObject()` has finished. [#10587](https://github.com/ClickHouse/ClickHouse/pull/10587) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Disable `GROUP BY` sharding_key optimization by default (`optimize_distributed_group_by_sharding_key` had been introduced and turned of by default, due to trickery of sharding_key analyzing, simple example is `if` in sharding key) and fix it for `WITH ROLLUP/CUBE/TOTALS`. [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#10263](https://github.com/ClickHouse/ClickHouse/issues/10263). [#10486](https://github.com/ClickHouse/ClickHouse/pull/10486) ([Azat Khuzhin](https://github.com/azat)). +* Added tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added backward compatibility for create bloom filter index. [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). + +### ClickHouse release v20.4.2.9, 2020-05-12 + +#### Backward Incompatible Change +* System tables (e.g. system.query_log, system.trace_log, system.metric_log) are using compact data part format for parts smaller than 10 MiB in size. Compact data part format is supported since version 20.3. If you are going to downgrade to version less than 20.3, you should manually delete table data for system logs in `/var/lib/clickhouse/data/system/`. +* When string comparison involves FixedString and compared arguments are of different sizes, do comparison as if smaller string is padded to the length of the larger. This is intented for SQL compatibility if we imagine that FixedString data type corresponds to SQL CHAR. This closes [#9272](https://github.com/ClickHouse/ClickHouse/issues/9272). [#10363](https://github.com/ClickHouse/ClickHouse/pull/10363) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Make SHOW CREATE TABLE multiline. Now it is more readable and more like MySQL. [#10049](https://github.com/ClickHouse/ClickHouse/pull/10049) ([Azat Khuzhin](https://github.com/azat)) +* Added a setting `validate_polygons` that is used in `pointInPolygon` function and enabled by default. [#9857](https://github.com/ClickHouse/ClickHouse/pull/9857) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### New Feature +* Add support for secured connection from ClickHouse to Zookeeper [#10184](https://github.com/ClickHouse/ClickHouse/pull/10184) ([Konstantin Lebedev](https://github.com/xzkostyan)) +* Support custom HTTP handlers. See [#5436](https://github.com/ClickHouse/ClickHouse/issues/5436) for description. [#7572](https://github.com/ClickHouse/ClickHouse/pull/7572) ([Winter Zhang](https://github.com/zhang2014)) +* Add MessagePack Input/Output format. [#9889](https://github.com/ClickHouse/ClickHouse/pull/9889) ([Kruglov Pavel](https://github.com/Avogar)) +* Add Regexp input format. [#9196](https://github.com/ClickHouse/ClickHouse/pull/9196) ([Kruglov Pavel](https://github.com/Avogar)) +* Added output format `Markdown` for embedding tables in markdown documents. [#10317](https://github.com/ClickHouse/ClickHouse/pull/10317) ([Kruglov Pavel](https://github.com/Avogar)) +* Added support for custom settings section in dictionaries. Also fixes issue [#2829](https://github.com/ClickHouse/ClickHouse/issues/2829). [#10137](https://github.com/ClickHouse/ClickHouse/pull/10137) ([Artem Streltsov](https://github.com/kekekekule)) +* Added custom settings support in DDL-queries for `CREATE DICTIONARY` [#10465](https://github.com/ClickHouse/ClickHouse/pull/10465) ([Artem Streltsov](https://github.com/kekekekule)) +* Add simple server-wide memory profiler that will collect allocation contexts when server memory usage becomes higher than the next allocation threshold. [#10444](https://github.com/ClickHouse/ClickHouse/pull/10444) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add setting `always_fetch_merged_part` which restrict replica to merge parts by itself and always prefer dowloading from other replicas. [#10379](https://github.com/ClickHouse/ClickHouse/pull/10379) ([alesapin](https://github.com/alesapin)) +* Add function `JSONExtractKeysAndValuesRaw` which extracts raw data from JSON objects [#10378](https://github.com/ClickHouse/ClickHouse/pull/10378) ([hcz](https://github.com/hczhcz)) +* Add memory usage from OS to `system.asynchronous_metrics`. [#10361](https://github.com/ClickHouse/ClickHouse/pull/10361) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added generic variants for functions `least` and `greatest`. Now they work with arbitrary number of arguments of arbitrary types. This fixes [#4767](https://github.com/ClickHouse/ClickHouse/issues/4767) [#10318](https://github.com/ClickHouse/ClickHouse/pull/10318) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Now ClickHouse controls timeouts of dictionary sources on its side. Two new settings added to cache dictionary configuration: `strict_max_lifetime_seconds`, which is `max_lifetime` by default, and `query_wait_timeout_milliseconds`, which is one minute by default. The first settings is also useful with `allow_read_expired_keys` settings (to forbid reading very expired keys). [#10337](https://github.com/ClickHouse/ClickHouse/pull/10337) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Add log_queries_min_type to filter which entries will be written to query_log [#10053](https://github.com/ClickHouse/ClickHouse/pull/10053) ([Azat Khuzhin](https://github.com/azat)) +* Added function `isConstant`. This function checks whether its argument is constant expression and returns 1 or 0. It is intended for development, debugging and demonstration purposes. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* add joinGetOrNull to return NULL when key is missing instead of returning the default value. [#10094](https://github.com/ClickHouse/ClickHouse/pull/10094) ([Amos Bird](https://github.com/amosbird)) +* Consider `NULL` to be equal to `NULL` in `IN` operator, if the option `transform_null_in` is set. [#10085](https://github.com/ClickHouse/ClickHouse/pull/10085) ([achimbab](https://github.com/achimbab)) +* Add `ALTER TABLE ... RENAME COLUMN` for MergeTree table engines family. [#9948](https://github.com/ClickHouse/ClickHouse/pull/9948) ([alesapin](https://github.com/alesapin)) +* Support parallel distributed INSERT SELECT. [#9759](https://github.com/ClickHouse/ClickHouse/pull/9759) ([vxider](https://github.com/Vxider)) +* Add ability to query Distributed over Distributed (w/o `distributed_group_by_no_merge`) ... [#9923](https://github.com/ClickHouse/ClickHouse/pull/9923) ([Azat Khuzhin](https://github.com/azat)) +* Add function `arrayReduceInRanges` which aggregates array elements in given ranges. [#9598](https://github.com/ClickHouse/ClickHouse/pull/9598) ([hcz](https://github.com/hczhcz)) +* Add Dictionary Status on prometheus exporter. [#9622](https://github.com/ClickHouse/ClickHouse/pull/9622) ([Guillaume Tassery](https://github.com/YiuRULE)) +* Add function `arrayAUC` [#8698](https://github.com/ClickHouse/ClickHouse/pull/8698) ([taiyang-li](https://github.com/taiyang-li)) +* Support `DROP VIEW` statement for better TPC-H compatibility. [#9831](https://github.com/ClickHouse/ClickHouse/pull/9831) ([Amos Bird](https://github.com/amosbird)) +* Add 'strict_order' option to windowFunnel() [#9773](https://github.com/ClickHouse/ClickHouse/pull/9773) ([achimbab](https://github.com/achimbab)) +* Support `DATE` and `TIMESTAMP` SQL operators, e.g. `SELECT date '2001-01-01'` [#9691](https://github.com/ClickHouse/ClickHouse/pull/9691) ([Artem Zuikov](https://github.com/4ertus2)) + +#### Experimental Feature +* Added experimental database engine Atomic. It supports non-blocking `DROP` and `RENAME TABLE` queries and atomic `EXCHANGE TABLES t1 AND t2` query [#7512](https://github.com/ClickHouse/ClickHouse/pull/7512) ([tavplubix](https://github.com/tavplubix)) +* Initial support for ReplicatedMergeTree over S3 (it works in suboptimal way) [#10126](https://github.com/ClickHouse/ClickHouse/pull/10126) ([Pavel Kovalenko](https://github.com/Jokser)) + +#### Bug Fix +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed bug, which caused HTTP requests to get stuck on client closing connection when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)) +* Fix segfault in StorageBuffer when exception is thrown on server startup. Fixes [#10550](https://github.com/ClickHouse/ClickHouse/issues/10550) [#10609](https://github.com/ClickHouse/ClickHouse/pull/10609) ([tavplubix](https://github.com/tavplubix)) +* The query`SYSTEM DROP DNS CACHE` now also drops caches used to check if user is allowed to connect from some IP addresses [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)) +* Fix usage of multiple `IN` operators with an identical set in one query. Fixes [#10539](https://github.com/ClickHouse/ClickHouse/issues/10539) [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)) +* Fix crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix logic for aggregation_memory_efficient_merge_threads setting. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)) +* Fix disappearing totals. Totals could have being filtered if query had `JOIN` or subquery with external `WHERE` condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674) [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655) [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709) [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix index corruption, which may occur in some cases after merging compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)) +* Fix the situation, when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)) +* Fix overflow at beginning of unix epoch for timezones with fractional offset from UTC. Fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Better diagnostics for input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204) [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)) +* Fix numeric overflow in `simpleLinearRegression()` over large integers [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)) +* Fix use-after-free in Distributed shutdown, avoid waiting for sending all batches [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)) +* Add CA certificates to clickhouse-server docker image [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)) +* Fix a rare endless loop that might have occurred when using the `addressToLine` function or AggregateFunctionState columns. [#10466](https://github.com/ClickHouse/ClickHouse/pull/10466) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Handle zookeeper "no node error" during distributed query [#10050](https://github.com/ClickHouse/ClickHouse/pull/10050) ([Daniel Chen](https://github.com/Phantomape)) +* Fix bug when server cannot attach table after column's default was altered. [#10441](https://github.com/ClickHouse/ClickHouse/pull/10441) ([alesapin](https://github.com/alesapin)) +* Implicitly cast the default expression type to the column type for the ALIAS columns [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)) +* Don't remove metadata directory if `ATTACH DATABASE` fails [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)) +* Avoid dependency on system tzdata. Fixes loading of `Africa/Casablanca` timezone on CentOS 8. Fixes [#10211](https://github.com/ClickHouse/ClickHouse/issues/10211) [#10425](https://github.com/ClickHouse/ClickHouse/pull/10425) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix some issues if data is inserted with quorum and then gets deleted (DROP PARTITION, TTL, etc.). It led to stuck of INSERTs or false-positive exceptions in SELECTs. Fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946) [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Check the number and type of arguments when creating BloomFilter index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623) [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)) +* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`, otherwise when both settings specified and there are no up-to-date replicas the query will fail (patch from @alex-zaitsev ) [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)) +* Fix the issue when a query with ARRAY JOIN, ORDER BY and LIMIT may return incomplete result. Fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([Vadim Plakhtinskiy](https://github.com/VadimPlh)) +* Add database name to dictionary name after DETACH/ATTACH. Fixes system.dictionaries table and `SYSTEM RELOAD` query [#10415](https://github.com/ClickHouse/ClickHouse/pull/10415) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible incorrect result for extremes in processors pipeline. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix possible segfault when the setting `distributed_group_by_no_merge` is enabled (introduced in 20.3.7.46 by [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131)). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix wrong flattening of `Array(Tuple(...))` data types. Fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259) [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN [#9950](https://github.com/ClickHouse/ClickHouse/pull/9950) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Fix order of columns after Block::sortColumns() [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible `Pipeline stuck` error in `ConcatProcessor` which may happen in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Don't make disk reservations for aggregations. Fixes [#9241](https://github.com/ClickHouse/ClickHouse/issues/9241) [#10375](https://github.com/ClickHouse/ClickHouse/pull/10375) ([Azat Khuzhin](https://github.com/azat)) +* Fix wrong behaviour of datetime functions for timezones that has altered between positive and negative offsets from UTC (e.g. Pacific/Kiritimati). Fixes [#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Avoid infinite loop in `dictIsIn` function. Fixes #515 [#10365](https://github.com/ClickHouse/ClickHouse/pull/10365) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Disable GROUP BY sharding_key optimization by default and fix it for WITH ROLLUP/CUBE/TOTALS [#10516](https://github.com/ClickHouse/ClickHouse/pull/10516) ([Azat Khuzhin](https://github.com/azat)) +* Check for error code when checking parts and don't mark part as broken if the error is like "not enough memory". Fixes [#6269](https://github.com/ClickHouse/ClickHouse/issues/6269) [#10364](https://github.com/ClickHouse/ClickHouse/pull/10364) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Show information about not loaded dictionaries in system tables. [#10234](https://github.com/ClickHouse/ClickHouse/pull/10234) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix nullptr dereference in StorageBuffer if server was shutdown before table startup. [#10641](https://github.com/ClickHouse/ClickHouse/pull/10641) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed `DROP` vs `OPTIMIZE` race in `ReplicatedMergeTree`. `DROP` could left some garbage in replica path in ZooKeeper if there was concurrent `OPTIMIZE` query. [#10312](https://github.com/ClickHouse/ClickHouse/pull/10312) ([tavplubix](https://github.com/tavplubix)) +* Fix 'Logical error: CROSS JOIN has expressions' error for queries with comma and names joins mix. Fixes [#9910](https://github.com/ClickHouse/ClickHouse/issues/9910) [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix queries with `max_bytes_before_external_group_by`. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix the issue with limiting maximum recursion depth in parser in certain cases. This fixes [#10283](https://github.com/ClickHouse/ClickHouse/issues/10283) This fix may introduce minor incompatibility: long and deep queries via clickhouse-client may refuse to work, and you should adjust settings `max_query_size` and `max_parser_depth` accordingly. [#10295](https://github.com/ClickHouse/ClickHouse/pull/10295) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to use `count(*)` with multiple JOINs. Fixes [#9853](https://github.com/ClickHouse/ClickHouse/issues/9853) [#10291](https://github.com/ClickHouse/ClickHouse/pull/10291) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix 'Cannot add column' error while creating `range_hashed` dictionary using DDL query. Fixes [#10093](https://github.com/ClickHouse/ClickHouse/issues/10093). [#10235](https://github.com/ClickHouse/ClickHouse/pull/10235) ([alesapin](https://github.com/alesapin)) +* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed bug where ClickHouse would throw "Unknown function lambda." error message when user tries to run ALTER UPDATE/DELETE on tables with ENGINE = Replicated*. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)) +* Fixed reasonably rare segfault in StorageSystemTables that happens when SELECT ... FROM system.tables is run on a database with Lazy engine. [#10209](https://github.com/ClickHouse/ClickHouse/pull/10209) ([Alexander Kazakov](https://github.com/Akazz)) +* Fix possible infinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed "generateRandom" function for Date type. This fixes [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). Fix an edge case when dates with year 2106 are inserted to MergeTree tables with old-style partitioning but partitions are named with year 1970. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022) [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082) [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix move-to-prewhere optimization in presense of arrayJoin functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092) [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix issue with separator appearing in SCRAMBLE for native mysql-connector-java (JDBC) [#10140](https://github.com/ClickHouse/ClickHouse/pull/10140) ([BohuTANG](https://github.com/BohuTANG)) +* Fix using the current database for an access checking when the database isn't specified. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix ALTER of tables with compact parts. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)) +* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)) +* Fix `DROP TABLE` invoked for dictionary [#10165](https://github.com/ClickHouse/ClickHouse/pull/10165) ([Azat Khuzhin](https://github.com/azat)) +* Convert blocks if structure does not match when doing `INSERT` into Distributed table [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)) +* The number of rows was logged incorrectly (as sum across all parts) when inserted block is split by parts with partition key. [#10138](https://github.com/ClickHouse/ClickHouse/pull/10138) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add some arguments check and support identifier arguments for MySQL Database Engine [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)) +* Fix incorrect `index_granularity_bytes` check while creating new replica. Fixes [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)) +* Fix bug in `CHECK TABLE` query when table contain skip indices. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)) +* Fix Distributed-over-Distributed with the only one shard in a nested table [#9997](https://github.com/ClickHouse/ClickHouse/pull/9997) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). ... [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix bug in dictionary when local clickhouse server is used as source. It may caused memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)) +* Fixed replicated tables startup when updating from an old ClickHouse version where `/table/replicas/replica_name/metadata` node doesn't exist. Fixes [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)) +* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with mixed single and two-level aggregation from different shards. [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix deadlock when database with materialized view failed attach at start [#10054](https://github.com/ClickHouse/ClickHouse/pull/10054) ([Azat Khuzhin](https://github.com/azat)) +* Fix a segmentation fault that could occur in GROUP BY over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). ... [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Fix wrong results of distributed queries when alias could override qualified column name. Fixes [#9672](https://github.com/ClickHouse/ClickHouse/issues/9672) [#9714](https://github.com/ClickHouse/ClickHouse/issues/9714) [#9972](https://github.com/ClickHouse/ClickHouse/pull/9972) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix possible deadlock in `SYSTEM RESTART REPLICAS` [#9955](https://github.com/ClickHouse/ClickHouse/pull/9955) ([tavplubix](https://github.com/tavplubix)) +* Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965) [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed `DeleteOnDestroy` logic in `ATTACH PART` which could lead to automatic removal of attached part and added few tests [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)) +* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)) +* Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839) [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix parsing multiple hosts set in the CREATE USER command [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)) +* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)) +* Fix `DISTINCT` for Distributed when `optimize_skip_unused_shards` is set. [#9808](https://github.com/ClickHouse/ClickHouse/pull/9808) ([Azat Khuzhin](https://github.com/azat)) +* Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). ... [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)) +* Fix error with qualified names in `distributed_product_mode=\'local\'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756) [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)) +* For INSERT queries shards now do clamp the settings from the initiator to their constraints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)) +* Add some retries when commiting offsets to Kafka broker, since it can reject commit if during `offsets.commit.timeout.ms` there were no enough replicas available for the `__consumer_offsets` topic [#9884](https://github.com/ClickHouse/ClickHouse/pull/9884) ([filimonov](https://github.com/filimonov)) +* Fix Distributed engine behavior when virtual columns of the underlying table used in `WHERE` [#9847](https://github.com/ClickHouse/ClickHouse/pull/9847) ([Azat Khuzhin](https://github.com/azat)) +* Fixed some cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)) +* Fix 'Different expressions with the same alias' error when query has PREWHERE and WHERE on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)) +* Fix calculating grants for introspection functions from the setting `allow_introspection_functions`. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix max_distributed_connections (w/ and w/o Processors) [#9673](https://github.com/ClickHouse/ClickHouse/pull/9673) ([Azat Khuzhin](https://github.com/azat)) +* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). ... [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix 'COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query' error in case of subqueries with COMMA JOIN out of tables lists (i.e. in WHERE). Fixes [#9782](https://github.com/ClickHouse/ClickHouse/issues/9782) [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix server crashing when `optimize_skip_unused_shards` is set and expression for key can't be converted to its field type [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)) +* Fix empty string handling in `splitByString`. [#9767](https://github.com/ClickHouse/ClickHouse/pull/9767) ([hcz](https://github.com/hczhcz)) +* Fix broken `ALTER TABLE DELETE COLUMN` query for compact parts. [#9779](https://github.com/ClickHouse/ClickHouse/pull/9779) ([alesapin](https://github.com/alesapin)) +* Fixed missing `rows_before_limit_at_least` for queries over http (with processors pipeline). Fixes [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730) [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix excessive memory consumption in `ALTER` queries (mutations). This fixes [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) +* Fix possible permanent "Cannot schedule a task" error. [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) +* Fix bug in backquoting in external dictionaries DDL. Fixes [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) +* Fixed data race in `text_log`. It does not correspond to any real bug. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix bug in a replication that doesn't allow replication to work if the user has executed mutations on the previous version. This fixes [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)) +* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. It led to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)) +* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)) +* Fix RIGHT and FULL JOIN with LowCardinality in JOIN keys. [#9610](https://github.com/ClickHouse/ClickHouse/pull/9610) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) +* Allow `ALTER ON CLUSTER` of Distributed tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268) [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)) +* Fix issue when timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743) [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Improvement +* Use time zone when comparing DateTime with string literal. This fixes [#5206](https://github.com/ClickHouse/ClickHouse/issues/5206). [#10515](https://github.com/ClickHouse/ClickHouse/pull/10515) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Print verbose diagnostic info if Decimal value cannot be parsed from text input format. [#10205](https://github.com/ClickHouse/ClickHouse/pull/10205) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add tasks/memory metrics for distributed/buffer schedule pools [#10449](https://github.com/ClickHouse/ClickHouse/pull/10449) ([Azat Khuzhin](https://github.com/azat)) +* Display result as soon as it's ready for SELECT DISTINCT queries in clickhouse-local and HTTP interface. This fixes [#8951](https://github.com/ClickHouse/ClickHouse/issues/8951) [#9559](https://github.com/ClickHouse/ClickHouse/pull/9559) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to use `SAMPLE OFFSET` query instead of `cityHash64(PRIMARY KEY) % N == n` for splitting in `clickhouse-copier`. To use this feature, pass `--experimental-use-sample-offset 1` as a command line argument. [#10414](https://github.com/ClickHouse/ClickHouse/pull/10414) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Allow to parse BOM in TSV if the first column cannot contain BOM in its value. This fixes [#10301](https://github.com/ClickHouse/ClickHouse/issues/10301) [#10424](https://github.com/ClickHouse/ClickHouse/pull/10424) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add Avro nested fields insert support [#10354](https://github.com/ClickHouse/ClickHouse/pull/10354) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Allowed to alter column in non-modifying data mode when the same type is specified. [#10382](https://github.com/ClickHouse/ClickHouse/pull/10382) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Auto `distributed_group_by_no_merge` on GROUP BY sharding key (if `optimize_skip_unused_shards` is set) [#10341](https://github.com/ClickHouse/ClickHouse/pull/10341) ([Azat Khuzhin](https://github.com/azat)) +* Optimize queries with LIMIT/LIMIT BY/ORDER BY for distributed with GROUP BY sharding_key [#10373](https://github.com/ClickHouse/ClickHouse/pull/10373) ([Azat Khuzhin](https://github.com/azat)) +* Added a setting `max_server_memory_usage` to limit total memory usage of the server. The metric `MemoryTracking` is now calculated without a drift. The setting `max_memory_usage_for_all_queries` is now obsolete and does nothing. This closes [#10293](https://github.com/ClickHouse/ClickHouse/issues/10293). [#10362](https://github.com/ClickHouse/ClickHouse/pull/10362) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add config option `system_tables_lazy_load`. If it's set to false, then system tables with logs are loaded at the server startup. [Alexander Burmak](https://github.com/Alex-Burmak), [Svyatoslav Tkhon Il Pak](https://github.com/DeifyTheGod), [#9642](https://github.com/ClickHouse/ClickHouse/pull/9642) [#10359](https://github.com/ClickHouse/ClickHouse/pull/10359) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Use background thread pool (background_schedule_pool_size) for distributed sends [#10263](https://github.com/ClickHouse/ClickHouse/pull/10263) ([Azat Khuzhin](https://github.com/azat)) +* Use background thread pool for background buffer flushes. [#10315](https://github.com/ClickHouse/ClickHouse/pull/10315) ([Azat Khuzhin](https://github.com/azat)) +* Support for one special case of removing incompletely written parts. This fixes [#9940](https://github.com/ClickHouse/ClickHouse/issues/9940). [#10221](https://github.com/ClickHouse/ClickHouse/pull/10221) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Use isInjective() over manual list of such functions for GROUP BY optimization. [#10342](https://github.com/ClickHouse/ClickHouse/pull/10342) ([Azat Khuzhin](https://github.com/azat)) +* Avoid printing error message in log if client sends RST packet immediately on connect. It is typical behaviour of IPVS balancer with keepalived and VRRP. This fixes [#1851](https://github.com/ClickHouse/ClickHouse/issues/1851) [#10274](https://github.com/ClickHouse/ClickHouse/pull/10274) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to parse `+inf` for floating point types. This closes [#1839](https://github.com/ClickHouse/ClickHouse/issues/1839) [#10272](https://github.com/ClickHouse/ClickHouse/pull/10272) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Implemented `generateRandom` table function for Nested types. This closes [#9903](https://github.com/ClickHouse/ClickHouse/issues/9903) [#10219](https://github.com/ClickHouse/ClickHouse/pull/10219) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Provide `max_allowed_packed` in MySQL compatibility interface that will help some clients to communicate with ClickHouse via MySQL protocol. [#10199](https://github.com/ClickHouse/ClickHouse/pull/10199) ([BohuTANG](https://github.com/BohuTANG)) +* Allow literals for GLOBAL IN (i.e. `SELECT * FROM remote('localhost', system.one) WHERE dummy global in (0)`) [#10196](https://github.com/ClickHouse/ClickHouse/pull/10196) ([Azat Khuzhin](https://github.com/azat)) +* Fix various small issues in interactive mode of clickhouse-client [#10194](https://github.com/ClickHouse/ClickHouse/pull/10194) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Avoid superfluous dictionaries load (system.tables, DROP/SHOW CREATE TABLE) [#10164](https://github.com/ClickHouse/ClickHouse/pull/10164) ([Azat Khuzhin](https://github.com/azat)) +* Update to RWLock: timeout parameter for getLock() + implementation reworked to be phase fair [#10073](https://github.com/ClickHouse/ClickHouse/pull/10073) ([Alexander Kazakov](https://github.com/Akazz)) +* Enhanced compatibility with native mysql-connector-java(JDBC) [#10021](https://github.com/ClickHouse/ClickHouse/pull/10021) ([BohuTANG](https://github.com/BohuTANG)) +* The function `toString` is considered monotonic and can be used for index analysis even when applied in tautological cases with String or LowCardinality(String) argument. [#10110](https://github.com/ClickHouse/ClickHouse/pull/10110) ([Amos Bird](https://github.com/amosbird)) +* Add `ON CLUSTER` clause support to commands `{CREATE|DROP} USER/ROLE/ROW POLICY/SETTINGS PROFILE/QUOTA`, `GRANT`. [#9811](https://github.com/ClickHouse/ClickHouse/pull/9811) ([Vitaly Baranov](https://github.com/vitlibar)) +* Virtual hosted-style support for S3 URI [#9998](https://github.com/ClickHouse/ClickHouse/pull/9998) ([Pavel Kovalenko](https://github.com/Jokser)) +* Now layout type for dictionaries with no arguments can be specified without round brackets in dictionaries DDL-queries. Fixes [#10057](https://github.com/ClickHouse/ClickHouse/issues/10057). [#10064](https://github.com/ClickHouse/ClickHouse/pull/10064) ([alesapin](https://github.com/alesapin)) +* Add ability to use number ranges with leading zeros in filepath [#9989](https://github.com/ClickHouse/ClickHouse/pull/9989) ([Olga Khvostikova](https://github.com/stavrolia)) +* Better memory usage in CROSS JOIN. [#10029](https://github.com/ClickHouse/ClickHouse/pull/10029) ([Artem Zuikov](https://github.com/4ertus2)) +* Try to connect to all shards in cluster when getting structure of remote table and skip_unavailable_shards is set. [#7278](https://github.com/ClickHouse/ClickHouse/pull/7278) ([nvartolomei](https://github.com/nvartolomei)) +* Add `total_rows`/`total_bytes` into the `system.tables` table. [#9919](https://github.com/ClickHouse/ClickHouse/pull/9919) ([Azat Khuzhin](https://github.com/azat)) +* System log tables now use polymorpic parts by default. [#9905](https://github.com/ClickHouse/ClickHouse/pull/9905) ([Anton Popov](https://github.com/CurtizJ)) +* Add type column into system.settings/merge_tree_settings [#9909](https://github.com/ClickHouse/ClickHouse/pull/9909) ([Azat Khuzhin](https://github.com/azat)) +* Check for available CPU instructions at server startup as early as possible. [#9888](https://github.com/ClickHouse/ClickHouse/pull/9888) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove `ORDER BY` stage from mutations because we read from a single ordered part in a single thread. Also add check that the rows in mutation are ordered by sorting key and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)) +* Implement operator LIKE for FixedString at left hand side. This is needed to better support TPC-DS queries. [#9890](https://github.com/ClickHouse/ClickHouse/pull/9890) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add `force_optimize_skip_unused_shards_no_nested` that will disable `force_optimize_skip_unused_shards` for nested Distributed table [#9812](https://github.com/ClickHouse/ClickHouse/pull/9812) ([Azat Khuzhin](https://github.com/azat)) +* Now columns size is calculated only once for MergeTree data parts. [#9827](https://github.com/ClickHouse/ClickHouse/pull/9827) ([alesapin](https://github.com/alesapin)) +* Evaluate constant expressions for `optimize_skip_unused_shards` (i.e. `SELECT * FROM foo_dist WHERE key=xxHash32(0)`) [#8846](https://github.com/ClickHouse/ClickHouse/pull/8846) ([Azat Khuzhin](https://github.com/azat)) +* Check for using `Date` or `DateTime` column from TTL expressions was removed. [#9967](https://github.com/ClickHouse/ClickHouse/pull/9967) ([Vladimir Chebotarev](https://github.com/excitoon)) +* DiskS3 hard links optimal implementation. [#9760](https://github.com/ClickHouse/ClickHouse/pull/9760) ([Pavel Kovalenko](https://github.com/Jokser)) +* If `set multiple_joins_rewriter_version = 2` enables second version of multiple JOIN rewrites that keeps not clashed column names as is. It supports multiple JOINs with `USING` and allow `select *` for JOINs with subqueries. [#9739](https://github.com/ClickHouse/ClickHouse/pull/9739) ([Artem Zuikov](https://github.com/4ertus2)) +* Implementation of "non-blocking" alter for StorageMergeTree [#9606](https://github.com/ClickHouse/ClickHouse/pull/9606) ([alesapin](https://github.com/alesapin)) +* Add MergeTree full support for DiskS3 [#9646](https://github.com/ClickHouse/ClickHouse/pull/9646) ([Pavel Kovalenko](https://github.com/Jokser)) +* Extend `splitByString` to support empty strings as separators. [#9742](https://github.com/ClickHouse/ClickHouse/pull/9742) ([hcz](https://github.com/hczhcz)) +* Add a `timestamp_ns` column to `system.trace_log`. It contains a high-definition timestamp of the trace event, and allows to build timelines of thread profiles ("flame charts"). [#9696](https://github.com/ClickHouse/ClickHouse/pull/9696) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* When the setting `send_logs_level` is enabled, avoid intermixing of log messages and query progress. [#9634](https://github.com/ClickHouse/ClickHouse/pull/9634) ([Azat Khuzhin](https://github.com/azat)) +* Added support of `MATERIALIZE TTL IN PARTITION`. [#9581](https://github.com/ClickHouse/ClickHouse/pull/9581) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Support complex types inside Avro nested fields [#10502](https://github.com/ClickHouse/ClickHouse/pull/10502) ([Andrew Onyshchuk](https://github.com/oandrew)) + +#### Performance Improvement +* Better insert logic for right table for Partial MergeJoin. [#10467](https://github.com/ClickHouse/ClickHouse/pull/10467) ([Artem Zuikov](https://github.com/4ertus2)) +* Improved performance of row-oriented formats (more than 10% for CSV and more than 35% for Avro in case of narrow tables). [#10503](https://github.com/ClickHouse/ClickHouse/pull/10503) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Improved performance of queries with explicitly defined sets at right side of IN operator and tuples on the left side. [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) +* Use less memory for hash table in HashJoin. [#10416](https://github.com/ClickHouse/ClickHouse/pull/10416) ([Artem Zuikov](https://github.com/4ertus2)) +* Special HashJoin over StorageDictionary. Allow rewrite `dictGet()` functions with JOINs. It's not backward incompatible itself but could uncover [#8400](https://github.com/ClickHouse/ClickHouse/issues/8400) on some installations. [#10133](https://github.com/ClickHouse/ClickHouse/pull/10133) ([Artem Zuikov](https://github.com/4ertus2)) +* Enable parallel insert of materialized view when its target table supports. [#10052](https://github.com/ClickHouse/ClickHouse/pull/10052) ([vxider](https://github.com/Vxider)) +* Improved performance of index analysis with monotonic functions. [#9607](https://github.com/ClickHouse/ClickHouse/pull/9607)[#10026](https://github.com/ClickHouse/ClickHouse/pull/10026) ([Anton Popov](https://github.com/CurtizJ)) +* Using SSE2 or SSE4.2 SIMD intrinsics to speed up tokenization in bloom filters. [#9968](https://github.com/ClickHouse/ClickHouse/pull/9968) ([Vasily Nemkov](https://github.com/Enmk)) +* Improved performance of queries with explicitly defined sets at right side of `IN` operator. This fixes performance regression in version 20.3. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740) ([Anton Popov](https://github.com/CurtizJ)) +* Now clickhouse-copier splits each partition in number of pieces and copies them independently. [#9075](https://github.com/ClickHouse/ClickHouse/pull/9075) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Adding more aggregation methods. For example TPC-H query 1 will now pick `FixedHashMap` and gets 25% performance gain [#9829](https://github.com/ClickHouse/ClickHouse/pull/9829) ([Amos Bird](https://github.com/amosbird)) +* Use single row counter for multiple streams in pre-limit transform. This helps to avoid uniting pipeline streams in queries with `limit` but without `order by` (like `select f(x) from (select x from t limit 1000000000)`) and use multiple threads for further processing. [#9602](https://github.com/ClickHouse/ClickHouse/pull/9602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Build/Testing/Packaging Improvement +* Use a fork of AWS SDK libraries from ClickHouse-Extras [#10527](https://github.com/ClickHouse/ClickHouse/pull/10527) ([Pavel Kovalenko](https://github.com/Jokser)) +* Add integration tests for new ALTER RENAME COLUMN query. [#10654](https://github.com/ClickHouse/ClickHouse/pull/10654) ([vzakaznikov](https://github.com/vzakaznikov)) +* Fix possible signed integer overflow in invocation of function `now64` with wrong arguments. This fixes [#8973](https://github.com/ClickHouse/ClickHouse/issues/8973) [#10511](https://github.com/ClickHouse/ClickHouse/pull/10511) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Split fuzzer and sanitizer configurations to make build config compatible with Oss-fuzz. [#10494](https://github.com/ClickHouse/ClickHouse/pull/10494) ([kyprizel](https://github.com/kyprizel)) +* Fixes for clang-tidy on clang-10. [#10420](https://github.com/ClickHouse/ClickHouse/pull/10420) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Display absolute paths in error messages. Otherwise KDevelop fails to navigate to correct file and opens a new file instead. [#10434](https://github.com/ClickHouse/ClickHouse/pull/10434) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added `ASAN_OPTIONS` environment variable to investigate errors in CI stress tests with Address sanitizer. [#10440](https://github.com/ClickHouse/ClickHouse/pull/10440) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Enable ThinLTO for clang builds (experimental). [#10435](https://github.com/ClickHouse/ClickHouse/pull/10435) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove accidential dependency on Z3 that may be introduced if the system has Z3 solver installed. [#10426](https://github.com/ClickHouse/ClickHouse/pull/10426) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Move integration tests docker files to docker/ directory. [#10335](https://github.com/ClickHouse/ClickHouse/pull/10335) ([Ilya Yatsishin](https://github.com/qoega)) +* Allow to use `clang-10` in CI. It ensures that [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238) is fixed. [#10384](https://github.com/ClickHouse/ClickHouse/pull/10384) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update OpenSSL to upstream master. Fixed the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` and `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. The issue was present in version 20.1. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix clang-10 build. [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238) [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)) +* Add performance test for [Parallel INSERT for materialized view](https://github.com/ClickHouse/ClickHouse/pull/10052). [#10345](https://github.com/ClickHouse/ClickHouse/pull/10345) ([vxider](https://github.com/Vxider)) +* Fix flaky test `test_settings_constraints_distributed.test_insert_clamps_settings`. [#10346](https://github.com/ClickHouse/ClickHouse/pull/10346) ([Vitaly Baranov](https://github.com/vitlibar)) +* Add util to test results upload in CI ClickHouse [#10330](https://github.com/ClickHouse/ClickHouse/pull/10330) ([Ilya Yatsishin](https://github.com/qoega)) +* Convert test results to JSONEachRow format in junit_to_html tool [#10323](https://github.com/ClickHouse/ClickHouse/pull/10323) ([Ilya Yatsishin](https://github.com/qoega)) +* Update cctz. [#10215](https://github.com/ClickHouse/ClickHouse/pull/10215) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow to create HTML report from the purest JUnit XML report. [#10247](https://github.com/ClickHouse/ClickHouse/pull/10247) ([Ilya Yatsishin](https://github.com/qoega)) +* Update the check for minimal compiler version. Fix the root cause of the issue [#10250](https://github.com/ClickHouse/ClickHouse/issues/10250) [#10256](https://github.com/ClickHouse/ClickHouse/pull/10256) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Initial support for live view tables over distributed [#10179](https://github.com/ClickHouse/ClickHouse/pull/10179) ([vzakaznikov](https://github.com/vzakaznikov)) +* Fix (false) MSan report in MergeTreeIndexFullText. The issue first appeared in [#9968](https://github.com/ClickHouse/ClickHouse/issues/9968). [#10801](https://github.com/ClickHouse/ClickHouse/pull/10801) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* clickhouse-docker-util [#10151](https://github.com/ClickHouse/ClickHouse/pull/10151) ([filimonov](https://github.com/filimonov)) +* Update pdqsort to recent version [#10171](https://github.com/ClickHouse/ClickHouse/pull/10171) ([Ivan](https://github.com/abyss7)) +* Update libdivide to v3.0 [#10169](https://github.com/ClickHouse/ClickHouse/pull/10169) ([Ivan](https://github.com/abyss7)) +* Add check with enabled polymorphic parts. [#10086](https://github.com/ClickHouse/ClickHouse/pull/10086) ([Anton Popov](https://github.com/CurtizJ)) +* Add cross-compile build for FreeBSD. This fixes [#9465](https://github.com/ClickHouse/ClickHouse/issues/9465) [#9643](https://github.com/ClickHouse/ClickHouse/pull/9643) ([Ivan](https://github.com/abyss7)) +* Add performance test for [#6924](https://github.com/ClickHouse/ClickHouse/issues/6924) [#6980](https://github.com/ClickHouse/ClickHouse/pull/6980) ([filimonov](https://github.com/filimonov)) +* Add support of `/dev/null` in the `File` engine for better performance testing [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) +* Move all folders inside /dbms one level up [#9974](https://github.com/ClickHouse/ClickHouse/pull/9974) ([Ivan](https://github.com/abyss7)) +* Add a test that checks that read from MergeTree with single thread is performed in order. Addition to [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670) [#9762](https://github.com/ClickHouse/ClickHouse/pull/9762) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix the `00964_live_view_watch_events_heartbeat.py` test to avoid race condition. [#9944](https://github.com/ClickHouse/ClickHouse/pull/9944) ([vzakaznikov](https://github.com/vzakaznikov)) +* Fix integration test `test_settings_constraints` [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)) +* Every function in its own file, part 12. [#9922](https://github.com/ClickHouse/ClickHouse/pull/9922) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added performance test for the case of extremely slow analysis of array of tuples. [#9872](https://github.com/ClickHouse/ClickHouse/pull/9872) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update zstd to 1.4.4. It has some minor improvements in performance and compression ratio. If you run replicas with different versions of ClickHouse you may see reasonable error messages `Data after merge is not byte-identical to data on another replicas.` with explanation. These messages are Ok and you should not worry. [#10663](https://github.com/ClickHouse/ClickHouse/pull/10663) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix TSan report in `system.stack_trace`. [#9832](https://github.com/ClickHouse/ClickHouse/pull/9832) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Removed dependency on `clock_getres`. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added identifier names check with clang-tidy. [#9799](https://github.com/ClickHouse/ClickHouse/pull/9799) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update "builder" docker image. This image is not used in CI but is useful for developers. [#9809](https://github.com/ClickHouse/ClickHouse/pull/9809) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove old `performance-test` tool that is no longer used in CI. `clickhouse-performance-test` is great but now we are using way superior tool that is doing comparison testing with sophisticated statistical formulas to achieve confident results regardless to various changes in environment. [#9796](https://github.com/ClickHouse/ClickHouse/pull/9796) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added most of clang-static-analyzer checks. [#9765](https://github.com/ClickHouse/ClickHouse/pull/9765) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update Poco to 1.9.3 in preparation for MongoDB URI support. [#6892](https://github.com/ClickHouse/ClickHouse/pull/6892) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Fix build with `-DUSE_STATIC_LIBRARIES=0 -DENABLE_JEMALLOC=0` [#9651](https://github.com/ClickHouse/ClickHouse/pull/9651) ([Artem Zuikov](https://github.com/4ertus2)) +* For change log script, if merge commit was cherry-picked to release branch, take PR name from commit description. [#9708](https://github.com/ClickHouse/ClickHouse/pull/9708) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Support `vX.X-conflicts` tag in backport script. [#9705](https://github.com/ClickHouse/ClickHouse/pull/9705) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix `auto-label` for backporting script. [#9685](https://github.com/ClickHouse/ClickHouse/pull/9685) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Use libc++ in Darwin cross-build to make it consistent with native build. [#9665](https://github.com/ClickHouse/ClickHouse/pull/9665) ([Hui Wang](https://github.com/huiwang)) +* Fix flacky test `01017_uniqCombined_memory_usage`. Continuation of [#7236](https://github.com/ClickHouse/ClickHouse/issues/7236). [#9667](https://github.com/ClickHouse/ClickHouse/pull/9667) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix build for native MacOS Clang compiler [#9649](https://github.com/ClickHouse/ClickHouse/pull/9649) ([Ivan](https://github.com/abyss7)) +* Allow to add various glitches around `pthread_mutex_lock`, `pthread_mutex_unlock` functions. [#9635](https://github.com/ClickHouse/ClickHouse/pull/9635) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add support for `clang-tidy` in `packager` script. [#9625](https://github.com/ClickHouse/ClickHouse/pull/9625) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add ability to use unbundled msgpack. [#10168](https://github.com/ClickHouse/ClickHouse/pull/10168) ([Azat Khuzhin](https://github.com/azat)) + + +## ClickHouse release v20.3 + + +### ClickHouse release v20.3.21.2-lts, 2020-11-02 + +#### Bug Fix + +* Fix dictGet in sharding_key (and similar places, i.e. when the function context is stored permanently). [#16205](https://github.com/ClickHouse/ClickHouse/pull/16205) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect empty result for query from `Distributed` table if query has `WHERE`, `PREWHERE` and `GLOBAL IN`. Fixes [#15792](https://github.com/ClickHouse/ClickHouse/issues/15792). [#15933](https://github.com/ClickHouse/ClickHouse/pull/15933) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix missing or excessive headers in `TSV/CSVWithNames` formats. This fixes [#12504](https://github.com/ClickHouse/ClickHouse/issues/12504). [#13343](https://github.com/ClickHouse/ClickHouse/pull/13343) ([Azat Khuzhin](https://github.com/azat)). + + +### ClickHouse release v20.3.20.6-lts, 2020-10-09 + +#### Bug Fix + +* Mutation might hang waiting for some non-existent part after `MOVE` or `REPLACE PARTITION` or, in rare cases, after `DETACH` or `DROP PARTITION`. It's fixed. [#15724](https://github.com/ClickHouse/ClickHouse/pull/15724), [#15537](https://github.com/ClickHouse/ClickHouse/pull/15537) ([tavplubix](https://github.com/tavplubix)). +* Fix hang of queries with a lot of subqueries to same table of `MySQL` engine. Previously, if there were more than 16 subqueries to same `MySQL` table in query, it hang forever. [#15299](https://github.com/ClickHouse/ClickHouse/pull/15299) ([Anton Popov](https://github.com/CurtizJ)). +* Fix 'Unknown identifier' in GROUP BY when query has JOIN over Merge table. [#15242](https://github.com/ClickHouse/ClickHouse/pull/15242) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix to make predicate push down work when subquery contains finalizeAggregation function. Fixes [#14847](https://github.com/ClickHouse/ClickHouse/issues/14847). [#14937](https://github.com/ClickHouse/ClickHouse/pull/14937) ([filimonov](https://github.com/filimonov)). +* Concurrent `ALTER ... REPLACE/MOVE PARTITION ...` queries might cause deadlock. It's fixed. [#13626](https://github.com/ClickHouse/ClickHouse/pull/13626) ([tavplubix](https://github.com/tavplubix)). + + +### ClickHouse release v20.3.19.4-lts, 2020-09-18 + +#### Bug Fix + +* Fix rare error in `SELECT` queries when the queried column has `DEFAULT` expression which depends on the other column which also has `DEFAULT` and not present in select query and not exists on disk. Partially fixes [#14531](https://github.com/ClickHouse/ClickHouse/issues/14531). [#14845](https://github.com/ClickHouse/ClickHouse/pull/14845) ([alesapin](https://github.com/alesapin)). +* Fix bug when `ALTER UPDATE` mutation with Nullable column in assignment expression and constant value (like `UPDATE x = 42`) leads to incorrect value in column or segfault. Fixes [#13634](https://github.com/ClickHouse/ClickHouse/issues/13634), [#14045](https://github.com/ClickHouse/ClickHouse/issues/14045). [#14646](https://github.com/ClickHouse/ClickHouse/pull/14646) ([alesapin](https://github.com/alesapin)). +* Fix wrong Decimal multiplication result caused wrong decimal scale of result column. [#14603](https://github.com/ClickHouse/ClickHouse/pull/14603) ([Artem Zuikov](https://github.com/4ertus2)). + +#### Improvement + +* Support custom codecs in compact parts. [#12183](https://github.com/ClickHouse/ClickHouse/pull/12183) ([Anton Popov](https://github.com/CurtizJ)). + + +### ClickHouse release v20.3.18.10-lts, 2020-09-08 + +#### Bug Fix + +* Stop query execution if exception happened in `PipelineExecutor` itself. This could prevent rare possible query hung. Continuation of [#14334](https://github.com/ClickHouse/ClickHouse/issues/14334). [#14402](https://github.com/ClickHouse/ClickHouse/pull/14402) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the behaviour when sometimes cache-dictionary returned default value instead of present value from source. [#13624](https://github.com/ClickHouse/ClickHouse/pull/13624) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix parsing row policies from users.xml when names of databases or tables contain dots. This fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779), [#12527](https://github.com/ClickHouse/ClickHouse/issues/12527). [#13199](https://github.com/ClickHouse/ClickHouse/pull/13199) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix CAST(Nullable(String), Enum()). [#12745](https://github.com/ClickHouse/ClickHouse/pull/12745) ([Azat Khuzhin](https://github.com/azat)). +* Fixed data race in `text_log`. It does not correspond to any real bug. [#9726](https://github.com/ClickHouse/ClickHouse/pull/9726) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Improvement + +* Fix wrong error for long queries. It was possible to get syntax error other than `Max query size exceeded` for correct query. [#13928](https://github.com/ClickHouse/ClickHouse/pull/13928) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* 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)). + +#### Performance Improvement + +* Slightly optimize very short queries with LowCardinality. [#14129](https://github.com/ClickHouse/ClickHouse/pull/14129) ([Anton Popov](https://github.com/CurtizJ)). + +#### Build/Testing/Packaging Improvement + +* Fix UBSan report (adding zero to nullptr) in HashTable that appeared after migration to clang-10. [#10638](https://github.com/ClickHouse/ClickHouse/pull/10638) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + +### ClickHouse release v20.3.17.173-lts, 2020-08-15 + +#### Bug Fix + +* Fix crash in JOIN with StorageMerge and `set enable_optimize_predicate_expression=1`. [#13679](https://github.com/ClickHouse/ClickHouse/pull/13679) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix invalid return type for comparison of tuples with `NULL` elements. Fixes [#12461](https://github.com/ClickHouse/ClickHouse/issues/12461). [#13420](https://github.com/ClickHouse/ClickHouse/pull/13420) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix queries with constant columns and `ORDER BY` prefix of primary key. [#13396](https://github.com/ClickHouse/ClickHouse/pull/13396) ([Anton Popov](https://github.com/CurtizJ)). +* Return passed number for numbers with MSB set in roundUpToPowerOfTwoOrZero(). [#13234](https://github.com/ClickHouse/ClickHouse/pull/13234) ([Azat Khuzhin](https://github.com/azat)). + + +### ClickHouse release v20.3.16.165-lts 2020-08-10 + +#### Bug Fix + +* Fixed error in `parseDateTimeBestEffort` function when unix timestamp was passed as an argument. This fixes [#13362](https://github.com/ClickHouse/ClickHouse/issues/13362). [#13441](https://github.com/ClickHouse/ClickHouse/pull/13441) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potentially low performance and slightly incorrect result for `uniqExact`, `topK`, `sumDistinct` and similar aggregate functions called on Float types with `NaN` values. It also triggered assert in debug build. This fixes [#12491](https://github.com/ClickHouse/ClickHouse/issues/12491). [#13254](https://github.com/ClickHouse/ClickHouse/pull/13254) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed function if with nullable constexpr as cond that is not literal NULL. Fixes [#12463](https://github.com/ClickHouse/ClickHouse/issues/12463). [#13226](https://github.com/ClickHouse/ClickHouse/pull/13226) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed assert in `arrayElement` function in case of array elements are Nullable and array subscript is also Nullable. This fixes [#12172](https://github.com/ClickHouse/ClickHouse/issues/12172). [#13224](https://github.com/ClickHouse/ClickHouse/pull/13224) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed unnecessary limiting for the number of threads for selects from local replica. [#12840](https://github.com/ClickHouse/ClickHouse/pull/12840) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible extra overflow row in data which could appear for queries `WITH TOTALS`. [#12747](https://github.com/ClickHouse/ClickHouse/pull/12747) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed performance with large tuples, which are interpreted as functions in `IN` section. The case when user write `WHERE x IN tuple(1, 2, ...)` instead of `WHERE x IN (1, 2, ...)` for some obscure reason. [#12700](https://github.com/ClickHouse/ClickHouse/pull/12700) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed memory tracking for input_format_parallel_parsing (by attaching thread to group). [#12672](https://github.com/ClickHouse/ClickHouse/pull/12672) ([Azat Khuzhin](https://github.com/azat)). +* Fixed [#12293](https://github.com/ClickHouse/ClickHouse/issues/12293) allow push predicate when subquery contains with clause. [#12663](https://github.com/ClickHouse/ClickHouse/pull/12663) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed [#10572](https://github.com/ClickHouse/ClickHouse/issues/10572) fix bloom filter index with const expression. [#12659](https://github.com/ClickHouse/ClickHouse/pull/12659) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed SIGSEGV in StorageKafka when broker is unavailable (and not only). [#12658](https://github.com/ClickHouse/ClickHouse/pull/12658) ([Azat Khuzhin](https://github.com/azat)). +* Fixed race condition in external dictionaries with cache layout which can lead server crash. [#12566](https://github.com/ClickHouse/ClickHouse/pull/12566) ([alesapin](https://github.com/alesapin)). +* Fixed bug which lead to broken old parts after `ALTER DELETE` query when `enable_mixed_granularity_parts=1`. Fixes [#12536](https://github.com/ClickHouse/ClickHouse/issues/12536). [#12543](https://github.com/ClickHouse/ClickHouse/pull/12543) ([alesapin](https://github.com/alesapin)). +* Better exception for function `in` with invalid number of arguments. [#12529](https://github.com/ClickHouse/ClickHouse/pull/12529) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed performance issue, while reading from compact parts. [#12492](https://github.com/ClickHouse/ClickHouse/pull/12492) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed the deadlock if `text_log` is enabled. [#12452](https://github.com/ClickHouse/ClickHouse/pull/12452) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed possible segfault if StorageMerge. Closes [#12054](https://github.com/ClickHouse/ClickHouse/issues/12054). [#12401](https://github.com/ClickHouse/ClickHouse/pull/12401) ([tavplubix](https://github.com/tavplubix)). +* Fixed `TOTALS/ROLLUP/CUBE` for aggregate functions with `-State` and `Nullable` arguments. This fixes [#12163](https://github.com/ClickHouse/ClickHouse/issues/12163). [#12376](https://github.com/ClickHouse/ClickHouse/pull/12376) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed order of columns in `WITH FILL` modifier. Previously order of columns of `ORDER BY` statement wasn't respected. [#12306](https://github.com/ClickHouse/ClickHouse/pull/12306) ([Anton Popov](https://github.com/CurtizJ)). +* Avoid "bad cast" exception when there is an expression that filters data by virtual columns (like `_table` in `Merge` tables) or by "index" columns in system tables such as filtering by database name when querying from `system.tables`, and this expression returns `Nullable` type. This fixes [#12166](https://github.com/ClickHouse/ClickHouse/issues/12166). [#12305](https://github.com/ClickHouse/ClickHouse/pull/12305) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Show error after `TrieDictionary` failed to load. [#12290](https://github.com/ClickHouse/ClickHouse/pull/12290) ([Vitaly Baranov](https://github.com/vitlibar)). +* The function `arrayFill` worked incorrectly for empty arrays that may lead to crash. This fixes [#12263](https://github.com/ClickHouse/ClickHouse/issues/12263). [#12279](https://github.com/ClickHouse/ClickHouse/pull/12279) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Implement conversions to the common type for `LowCardinality` types. This allows to execute UNION ALL of tables with columns of LowCardinality and other columns. This fixes [#8212](https://github.com/ClickHouse/ClickHouse/issues/8212). This fixes [#4342](https://github.com/ClickHouse/ClickHouse/issues/4342). [#12275](https://github.com/ClickHouse/ClickHouse/pull/12275) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behaviour when during multiple sequential inserts in `StorageFile` header for some special types was written more than once. This fixed [#6155](https://github.com/ClickHouse/ClickHouse/issues/6155). [#12197](https://github.com/ClickHouse/ClickHouse/pull/12197) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed logical functions for UInt8 values when they are not equal to 0 or 1. [#12196](https://github.com/ClickHouse/ClickHouse/pull/12196) ([Alexander Kazakov](https://github.com/Akazz)). +* Fixed `dictGet` arguments check during GROUP BY injective functions elimination. [#12179](https://github.com/ClickHouse/ClickHouse/pull/12179) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong logic in `ALTER DELETE` that leads to deleting of records when condition evaluates to NULL. This fixes [#9088](https://github.com/ClickHouse/ClickHouse/issues/9088). This closes [#12106](https://github.com/ClickHouse/ClickHouse/issues/12106). [#12153](https://github.com/ClickHouse/ClickHouse/pull/12153) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed transform of query to send to external DBMS (e.g. MySQL, ODBC) in presense of aliases. This fixes [#12032](https://github.com/ClickHouse/ClickHouse/issues/12032). [#12151](https://github.com/ClickHouse/ClickHouse/pull/12151) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential overflow in integer division. This fixes [#12119](https://github.com/ClickHouse/ClickHouse/issues/12119). [#12140](https://github.com/ClickHouse/ClickHouse/pull/12140) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential infinite loop in `greatCircleDistance`, `geoDistance`. This fixes [#12117](https://github.com/ClickHouse/ClickHouse/issues/12117). [#12137](https://github.com/ClickHouse/ClickHouse/pull/12137) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Avoid `There is no query` exception for materialized views with joins or with subqueries attached to system logs (system.query_log, metric_log, etc) or to engine=Buffer underlying table. [#12120](https://github.com/ClickHouse/ClickHouse/pull/12120) ([filimonov](https://github.com/filimonov)). +* Fixed performance for selects with `UNION` caused by wrong limit for the total number of threads. Fixes [#12030](https://github.com/ClickHouse/ClickHouse/issues/12030). [#12103](https://github.com/ClickHouse/ClickHouse/pull/12103) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed segfault with `-StateResample` combinators. [#12092](https://github.com/ClickHouse/ClickHouse/pull/12092) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed unnecessary limiting the number of threads for selects from `VIEW`. Fixes [#11937](https://github.com/ClickHouse/ClickHouse/issues/11937). [#12085](https://github.com/ClickHouse/ClickHouse/pull/12085) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible crash while using wrong type for `PREWHERE`. Fixes [#12053](https://github.com/ClickHouse/ClickHouse/issues/12053), [#12060](https://github.com/ClickHouse/ClickHouse/issues/12060). [#12060](https://github.com/ClickHouse/ClickHouse/pull/12060) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Expected single dictionary argument for function` for function `defaultValueOfArgumentType` with `LowCardinality` type. Fixes [#11808](https://github.com/ClickHouse/ClickHouse/issues/11808). [#12056](https://github.com/ClickHouse/ClickHouse/pull/12056) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Cannot capture column` for higher-order functions with `Tuple(LowCardinality)` argument. Fixes [#9766](https://github.com/ClickHouse/ClickHouse/issues/9766). [#12055](https://github.com/ClickHouse/ClickHouse/pull/12055) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Parse tables metadata in parallel when loading database. This fixes slow server startup when there are large number of tables. [#12045](https://github.com/ClickHouse/ClickHouse/pull/12045) ([tavplubix](https://github.com/tavplubix)). +* Make `topK` aggregate function return Enum for Enum types. This fixes [#3740](https://github.com/ClickHouse/ClickHouse/issues/3740). [#12043](https://github.com/ClickHouse/ClickHouse/pull/12043) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed constraints check if constraint is a constant expression. This fixes [#11360](https://github.com/ClickHouse/ClickHouse/issues/11360). [#12042](https://github.com/ClickHouse/ClickHouse/pull/12042) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect comparison of tuples with `Nullable` columns. Fixes [#11985](https://github.com/ClickHouse/ClickHouse/issues/11985). [#12039](https://github.com/ClickHouse/ClickHouse/pull/12039) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed wrong result and potential crash when invoking function `if` with arguments of type `FixedString` with different sizes. This fixes [#11362](https://github.com/ClickHouse/ClickHouse/issues/11362). [#12021](https://github.com/ClickHouse/ClickHouse/pull/12021) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* A query with function `neighbor` as the only returned expression may return empty result if the function is called with offset `-9223372036854775808`. This fixes [#11367](https://github.com/ClickHouse/ClickHouse/issues/11367). [#12019](https://github.com/ClickHouse/ClickHouse/pull/12019) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential array size overflow in generateRandom that may lead to crash. This fixes [#11371](https://github.com/ClickHouse/ClickHouse/issues/11371). [#12013](https://github.com/ClickHouse/ClickHouse/pull/12013) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential floating point exception. This closes [#11378](https://github.com/ClickHouse/ClickHouse/issues/11378). [#12005](https://github.com/ClickHouse/ClickHouse/pull/12005) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong setting name in log message at server startup. [#11997](https://github.com/ClickHouse/ClickHouse/pull/11997) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Query parameter was not set` in `Values` format. Fixes [#11918](https://github.com/ClickHouse/ClickHouse/issues/11918). [#11936](https://github.com/ClickHouse/ClickHouse/pull/11936) ([tavplubix](https://github.com/tavplubix)). +* Keep aliases for substitutions in query (parametrized queries). This fixes [#11914](https://github.com/ClickHouse/ClickHouse/issues/11914). [#11916](https://github.com/ClickHouse/ClickHouse/pull/11916) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed potential floating point exception when parsing DateTime64. This fixes [#11374](https://github.com/ClickHouse/ClickHouse/issues/11374). [#11875](https://github.com/ClickHouse/ClickHouse/pull/11875) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed memory accounting via `HTTP` interface (can be significant with `wait_end_of_query=1`). [#11840](https://github.com/ClickHouse/ClickHouse/pull/11840) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong result for `if()` with NULLs in condition. [#11807](https://github.com/ClickHouse/ClickHouse/pull/11807) ([Artem Zuikov](https://github.com/4ertus2)). +* Parse metadata stored in zookeeper before checking for equality. [#11739](https://github.com/ClickHouse/ClickHouse/pull/11739) ([Azat Khuzhin](https://github.com/azat)). +* Fixed `LIMIT n WITH TIES` usage together with `ORDER BY` statement, which contains aliases. [#11689](https://github.com/ClickHouse/ClickHouse/pull/11689) ([Anton Popov](https://github.com/CurtizJ)). +* Fix potential read of uninitialized memory in cache dictionary. [#10834](https://github.com/ClickHouse/ClickHouse/pull/10834) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Performance Improvement + +* Index not used for IN operator with literals, performance regression introduced around v19.3. This fixes [#10574](https://github.com/ClickHouse/ClickHouse/issues/10574). [#12062](https://github.com/ClickHouse/ClickHouse/pull/12062) ([nvartolomei](https://github.com/nvartolomei)). + + +### ClickHouse release v20.3.12.112-lts 2020-06-25 + +#### Bug Fix + +* Fix rare crash caused by using `Nullable` column in prewhere condition. Continuation of [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix using too many threads for queries. [#11788](https://github.com/ClickHouse/ClickHouse/pull/11788) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). +* Now replicated fetches will be cancelled during metadata alter. [#11744](https://github.com/ClickHouse/ClickHouse/pull/11744) ([alesapin](https://github.com/alesapin)). +* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). +* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). +* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix error which leads to an incorrect state of `system.mutations`. It may show that whole mutation is already done but the server still has `MUTATE_PART` tasks in the replication queue and tries to execute them. This fixes [#11611](https://github.com/ClickHouse/ClickHouse/issues/11611). [#11681](https://github.com/ClickHouse/ClickHouse/pull/11681) ([alesapin](https://github.com/alesapin)). +* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove trivial count query optimization if row-level security is set. In previous versions the user get total count of records in a table instead filtered. This fixes [#11352](https://github.com/ClickHouse/ClickHouse/issues/11352). [#11644](https://github.com/ClickHouse/ClickHouse/pull/11644) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare crash caused by using `Nullable` column in prewhere condition. (Probably it is connected with [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) somehow). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix error `Block structure mismatch` for queries with sampling reading from `Buffer` table. [#11602](https://github.com/ClickHouse/ClickHouse/pull/11602) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix wrong exit code of the clickhouse-client, when exception.code() % 256 = 0. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). +* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* All queries in HTTP session have had the same query_id. It is fixed. [#11578](https://github.com/ClickHouse/ClickHouse/pull/11578) ([tavplubix](https://github.com/tavplubix)). +* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). +* Fix shard_num/replica_num for `` (breaks use_compact_format_in_distributed_parts_names). [#11528](https://github.com/ClickHouse/ClickHouse/pull/11528) ([Azat Khuzhin](https://github.com/azat)). +* Fix memory leak when exception is thrown in the middle of aggregation with -State functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix wrong results of distributed queries when alias could override qualified column name. Fixes [#9672](https://github.com/ClickHouse/ClickHouse/issues/9672) [#9714](https://github.com/ClickHouse/ClickHouse/issues/9714). [#9972](https://github.com/ClickHouse/ClickHouse/pull/9972) ([Artem Zuikov](https://github.com/4ertus2)). + + +### ClickHouse release v20.3.11.97-lts 2020-06-10 + +#### New Feature + +* Now ClickHouse controls timeouts of dictionary sources on its side. Two new settings added to cache dictionary configuration: `strict_max_lifetime_seconds`, which is `max_lifetime` by default and `query_wait_timeout_milliseconds`, which is one minute by default. The first settings is also useful with `allow_read_expired_keys` settings (to forbid reading very expired keys). [#10337](https://github.com/ClickHouse/ClickHouse/pull/10337) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). + +#### Bug Fix + +* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). +* Fix pointInPolygon with nan as point. Fixes [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). +* Fix crash in JOIN over LowCarinality(T) and Nullable(T). [#11380](https://github.com/ClickHouse/ClickHouse/issues/11380). [#11414](https://github.com/ClickHouse/ClickHouse/pull/11414) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix error code for wrong `USING` key. [#11373](https://github.com/ClickHouse/ClickHouse/issues/11373). [#11404](https://github.com/ClickHouse/ClickHouse/pull/11404) ([Artem Zuikov](https://github.com/4ertus2)). +* Fixed geohashesInBox with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). +* Better errors for `joinGet()` functions. [#11389](https://github.com/ClickHouse/ClickHouse/pull/11389) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Remove redundant lock during parts send in ReplicatedMergeTree. [#11354](https://github.com/ClickHouse/ClickHouse/pull/11354) ([alesapin](https://github.com/alesapin)). +* Fix support for `\G` (vertical output) in clickhouse-client in multiline mode. This closes [#9933](https://github.com/ClickHouse/ClickHouse/issues/9933). [#11350](https://github.com/ClickHouse/ClickHouse/pull/11350) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash in direct selects from StorageJoin (without JOIN) and wrong nullability. [#11340](https://github.com/ClickHouse/ClickHouse/pull/11340) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Now merges stopped before change metadata in `ALTER` queries. [#11335](https://github.com/ClickHouse/ClickHouse/pull/11335) ([alesapin](https://github.com/alesapin)). +* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix visitParamExtractRaw when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). +* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash while reading malformed data in Protobuf format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed a bug when cache-dictionary could return default value instead of normal (when there are only expired keys). This affects only string fields. [#11233](https://github.com/ClickHouse/ClickHouse/pull/11233) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix error `Block structure mismatch in QueryPipeline` while reading from `VIEW` with constants in inner query. Fixes [#11181](https://github.com/ClickHouse/ClickHouse/issues/11181). [#11205](https://github.com/ClickHouse/ClickHouse/pull/11205) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible exception `Invalid status for associated output`. [#11200](https://github.com/ClickHouse/ClickHouse/pull/11200) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed S3 globbing which could fail in case of more than 1000 keys and some backends. [#11179](https://github.com/ClickHouse/ClickHouse/pull/11179) ([Vladimir Chebotarev](https://github.com/excitoon)). +* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). +* Fix excessive reserving of threads for simple queries (optimization for reducing the number of threads, which was partly broken after changes in pipeline). [#11114](https://github.com/ClickHouse/ClickHouse/pull/11114) ([Azat Khuzhin](https://github.com/azat)). +* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). +* Introduce commit retry logic to decrease the possibility of getting duplicates from Kafka in rare cases when offset commit was failed. [#9884](https://github.com/ClickHouse/ClickHouse/pull/9884) ([filimonov](https://github.com/filimonov)). + +#### Performance Improvement + +* Get dictionary and check access rights only once per each call of any function reading external dictionaries. [#10928](https://github.com/ClickHouse/ClickHouse/pull/10928) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Build/Testing/Packaging Improvement + +* Fix several flaky integration tests. [#11355](https://github.com/ClickHouse/ClickHouse/pull/11355) ([alesapin](https://github.com/alesapin)). + +### ClickHouse release v20.3.10.75-lts 2020-05-23 + +#### Bug Fix + +* Removed logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* Fixed `parseDateTime64BestEffort` argument resolution bugs. [#11038](https://github.com/ClickHouse/ClickHouse/pull/11038) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed incorrect raw data size in method `getRawData()`. [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of `GROUP BY` result is large and aggregation is performed by a single `String` field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed backward compatibility with tuples in `Distributed` tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed `SIGSEGV` in `StringHashTable` if such a key does not exist. [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Fixed columns order after `Block::sortColumns()`. [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fixed the issue with `ODBC` bridge when no quoting of identifiers is requested. Fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `UBSan` and `MSan` report in `DateLUT`. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incorrect type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Fixed `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed combinator -`OrNull` and `-OrDefault` when combined with `-State`. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fixed crash in `generateRandom` with nested types. Fixes [#10583](https://github.com/ClickHouse/ClickHouse/issues/10583). [#10734](https://github.com/ClickHouse/ClickHouse/pull/10734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible buffer overflow in function `h3EdgeAngle`. [#10711](https://github.com/ClickHouse/ClickHouse/pull/10711) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). +* Fixed order of parameters in `AggregateTransform` constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). +* Fixed the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed possible incorrect number of rows for queries with `LIMIT`. Fixes [#10566](https://github.com/ClickHouse/ClickHouse/issues/10566), [#10709](https://github.com/ClickHouse/ClickHouse/issues/10709). [#10660](https://github.com/ClickHouse/ClickHouse/pull/10660) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed a bug which locks concurrent alters when table has a lot of parts. [#10659](https://github.com/ClickHouse/ClickHouse/pull/10659) ([alesapin](https://github.com/alesapin)). +* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `SELECT` of column `ALIAS` which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* Implemented comparison between DateTime64 and String values. [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed index corruption, which may occur in some cases after merge compact parts into another compact part. [#10531](https://github.com/ClickHouse/ClickHouse/pull/10531) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed the situation, when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). +* Fixed overflow at beginning of unix epoch for timezones with fractional offset from `UTC`. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed improper shutdown of `Distributed` storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). +* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). + + +#### Build/Testing/Packaging Improvement + +* Fix UBSan report in LZ4 library. [#10631](https://github.com/ClickHouse/ClickHouse/pull/10631) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix clang-10 build. [#10238](https://github.com/ClickHouse/ClickHouse/issues/10238). [#10370](https://github.com/ClickHouse/ClickHouse/pull/10370) ([Amos Bird](https://github.com/amosbird)). +* Added failing tests about `max_rows_to_sort` setting. [#10268](https://github.com/ClickHouse/ClickHouse/pull/10268) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added some improvements in printing diagnostic info in input formats. Fixes [#10204](https://github.com/ClickHouse/ClickHouse/issues/10204). [#10418](https://github.com/ClickHouse/ClickHouse/pull/10418) ([tavplubix](https://github.com/tavplubix)). +* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). + +#### Bug fix + +* Fix error `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). + + +### ClickHouse release v20.3.8.53, 2020-04-23 + +#### Bug Fix +* Fixed wrong behaviour of datetime functions for timezones that has altered between positive and negative offsets from UTC (e.g. Pacific/Kiritimati). This fixes [#7202](https://github.com/ClickHouse/ClickHouse/issues/7202) [#10369](https://github.com/ClickHouse/ClickHouse/pull/10369) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix possible segfault with `distributed_group_by_no_merge` enabled (introduced in 20.3.7.46 by [#10131](https://github.com/ClickHouse/ClickHouse/issues/10131)). [#10399](https://github.com/ClickHouse/ClickHouse/pull/10399) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix wrong flattening of `Array(Tuple(...))` data types. This fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259) [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Drop disks reservation in Aggregator. This fixes bug in disk space reservation, which may cause big external aggregation to fail even if it could be completed successfully [#10375](https://github.com/ClickHouse/ClickHouse/pull/10375) ([Azat Khuzhin](https://github.com/azat)) +* Fixed `DROP` vs `OPTIMIZE` race in `ReplicatedMergeTree`. `DROP` could left some garbage in replica path in ZooKeeper if there was concurrent `OPTIMIZE` query. [#10312](https://github.com/ClickHouse/ClickHouse/pull/10312) ([tavplubix](https://github.com/tavplubix)) +* Fix bug when server cannot attach table after column default was altered. [#10441](https://github.com/ClickHouse/ClickHouse/pull/10441) ([alesapin](https://github.com/alesapin)) +* Do not remove metadata directory when attach database fails before loading tables. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)) +* Fixed several bugs when some data was inserted with quorum, then deleted somehow (DROP PARTITION, TTL) and this leaded to the stuck of INSERTs or false-positive exceptions in SELECTs. This fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946) [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Fix possible `Pipeline stuck` error in `ConcatProcessor` which could have happened in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed wrong behavior in HashTable that caused compilation error when trying to read HashMap from buffer. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)) +* Allow to use `count(*)` with multiple JOINs. Fixes [#9853](https://github.com/ClickHouse/ClickHouse/issues/9853) [#10291](https://github.com/ClickHouse/ClickHouse/pull/10291) ([Artem Zuikov](https://github.com/4ertus2)) +* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`, otherwise when both settings specified and there are no up-to-date replicas the query will fail (patch from @alex-zaitsev). Fixes: [#2564](https://github.com/ClickHouse/ClickHouse/issues/2564). [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)) +* Fix the issue when a query with ARRAY JOIN, ORDER BY and LIMIT may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). Author: [Vadim Plakhtinskiy](https://github.com/VadimPlh). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Check the number and type of arguments when creating BloomFilter index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623) [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)) + +#### Performance Improvement +* Improved performance of queries with explicitly defined sets at right side of `IN` operator and tuples in the left side. This fixes performance regression in version 20.3. [#9740](https://github.com/ClickHouse/ClickHouse/pull/9740), [#10385](https://github.com/ClickHouse/ClickHouse/pull/10385) ([Anton Popov](https://github.com/CurtizJ)) + +### ClickHouse release v20.3.7.46, 2020-04-17 + +#### Bug Fix + +* Fix `Logical error: CROSS JOIN has expressions` error for queries with comma and names joins mix. [#10311](https://github.com/ClickHouse/ClickHouse/pull/10311) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix queries with `max_bytes_before_external_group_by`. [#10302](https://github.com/ClickHouse/ClickHouse/pull/10302) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix move-to-prewhere optimization in presense of arrayJoin functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). + +### ClickHouse release v20.3.6.40, 2020-04-16 + +#### New Feature + +* Added function `isConstant`. This function checks whether its argument is constant expression and returns 1 or 0. It is intended for development, debugging and demonstration purposes. [#10198](https://github.com/ClickHouse/ClickHouse/pull/10198) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix + +* Fix error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug where ClickHouse would throw "Unknown function lambda." error message when user tries to run ALTER UPDATE/DELETE on tables with ENGINE = Replicated*. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). +* Fixed "generateRandom" function for Date type. This fixes [#9973](https://github.com/ClickHouse/ClickHouse/issues/9973). Fix an edge case when dates with year 2106 are inserted to MergeTree tables with old-style partitioning but partitions are named with year 1970. [#10218](https://github.com/ClickHouse/ClickHouse/pull/10218) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Convert types if the table definition of a View does not correspond to the SELECT query. This fixes [#10180](https://github.com/ClickHouse/ClickHouse/issues/10180) and [#10022](https://github.com/ClickHouse/ClickHouse/issues/10022). [#10217](https://github.com/ClickHouse/ClickHouse/pull/10217) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix column names of constants inside JOIN that may clash with names of constants outside of JOIN. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible inifinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix using the current database for access checking when the database isn't specified. [#10192](https://github.com/ClickHouse/ClickHouse/pull/10192) ([Vitaly Baranov](https://github.com/vitlibar)). +* Convert blocks if structure does not match on INSERT into Distributed(). [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible incorrect result for extremes in processors pipeline. [#10131](https://github.com/ClickHouse/ClickHouse/pull/10131) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix some kinds of alters with compact parts. [#10130](https://github.com/ClickHouse/ClickHouse/pull/10130) ([Anton Popov](https://github.com/CurtizJ)). +* Fix incorrect `index_granularity_bytes` check while creating new replica. Fixes [#10098](https://github.com/ClickHouse/ClickHouse/issues/10098). [#10121](https://github.com/ClickHouse/ClickHouse/pull/10121) ([alesapin](https://github.com/alesapin)). +* Fix SIGSEGV on INSERT into Distributed table when its structure differs from the underlying tables. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed replicated tables startup when updating from an old ClickHouse version where `/table/replicas/replica_name/metadata` node doesn't exist. Fixes [#10037](https://github.com/ClickHouse/ClickHouse/issues/10037). [#10095](https://github.com/ClickHouse/ClickHouse/pull/10095) ([alesapin](https://github.com/alesapin)). +* Add some arguments check and support identifier arguments for MySQL Database Engine. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). +* Fix bug in clickhouse dictionary source from localhost clickhouse server. The bug may lead to memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). +* Fix bug in `CHECK TABLE` query when table contain skip indices. [#10068](https://github.com/ClickHouse/ClickHouse/pull/10068) ([alesapin](https://github.com/alesapin)). +* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with different level from different shards (mixed single and two level aggregation). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a segmentation fault that could occur in GROUP BY over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix the number of threads used for remote query execution (performance regression, since 20.3). This happened when query from `Distributed` table was executed simultaneously on local and remote shards. Fixes [#9965](https://github.com/ClickHouse/ClickHouse/issues/9965). [#9971](https://github.com/ClickHouse/ClickHouse/pull/9971) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). +* Fix 'Not found column in block' error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). +* Fix parsing multiple hosts set in the CREATE USER command, e.g. `CREATE USER user6 HOST NAME REGEXP 'lo.?*host', NAME REGEXP 'lo*host'`. [#9924](https://github.com/ClickHouse/ClickHouse/pull/9924) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). +* Fix "scalar doesn't exist" error in ALTERs ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). +* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). +* Fix error with qualified names in `distributed_product_mode='local'`. Fixes [#4756](https://github.com/ClickHouse/ClickHouse/issues/4756). [#9891](https://github.com/ClickHouse/ClickHouse/pull/9891) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix calculating grants for introspection functions from the setting 'allow_introspection_functions'. [#9840](https://github.com/ClickHouse/ClickHouse/pull/9840) ([Vitaly Baranov](https://github.com/vitlibar)). + +#### Build/Testing/Packaging Improvement + +* Fix integration test `test_settings_constraints`. [#9962](https://github.com/ClickHouse/ClickHouse/pull/9962) ([Vitaly Baranov](https://github.com/vitlibar)). +* Removed dependency on `clock_getres`. [#9833](https://github.com/ClickHouse/ClickHouse/pull/9833) ([alexey-milovidov](https://github.com/alexey-milovidov)). + + +### ClickHouse release v20.3.5.21, 2020-03-27 + +#### Bug Fix + +* Fix 'Different expressions with the same alias' error when query has PREWHERE and WHERE on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). +* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send INSERT queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix 'COMMA to CROSS JOIN rewriter is not enabled or cannot rewrite query' error in case of subqueries with COMMA JOIN out of tables lists (i.e. in WHERE). Fixes [#9782](https://github.com/ClickHouse/ClickHouse/issues/9782). [#9830](https://github.com/ClickHouse/ClickHouse/pull/9830) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix SIGSEGV with optimize_skip_unused_shards when type cannot be converted. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). +* Fix broken `ALTER TABLE DELETE COLUMN` query for compact parts. [#9779](https://github.com/ClickHouse/ClickHouse/pull/9779) ([alesapin](https://github.com/alesapin)). +* Fix max_distributed_connections (w/ and w/o Processors). [#9673](https://github.com/ClickHouse/ClickHouse/pull/9673) ([Azat Khuzhin](https://github.com/azat)). +* Fixed a few cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)). + +#### Improvement + +* Remove order by stage from mutations because we read from a single ordered part in a single thread. Also add check that the order of rows in mutation is ordered in sorting key order and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)). + + +### ClickHouse release v20.3.4.10, 2020-03-20 + +#### Bug Fix +* This release also contains all bug fixes from 20.1.8.41 +* Fix missing `rows_before_limit_at_least` for queries over http (with processors pipeline). This fixes [#9730](https://github.com/ClickHouse/ClickHouse/issues/9730). [#9757](https://github.com/ClickHouse/ClickHouse/pull/9757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + + +### ClickHouse release v20.3.3.6, 2020-03-17 + +#### Bug Fix +* This release also contains all bug fixes from 20.1.7.38 +* Fix bug in a replication that doesn't allow replication to work if the user has executed mutations on the previous version. This fixes [#9645](https://github.com/ClickHouse/ClickHouse/issues/9645). [#9652](https://github.com/ClickHouse/ClickHouse/pull/9652) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again. +* Add setting `use_compact_format_in_distributed_parts_names` which allows to write files for `INSERT` queries into `Distributed` table with more compact format. This fixes [#9647](https://github.com/ClickHouse/ClickHouse/issues/9647). [#9653](https://github.com/ClickHouse/ClickHouse/pull/9653) ([alesapin](https://github.com/alesapin)). It makes version 20.3 backward compatible again. + +### ClickHouse release v20.3.2.1, 2020-03-12 + +#### Backward Incompatible Change + +* Fixed the issue `file name too long` when sending data for `Distributed` tables for a large number of replicas. Fixed the issue that replica credentials were exposed in the server log. The format of directory name on disk was changed to `[shard{shard_index}[_replica{replica_index}]]`. [#8911](https://github.com/ClickHouse/ClickHouse/pull/8911) ([Mikhail Korotov](https://github.com/millb)) After you upgrade to the new version, you will not be able to downgrade without manual intervention, because old server version does not recognize the new directory format. If you want to downgrade, you have to manually rename the corresponding directories to the old format. This change is relevant only if you have used asynchronous `INSERT`s to `Distributed` tables. In the version 20.3.3 we will introduce a setting that will allow you to enable the new format gradually. +* Changed the format of replication log entries for mutation commands. You have to wait for old mutations to process before installing the new version. +* Implement simple memory profiler that dumps stacktraces to `system.trace_log` every N bytes over soft allocation limit [#8765](https://github.com/ClickHouse/ClickHouse/pull/8765) ([Ivan](https://github.com/abyss7)) [#9472](https://github.com/ClickHouse/ClickHouse/pull/9472) ([alexey-milovidov](https://github.com/alexey-milovidov)) The column of `system.trace_log` was renamed from `timer_type` to `trace_type`. This will require changes in third-party performance analysis and flamegraph processing tools. +* Use OS thread id everywhere instead of internal thread number. This fixes [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477) Old `clickhouse-client` cannot receive logs that are send from the server when the setting `send_logs_level` is enabled, because the names and types of the structured log messages were changed. On the other hand, different server versions can send logs with different types to each other. When you don't use the `send_logs_level` setting, you should not care. [#8954](https://github.com/ClickHouse/ClickHouse/pull/8954) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove `indexHint` function [#9542](https://github.com/ClickHouse/ClickHouse/pull/9542) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove `findClusterIndex`, `findClusterValue` functions. This fixes [#8641](https://github.com/ClickHouse/ClickHouse/issues/8641). If you were using these functions, send an email to `clickhouse-feedback@yandex-team.com` [#9543](https://github.com/ClickHouse/ClickHouse/pull/9543) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Now it's not allowed to create columns or add columns with `SELECT` subquery as default expression. [#9481](https://github.com/ClickHouse/ClickHouse/pull/9481) ([alesapin](https://github.com/alesapin)) +* Require aliases for subqueries in JOIN. [#9274](https://github.com/ClickHouse/ClickHouse/pull/9274) ([Artem Zuikov](https://github.com/4ertus2)) +* Improved `ALTER MODIFY/ADD` queries logic. Now you cannot `ADD` column without type, `MODIFY` default expression doesn't change type of column and `MODIFY` type doesn't loose default expression value. Fixes [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). [#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) ([alesapin](https://github.com/alesapin)) +* Require server to be restarted to apply the changes in logging configuration. This is a temporary workaround to avoid the bug where the server logs to a deleted log file (see [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* The setting `experimental_use_processors` is enabled by default. This setting enables usage of the new query pipeline. This is internal refactoring and we expect no visible changes. If you will see any issues, set it to back zero. [#8768](https://github.com/ClickHouse/ClickHouse/pull/8768) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### New Feature +* Add `Avro` and `AvroConfluent` input/output formats [#8571](https://github.com/ClickHouse/ClickHouse/pull/8571) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8957](https://github.com/ClickHouse/ClickHouse/pull/8957) ([Andrew Onyshchuk](https://github.com/oandrew)) [#8717](https://github.com/ClickHouse/ClickHouse/pull/8717) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Multi-threaded and non-blocking updates of expired keys in `cache` dictionaries (with optional permission to read old ones). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Add query `ALTER ... MATERIALIZE TTL`. It runs mutation that forces to remove expired data by TTL and recalculates meta-information about TTL in all parts. [#8775](https://github.com/ClickHouse/ClickHouse/pull/8775) ([Anton Popov](https://github.com/CurtizJ)) +* Switch from HashJoin to MergeJoin (on disk) if needed [#9082](https://github.com/ClickHouse/ClickHouse/pull/9082) ([Artem Zuikov](https://github.com/4ertus2)) +* Added `MOVE PARTITION` command for `ALTER TABLE` [#4729](https://github.com/ClickHouse/ClickHouse/issues/4729) [#6168](https://github.com/ClickHouse/ClickHouse/pull/6168) ([Guillaume Tassery](https://github.com/YiuRULE)) +* Reloading storage configuration from configuration file on the fly. [#8594](https://github.com/ClickHouse/ClickHouse/pull/8594) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Allowed to change `storage_policy` to not less rich one. [#8107](https://github.com/ClickHouse/ClickHouse/pull/8107) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Added support for globs/wildcards for S3 storage and table function. [#8851](https://github.com/ClickHouse/ClickHouse/pull/8851) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Implement `bitAnd`, `bitOr`, `bitXor`, `bitNot` for `FixedString(N)` datatype. [#9091](https://github.com/ClickHouse/ClickHouse/pull/9091) ([Guillaume Tassery](https://github.com/YiuRULE)) +* Added function `bitCount`. This fixes [#8702](https://github.com/ClickHouse/ClickHouse/issues/8702). [#8708](https://github.com/ClickHouse/ClickHouse/pull/8708) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#8749](https://github.com/ClickHouse/ClickHouse/pull/8749) ([ikopylov](https://github.com/ikopylov)) +* Add `generateRandom` table function to generate random rows with given schema. Allows to populate arbitrary test table with data. [#8994](https://github.com/ClickHouse/ClickHouse/pull/8994) ([Ilya Yatsishin](https://github.com/qoega)) +* `JSONEachRowFormat`: support special case when objects enclosed in top-level array. [#8860](https://github.com/ClickHouse/ClickHouse/pull/8860) ([Kruglov Pavel](https://github.com/Avogar)) +* Now it's possible to create a column with `DEFAULT` expression which depends on a column with default `ALIAS` expression. [#9489](https://github.com/ClickHouse/ClickHouse/pull/9489) ([alesapin](https://github.com/alesapin)) +* Allow to specify `--limit` more than the source data size in `clickhouse-obfuscator`. The data will repeat itself with different random seed. [#9155](https://github.com/ClickHouse/ClickHouse/pull/9155) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added `groupArraySample` function (similar to `groupArray`) with reservior sampling algorithm. [#8286](https://github.com/ClickHouse/ClickHouse/pull/8286) ([Amos Bird](https://github.com/amosbird)) +* Now you can monitor the size of update queue in `cache`/`complex_key_cache` dictionaries via system metrics. [#9413](https://github.com/ClickHouse/ClickHouse/pull/9413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Allow to use CRLF as a line separator in CSV output format with setting `output_format_csv_crlf_end_of_line` is set to 1 [#8934](https://github.com/ClickHouse/ClickHouse/pull/8934) [#8935](https://github.com/ClickHouse/ClickHouse/pull/8935) [#8963](https://github.com/ClickHouse/ClickHouse/pull/8963) ([Mikhail Korotov](https://github.com/millb)) +* Implement more functions of the [H3](https://github.com/uber/h3) API: `h3GetBaseCell`, `h3HexAreaM2`, `h3IndexesAreNeighbors`, `h3ToChildren`, `h3ToString` and `stringToH3` [#8938](https://github.com/ClickHouse/ClickHouse/pull/8938) ([Nico Mandery](https://github.com/nmandery)) +* New setting introduced: `max_parser_depth` to control maximum stack size and allow large complex queries. This fixes [#6681](https://github.com/ClickHouse/ClickHouse/issues/6681) and [#7668](https://github.com/ClickHouse/ClickHouse/issues/7668). [#8647](https://github.com/ClickHouse/ClickHouse/pull/8647) ([Maxim Smirnov](https://github.com/qMBQx8GH)) +* Add a setting `force_optimize_skip_unused_shards` setting to throw if skipping of unused shards is not possible [#8805](https://github.com/ClickHouse/ClickHouse/pull/8805) ([Azat Khuzhin](https://github.com/azat)) +* Allow to configure multiple disks/volumes for storing data for send in `Distributed` engine [#8756](https://github.com/ClickHouse/ClickHouse/pull/8756) ([Azat Khuzhin](https://github.com/azat)) +* Support storage policy (``) for storing temporary data. [#8750](https://github.com/ClickHouse/ClickHouse/pull/8750) ([Azat Khuzhin](https://github.com/azat)) +* Added `X-ClickHouse-Exception-Code` HTTP header that is set if exception was thrown before sending data. This implements [#4971](https://github.com/ClickHouse/ClickHouse/issues/4971). [#8786](https://github.com/ClickHouse/ClickHouse/pull/8786) ([Mikhail Korotov](https://github.com/millb)) +* Added function `ifNotFinite`. It is just a syntactic sugar: `ifNotFinite(x, y) = isFinite(x) ? x : y`. [#8710](https://github.com/ClickHouse/ClickHouse/pull/8710) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added `last_successful_update_time` column in `system.dictionaries` table [#9394](https://github.com/ClickHouse/ClickHouse/pull/9394) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Add `blockSerializedSize` function (size on disk without compression) [#8952](https://github.com/ClickHouse/ClickHouse/pull/8952) ([Azat Khuzhin](https://github.com/azat)) +* Add function `moduloOrZero` [#9358](https://github.com/ClickHouse/ClickHouse/pull/9358) ([hcz](https://github.com/hczhcz)) +* Added system tables `system.zeros` and `system.zeros_mt` as well as tale functions `zeros()` and `zeros_mt()`. Tables (and table functions) contain single column with name `zero` and type `UInt8`. This column contains zeros. It is needed for test purposes as the fastest method to generate many rows. This fixes [#6604](https://github.com/ClickHouse/ClickHouse/issues/6604) [#9593](https://github.com/ClickHouse/ClickHouse/pull/9593) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) + +#### Experimental Feature +* Add new compact format of parts in `MergeTree`-family tables in which all columns are stored in one file. It helps to increase performance of small and frequent inserts. The old format (one file per column) is now called wide. Data storing format is controlled by settings `min_bytes_for_wide_part` and `min_rows_for_wide_part`. [#8290](https://github.com/ClickHouse/ClickHouse/pull/8290) ([Anton Popov](https://github.com/CurtizJ)) +* Support for S3 storage for `Log`, `TinyLog` and `StripeLog` tables. [#8862](https://github.com/ClickHouse/ClickHouse/pull/8862) ([Pavel Kovalenko](https://github.com/Jokser)) + +#### Bug Fix +* Fixed inconsistent whitespaces in log messages. [#9322](https://github.com/ClickHouse/ClickHouse/pull/9322) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix bug in which arrays of unnamed tuples were flattened as Nested structures on table creation. [#8866](https://github.com/ClickHouse/ClickHouse/pull/8866) ([achulkov2](https://github.com/achulkov2)) +* Fixed the issue when "Too many open files" error may happen if there are too many files matching glob pattern in `File` table or `file` table function. Now files are opened lazily. This fixes [#8857](https://github.com/ClickHouse/ClickHouse/issues/8857) [#8861](https://github.com/ClickHouse/ClickHouse/pull/8861) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* DROP TEMPORARY TABLE now drops only temporary table. [#8907](https://github.com/ClickHouse/ClickHouse/pull/8907) ([Vitaly Baranov](https://github.com/vitlibar)) +* Remove outdated partition when we shutdown the server or DETACH/ATTACH a table. [#8602](https://github.com/ClickHouse/ClickHouse/pull/8602) ([Guillaume Tassery](https://github.com/YiuRULE)) +* For how the default disk calculates the free space from `data` subdirectory. Fixed the issue when the amount of free space is not calculated correctly if the `data` directory is mounted to a separate device (rare case). This fixes [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) [#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) ([Mikhail Korotov](https://github.com/millb)) +* Allow comma (cross) join with IN () inside. [#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) ([Artem Zuikov](https://github.com/4ertus2)) +* Allow to rewrite CROSS to INNER JOIN if there's [NOT] LIKE operator in WHERE section. [#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix possible incorrect result after `GROUP BY` with enabled setting `distributed_aggregation_memory_efficient`. Fixes [#9134](https://github.com/ClickHouse/ClickHouse/issues/9134). [#9289](https://github.com/ClickHouse/ClickHouse/pull/9289) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Found keys were counted as missed in metrics of cache dictionaries. [#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Fix replication protocol incompatibility introduced in [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) +* Fixed race condition on `queue_task_handle` at the startup of `ReplicatedMergeTree` tables. [#9552](https://github.com/ClickHouse/ClickHouse/pull/9552) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* The token `NOT` didn't work in `SHOW TABLES NOT LIKE` query [#8727](https://github.com/ClickHouse/ClickHouse/issues/8727) [#8940](https://github.com/ClickHouse/ClickHouse/pull/8940) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added range check to function `h3EdgeLengthM`. Without this check, buffer overflow is possible. [#8945](https://github.com/ClickHouse/ClickHouse/pull/8945) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed up a bug in batched calculations of ternary logical OPs on multiple arguments (more than 10). [#8718](https://github.com/ClickHouse/ClickHouse/pull/8718) ([Alexander Kazakov](https://github.com/Akazz)) +* Fix error of PREWHERE optimization, which could lead to segfaults or `Inconsistent number of columns got from MergeTreeRangeReader` exception. [#9024](https://github.com/ClickHouse/ClickHouse/pull/9024) ([Anton Popov](https://github.com/CurtizJ)) +* Fix unexpected `Timeout exceeded while reading from socket` exception, which randomly happens on secure connection before timeout actually exceeded and when query profiler is enabled. Also add `connect_timeout_with_failover_secure_ms` settings (default 100ms), which is similar to `connect_timeout_with_failover_ms`, but is used for secure connections (because SSL handshake is slower, than ordinary TCP connection) [#9026](https://github.com/ClickHouse/ClickHouse/pull/9026) ([tavplubix](https://github.com/tavplubix)) +* Fix bug with mutations finalization, when mutation may hang in state with `parts_to_do=0` and `is_done=0`. [#9022](https://github.com/ClickHouse/ClickHouse/pull/9022) ([alesapin](https://github.com/alesapin)) +* Use new ANY JOIN logic with `partial_merge_join` setting. It's possible to make `ANY|ALL|SEMI LEFT` and `ALL INNER` joins with `partial_merge_join=1` now. [#8932](https://github.com/ClickHouse/ClickHouse/pull/8932) ([Artem Zuikov](https://github.com/4ertus2)) +* Shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send queries to a shard with another constraints. [#9447](https://github.com/ClickHouse/ClickHouse/pull/9447) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fixed memory management problem in `MergeTreeReadPool`. [#8791](https://github.com/ClickHouse/ClickHouse/pull/8791) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix `toDecimal*OrNull()` functions family when called with string `e`. Fixes [#8312](https://github.com/ClickHouse/ClickHouse/issues/8312) [#8764](https://github.com/ClickHouse/ClickHouse/pull/8764) ([Artem Zuikov](https://github.com/4ertus2)) +* Make sure that `FORMAT Null` sends no data to the client. [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Fix bug that timestamp in `LiveViewBlockInputStream` will not updated. `LIVE VIEW` is an experimental feature. [#8644](https://github.com/ClickHouse/ClickHouse/pull/8644) ([vxider](https://github.com/Vxider)) [#8625](https://github.com/ClickHouse/ClickHouse/pull/8625) ([vxider](https://github.com/Vxider)) +* Fixed `ALTER MODIFY TTL` wrong behavior which did not allow to delete old TTL expressions. [#8422](https://github.com/ClickHouse/ClickHouse/pull/8422) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fixed UBSan report in MergeTreeIndexSet. This fixes [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) [#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed the behaviour of `match` and `extract` functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. This fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Avoid throwing from destructor in Apache Avro 3rd-party library. [#9066](https://github.com/ClickHouse/ClickHouse/pull/9066) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Don't commit a batch polled from `Kafka` partially as it can lead to holes in data. [#8876](https://github.com/ClickHouse/ClickHouse/pull/8876) ([filimonov](https://github.com/filimonov)) +* Fix `joinGet` with nullable return types. [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) [#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) ([Amos Bird](https://github.com/amosbird)) +* Fix data incompatibility when compressed with `T64` codec. [#9016](https://github.com/ClickHouse/ClickHouse/pull/9016) ([Artem Zuikov](https://github.com/4ertus2)) Fix data type ids in `T64` compression codec that leads to wrong (de)compression in affected versions. [#9033](https://github.com/ClickHouse/ClickHouse/pull/9033) ([Artem Zuikov](https://github.com/4ertus2)) +* Add setting `enable_early_constant_folding` and disable it in some cases that leads to errors. [#9010](https://github.com/ClickHouse/ClickHouse/pull/9010) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix pushdown predicate optimizer with VIEW and enable the test [#9011](https://github.com/ClickHouse/ClickHouse/pull/9011) ([Winter Zhang](https://github.com/zhang2014)) +* Fix segfault in `Merge` tables, that can happen when reading from `File` storages [#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) ([tavplubix](https://github.com/tavplubix)) +* Added a check for storage policy in `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE`. Otherwise it could make data of part inaccessible after restart and prevent ClickHouse to start. [#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix alters if there is TTL set for table. [#8800](https://github.com/ClickHouse/ClickHouse/pull/8800) ([Anton Popov](https://github.com/CurtizJ)) +* Fix race condition that can happen when `SYSTEM RELOAD ALL DICTIONARIES` is executed while some dictionary is being modified/added/removed. [#8801](https://github.com/ClickHouse/ClickHouse/pull/8801) ([Vitaly Baranov](https://github.com/vitlibar)) +* In previous versions `Memory` database engine use empty data path, so tables are created in `path` directory (e.g. `/var/lib/clickhouse/`), not in data directory of database (e.g. `/var/lib/clickhouse/db_name`). [#8753](https://github.com/ClickHouse/ClickHouse/pull/8753) ([tavplubix](https://github.com/tavplubix)) +* Fixed wrong log messages about missing default disk or policy. [#9530](https://github.com/ClickHouse/ClickHouse/pull/9530) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix not(has()) for the bloom_filter index of array types. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) +* Allow first column(s) in a table with `Log` engine be an alias [#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) ([Ivan](https://github.com/abyss7)) +* Fix order of ranges while reading from `MergeTree` table in one thread. It could lead to exceptions from `MergeTreeRangeReader` or wrong query results. [#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) ([Anton Popov](https://github.com/CurtizJ)) +* Make `reinterpretAsFixedString` to return `FixedString` instead of `String`. [#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) ([Andrew Onyshchuk](https://github.com/oandrew)) +* Avoid extremely rare cases when the user can get wrong error message (`Success` instead of detailed error description). [#9457](https://github.com/ClickHouse/ClickHouse/pull/9457) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Do not crash when using `Template` format with empty row template. [#8785](https://github.com/ClickHouse/ClickHouse/pull/8785) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Metadata files for system tables could be created in wrong place [#8653](https://github.com/ClickHouse/ClickHouse/pull/8653) ([tavplubix](https://github.com/tavplubix)) Fixes [#8581](https://github.com/ClickHouse/ClickHouse/issues/8581). +* Fix data race on exception_ptr in cache dictionary [#8303](https://github.com/ClickHouse/ClickHouse/issues/8303). [#9379](https://github.com/ClickHouse/ClickHouse/pull/9379) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Do not throw an exception for query `ATTACH TABLE IF NOT EXISTS`. Previously it was thrown if table already exists, despite the `IF NOT EXISTS` clause. [#8967](https://github.com/ClickHouse/ClickHouse/pull/8967) ([Anton Popov](https://github.com/CurtizJ)) +* Fixed missing closing paren in exception message. [#8811](https://github.com/ClickHouse/ClickHouse/pull/8811) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Avoid message `Possible deadlock avoided` at the startup of clickhouse-client in interactive mode. [#9455](https://github.com/ClickHouse/ClickHouse/pull/9455) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed the issue when padding at the end of base64 encoded value can be malformed. Update base64 library. This fixes [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378) [#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) ([filimonov](https://github.com/filimonov)) +* Fixed exception in `DROP TABLE IF EXISTS` [#8663](https://github.com/ClickHouse/ClickHouse/pull/8663) ([Nikita Vasilev](https://github.com/nikvas0)) +* Fix crash when a user tries to `ALTER MODIFY SETTING` for old-formated `MergeTree` table engines family. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) +* Support for UInt64 numbers that don't fit in Int64 in JSON-related functions. Update SIMDJSON to master. This fixes [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) [#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed execution of inversed predicates when non-strictly monotinic functional index is used. [#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) ([Alexander Kazakov](https://github.com/Akazz)) +* Don't try to fold `IN` constant in `GROUP BY` [#8868](https://github.com/ClickHouse/ClickHouse/pull/8868) ([Amos Bird](https://github.com/amosbird)) +* Fix bug in `ALTER DELETE` mutations which leads to index corruption. This fixes [#9019](https://github.com/ClickHouse/ClickHouse/issues/9019) and [#8982](https://github.com/ClickHouse/ClickHouse/issues/8982). Additionally fix extremely rare race conditions in `ReplicatedMergeTree` `ALTER` queries. [#9048](https://github.com/ClickHouse/ClickHouse/pull/9048) ([alesapin](https://github.com/alesapin)) +* When the setting `compile_expressions` is enabled, you can get `unexpected column` in `LLVMExecutableFunction` when we use `Nullable` type [#8910](https://github.com/ClickHouse/ClickHouse/pull/8910) ([Guillaume Tassery](https://github.com/YiuRULE)) +* Multiple fixes for `Kafka` engine: 1) fix duplicates that were appearing during consumer group rebalance. 2) Fix rare 'holes' appeared when data were polled from several partitions with one poll and committed partially (now we always process / commit the whole polled block of messages). 3) Fix flushes by block size (before that only flushing by timeout was working properly). 4) better subscription procedure (with assignment feedback). 5) Make tests work faster (with default intervals and timeouts). Due to the fact that data was not flushed by block size before (as it should according to documentation), that PR may lead to some performance degradation with default settings (due to more often & tinier flushes which are less optimal). If you encounter the performance issue after that change - please increase `kafka_max_block_size` in the table to the bigger value ( for example `CREATE TABLE ...Engine=Kafka ... SETTINGS ... kafka_max_block_size=524288`). Fixes [#7259](https://github.com/ClickHouse/ClickHouse/issues/7259) [#8917](https://github.com/ClickHouse/ClickHouse/pull/8917) ([filimonov](https://github.com/filimonov)) +* Fix `Parameter out of bound` exception in some queries after PREWHERE optimizations. [#8914](https://github.com/ClickHouse/ClickHouse/pull/8914) ([Baudouin Giard](https://github.com/bgiard)) +* Fixed the case of mixed-constness of arguments of function `arrayZip`. [#8705](https://github.com/ClickHouse/ClickHouse/pull/8705) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492) [#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) ([tavplubix](https://github.com/tavplubix)) +* Now it's not possible to create or add columns with simple cyclic aliases like `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) +* Fixed a bug with double move which may corrupt original part. This is relevant if you use `ALTER TABLE MOVE` [#8680](https://github.com/ClickHouse/ClickHouse/pull/8680) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Allow `interval` identifier to correctly parse without backticks. Fixed issue when a query cannot be executed even if the `interval` identifier is enclosed in backticks or double quotes. This fixes [#9124](https://github.com/ClickHouse/ClickHouse/issues/9124). [#9142](https://github.com/ClickHouse/ClickHouse/pull/9142) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed fuzz test and incorrect behaviour of `bitTestAll`/`bitTestAny` functions. [#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix possible crash/wrong number of rows in `LIMIT n WITH TIES` when there are a lot of rows equal to n'th row. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) +* Fix mutations with parts written with enabled `insert_quorum`. [#9463](https://github.com/ClickHouse/ClickHouse/pull/9463) ([alesapin](https://github.com/alesapin)) +* Fix data race at destruction of `Poco::HTTPServer`. It could happen when server is started and immediately shut down. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) +* Fix bug in which a misleading error message was shown when running `SHOW CREATE TABLE a_table_that_does_not_exist`. [#8899](https://github.com/ClickHouse/ClickHouse/pull/8899) ([achulkov2](https://github.com/achulkov2)) +* Fixed `Parameters are out of bound` exception in some rare cases when we have a constant in the `SELECT` clause when we have an `ORDER BY` and a `LIMIT` clause. [#8892](https://github.com/ClickHouse/ClickHouse/pull/8892) ([Guillaume Tassery](https://github.com/YiuRULE)) +* Fix mutations finalization, when already done mutation can have status `is_done=0`. [#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) ([alesapin](https://github.com/alesapin)) +* Prevent from executing `ALTER ADD INDEX` for MergeTree tables with old syntax, because it doesn't work. [#8822](https://github.com/ClickHouse/ClickHouse/pull/8822) ([Mikhail Korotov](https://github.com/millb)) +* During server startup do not access table, which `LIVE VIEW` depends on, so server will be able to start. Also remove `LIVE VIEW` dependencies when detaching `LIVE VIEW`. `LIVE VIEW` is an experimental feature. [#8824](https://github.com/ClickHouse/ClickHouse/pull/8824) ([tavplubix](https://github.com/tavplubix)) +* Fix possible segfault in `MergeTreeRangeReader`, while executing `PREWHERE`. [#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) ([Anton Popov](https://github.com/CurtizJ)) +* Fix possible mismatched checksums with column TTLs. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) +* Fixed a bug when parts were not being moved in background by TTL rules in case when there is only one volume. [#8672](https://github.com/ClickHouse/ClickHouse/pull/8672) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fixed the issue `Method createColumn() is not implemented for data type Set`. This fixes [#7799](https://github.com/ClickHouse/ClickHouse/issues/7799). [#8674](https://github.com/ClickHouse/ClickHouse/pull/8674) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Now we will try finalize mutations more frequently. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) +* Fix `intDiv` by minus one constant [#9351](https://github.com/ClickHouse/ClickHouse/pull/9351) ([hcz](https://github.com/hczhcz)) +* Fix possible race condition in `BlockIO`. [#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. [#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) ([filimonov](https://github.com/filimonov)) +* Added workaround if OS returns wrong result for `timer_create` function. [#8837](https://github.com/ClickHouse/ClickHouse/pull/8837) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed error in usage of `min_marks_for_seek` parameter. Fixed the error message when there is no sharding key in Distributed table and we try to skip unused shards. [#8908](https://github.com/ClickHouse/ClickHouse/pull/8908) ([Azat Khuzhin](https://github.com/azat)) + +#### Improvement +* Implement `ALTER MODIFY/DROP` queries on top of mutations for `ReplicatedMergeTree*` engines family. Now `ALTERS` blocks only at the metadata update stage, and don't block after that. [#8701](https://github.com/ClickHouse/ClickHouse/pull/8701) ([alesapin](https://github.com/alesapin)) +* Add ability to rewrite CROSS to INNER JOINs with `WHERE` section containing unqialified names. [#9512](https://github.com/ClickHouse/ClickHouse/pull/9512) ([Artem Zuikov](https://github.com/4ertus2)) +* Make `SHOW TABLES` and `SHOW DATABASES` queries support the `WHERE` expressions and `FROM`/`IN` [#9076](https://github.com/ClickHouse/ClickHouse/pull/9076) ([sundyli](https://github.com/sundy-li)) +* Added a setting `deduplicate_blocks_in_dependent_materialized_views`. [#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) ([urykhy](https://github.com/urykhy)) +* After recent changes MySQL client started to print binary strings in hex thereby making them not readable ([#9032](https://github.com/ClickHouse/ClickHouse/issues/9032)). The workaround in ClickHouse is to mark string columns as UTF-8, which is not always, but usually the case. [#9079](https://github.com/ClickHouse/ClickHouse/pull/9079) ([Yuriy Baranov](https://github.com/yurriy)) +* Add support of String and FixedString keys for `sumMap` [#8903](https://github.com/ClickHouse/ClickHouse/pull/8903) ([Baudouin Giard](https://github.com/bgiard)) +* Support string keys in SummingMergeTree maps [#8933](https://github.com/ClickHouse/ClickHouse/pull/8933) ([Baudouin Giard](https://github.com/bgiard)) +* Signal termination of thread to the thread pool even if the thread has thrown exception [#8736](https://github.com/ClickHouse/ClickHouse/pull/8736) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) +* Allow to set `query_id` in `clickhouse-benchmark` [#9416](https://github.com/ClickHouse/ClickHouse/pull/9416) ([Anton Popov](https://github.com/CurtizJ)) +* Don't allow strange expressions in `ALTER TABLE ... PARTITION partition` query. This addresses [#7192](https://github.com/ClickHouse/ClickHouse/issues/7192) [#8835](https://github.com/ClickHouse/ClickHouse/pull/8835) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* The table `system.table_engines` now provides information about feature support (like `supports_ttl` or `supports_sort_order`). [#8830](https://github.com/ClickHouse/ClickHouse/pull/8830) ([Max Akhmedov](https://github.com/zlobober)) +* Enable `system.metric_log` by default. It will contain rows with values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval (one second by default). The table is very small (usually in order of megabytes) and collecting this data by default is reasonable. [#9225](https://github.com/ClickHouse/ClickHouse/pull/9225) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries. Fixes [#6964](https://github.com/ClickHouse/ClickHouse/issues/6964) [#8874](https://github.com/ClickHouse/ClickHouse/pull/8874) ([Ivan](https://github.com/abyss7)) +* Now temporary `LIVE VIEW` is created by `CREATE LIVE VIEW name WITH TIMEOUT [42] ...` instead of `CREATE TEMPORARY LIVE VIEW ...`, because the previous syntax was not consistent with `CREATE TEMPORARY TABLE ...` [#9131](https://github.com/ClickHouse/ClickHouse/pull/9131) ([tavplubix](https://github.com/tavplubix)) +* Add text_log.level configuration parameter to limit entries that goes to `system.text_log` table [#8809](https://github.com/ClickHouse/ClickHouse/pull/8809) ([Azat Khuzhin](https://github.com/azat)) +* Allow to put downloaded part to a disks/volumes according to TTL rules [#8598](https://github.com/ClickHouse/ClickHouse/pull/8598) ([Vladimir Chebotarev](https://github.com/excitoon)) +* For external MySQL dictionaries, allow to mutualize MySQL connection pool to "share" them among dictionaries. This option significantly reduces the number of connections to MySQL servers. [#9409](https://github.com/ClickHouse/ClickHouse/pull/9409) ([Clément Rodriguez](https://github.com/clemrodriguez)) +* Show nearest query execution time for quantiles in `clickhouse-benchmark` output instead of interpolated values. It's better to show values that correspond to the execution time of some queries. [#8712](https://github.com/ClickHouse/ClickHouse/pull/8712) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Possibility to add key & timestamp for the message when inserting data to Kafka. Fixes [#7198](https://github.com/ClickHouse/ClickHouse/issues/7198) [#8969](https://github.com/ClickHouse/ClickHouse/pull/8969) ([filimonov](https://github.com/filimonov)) +* If server is run from terminal, highlight thread number, query id and log priority by colors. This is for improved readability of correlated log messages for developers. [#8961](https://github.com/ClickHouse/ClickHouse/pull/8961) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Better exception message while loading tables for `Ordinary` database. [#9527](https://github.com/ClickHouse/ClickHouse/pull/9527) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Implement `arraySlice` for arrays with aggregate function states. This fixes [#9388](https://github.com/ClickHouse/ClickHouse/issues/9388) [#9391](https://github.com/ClickHouse/ClickHouse/pull/9391) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Allow constant functions and constant arrays to be used on the right side of IN operator. [#8813](https://github.com/ClickHouse/ClickHouse/pull/8813) ([Anton Popov](https://github.com/CurtizJ)) +* If zookeeper exception has happened while fetching data for system.replicas, display it in a separate column. This implements [#9137](https://github.com/ClickHouse/ClickHouse/issues/9137) [#9138](https://github.com/ClickHouse/ClickHouse/pull/9138) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Atomically remove MergeTree data parts on destroy. [#8402](https://github.com/ClickHouse/ClickHouse/pull/8402) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Support row-level security for Distributed tables. [#8926](https://github.com/ClickHouse/ClickHouse/pull/8926) ([Ivan](https://github.com/abyss7)) +* Now we recognize suffix (like KB, KiB...) in settings values. [#8072](https://github.com/ClickHouse/ClickHouse/pull/8072) ([Mikhail Korotov](https://github.com/millb)) +* Prevent out of memory while constructing result of a large JOIN. [#8637](https://github.com/ClickHouse/ClickHouse/pull/8637) ([Artem Zuikov](https://github.com/4ertus2)) +* Added names of clusters to suggestions in interactive mode in `clickhouse-client`. [#8709](https://github.com/ClickHouse/ClickHouse/pull/8709) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Initialize query profiler for all threads in a group, e.g. it allows to fully profile insert-queries [#8820](https://github.com/ClickHouse/ClickHouse/pull/8820) ([Ivan](https://github.com/abyss7)) +* Added column `exception_code` in `system.query_log` table. [#8770](https://github.com/ClickHouse/ClickHouse/pull/8770) ([Mikhail Korotov](https://github.com/millb)) +* Enabled MySQL compatibility server on port `9004` in the default server configuration file. Fixed password generation command in the example in configuration. [#8771](https://github.com/ClickHouse/ClickHouse/pull/8771) ([Yuriy Baranov](https://github.com/yurriy)) +* Prevent abort on shutdown if the filesystem is readonly. This fixes [#9094](https://github.com/ClickHouse/ClickHouse/issues/9094) [#9100](https://github.com/ClickHouse/ClickHouse/pull/9100) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Better exception message when length is required in HTTP POST query. [#9453](https://github.com/ClickHouse/ClickHouse/pull/9453) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add `_path` and `_file` virtual columns to `HDFS` and `File` engines and `hdfs` and `file` table functions [#8489](https://github.com/ClickHouse/ClickHouse/pull/8489) ([Olga Khvostikova](https://github.com/stavrolia)) +* Fix error `Cannot find column` while inserting into `MATERIALIZED VIEW` in case if new column was added to view's internal table. [#8766](https://github.com/ClickHouse/ClickHouse/pull/8766) [#8788](https://github.com/ClickHouse/ClickHouse/pull/8788) ([vzakaznikov](https://github.com/vzakaznikov)) [#8788](https://github.com/ClickHouse/ClickHouse/issues/8788) [#8806](https://github.com/ClickHouse/ClickHouse/pull/8806) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8803](https://github.com/ClickHouse/ClickHouse/pull/8803) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix progress over native client-server protocol, by send progress after final update (like logs). This may be relevant only to some third-party tools that are using native protocol. [#9495](https://github.com/ClickHouse/ClickHouse/pull/9495) ([Azat Khuzhin](https://github.com/azat)) +* Add a system metric tracking the number of client connections using MySQL protocol ([#9013](https://github.com/ClickHouse/ClickHouse/issues/9013)). [#9015](https://github.com/ClickHouse/ClickHouse/pull/9015) ([Eugene Klimov](https://github.com/Slach)) +* From now on, HTTP responses will have `X-ClickHouse-Timezone` header set to the same timezone value that `SELECT timezone()` would report. [#9493](https://github.com/ClickHouse/ClickHouse/pull/9493) ([Denis Glazachev](https://github.com/traceon)) + +#### Performance Improvement +* Improve performance of analysing index with IN [#9261](https://github.com/ClickHouse/ClickHouse/pull/9261) ([Anton Popov](https://github.com/CurtizJ)) +* Simpler and more efficient code in Logical Functions + code cleanups. A followup to [#8718](https://github.com/ClickHouse/ClickHouse/issues/8718) [#8728](https://github.com/ClickHouse/ClickHouse/pull/8728) ([Alexander Kazakov](https://github.com/Akazz)) +* Overall performance improvement (in range of 5%..200% for affected queries) by ensuring even more strict aliasing with C++20 features. [#9304](https://github.com/ClickHouse/ClickHouse/pull/9304) ([Amos Bird](https://github.com/amosbird)) +* More strict aliasing for inner loops of comparison functions. [#9327](https://github.com/ClickHouse/ClickHouse/pull/9327) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* More strict aliasing for inner loops of arithmetic functions. [#9325](https://github.com/ClickHouse/ClickHouse/pull/9325) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* A ~3 times faster implementation for ColumnVector::replicate(), via which ColumnConst::convertToFullColumn() is implemented. Also will be useful in tests when materializing constants. [#9293](https://github.com/ClickHouse/ClickHouse/pull/9293) ([Alexander Kazakov](https://github.com/Akazz)) +* Another minor performance improvement to `ColumnVector::replicate()` (this speeds up the `materialize` function and higher order functions) an even further improvement to [#9293](https://github.com/ClickHouse/ClickHouse/issues/9293) [#9442](https://github.com/ClickHouse/ClickHouse/pull/9442) ([Alexander Kazakov](https://github.com/Akazz)) +* Improved performance of `stochasticLinearRegression` aggregate function. This patch is contributed by Intel. [#8652](https://github.com/ClickHouse/ClickHouse/pull/8652) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Improve performance of `reinterpretAsFixedString` function. [#9342](https://github.com/ClickHouse/ClickHouse/pull/9342) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Do not send blocks to client for `Null` format in processors pipeline. [#8797](https://github.com/ClickHouse/ClickHouse/pull/8797) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#8767](https://github.com/ClickHouse/ClickHouse/pull/8767) ([Alexander Kuzmenkov](https://github.com/akuzm)) + +#### Build/Testing/Packaging Improvement +* Exception handling now works correctly on Windows Subsystem for Linux. See https://github.com/ClickHouse-Extras/libunwind/pull/3 This fixes [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) +* Replace `readline` with `replxx` for interactive line editing in `clickhouse-client` [#8416](https://github.com/ClickHouse/ClickHouse/pull/8416) ([Ivan](https://github.com/abyss7)) +* Better build time and less template instantiations in FunctionsComparison. [#9324](https://github.com/ClickHouse/ClickHouse/pull/9324) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added integration with `clang-tidy` in CI. See also [#6044](https://github.com/ClickHouse/ClickHouse/issues/6044) [#9566](https://github.com/ClickHouse/ClickHouse/pull/9566) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Now we link ClickHouse in CI using `lld` even for `gcc`. [#9049](https://github.com/ClickHouse/ClickHouse/pull/9049) ([alesapin](https://github.com/alesapin)) +* Allow to randomize thread scheduling and insert glitches when `THREAD_FUZZER_*` environment variables are set. This helps testing. [#9459](https://github.com/ClickHouse/ClickHouse/pull/9459) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Enable secure sockets in stateless tests [#9288](https://github.com/ClickHouse/ClickHouse/pull/9288) ([tavplubix](https://github.com/tavplubix)) +* Make SPLIT_SHARED_LIBRARIES=OFF more robust [#9156](https://github.com/ClickHouse/ClickHouse/pull/9156) ([Azat Khuzhin](https://github.com/azat)) +* Make "performance_introspection_and_logging" test reliable to random server stuck. This may happen in CI environment. See also [#9515](https://github.com/ClickHouse/ClickHouse/issues/9515) [#9528](https://github.com/ClickHouse/ClickHouse/pull/9528) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Validate XML in style check. [#9550](https://github.com/ClickHouse/ClickHouse/pull/9550) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed race condition in test `00738_lock_for_inner_table`. This test relied on sleep. [#9555](https://github.com/ClickHouse/ClickHouse/pull/9555) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove performance tests of type `once`. This is needed to run all performance tests in statistical comparison mode (more reliable). [#9557](https://github.com/ClickHouse/ClickHouse/pull/9557) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added performance test for arithmetic functions. [#9326](https://github.com/ClickHouse/ClickHouse/pull/9326) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added performance test for `sumMap` and `sumMapWithOverflow` aggregate functions. Follow-up for [#8933](https://github.com/ClickHouse/ClickHouse/issues/8933) [#8947](https://github.com/ClickHouse/ClickHouse/pull/8947) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Ensure style of ErrorCodes by style check. [#9370](https://github.com/ClickHouse/ClickHouse/pull/9370) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add script for tests history. [#8796](https://github.com/ClickHouse/ClickHouse/pull/8796) ([alesapin](https://github.com/alesapin)) +* Add GCC warning `-Wsuggest-override` to locate and fix all places where `override` keyword must be used. [#8760](https://github.com/ClickHouse/ClickHouse/pull/8760) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) +* Ignore weak symbol under Mac OS X because it must be defined [#9538](https://github.com/ClickHouse/ClickHouse/pull/9538) ([Deleted user](https://github.com/ghost)) +* Normalize running time of some queries in performance tests. This is done in preparation to run all the performance tests in comparison mode. [#9565](https://github.com/ClickHouse/ClickHouse/pull/9565) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix some tests to support pytest with query tests [#9062](https://github.com/ClickHouse/ClickHouse/pull/9062) ([Ivan](https://github.com/abyss7)) +* Enable SSL in build with MSan, so server will not fail at startup when running stateless tests [#9531](https://github.com/ClickHouse/ClickHouse/pull/9531) ([tavplubix](https://github.com/tavplubix)) +* Fix database substitution in test results [#9384](https://github.com/ClickHouse/ClickHouse/pull/9384) ([Ilya Yatsishin](https://github.com/qoega)) +* Build fixes for miscellaneous platforms [#9381](https://github.com/ClickHouse/ClickHouse/pull/9381) ([proller](https://github.com/proller)) [#8755](https://github.com/ClickHouse/ClickHouse/pull/8755) ([proller](https://github.com/proller)) [#8631](https://github.com/ClickHouse/ClickHouse/pull/8631) ([proller](https://github.com/proller)) +* Added disks section to stateless-with-coverage test docker image [#9213](https://github.com/ClickHouse/ClickHouse/pull/9213) ([Pavel Kovalenko](https://github.com/Jokser)) +* Get rid of in-source-tree files when building with GRPC [#9588](https://github.com/ClickHouse/ClickHouse/pull/9588) ([Amos Bird](https://github.com/amosbird)) +* Slightly faster build time by removing SessionCleaner from Context. Make the code of SessionCleaner more simple. [#9232](https://github.com/ClickHouse/ClickHouse/pull/9232) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Updated checking for hung queries in clickhouse-test script [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz)) +* Removed some useless files from repository. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Changed type of math perftests from `once` to `loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Add docker image which allows to build interactive code browser HTML report for our codebase. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/index.html) +* Suppress some test failures under MSan. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Speedup "exception while insert" test. This test often time out in debug-with-coverage build. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Updated `libcxx` and `libcxxabi` to master. In preparation to [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix flacky test `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Clean up duplicated linker flags. Make sure the linker won't look up an unexpected symbol. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)) +* Add `clickhouse-odbc` driver into test images. This allows to test interaction of ClickHouse with ClickHouse via its own ODBC driver. [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) +* Fix several bugs in unit tests. [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([alesapin](https://github.com/alesapin)) +* Enable `-Wmissing-include-dirs` GCC warning to eliminate all non-existing includes - mostly as a result of CMake scripting errors [#8704](https://github.com/ClickHouse/ClickHouse/pull/8704) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) +* Describe reasons if query profiler cannot work. This is intended for [#9049](https://github.com/ClickHouse/ClickHouse/issues/9049) [#9144](https://github.com/ClickHouse/ClickHouse/pull/9144) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update OpenSSL to upstream master. Fixed the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error` and `SSL Exception: error:2400006E:random number generator::error retrieving entropy`. The issue was present in version 20.1. [#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update Dockerfile for server [#8893](https://github.com/ClickHouse/ClickHouse/pull/8893) ([Ilya Mazaev](https://github.com/ne-ray)) +* Minor fixes in build-gcc-from-sources script [#8774](https://github.com/ClickHouse/ClickHouse/pull/8774) ([Michael Nacharov](https://github.com/mnach)) +* Replace `numbers` to `zeros` in perftests where `number` column is not used. This will lead to more clean test results. [#9600](https://github.com/ClickHouse/ClickHouse/pull/9600) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix stack overflow issue when using initializer_list in Column constructors. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)) +* Upgrade librdkafka to v1.3.0. Enable bundled `rdkafka` and `gsasl` libraries on Mac OS X. [#9000](https://github.com/ClickHouse/ClickHouse/pull/9000) ([Andrew Onyshchuk](https://github.com/oandrew)) +* build fix on GCC 9.2.0 [#9306](https://github.com/ClickHouse/ClickHouse/pull/9306) ([vxider](https://github.com/Vxider)) + + +## ClickHouse release v20.1 + +### ClickHouse release v20.1.16.120-stable 2020-60-26 + +#### Bug Fix + +* Fix rare crash caused by using `Nullable` column in prewhere condition. Continuation of [#11608](https://github.com/ClickHouse/ClickHouse/issues/11608). [#11869](https://github.com/ClickHouse/ClickHouse/pull/11869) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Don't allow arrayJoin inside higher order functions. It was leading to broken protocol synchronization. This closes [#3933](https://github.com/ClickHouse/ClickHouse/issues/3933). [#11846](https://github.com/ClickHouse/ClickHouse/pull/11846) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix unexpected behaviour of queries like `SELECT *, xyz.*` which were success while an error expected. [#11753](https://github.com/ClickHouse/ClickHouse/pull/11753) ([hexiaoting](https://github.com/hexiaoting)). +* Fixed LOGICAL_ERROR caused by wrong type deduction of complex literals in Values input format. [#11732](https://github.com/ClickHouse/ClickHouse/pull/11732) ([tavplubix](https://github.com/tavplubix)). +* Fix `ORDER BY ... WITH FILL` over const columns. [#11697](https://github.com/ClickHouse/ClickHouse/pull/11697) ([Anton Popov](https://github.com/CurtizJ)). +* Pass proper timeouts when communicating with XDBC bridge. Recently timeouts were not respected when checking bridge liveness and receiving meta info. [#11690](https://github.com/ClickHouse/ClickHouse/pull/11690) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add support for regular expressions with case-insensitive flags. This fixes [#11101](https://github.com/ClickHouse/ClickHouse/issues/11101) and fixes [#11506](https://github.com/ClickHouse/ClickHouse/issues/11506). [#11649](https://github.com/ClickHouse/ClickHouse/pull/11649) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bloom filters for String (data skipping indices). [#11638](https://github.com/ClickHouse/ClickHouse/pull/11638) ([Azat Khuzhin](https://github.com/azat)). +* Fix rare crash caused by using `Nullable` column in prewhere condition. (Probably it is connected with [#11572](https://github.com/ClickHouse/ClickHouse/issues/11572) somehow). [#11608](https://github.com/ClickHouse/ClickHouse/pull/11608) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix wrong exit code of the clickhouse-client, when exception.code() % 256 = 0. [#11601](https://github.com/ClickHouse/ClickHouse/pull/11601) ([filimonov](https://github.com/filimonov)). +* Fix trivial error in log message about "Mark cache size was lowered" at server startup. This closes [#11399](https://github.com/ClickHouse/ClickHouse/issues/11399). [#11589](https://github.com/ClickHouse/ClickHouse/pull/11589) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now clickhouse-server docker container will prefer IPv6 checking server aliveness. [#11550](https://github.com/ClickHouse/ClickHouse/pull/11550) ([Ivan Starkov](https://github.com/istarkov)). +* Fix memory leak when exception is thrown in the middle of aggregation with -State functions. This fixes [#8995](https://github.com/ClickHouse/ClickHouse/issues/8995). [#11496](https://github.com/ClickHouse/ClickHouse/pull/11496) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix usage of primary key wrapped into a function with 'FINAL' modifier and 'ORDER BY' optimization. [#10715](https://github.com/ClickHouse/ClickHouse/pull/10715) ([Anton Popov](https://github.com/CurtizJ)). + + +### ClickHouse release v20.1.15.109-stable 2020-06-19 + +#### Bug Fix + +* Fix excess lock for structure during alter. [#11790](https://github.com/ClickHouse/ClickHouse/pull/11790) ([alesapin](https://github.com/alesapin)). + + +### ClickHouse release v20.1.14.107-stable 2020-06-11 + +#### Bug Fix + +* Fix error `Size of offsets doesn't match size of column` for queries with `PREWHERE column in (subquery)` and `ARRAY JOIN`. [#11580](https://github.com/ClickHouse/ClickHouse/pull/11580) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + + +### ClickHouse release v20.1.13.105-stable 2020-06-10 + +#### Bug Fix + +* Fix the error `Data compressed with different methods` that can happen if `min_bytes_to_use_direct_io` is enabled and PREWHERE is active and using SAMPLE or high number of threads. This fixes [#11539](https://github.com/ClickHouse/ClickHouse/issues/11539). [#11540](https://github.com/ClickHouse/ClickHouse/pull/11540) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix return compressed size for codecs. [#11448](https://github.com/ClickHouse/ClickHouse/pull/11448) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix server crash when a column has compression codec with non-literal arguments. Fixes [#11365](https://github.com/ClickHouse/ClickHouse/issues/11365). [#11431](https://github.com/ClickHouse/ClickHouse/pull/11431) ([alesapin](https://github.com/alesapin)). +* Fix pointInPolygon with nan as point. Fixes [#11375](https://github.com/ClickHouse/ClickHouse/issues/11375). [#11421](https://github.com/ClickHouse/ClickHouse/pull/11421) ([Alexey Ilyukhov](https://github.com/livace)). +* Fixed geohashesInBox with arguments outside of latitude/longitude range. [#11403](https://github.com/ClickHouse/ClickHouse/pull/11403) ([Vasily Nemkov](https://github.com/Enmk)). +* Fix possible `Pipeline stuck` error for queries with external sort and limit. Fixes [#11359](https://github.com/ClickHouse/ClickHouse/issues/11359). [#11366](https://github.com/ClickHouse/ClickHouse/pull/11366) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix crash in `quantilesExactWeightedArray`. [#11337](https://github.com/ClickHouse/ClickHouse/pull/11337) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Make writing to `MATERIALIZED VIEW` with setting `parallel_view_processing = 1` parallel again. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#11330](https://github.com/ClickHouse/ClickHouse/pull/11330) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix visitParamExtractRaw when extracted JSON has strings with unbalanced { or [. [#11318](https://github.com/ClickHouse/ClickHouse/pull/11318) ([Ewout](https://github.com/devwout)). +* Fix very rare race condition in ThreadPool. [#11314](https://github.com/ClickHouse/ClickHouse/pull/11314) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix potential uninitialized memory in conversion. Example: `SELECT toIntervalSecond(now64())`. [#11311](https://github.com/ClickHouse/ClickHouse/pull/11311) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the issue when index analysis cannot work if a table has Array column in primary key and if a query is filtering by this column with `empty` or `notEmpty` functions. This fixes [#11286](https://github.com/ClickHouse/ClickHouse/issues/11286). [#11303](https://github.com/ClickHouse/ClickHouse/pull/11303) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug when query speed estimation can be incorrect and the limit of `min_execution_speed` may not work or work incorrectly if the query is throttled by `max_network_bandwidth`, `max_execution_speed` or `priority` settings. Change the default value of `timeout_before_checking_execution_speed` to non-zero, because otherwise the settings `min_execution_speed` and `max_execution_speed` have no effect. This fixes [#11297](https://github.com/ClickHouse/ClickHouse/issues/11297). This fixes [#5732](https://github.com/ClickHouse/ClickHouse/issues/5732). This fixes [#6228](https://github.com/ClickHouse/ClickHouse/issues/6228). Usability improvement: avoid concatenation of exception message with progress bar in `clickhouse-client`. [#11296](https://github.com/ClickHouse/ClickHouse/pull/11296) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix crash while reading malformed data in Protobuf format. This fixes [#5957](https://github.com/ClickHouse/ClickHouse/issues/5957), fixes [#11203](https://github.com/ClickHouse/ClickHouse/issues/11203). [#11258](https://github.com/ClickHouse/ClickHouse/pull/11258) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix possible error `Cannot capture column` for higher-order functions with `Array(Array(LowCardinality))` captured argument. [#11185](https://github.com/ClickHouse/ClickHouse/pull/11185) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* If data skipping index is dependent on columns that are going to be modified during background merge (for SummingMergeTree, AggregatingMergeTree as well as for TTL GROUP BY), it was calculated incorrectly. This issue is fixed by moving index calculation after merge so the index is calculated on merged data. [#11162](https://github.com/ClickHouse/ClickHouse/pull/11162) ([Azat Khuzhin](https://github.com/azat)). +* Remove logging from mutation finalization task if nothing was finalized. [#11109](https://github.com/ClickHouse/ClickHouse/pull/11109) ([alesapin](https://github.com/alesapin)). +* 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)). +* Fix incorrect raw data size in method getRawData(). [#10964](https://github.com/ClickHouse/ClickHouse/pull/10964) ([Igr](https://github.com/ObjatieGroba)). +* Fix backward compatibility with tuples in Distributed tables. [#10889](https://github.com/ClickHouse/ClickHouse/pull/10889) ([Anton Popov](https://github.com/CurtizJ)). +* Fix SIGSEGV in StringHashTable (if such key does not exist). [#10870](https://github.com/ClickHouse/ClickHouse/pull/10870) ([Azat Khuzhin](https://github.com/azat)). +* Fixed bug in `ReplicatedMergeTree` which might cause some `ALTER` on `OPTIMIZE` query to hang waiting for some replica after it become inactive. [#10849](https://github.com/ClickHouse/ClickHouse/pull/10849) ([tavplubix](https://github.com/tavplubix)). +* Fix columns order after Block::sortColumns() (also add a test that shows that it affects some real use case - Buffer engine). [#10826](https://github.com/ClickHouse/ClickHouse/pull/10826) ([Azat Khuzhin](https://github.com/azat)). +* Fix the issue with ODBC bridge when no quoting of identifiers is requested. This fixes [#7984](https://github.com/ClickHouse/ClickHouse/issues/7984). [#10821](https://github.com/ClickHouse/ClickHouse/pull/10821) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix UBSan and MSan report in DateLUT. [#10798](https://github.com/ClickHouse/ClickHouse/pull/10798) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* - Make use of `src_type` for correct type conversion in key conditions. Fixes [#6287](https://github.com/ClickHouse/ClickHouse/issues/6287). [#10791](https://github.com/ClickHouse/ClickHouse/pull/10791) ([Andrew Onyshchuk](https://github.com/oandrew)). +* Fix `parallel_view_processing` behavior. Now all insertions into `MATERIALIZED VIEW` without exception should be finished if exception happened. Fixes [#10241](https://github.com/ClickHouse/ClickHouse/issues/10241). [#10757](https://github.com/ClickHouse/ClickHouse/pull/10757) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix combinator -OrNull and -OrDefault when combined with -State. [#10741](https://github.com/ClickHouse/ClickHouse/pull/10741) ([hcz](https://github.com/hczhcz)). +* Fix disappearing totals. Totals could have being filtered if query had had join or subquery with external where condition. Fixes [#10674](https://github.com/ClickHouse/ClickHouse/issues/10674). [#10698](https://github.com/ClickHouse/ClickHouse/pull/10698) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix multiple usages of `IN` operator with the identical set in one query. [#10686](https://github.com/ClickHouse/ClickHouse/pull/10686) ([Anton Popov](https://github.com/CurtizJ)). +* Fix order of parameters in AggregateTransform constructor. [#10667](https://github.com/ClickHouse/ClickHouse/pull/10667) ([palasonic1](https://github.com/palasonic1)). +* Fix the lack of parallel execution of remote queries with `distributed_aggregation_memory_efficient` enabled. Fixes [#10655](https://github.com/ClickHouse/ClickHouse/issues/10655). [#10664](https://github.com/ClickHouse/ClickHouse/pull/10664) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix predicates optimization for distributed queries (`enable_optimize_predicate_expression=1`) for queries with `HAVING` section (i.e. when filtering on the server initiator is required), by preserving the order of expressions (and this is enough to fix), and also force aggregator use column names over indexes. Fixes: [#10613](https://github.com/ClickHouse/ClickHouse/issues/10613), [#11413](https://github.com/ClickHouse/ClickHouse/issues/11413). [#10621](https://github.com/ClickHouse/ClickHouse/pull/10621) ([Azat Khuzhin](https://github.com/azat)). +* Fix error `the BloomFilter false positive must be a double number between 0 and 1` [#10551](https://github.com/ClickHouse/ClickHouse/issues/10551). [#10569](https://github.com/ClickHouse/ClickHouse/pull/10569) ([Winter Zhang](https://github.com/zhang2014)). +* Fix SELECT of column ALIAS which default expression type different from column type. [#10563](https://github.com/ClickHouse/ClickHouse/pull/10563) ([Azat Khuzhin](https://github.com/azat)). +* * Implemented comparison between DateTime64 and String values (just like for DateTime). [#10560](https://github.com/ClickHouse/ClickHouse/pull/10560) ([Vasily Nemkov](https://github.com/Enmk)). + + +### ClickHouse release v20.1.12.86, 2020-05-26 + +#### Bug Fix + +* Fixed incompatibility of two-level aggregation between versions 20.1 and earlier. This incompatibility happens when different versions of ClickHouse are used on initiator node and remote nodes and the size of GROUP BY result is large and aggregation is performed by a single String field. It leads to several unmerged rows for a single key in result. [#10952](https://github.com/ClickHouse/ClickHouse/pull/10952) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed data corruption for `LowCardinality(FixedString)` key column in `SummingMergeTree` which could have happened after merge. Fixes [#10489](https://github.com/ClickHouse/ClickHouse/issues/10489). [#10721](https://github.com/ClickHouse/ClickHouse/pull/10721) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug, which causes http requests stuck on client close when `readonly=2` and `cancel_http_readonly_queries_on_client_close=1`. Fixes [#7939](https://github.com/ClickHouse/ClickHouse/issues/7939), [#7019](https://github.com/ClickHouse/ClickHouse/issues/7019), [#7736](https://github.com/ClickHouse/ClickHouse/issues/7736), [#7091](https://github.com/ClickHouse/ClickHouse/issues/7091). [#10684](https://github.com/ClickHouse/ClickHouse/pull/10684) ([tavplubix](https://github.com/tavplubix)). +* Fixed a bug when on `SYSTEM DROP DNS CACHE` query also drop caches, which are used to check if user is allowed to connect from some IP addresses. [#10608](https://github.com/ClickHouse/ClickHouse/pull/10608) ([tavplubix](https://github.com/tavplubix)). +* Fixed incorrect scalar results inside inner query of `MATERIALIZED VIEW` in case if this query contained dependent table. [#10603](https://github.com/ClickHouse/ClickHouse/pull/10603) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed the situation when mutation finished all parts, but hung up in `is_done=0`. [#10526](https://github.com/ClickHouse/ClickHouse/pull/10526) ([alesapin](https://github.com/alesapin)). +* Fixed overflow at beginning of unix epoch for timezones with fractional offset from UTC. This fixes [#9335](https://github.com/ClickHouse/ClickHouse/issues/9335). [#10513](https://github.com/ClickHouse/ClickHouse/pull/10513) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed improper shutdown of Distributed storage. [#10491](https://github.com/ClickHouse/ClickHouse/pull/10491) ([Azat Khuzhin](https://github.com/azat)). +* Fixed numeric overflow in `simpleLinearRegression` over large integers. [#10474](https://github.com/ClickHouse/ClickHouse/pull/10474) ([hcz](https://github.com/hczhcz)). +* Fixed removing metadata directory when attach database fails. [#10442](https://github.com/ClickHouse/ClickHouse/pull/10442) ([Winter Zhang](https://github.com/zhang2014)). +* Added a check of number and type of arguments when creating `BloomFilter` index [#9623](https://github.com/ClickHouse/ClickHouse/issues/9623). [#10431](https://github.com/ClickHouse/ClickHouse/pull/10431) ([Winter Zhang](https://github.com/zhang2014)). +* Fixed the issue when a query with `ARRAY JOIN`, `ORDER BY` and `LIMIT` may return incomplete result. This fixes [#10226](https://github.com/ClickHouse/ClickHouse/issues/10226). [#10427](https://github.com/ClickHouse/ClickHouse/pull/10427) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Prefer `fallback_to_stale_replicas` over `skip_unavailable_shards`. [#10422](https://github.com/ClickHouse/ClickHouse/pull/10422) ([Azat Khuzhin](https://github.com/azat)). +* Fixed wrong flattening of `Array(Tuple(...))` data types. This fixes [#10259](https://github.com/ClickHouse/ClickHouse/issues/10259). [#10390](https://github.com/ClickHouse/ClickHouse/pull/10390) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed wrong behavior in `HashTable` that caused compilation error when trying to read HashMap from buffer. [#10386](https://github.com/ClickHouse/ClickHouse/pull/10386) ([palasonic1](https://github.com/palasonic1)). +* Fixed possible `Pipeline stuck` error in `ConcatProcessor` which could have happened in remote query. [#10381](https://github.com/ClickHouse/ClickHouse/pull/10381) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed error `Pipeline stuck` with `max_rows_to_group_by` and `group_by_overflow_mode = 'break'`. [#10279](https://github.com/ClickHouse/ClickHouse/pull/10279) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed several bugs when some data was inserted with quorum, then deleted somehow (DROP PARTITION, TTL) and this leaded to the stuck of INSERTs or false-positive exceptions in SELECTs. This fixes [#9946](https://github.com/ClickHouse/ClickHouse/issues/9946). [#10188](https://github.com/ClickHouse/ClickHouse/pull/10188) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed incompatibility when versions prior to 18.12.17 are used on remote servers and newer is used on initiating server, and GROUP BY both fixed and non-fixed keys, and when two-level group by method is activated. [#3254](https://github.com/ClickHouse/ClickHouse/pull/3254) ([alexey-milovidov](https://github.com/alexey-milovidov)). + +#### Build/Testing/Packaging Improvement + +* Added CA certificates to clickhouse-server docker image. [#10476](https://github.com/ClickHouse/ClickHouse/pull/10476) ([filimonov](https://github.com/filimonov)). + + +### ClickHouse release v20.1.10.70, 2020-04-17 + +#### Bug Fix + +* Fix rare possible exception `Cannot drain connections: cancel first`. [#10239](https://github.com/ClickHouse/ClickHouse/pull/10239) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed bug where ClickHouse would throw `'Unknown function lambda.'` error message when user tries to run `ALTER UPDATE/DELETE` on tables with `ENGINE = Replicated*`. Check for nondeterministic functions now handles lambda expressions correctly. [#10237](https://github.com/ClickHouse/ClickHouse/pull/10237) ([Alexander Kazakov](https://github.com/Akazz)). +* Fix `parseDateTimeBestEffort` for strings in RFC-2822 when day of week is Tuesday or Thursday. This fixes [#10082](https://github.com/ClickHouse/ClickHouse/issues/10082). [#10214](https://github.com/ClickHouse/ClickHouse/pull/10214) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix column names of constants inside `JOIN` that may clash with names of constants outside of `JOIN`. [#10207](https://github.com/ClickHouse/ClickHouse/pull/10207) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix possible inifinite query execution when the query actually should stop on LIMIT, while reading from infinite source like `system.numbers` or `system.zeros`. [#10206](https://github.com/ClickHouse/ClickHouse/pull/10206) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix move-to-prewhere optimization in presense of `arrayJoin` functions (in certain cases). This fixes [#10092](https://github.com/ClickHouse/ClickHouse/issues/10092). [#10195](https://github.com/ClickHouse/ClickHouse/pull/10195) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add the ability to relax the restriction on non-deterministic functions usage in mutations with `allow_nondeterministic_mutations` setting. [#10186](https://github.com/ClickHouse/ClickHouse/pull/10186) ([filimonov](https://github.com/filimonov)). +* Convert blocks if structure does not match on `INSERT` into table with `Distributed` engine. [#10135](https://github.com/ClickHouse/ClickHouse/pull/10135) ([Azat Khuzhin](https://github.com/azat)). +* Fix `SIGSEGV` on `INSERT` into `Distributed` table when its structure differs from the underlying tables. [#10105](https://github.com/ClickHouse/ClickHouse/pull/10105) ([Azat Khuzhin](https://github.com/azat)). +* Fix possible rows loss for queries with `JOIN` and `UNION ALL`. Fixes [#9826](https://github.com/ClickHouse/ClickHouse/issues/9826), [#10113](https://github.com/ClickHouse/ClickHouse/issues/10113). [#10099](https://github.com/ClickHouse/ClickHouse/pull/10099) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Add arguments check and support identifier arguments for MySQL Database Engine. [#10077](https://github.com/ClickHouse/ClickHouse/pull/10077) ([Winter Zhang](https://github.com/zhang2014)). +* Fix bug in clickhouse dictionary source from localhost clickhouse server. The bug may lead to memory corruption if types in dictionary and source are not compatible. [#10071](https://github.com/ClickHouse/ClickHouse/pull/10071) ([alesapin](https://github.com/alesapin)). +* Fix error `Cannot clone block with columns because block has 0 columns ... While executing GroupingAggregatedTransform`. It happened when setting `distributed_aggregation_memory_efficient` was enabled, and distributed query read aggregating data with different level from different shards (mixed single and two level aggregation). [#10063](https://github.com/ClickHouse/ClickHouse/pull/10063) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix a segmentation fault that could occur in `GROUP BY` over string keys containing trailing zero bytes ([#8636](https://github.com/ClickHouse/ClickHouse/issues/8636), [#8925](https://github.com/ClickHouse/ClickHouse/issues/8925)). [#10025](https://github.com/ClickHouse/ClickHouse/pull/10025) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Fix bug in which the necessary tables weren't retrieved at one of the processing stages of queries to some databases. Fixes [#9699](https://github.com/ClickHouse/ClickHouse/issues/9699). [#9949](https://github.com/ClickHouse/ClickHouse/pull/9949) ([achulkov2](https://github.com/achulkov2)). +* Fix `'Not found column in block'` error when `JOIN` appears with `TOTALS`. Fixes [#9839](https://github.com/ClickHouse/ClickHouse/issues/9839). [#9939](https://github.com/ClickHouse/ClickHouse/pull/9939) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix a bug with `ON CLUSTER` DDL queries freezing on server startup. [#9927](https://github.com/ClickHouse/ClickHouse/pull/9927) ([Gagan Arneja](https://github.com/garneja)). +* Fix `TRUNCATE` for Join table engine ([#9917](https://github.com/ClickHouse/ClickHouse/issues/9917)). [#9920](https://github.com/ClickHouse/ClickHouse/pull/9920) ([Amos Bird](https://github.com/amosbird)). +* Fix `'scalar doesn't exist'` error in ALTER queries ([#9878](https://github.com/ClickHouse/ClickHouse/issues/9878)). [#9904](https://github.com/ClickHouse/ClickHouse/pull/9904) ([Amos Bird](https://github.com/amosbird)). +* Fix race condition between drop and optimize in `ReplicatedMergeTree`. [#9901](https://github.com/ClickHouse/ClickHouse/pull/9901) ([alesapin](https://github.com/alesapin)). +* Fixed `DeleteOnDestroy` logic in `ATTACH PART` which could lead to automatic removal of attached part and added few tests. [#9410](https://github.com/ClickHouse/ClickHouse/pull/9410) ([Vladimir Chebotarev](https://github.com/excitoon)). + +#### Build/Testing/Packaging Improvement + +* Fix unit test `collapsing_sorted_stream`. [#9367](https://github.com/ClickHouse/ClickHouse/pull/9367) ([Deleted user](https://github.com/ghost)). + +### ClickHouse release v20.1.9.54, 2020-03-28 + +#### Bug Fix + +* Fix `'Different expressions with the same alias'` error when query has `PREWHERE` and `WHERE` on distributed table and `SET distributed_product_mode = 'local'`. [#9871](https://github.com/ClickHouse/ClickHouse/pull/9871) ([Artem Zuikov](https://github.com/4ertus2)). +* Fix mutations excessive memory consumption for tables with a composite primary key. This fixes [#9850](https://github.com/ClickHouse/ClickHouse/issues/9850). [#9860](https://github.com/ClickHouse/ClickHouse/pull/9860) ([alesapin](https://github.com/alesapin)). +* For INSERT queries shard now clamps the settings got from the initiator to the shard's constaints instead of throwing an exception. This fix allows to send `INSERT` queries to a shard with another constraints. This change improves fix [#9447](https://github.com/ClickHouse/ClickHouse/issues/9447). [#9852](https://github.com/ClickHouse/ClickHouse/pull/9852) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix possible exception `Got 0 in totals chunk, expected 1` on client. It happened for queries with `JOIN` in case if right joined table had zero rows. Example: `select * from system.one t1 join system.one t2 on t1.dummy = t2.dummy limit 0 FORMAT TabSeparated;`. Fixes [#9777](https://github.com/ClickHouse/ClickHouse/issues/9777). [#9823](https://github.com/ClickHouse/ClickHouse/pull/9823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix `SIGSEGV` with `optimize_skip_unused_shards` when type cannot be converted. [#9804](https://github.com/ClickHouse/ClickHouse/pull/9804) ([Azat Khuzhin](https://github.com/azat)). +* Fixed a few cases when timezone of the function argument wasn't used properly. [#9574](https://github.com/ClickHouse/ClickHouse/pull/9574) ([Vasily Nemkov](https://github.com/Enmk)). + +#### Improvement + +* Remove `ORDER BY` stage from mutations because we read from a single ordered part in a single thread. Also add check that the order of rows in mutation is ordered in sorting key order and this order is not violated. [#9886](https://github.com/ClickHouse/ClickHouse/pull/9886) ([alesapin](https://github.com/alesapin)). + +#### Build/Testing/Packaging Improvement + +* Clean up duplicated linker flags. Make sure the linker won't look up an unexpected symbol. [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([Amos Bird](https://github.com/amosbird)). + +### ClickHouse release v20.1.8.41, 2020-03-20 + +#### Bug Fix +* Fix possible permanent `Cannot schedule a task` error (due to unhandled exception in `ParallelAggregatingBlockInputStream::Handler::onFinish/onFinishThread`). This fixes [#6833](https://github.com/ClickHouse/ClickHouse/issues/6833). [#9154](https://github.com/ClickHouse/ClickHouse/pull/9154) ([Azat Khuzhin](https://github.com/azat)) +* Fix excessive memory consumption in `ALTER` queries (mutations). This fixes [#9533](https://github.com/ClickHouse/ClickHouse/issues/9533) and [#9670](https://github.com/ClickHouse/ClickHouse/issues/9670). [#9754](https://github.com/ClickHouse/ClickHouse/pull/9754) ([alesapin](https://github.com/alesapin)) +* Fix bug in backquoting in external dictionaries DDL. This fixes [#9619](https://github.com/ClickHouse/ClickHouse/issues/9619). [#9734](https://github.com/ClickHouse/ClickHouse/pull/9734) ([alesapin](https://github.com/alesapin)) + +### ClickHouse release v20.1.7.38, 2020-03-18 + +#### Bug Fix +* Fixed incorrect internal function names for `sumKahan` and `sumWithOverflow`. I lead to exception while using this functions in remote queries. [#9636](https://github.com/ClickHouse/ClickHouse/pull/9636) ([Azat Khuzhin](https://github.com/azat)). This issue was in all ClickHouse releases. +* Allow `ALTER ON CLUSTER` of `Distributed` tables with internal replication. This fixes [#3268](https://github.com/ClickHouse/ClickHouse/issues/3268). [#9617](https://github.com/ClickHouse/ClickHouse/pull/9617) ([shinoi2](https://github.com/shinoi2)). This issue was in all ClickHouse releases. +* Fix possible exceptions `Size of filter doesn't match size of column` and `Invalid number of rows in Chunk` in `MergeTreeRangeReader`. They could appear while executing `PREWHERE` in some cases. Fixes [#9132](https://github.com/ClickHouse/ClickHouse/issues/9132). [#9612](https://github.com/ClickHouse/ClickHouse/pull/9612) ([Anton Popov](https://github.com/CurtizJ)) +* Fixed the issue: timezone was not preserved if you write a simple arithmetic expression like `time + 1` (in contrast to an expression like `time + INTERVAL 1 SECOND`). This fixes [#5743](https://github.com/ClickHouse/ClickHouse/issues/5743). [#9323](https://github.com/ClickHouse/ClickHouse/pull/9323) ([alexey-milovidov](https://github.com/alexey-milovidov)). This issue was in all ClickHouse releases. +* Now it's not possible to create or add columns with simple cyclic aliases like `a DEFAULT b, b DEFAULT a`. [#9603](https://github.com/ClickHouse/ClickHouse/pull/9603) ([alesapin](https://github.com/alesapin)) +* Fixed the issue when padding at the end of base64 encoded value can be malformed. Update base64 library. This fixes [#9491](https://github.com/ClickHouse/ClickHouse/issues/9491), closes [#9492](https://github.com/ClickHouse/ClickHouse/issues/9492) [#9500](https://github.com/ClickHouse/ClickHouse/pull/9500) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix data race at destruction of `Poco::HTTPServer`. It could happen when server is started and immediately shut down. [#9468](https://github.com/ClickHouse/ClickHouse/pull/9468) ([Anton Popov](https://github.com/CurtizJ)) +* Fix possible crash/wrong number of rows in `LIMIT n WITH TIES` when there are a lot of rows equal to n'th row. [#9464](https://github.com/ClickHouse/ClickHouse/pull/9464) ([tavplubix](https://github.com/tavplubix)) +* Fix possible mismatched checksums with column TTLs. [#9451](https://github.com/ClickHouse/ClickHouse/pull/9451) ([Anton Popov](https://github.com/CurtizJ)) +* Fix crash when a user tries to `ALTER MODIFY SETTING` for old-formated `MergeTree` table engines family. [#9435](https://github.com/ClickHouse/ClickHouse/pull/9435) ([alesapin](https://github.com/alesapin)) +* Now we will try finalize mutations more frequently. [#9427](https://github.com/ClickHouse/ClickHouse/pull/9427) ([alesapin](https://github.com/alesapin)) +* Fix replication protocol incompatibility introduced in [#8598](https://github.com/ClickHouse/ClickHouse/issues/8598). [#9412](https://github.com/ClickHouse/ClickHouse/pull/9412) ([alesapin](https://github.com/alesapin)) +* Fix not(has()) for the bloom_filter index of array types. [#9407](https://github.com/ClickHouse/ClickHouse/pull/9407) ([achimbab](https://github.com/achimbab)) +* Fixed the behaviour of `match` and `extract` functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. This fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) [#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#9345](https://github.com/ClickHouse/ClickHouse/pull/9345) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Build/Testing/Packaging Improvement + +* Exception handling now works correctly on Windows Subsystem for Linux. See https://github.com/ClickHouse-Extras/libunwind/pull/3 This fixes [#6480](https://github.com/ClickHouse/ClickHouse/issues/6480) [#9564](https://github.com/ClickHouse/ClickHouse/pull/9564) ([sobolevsv](https://github.com/sobolevsv)) + + +### ClickHouse release v20.1.6.30, 2020-03-05 + +#### Bug Fix + +* Fix data incompatibility when compressed with `T64` codec. +[#9039](https://github.com/ClickHouse/ClickHouse/pull/9039) [(abyss7)](https://github.com/abyss7) +* Fix order of ranges while reading from MergeTree table in one thread. Fixes [#8964](https://github.com/ClickHouse/ClickHouse/issues/8964). +[#9050](https://github.com/ClickHouse/ClickHouse/pull/9050) [(CurtizJ)](https://github.com/CurtizJ) +* Fix possible segfault in `MergeTreeRangeReader`, while executing `PREWHERE`. Fixes [#9064](https://github.com/ClickHouse/ClickHouse/issues/9064). +[#9106](https://github.com/ClickHouse/ClickHouse/pull/9106) [(CurtizJ)](https://github.com/CurtizJ) +* Fix `reinterpretAsFixedString` to return `FixedString` instead of `String`. +[#9052](https://github.com/ClickHouse/ClickHouse/pull/9052) [(oandrew)](https://github.com/oandrew) +* Fix `joinGet` with nullable return types. Fixes [#8919](https://github.com/ClickHouse/ClickHouse/issues/8919) +[#9014](https://github.com/ClickHouse/ClickHouse/pull/9014) [(amosbird)](https://github.com/amosbird) +* Fix fuzz test and incorrect behaviour of bitTestAll/bitTestAny functions. +[#9143](https://github.com/ClickHouse/ClickHouse/pull/9143) [(alexey-milovidov)](https://github.com/alexey-milovidov) +* Fix the behaviour of match and extract functions when haystack has zero bytes. The behaviour was wrong when haystack was constant. Fixes [#9160](https://github.com/ClickHouse/ClickHouse/issues/9160) +[#9163](https://github.com/ClickHouse/ClickHouse/pull/9163) [(alexey-milovidov)](https://github.com/alexey-milovidov) +* Fixed execution of inversed predicates when non-strictly monotinic functional index is used. Fixes [#9034](https://github.com/ClickHouse/ClickHouse/issues/9034) +[#9223](https://github.com/ClickHouse/ClickHouse/pull/9223) [(Akazz)](https://github.com/Akazz) +* Allow to rewrite `CROSS` to `INNER JOIN` if there's `[NOT] LIKE` operator in `WHERE` section. Fixes [#9191](https://github.com/ClickHouse/ClickHouse/issues/9191) +[#9229](https://github.com/ClickHouse/ClickHouse/pull/9229) [(4ertus2)](https://github.com/4ertus2) +* Allow first column(s) in a table with Log engine be an alias. +[#9231](https://github.com/ClickHouse/ClickHouse/pull/9231) [(abyss7)](https://github.com/abyss7) +* Allow comma join with `IN()` inside. Fixes [#7314](https://github.com/ClickHouse/ClickHouse/issues/7314). +[#9251](https://github.com/ClickHouse/ClickHouse/pull/9251) [(4ertus2)](https://github.com/4ertus2) +* Improve `ALTER MODIFY/ADD` queries logic. Now you cannot `ADD` column without type, `MODIFY` default expression doesn't change type of column and `MODIFY` type doesn't loose default expression value. Fixes [#8669](https://github.com/ClickHouse/ClickHouse/issues/8669). +[#9227](https://github.com/ClickHouse/ClickHouse/pull/9227) [(alesapin)](https://github.com/alesapin) +* Fix mutations finalization, when already done mutation can have status is_done=0. +[#9217](https://github.com/ClickHouse/ClickHouse/pull/9217) [(alesapin)](https://github.com/alesapin) +* Support "Processors" pipeline for system.numbers and system.numbers_mt. This also fixes the bug when `max_execution_time` is not respected. +[#7796](https://github.com/ClickHouse/ClickHouse/pull/7796) [(KochetovNicolai)](https://github.com/KochetovNicolai) +* Fix wrong counting of `DictCacheKeysRequestedFound` metric. +[#9411](https://github.com/ClickHouse/ClickHouse/pull/9411) [(nikitamikhaylov)](https://github.com/nikitamikhaylov) +* Added a check for storage policy in `ATTACH PARTITION FROM`, `REPLACE PARTITION`, `MOVE TO TABLE` which otherwise could make data of part inaccessible after restart and prevent ClickHouse to start. +[#9383](https://github.com/ClickHouse/ClickHouse/pull/9383) [(excitoon)](https://github.com/excitoon) +* Fixed UBSan report in `MergeTreeIndexSet`. This fixes [#9250](https://github.com/ClickHouse/ClickHouse/issues/9250) +[#9365](https://github.com/ClickHouse/ClickHouse/pull/9365) [(alexey-milovidov)](https://github.com/alexey-milovidov) +* Fix possible datarace in BlockIO. +[#9356](https://github.com/ClickHouse/ClickHouse/pull/9356) [(KochetovNicolai)](https://github.com/KochetovNicolai) +* Support for `UInt64` numbers that don't fit in Int64 in JSON-related functions. Update `SIMDJSON` to master. This fixes [#9209](https://github.com/ClickHouse/ClickHouse/issues/9209) +[#9344](https://github.com/ClickHouse/ClickHouse/pull/9344) [(alexey-milovidov)](https://github.com/alexey-milovidov) +* Fix the issue when the amount of free space is not calculated correctly if the data directory is mounted to a separate device. For default disk calculate the free space from data subdirectory. This fixes [#7441](https://github.com/ClickHouse/ClickHouse/issues/7441) +[#9257](https://github.com/ClickHouse/ClickHouse/pull/9257) [(millb)](https://github.com/millb) +* Fix the issue when TLS connections may fail with the message `OpenSSL SSL_read: error:14094438:SSL routines:ssl3_read_bytes:tlsv1 alert internal error and SSL Exception: error:2400006E:random number generator::error retrieving entropy.` Update OpenSSL to upstream master. +[#8956](https://github.com/ClickHouse/ClickHouse/pull/8956) [(alexey-milovidov)](https://github.com/alexey-milovidov) +* When executing `CREATE` query, fold constant expressions in storage engine arguments. Replace empty database name with current database. Fixes [#6508](https://github.com/ClickHouse/ClickHouse/issues/6508), [#3492](https://github.com/ClickHouse/ClickHouse/issues/3492). Also fix check for local address in ClickHouseDictionarySource. +[#9262](https://github.com/ClickHouse/ClickHouse/pull/9262) [(tabplubix)](https://github.com/tavplubix) +* Fix segfault in `StorageMerge`, which can happen when reading from StorageFile. +[#9387](https://github.com/ClickHouse/ClickHouse/pull/9387) [(tabplubix)](https://github.com/tavplubix) +* Prevent losing data in `Kafka` in rare cases when exception happens after reading suffix but before commit. Fixes [#9378](https://github.com/ClickHouse/ClickHouse/issues/9378). Related: [#7175](https://github.com/ClickHouse/ClickHouse/issues/7175) +[#9507](https://github.com/ClickHouse/ClickHouse/pull/9507) [(filimonov)](https://github.com/filimonov) +* Fix bug leading to server termination when trying to use / drop `Kafka` table created with wrong parameters. Fixes [#9494](https://github.com/ClickHouse/ClickHouse/issues/9494). Incorporates [#9507](https://github.com/ClickHouse/ClickHouse/issues/9507). +[#9513](https://github.com/ClickHouse/ClickHouse/pull/9513) [(filimonov)](https://github.com/filimonov) + +#### New Feature +* Add `deduplicate_blocks_in_dependent_materialized_views` option to control the behaviour of idempotent inserts into tables with materialized views. This new feature was added to the bugfix release by a special request from Altinity. +[#9070](https://github.com/ClickHouse/ClickHouse/pull/9070) [(urykhy)](https://github.com/urykhy) + +### ClickHouse release v20.1.2.4, 2020-01-22 + +#### Backward Incompatible Change +* Make the setting `merge_tree_uniform_read_distribution` obsolete. The server still recognizes this setting but it has no effect. [#8308](https://github.com/ClickHouse/ClickHouse/pull/8308) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Changed return type of the function `greatCircleDistance` to `Float32` because now the result of calculation is `Float32`. [#7993](https://github.com/ClickHouse/ClickHouse/pull/7993) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Now it's expected that query parameters are represented in "escaped" format. For example, to pass string `ab` you have to write `a\tb` or `a\b` and respectively, `a%5Ctb` or `a%5C%09b` in URL. This is needed to add the possibility to pass NULL as `\N`. This fixes [#7488](https://github.com/ClickHouse/ClickHouse/issues/7488). [#8517](https://github.com/ClickHouse/ClickHouse/pull/8517) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Enable `use_minimalistic_part_header_in_zookeeper` setting for `ReplicatedMergeTree` by default. This will significantly reduce amount of data stored in ZooKeeper. This setting is supported since version 19.1 and we already use it in production in multiple services without any issues for more than half a year. Disable this setting if you have a chance to downgrade to versions older than 19.1. [#6850](https://github.com/ClickHouse/ClickHouse/pull/6850) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Data skipping indices are production ready and enabled by default. The settings `allow_experimental_data_skipping_indices`, `allow_experimental_cross_to_join_conversion` and `allow_experimental_multiple_joins_emulation` are now obsolete and do nothing. [#7974](https://github.com/ClickHouse/ClickHouse/pull/7974) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add new `ANY JOIN` logic for `StorageJoin` consistent with `JOIN` operation. To upgrade without changes in behaviour you need add `SETTINGS any_join_distinct_right_table_keys = 1` to Engine Join tables metadata or recreate these tables after upgrade. [#8400](https://github.com/ClickHouse/ClickHouse/pull/8400) ([Artem Zuikov](https://github.com/4ertus2)) +* Require server to be restarted to apply the changes in logging configuration. This is a temporary workaround to avoid the bug where the server logs to a deleted log file (see [#8696](https://github.com/ClickHouse/ClickHouse/issues/8696)). [#8707](https://github.com/ClickHouse/ClickHouse/pull/8707) ([Alexander Kuzmenkov](https://github.com/akuzm)) + +#### New Feature +* Added information about part paths to `system.merges`. [#8043](https://github.com/ClickHouse/ClickHouse/pull/8043) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Add ability to execute `SYSTEM RELOAD DICTIONARY` query in `ON CLUSTER` mode. [#8288](https://github.com/ClickHouse/ClickHouse/pull/8288) ([Guillaume Tassery](https://github.com/YiuRULE)) +* Add ability to execute `CREATE DICTIONARY` queries in `ON CLUSTER` mode. [#8163](https://github.com/ClickHouse/ClickHouse/pull/8163) ([alesapin](https://github.com/alesapin)) +* Now user's profile in `users.xml` can inherit multiple profiles. [#8343](https://github.com/ClickHouse/ClickHouse/pull/8343) ([Mikhail f. Shiryaev](https://github.com/Felixoid)) +* Added `system.stack_trace` table that allows to look at stack traces of all server threads. This is useful for developers to introspect server state. This fixes [#7576](https://github.com/ClickHouse/ClickHouse/issues/7576). [#8344](https://github.com/ClickHouse/ClickHouse/pull/8344) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add `DateTime64` datatype with configurable sub-second precision. [#7170](https://github.com/ClickHouse/ClickHouse/pull/7170) ([Vasily Nemkov](https://github.com/Enmk)) +* Add table function `clusterAllReplicas` which allows to query all the nodes in the cluster. [#8493](https://github.com/ClickHouse/ClickHouse/pull/8493) ([kiran sunkari](https://github.com/kiransunkari)) +* Add aggregate function `categoricalInformationValue` which calculates the information value of a discrete feature. [#8117](https://github.com/ClickHouse/ClickHouse/pull/8117) ([hcz](https://github.com/hczhcz)) +* Speed up parsing of data files in `CSV`, `TSV` and `JSONEachRow` format by doing it in parallel. [#7780](https://github.com/ClickHouse/ClickHouse/pull/7780) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Add function `bankerRound` which performs banker's rounding. [#8112](https://github.com/ClickHouse/ClickHouse/pull/8112) ([hcz](https://github.com/hczhcz)) +* Support more languages in embedded dictionary for region names: 'ru', 'en', 'ua', 'uk', 'by', 'kz', 'tr', 'de', 'uz', 'lv', 'lt', 'et', 'pt', 'he', 'vi'. [#8189](https://github.com/ClickHouse/ClickHouse/pull/8189) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Improvements in consistency of `ANY JOIN` logic. Now `t1 ANY LEFT JOIN t2` equals `t2 ANY RIGHT JOIN t1`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +* Add setting `any_join_distinct_right_table_keys` which enables old behaviour for `ANY INNER JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +* Add new `SEMI` and `ANTI JOIN`. Old `ANY INNER JOIN` behaviour now available as `SEMI LEFT JOIN`. [#7665](https://github.com/ClickHouse/ClickHouse/pull/7665) ([Artem Zuikov](https://github.com/4ertus2)) +* Added `Distributed` format for `File` engine and `file` table function which allows to read from `.bin` files generated by asynchronous inserts into `Distributed` table. [#8535](https://github.com/ClickHouse/ClickHouse/pull/8535) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Add optional reset column argument for `runningAccumulate` which allows to reset aggregation results for each new key value. [#8326](https://github.com/ClickHouse/ClickHouse/pull/8326) ([Sergey Kononenko](https://github.com/kononencheg)) +* Add ability to use ClickHouse as Prometheus endpoint. [#7900](https://github.com/ClickHouse/ClickHouse/pull/7900) ([vdimir](https://github.com/Vdimir)) +* Add section `` in `config.xml` which restricts allowed hosts for remote table engines and table functions `URL`, `S3`, `HDFS`. [#7154](https://github.com/ClickHouse/ClickHouse/pull/7154) ([Mikhail Korotov](https://github.com/millb)) +* Added function `greatCircleAngle` which calculates the distance on a sphere in degrees. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Changed Earth radius to be consistent with H3 library. [#8105](https://github.com/ClickHouse/ClickHouse/pull/8105) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added `JSONCompactEachRow` and `JSONCompactEachRowWithNamesAndTypes` formats for input and output. [#7841](https://github.com/ClickHouse/ClickHouse/pull/7841) ([Mikhail Korotov](https://github.com/millb)) +* Added feature for file-related table engines and table functions (`File`, `S3`, `URL`, `HDFS`) which allows to read and write `gzip` files based on additional engine parameter or file extension. [#7840](https://github.com/ClickHouse/ClickHouse/pull/7840) ([Andrey Bodrov](https://github.com/apbodrov)) +* Added the `randomASCII(length)` function, generating a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. [#8401](https://github.com/ClickHouse/ClickHouse/pull/8401) ([BayoNet](https://github.com/BayoNet)) +* Added function `JSONExtractArrayRaw` which returns an array on unparsed json array elements from `JSON` string. [#8081](https://github.com/ClickHouse/ClickHouse/pull/8081) ([Oleg Matrokhin](https://github.com/errx)) +* Add `arrayZip` function which allows to combine multiple arrays of equal lengths into one array of tuples. [#8149](https://github.com/ClickHouse/ClickHouse/pull/8149) ([Winter Zhang](https://github.com/zhang2014)) +* Add ability to move data between disks according to configured `TTL`-expressions for `*MergeTree` table engines family. [#8140](https://github.com/ClickHouse/ClickHouse/pull/8140) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Added new aggregate function `avgWeighted` which allows to calculate weighted average. [#7898](https://github.com/ClickHouse/ClickHouse/pull/7898) ([Andrey Bodrov](https://github.com/apbodrov)) +* Now parallel parsing is enabled by default for `TSV`, `TSKV`, `CSV` and `JSONEachRow` formats. [#7894](https://github.com/ClickHouse/ClickHouse/pull/7894) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Add several geo functions from `H3` library: `h3GetResolution`, `h3EdgeAngle`, `h3EdgeLength`, `h3IsValid` and `h3kRing`. [#8034](https://github.com/ClickHouse/ClickHouse/pull/8034) ([Konstantin Malanchev](https://github.com/hombit)) +* Added support for brotli (`br`) compression in file-related storages and table functions. This fixes [#8156](https://github.com/ClickHouse/ClickHouse/issues/8156). [#8526](https://github.com/ClickHouse/ClickHouse/pull/8526) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add `groupBit*` functions for the `SimpleAggregationFunction` type. [#8485](https://github.com/ClickHouse/ClickHouse/pull/8485) ([Guillaume Tassery](https://github.com/YiuRULE)) + +#### Bug Fix +* Fix rename of tables with `Distributed` engine. Fixes issue [#7868](https://github.com/ClickHouse/ClickHouse/issues/7868). [#8306](https://github.com/ClickHouse/ClickHouse/pull/8306) ([tavplubix](https://github.com/tavplubix)) +* Now dictionaries support `EXPRESSION` for attributes in arbitrary string in non-ClickHouse SQL dialect. [#8098](https://github.com/ClickHouse/ClickHouse/pull/8098) ([alesapin](https://github.com/alesapin)) +* Fix broken `INSERT SELECT FROM mysql(...)` query. This fixes [#8070](https://github.com/ClickHouse/ClickHouse/issues/8070) and [#7960](https://github.com/ClickHouse/ClickHouse/issues/7960). [#8234](https://github.com/ClickHouse/ClickHouse/pull/8234) ([tavplubix](https://github.com/tavplubix)) +* Fix error "Mismatch column sizes" when inserting default `Tuple` from `JSONEachRow`. This fixes [#5653](https://github.com/ClickHouse/ClickHouse/issues/5653). [#8606](https://github.com/ClickHouse/ClickHouse/pull/8606) ([tavplubix](https://github.com/tavplubix)) +* Now an exception will be thrown in case of using `WITH TIES` alongside `LIMIT BY`. Also add ability to use `TOP` with `LIMIT BY`. This fixes [#7472](https://github.com/ClickHouse/ClickHouse/issues/7472). [#7637](https://github.com/ClickHouse/ClickHouse/pull/7637) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Fix unintendent dependency from fresh glibc version in `clickhouse-odbc-bridge` binary. [#8046](https://github.com/ClickHouse/ClickHouse/pull/8046) ([Amos Bird](https://github.com/amosbird)) +* Fix bug in check function of `*MergeTree` engines family. Now it doesn't fail in case when we have equal amount of rows in last granule and last mark (non-final). [#8047](https://github.com/ClickHouse/ClickHouse/pull/8047) ([alesapin](https://github.com/alesapin)) +* Fix insert into `Enum*` columns after `ALTER` query, when underlying numeric type is equal to table specified type. This fixes [#7836](https://github.com/ClickHouse/ClickHouse/issues/7836). [#7908](https://github.com/ClickHouse/ClickHouse/pull/7908) ([Anton Popov](https://github.com/CurtizJ)) +* Allowed non-constant negative "size" argument for function `substring`. It was not allowed by mistake. This fixes [#4832](https://github.com/ClickHouse/ClickHouse/issues/4832). [#7703](https://github.com/ClickHouse/ClickHouse/pull/7703) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix parsing bug when wrong number of arguments passed to `(O|J)DBC` table engine. [#7709](https://github.com/ClickHouse/ClickHouse/pull/7709) ([alesapin](https://github.com/alesapin)) +* Using command name of the running clickhouse process when sending logs to syslog. In previous versions, empty string was used instead of command name. [#8460](https://github.com/ClickHouse/ClickHouse/pull/8460) ([Michael Nacharov](https://github.com/mnach)) +* Fix check of allowed hosts for `localhost`. This PR fixes the solution provided in [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241). [#8342](https://github.com/ClickHouse/ClickHouse/pull/8342) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix rare crash in `argMin` and `argMax` functions for long string arguments, when result is used in `runningAccumulate` function. This fixes [#8325](https://github.com/ClickHouse/ClickHouse/issues/8325) [#8341](https://github.com/ClickHouse/ClickHouse/pull/8341) ([dinosaur](https://github.com/769344359)) +* Fix memory overcommit for tables with `Buffer` engine. [#8345](https://github.com/ClickHouse/ClickHouse/pull/8345) ([Azat Khuzhin](https://github.com/azat)) +* Fixed potential bug in functions that can take `NULL` as one of the arguments and return non-NULL. [#8196](https://github.com/ClickHouse/ClickHouse/pull/8196) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Better metrics calculations in thread pool for background processes for `MergeTree` table engines. [#8194](https://github.com/ClickHouse/ClickHouse/pull/8194) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix function `IN` inside `WHERE` statement when row-level table filter is present. Fixes [#6687](https://github.com/ClickHouse/ClickHouse/issues/6687) [#8357](https://github.com/ClickHouse/ClickHouse/pull/8357) ([Ivan](https://github.com/abyss7)) +* Now an exception is thrown if the integral value is not parsed completely for settings values. [#7678](https://github.com/ClickHouse/ClickHouse/pull/7678) ([Mikhail Korotov](https://github.com/millb)) +* Fix exception when aggregate function is used in query to distributed table with more than two local shards. [#8164](https://github.com/ClickHouse/ClickHouse/pull/8164) ([小路](https://github.com/nicelulu)) +* Now bloom filter can handle zero length arrays and doesn't perform redundant calculations. [#8242](https://github.com/ClickHouse/ClickHouse/pull/8242) ([achimbab](https://github.com/achimbab)) +* Fixed checking if a client host is allowed by matching the client host to `host_regexp` specified in `users.xml`. [#8241](https://github.com/ClickHouse/ClickHouse/pull/8241) ([Vitaly Baranov](https://github.com/vitlibar)) +* Relax ambiguous column check that leads to false positives in multiple `JOIN ON` section. [#8385](https://github.com/ClickHouse/ClickHouse/pull/8385) ([Artem Zuikov](https://github.com/4ertus2)) +* Fixed possible server crash (`std::terminate`) when the server cannot send or write data in `JSON` or `XML` format with values of `String` data type (that require `UTF-8` validation) or when compressing result data with Brotli algorithm or in some other rare cases. This fixes [#7603](https://github.com/ClickHouse/ClickHouse/issues/7603) [#8384](https://github.com/ClickHouse/ClickHouse/pull/8384) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix race condition in `StorageDistributedDirectoryMonitor` found by CI. This fixes [#8364](https://github.com/ClickHouse/ClickHouse/issues/8364). [#8383](https://github.com/ClickHouse/ClickHouse/pull/8383) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Now background merges in `*MergeTree` table engines family preserve storage policy volume order more accurately. [#8549](https://github.com/ClickHouse/ClickHouse/pull/8549) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Now table engine `Kafka` works properly with `Native` format. This fixes [#6731](https://github.com/ClickHouse/ClickHouse/issues/6731) [#7337](https://github.com/ClickHouse/ClickHouse/issues/7337) [#8003](https://github.com/ClickHouse/ClickHouse/issues/8003). [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) +* Fixed formats with headers (like `CSVWithNames`) which were throwing exception about EOF for table engine `Kafka`. [#8016](https://github.com/ClickHouse/ClickHouse/pull/8016) ([filimonov](https://github.com/filimonov)) +* Fixed a bug with making set from subquery in right part of `IN` section. This fixes [#5767](https://github.com/ClickHouse/ClickHouse/issues/5767) and [#2542](https://github.com/ClickHouse/ClickHouse/issues/2542). [#7755](https://github.com/ClickHouse/ClickHouse/pull/7755) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Fix possible crash while reading from storage `File`. [#7756](https://github.com/ClickHouse/ClickHouse/pull/7756) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed reading of the files in `Parquet` format containing columns of type `list`. [#8334](https://github.com/ClickHouse/ClickHouse/pull/8334) ([maxulan](https://github.com/maxulan)) +* Fix error `Not found column` for distributed queries with `PREWHERE` condition dependent on sampling key if `max_parallel_replicas > 1`. [#7913](https://github.com/ClickHouse/ClickHouse/pull/7913) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix error `Not found column` if query used `PREWHERE` dependent on table's alias and the result set was empty because of primary key condition. [#7911](https://github.com/ClickHouse/ClickHouse/pull/7911) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed return type for functions `rand` and `randConstant` in case of `Nullable` argument. Now functions always return `UInt32` and never `Nullable(UInt32)`. [#8204](https://github.com/ClickHouse/ClickHouse/pull/8204) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Disabled predicate push-down for `WITH FILL` expression. This fixes [#7784](https://github.com/ClickHouse/ClickHouse/issues/7784). [#7789](https://github.com/ClickHouse/ClickHouse/pull/7789) ([Winter Zhang](https://github.com/zhang2014)) +* Fixed incorrect `count()` result for `SummingMergeTree` when `FINAL` section is used. [#3280](https://github.com/ClickHouse/ClickHouse/issues/3280) [#7786](https://github.com/ClickHouse/ClickHouse/pull/7786) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Fix possible incorrect result for constant functions from remote servers. It happened for queries with functions like `version()`, `uptime()`, etc. which returns different constant values for different servers. This fixes [#7666](https://github.com/ClickHouse/ClickHouse/issues/7666). [#7689](https://github.com/ClickHouse/ClickHouse/pull/7689) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix complicated bug in push-down predicate optimization which leads to wrong results. This fixes a lot of issues on push-down predicate optimization. [#8503](https://github.com/ClickHouse/ClickHouse/pull/8503) ([Winter Zhang](https://github.com/zhang2014)) +* Fix crash in `CREATE TABLE .. AS dictionary` query. [#8508](https://github.com/ClickHouse/ClickHouse/pull/8508) ([Azat Khuzhin](https://github.com/azat)) +* Several improvements ClickHouse grammar in `.g4` file. [#8294](https://github.com/ClickHouse/ClickHouse/pull/8294) ([taiyang-li](https://github.com/taiyang-li)) +* Fix bug that leads to crashes in `JOIN`s with tables with engine `Join`. This fixes [#7556](https://github.com/ClickHouse/ClickHouse/issues/7556) [#8254](https://github.com/ClickHouse/ClickHouse/issues/8254) [#7915](https://github.com/ClickHouse/ClickHouse/issues/7915) [#8100](https://github.com/ClickHouse/ClickHouse/issues/8100). [#8298](https://github.com/ClickHouse/ClickHouse/pull/8298) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix redundant dictionaries reload on `CREATE DATABASE`. [#7916](https://github.com/ClickHouse/ClickHouse/pull/7916) ([Azat Khuzhin](https://github.com/azat)) +* Limit maximum number of streams for read from `StorageFile` and `StorageHDFS`. Fixes [#7650](https://github.com/ClickHouse/ClickHouse/issues/7650). [#7981](https://github.com/ClickHouse/ClickHouse/pull/7981) ([alesapin](https://github.com/alesapin)) +* Fix bug in `ALTER ... MODIFY ... CODEC` query, when user specify both default expression and codec. Fixes [8593](https://github.com/ClickHouse/ClickHouse/issues/8593). [#8614](https://github.com/ClickHouse/ClickHouse/pull/8614) ([alesapin](https://github.com/alesapin)) +* Fix error in background merge of columns with `SimpleAggregateFunction(LowCardinality)` type. [#8613](https://github.com/ClickHouse/ClickHouse/pull/8613) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed type check in function `toDateTime64`. [#8375](https://github.com/ClickHouse/ClickHouse/pull/8375) ([Vasily Nemkov](https://github.com/Enmk)) +* Now server do not crash on `LEFT` or `FULL JOIN` with and Join engine and unsupported `join_use_nulls` settings. [#8479](https://github.com/ClickHouse/ClickHouse/pull/8479) ([Artem Zuikov](https://github.com/4ertus2)) +* Now `DROP DICTIONARY IF EXISTS db.dict` query doesn't throw exception if `db` doesn't exist. [#8185](https://github.com/ClickHouse/ClickHouse/pull/8185) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix possible crashes in table functions (`file`, `mysql`, `remote`) caused by usage of reference to removed `IStorage` object. Fix incorrect parsing of columns specified at insertion into table function. [#7762](https://github.com/ClickHouse/ClickHouse/pull/7762) ([tavplubix](https://github.com/tavplubix)) +* Ensure network be up before starting `clickhouse-server`. This fixes [#7507](https://github.com/ClickHouse/ClickHouse/issues/7507). [#8570](https://github.com/ClickHouse/ClickHouse/pull/8570) ([Zhichang Yu](https://github.com/yuzhichang)) +* Fix timeouts handling for secure connections, so queries doesn't hang indefenitely. This fixes [#8126](https://github.com/ClickHouse/ClickHouse/issues/8126). [#8128](https://github.com/ClickHouse/ClickHouse/pull/8128) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix `clickhouse-copier`'s redundant contention between concurrent workers. [#7816](https://github.com/ClickHouse/ClickHouse/pull/7816) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) +* Now mutations doesn't skip attached parts, even if their mutation version were larger than current mutation version. [#7812](https://github.com/ClickHouse/ClickHouse/pull/7812) ([Zhichang Yu](https://github.com/yuzhichang)) [#8250](https://github.com/ClickHouse/ClickHouse/pull/8250) ([alesapin](https://github.com/alesapin)) +* Ignore redundant copies of `*MergeTree` data parts after move to another disk and server restart. [#7810](https://github.com/ClickHouse/ClickHouse/pull/7810) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix crash in `FULL JOIN` with `LowCardinality` in `JOIN` key. [#8252](https://github.com/ClickHouse/ClickHouse/pull/8252) ([Artem Zuikov](https://github.com/4ertus2)) +* Forbidden to use column name more than once in insert query like `INSERT INTO tbl (x, y, x)`. This fixes [#5465](https://github.com/ClickHouse/ClickHouse/issues/5465), [#7681](https://github.com/ClickHouse/ClickHouse/issues/7681). [#7685](https://github.com/ClickHouse/ClickHouse/pull/7685) ([alesapin](https://github.com/alesapin)) +* Added fallback for detection the number of physical CPU cores for unknown CPUs (using the number of logical CPU cores). This fixes [#5239](https://github.com/ClickHouse/ClickHouse/issues/5239). [#7726](https://github.com/ClickHouse/ClickHouse/pull/7726) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix `There's no column` error for materialized and alias columns. [#8210](https://github.com/ClickHouse/ClickHouse/pull/8210) ([Artem Zuikov](https://github.com/4ertus2)) +* Fixed sever crash when `EXISTS` query was used without `TABLE` or `DICTIONARY` qualifier. Just like `EXISTS t`. This fixes [#8172](https://github.com/ClickHouse/ClickHouse/issues/8172). This bug was introduced in version 19.17. [#8213](https://github.com/ClickHouse/ClickHouse/pull/8213) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix rare bug with error `"Sizes of columns doesn't match"` that might appear when using `SimpleAggregateFunction` column. [#7790](https://github.com/ClickHouse/ClickHouse/pull/7790) ([Boris Granveaud](https://github.com/bgranvea)) +* Fix bug where user with empty `allow_databases` got access to all databases (and same for `allow_dictionaries`). [#7793](https://github.com/ClickHouse/ClickHouse/pull/7793) ([DeifyTheGod](https://github.com/DeifyTheGod)) +* Fix client crash when server already disconnected from client. [#8071](https://github.com/ClickHouse/ClickHouse/pull/8071) ([Azat Khuzhin](https://github.com/azat)) +* Fix `ORDER BY` behaviour in case of sorting by primary key prefix and non primary key suffix. [#7759](https://github.com/ClickHouse/ClickHouse/pull/7759) ([Anton Popov](https://github.com/CurtizJ)) +* Check if qualified column present in the table. This fixes [#6836](https://github.com/ClickHouse/ClickHouse/issues/6836). [#7758](https://github.com/ClickHouse/ClickHouse/pull/7758) ([Artem Zuikov](https://github.com/4ertus2)) +* Fixed behavior with `ALTER MOVE` ran immediately after merge finish moves superpart of specified. Fixes [#8103](https://github.com/ClickHouse/ClickHouse/issues/8103). [#8104](https://github.com/ClickHouse/ClickHouse/pull/8104) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix possible server crash while using `UNION` with different number of columns. Fixes [#7279](https://github.com/ClickHouse/ClickHouse/issues/7279). [#7929](https://github.com/ClickHouse/ClickHouse/pull/7929) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix size of result substring for function `substr` with negative size. [#8589](https://github.com/ClickHouse/ClickHouse/pull/8589) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Now server does not execute part mutation in `MergeTree` if there are not enough free threads in background pool. [#8588](https://github.com/ClickHouse/ClickHouse/pull/8588) ([tavplubix](https://github.com/tavplubix)) +* Fix a minor typo on formatting `UNION ALL` AST. [#7999](https://github.com/ClickHouse/ClickHouse/pull/7999) ([litao91](https://github.com/litao91)) +* Fixed incorrect bloom filter results for negative numbers. This fixes [#8317](https://github.com/ClickHouse/ClickHouse/issues/8317). [#8566](https://github.com/ClickHouse/ClickHouse/pull/8566) ([Winter Zhang](https://github.com/zhang2014)) +* Fixed potential buffer overflow in decompress. Malicious user can pass fabricated compressed data that will cause read after buffer. This issue was found by Eldar Zaitov from Yandex information security team. [#8404](https://github.com/ClickHouse/ClickHouse/pull/8404) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix incorrect result because of integers overflow in `arrayIntersect`. [#7777](https://github.com/ClickHouse/ClickHouse/pull/7777) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Now `OPTIMIZE TABLE` query will not wait for offline replicas to perform the operation. [#8314](https://github.com/ClickHouse/ClickHouse/pull/8314) ([javi santana](https://github.com/javisantana)) +* Fixed `ALTER TTL` parser for `Replicated*MergeTree` tables. [#8318](https://github.com/ClickHouse/ClickHouse/pull/8318) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix communication between server and client, so server read temporary tables info after query failure. [#8084](https://github.com/ClickHouse/ClickHouse/pull/8084) ([Azat Khuzhin](https://github.com/azat)) +* Fix `bitmapAnd` function error when intersecting an aggregated bitmap and a scalar bitmap. [#8082](https://github.com/ClickHouse/ClickHouse/pull/8082) ([Yue Huang](https://github.com/moon03432)) +* Refine the definition of `ZXid` according to the ZooKeeper Programmer's Guide which fixes bug in `clickhouse-cluster-copier`. [#8088](https://github.com/ClickHouse/ClickHouse/pull/8088) ([Ding Xiang Fei](https://github.com/dingxiangfei2009)) +* `odbc` table function now respects `external_table_functions_use_nulls` setting. [#7506](https://github.com/ClickHouse/ClickHouse/pull/7506) ([Vasily Nemkov](https://github.com/Enmk)) +* Fixed bug that lead to a rare data race. [#8143](https://github.com/ClickHouse/ClickHouse/pull/8143) ([Alexander Kazakov](https://github.com/Akazz)) +* Now `SYSTEM RELOAD DICTIONARY` reloads a dictionary completely, ignoring `update_field`. This fixes [#7440](https://github.com/ClickHouse/ClickHouse/issues/7440). [#8037](https://github.com/ClickHouse/ClickHouse/pull/8037) ([Vitaly Baranov](https://github.com/vitlibar)) +* Add ability to check if dictionary exists in create query. [#8032](https://github.com/ClickHouse/ClickHouse/pull/8032) ([alesapin](https://github.com/alesapin)) +* Fix `Float*` parsing in `Values` format. This fixes [#7817](https://github.com/ClickHouse/ClickHouse/issues/7817). [#7870](https://github.com/ClickHouse/ClickHouse/pull/7870) ([tavplubix](https://github.com/tavplubix)) +* Fix crash when we cannot reserve space in some background operations of `*MergeTree` table engines family. [#7873](https://github.com/ClickHouse/ClickHouse/pull/7873) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix crash of merge operation when table contains `SimpleAggregateFunction(LowCardinality)` column. This fixes [#8515](https://github.com/ClickHouse/ClickHouse/issues/8515). [#8522](https://github.com/ClickHouse/ClickHouse/pull/8522) ([Azat Khuzhin](https://github.com/azat)) +* Restore support of all ICU locales and add the ability to apply collations for constant expressions. Also add language name to `system.collations` table. [#8051](https://github.com/ClickHouse/ClickHouse/pull/8051) ([alesapin](https://github.com/alesapin)) +* Fix bug when external dictionaries with zero minimal lifetime (`LIFETIME(MIN 0 MAX N)`, `LIFETIME(N)`) don't update in background. [#7983](https://github.com/ClickHouse/ClickHouse/pull/7983) ([alesapin](https://github.com/alesapin)) +* Fix crash when external dictionary with ClickHouse source has subquery in query. [#8351](https://github.com/ClickHouse/ClickHouse/pull/8351) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fix incorrect parsing of file extension in table with engine `URL`. This fixes [#8157](https://github.com/ClickHouse/ClickHouse/issues/8157). [#8419](https://github.com/ClickHouse/ClickHouse/pull/8419) ([Andrey Bodrov](https://github.com/apbodrov)) +* Fix `CHECK TABLE` query for `*MergeTree` tables without key. Fixes [#7543](https://github.com/ClickHouse/ClickHouse/issues/7543). [#7979](https://github.com/ClickHouse/ClickHouse/pull/7979) ([alesapin](https://github.com/alesapin)) +* Fixed conversion of `Float64` to MySQL type. [#8079](https://github.com/ClickHouse/ClickHouse/pull/8079) ([Yuriy Baranov](https://github.com/yurriy)) +* Now if table was not completely dropped because of server crash, server will try to restore and load it. [#8176](https://github.com/ClickHouse/ClickHouse/pull/8176) ([tavplubix](https://github.com/tavplubix)) +* Fixed crash in table function `file` while inserting into file that doesn't exist. Now in this case file would be created and then insert would be processed. [#8177](https://github.com/ClickHouse/ClickHouse/pull/8177) ([Olga Khvostikova](https://github.com/stavrolia)) +* Fix rare deadlock which can happen when `trace_log` is in enabled. [#7838](https://github.com/ClickHouse/ClickHouse/pull/7838) ([filimonov](https://github.com/filimonov)) +* Add ability to work with different types besides `Date` in `RangeHashed` external dictionary created from DDL query. Fixes [7899](https://github.com/ClickHouse/ClickHouse/issues/7899). [#8275](https://github.com/ClickHouse/ClickHouse/pull/8275) ([alesapin](https://github.com/alesapin)) +* Fixes crash when `now64()` is called with result of another function. [#8270](https://github.com/ClickHouse/ClickHouse/pull/8270) ([Vasily Nemkov](https://github.com/Enmk)) +* Fixed bug with detecting client IP for connections through mysql wire protocol. [#7743](https://github.com/ClickHouse/ClickHouse/pull/7743) ([Dmitry Muzyka](https://github.com/dmitriy-myz)) +* Fix empty array handling in `arraySplit` function. This fixes [#7708](https://github.com/ClickHouse/ClickHouse/issues/7708). [#7747](https://github.com/ClickHouse/ClickHouse/pull/7747) ([hcz](https://github.com/hczhcz)) +* Fixed the issue when `pid-file` of another running `clickhouse-server` may be deleted. [#8487](https://github.com/ClickHouse/ClickHouse/pull/8487) ([Weiqing Xu](https://github.com/weiqxu)) +* Fix dictionary reload if it has `invalidate_query`, which stopped updates and some exception on previous update tries. [#8029](https://github.com/ClickHouse/ClickHouse/pull/8029) ([alesapin](https://github.com/alesapin)) +* Fixed error in function `arrayReduce` that may lead to "double free" and error in aggregate function combinator `Resample` that may lead to memory leak. Added aggregate function `aggThrow`. This function can be used for testing purposes. [#8446](https://github.com/ClickHouse/ClickHouse/pull/8446) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Improvement +* Improved logging when working with `S3` table engine. [#8251](https://github.com/ClickHouse/ClickHouse/pull/8251) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) +* Printed help message when no arguments are passed when calling `clickhouse-local`. This fixes [#5335](https://github.com/ClickHouse/ClickHouse/issues/5335). [#8230](https://github.com/ClickHouse/ClickHouse/pull/8230) ([Andrey Nagorny](https://github.com/Melancholic)) +* Add setting `mutations_sync` which allows to wait `ALTER UPDATE/DELETE` queries synchronously. [#8237](https://github.com/ClickHouse/ClickHouse/pull/8237) ([alesapin](https://github.com/alesapin)) +* Allow to set up relative `user_files_path` in `config.xml` (in the way similar to `format_schema_path`). [#7632](https://github.com/ClickHouse/ClickHouse/pull/7632) ([hcz](https://github.com/hczhcz)) +* Add exception for illegal types for conversion functions with `-OrZero` postfix. [#7880](https://github.com/ClickHouse/ClickHouse/pull/7880) ([Andrey Konyaev](https://github.com/akonyaev90)) +* Simplify format of the header of data sending to a shard in a distributed query. [#8044](https://github.com/ClickHouse/ClickHouse/pull/8044) ([Vitaly Baranov](https://github.com/vitlibar)) +* `Live View` table engine refactoring. [#8519](https://github.com/ClickHouse/ClickHouse/pull/8519) ([vzakaznikov](https://github.com/vzakaznikov)) +* Add additional checks for external dictionaries created from DDL-queries. [#8127](https://github.com/ClickHouse/ClickHouse/pull/8127) ([alesapin](https://github.com/alesapin)) +* Fix error `Column ... already exists` while using `FINAL` and `SAMPLE` together, e.g. `select count() from table final sample 1/2`. Fixes [#5186](https://github.com/ClickHouse/ClickHouse/issues/5186). [#7907](https://github.com/ClickHouse/ClickHouse/pull/7907) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Now table the first argument of `joinGet` function can be table identifier. [#7707](https://github.com/ClickHouse/ClickHouse/pull/7707) ([Amos Bird](https://github.com/amosbird)) +* Allow using `MaterializedView` with subqueries above `Kafka` tables. [#8197](https://github.com/ClickHouse/ClickHouse/pull/8197) ([filimonov](https://github.com/filimonov)) +* Now background moves between disks run it the seprate thread pool. [#7670](https://github.com/ClickHouse/ClickHouse/pull/7670) ([Vladimir Chebotarev](https://github.com/excitoon)) +* `SYSTEM RELOAD DICTIONARY` now executes synchronously. [#8240](https://github.com/ClickHouse/ClickHouse/pull/8240) ([Vitaly Baranov](https://github.com/vitlibar)) +* Stack traces now display physical addresses (offsets in object file) instead of virtual memory addresses (where the object file was loaded). That allows the use of `addr2line` when binary is position independent and ASLR is active. This fixes [#8360](https://github.com/ClickHouse/ClickHouse/issues/8360). [#8387](https://github.com/ClickHouse/ClickHouse/pull/8387) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Support new syntax for row-level security filters: `…
`. Fixes [#5779](https://github.com/ClickHouse/ClickHouse/issues/5779). [#8381](https://github.com/ClickHouse/ClickHouse/pull/8381) ([Ivan](https://github.com/abyss7)) +* Now `cityHash` function can work with `Decimal` and `UUID` types. Fixes [#5184](https://github.com/ClickHouse/ClickHouse/issues/5184). [#7693](https://github.com/ClickHouse/ClickHouse/pull/7693) ([Mikhail Korotov](https://github.com/millb)) +* Removed fixed index granularity (it was 1024) from system logs because it's obsolete after implementation of adaptive granularity. [#7698](https://github.com/ClickHouse/ClickHouse/pull/7698) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Enabled MySQL compatibility server when ClickHouse is compiled without SSL. [#7852](https://github.com/ClickHouse/ClickHouse/pull/7852) ([Yuriy Baranov](https://github.com/yurriy)) +* Now server checksums distributed batches, which gives more verbose errors in case of corrupted data in batch. [#7914](https://github.com/ClickHouse/ClickHouse/pull/7914) ([Azat Khuzhin](https://github.com/azat)) +* Support `DROP DATABASE`, `DETACH TABLE`, `DROP TABLE` and `ATTACH TABLE` for `MySQL` database engine. [#8202](https://github.com/ClickHouse/ClickHouse/pull/8202) ([Winter Zhang](https://github.com/zhang2014)) +* Add authentication in S3 table function and table engine. [#7623](https://github.com/ClickHouse/ClickHouse/pull/7623) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Added check for extra parts of `MergeTree` at different disks, in order to not allow to miss data parts at undefined disks. [#8118](https://github.com/ClickHouse/ClickHouse/pull/8118) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Enable SSL support for Mac client and server. [#8297](https://github.com/ClickHouse/ClickHouse/pull/8297) ([Ivan](https://github.com/abyss7)) +* Now ClickHouse can work as MySQL federated server (see https://dev.mysql.com/doc/refman/5.7/en/federated-create-server.html). [#7717](https://github.com/ClickHouse/ClickHouse/pull/7717) ([Maxim Fedotov](https://github.com/MaxFedotov)) +* `clickhouse-client` now only enable `bracketed-paste` when multiquery is on and multiline is off. This fixes [#7757](https://github.com/ClickHouse/ClickHouse/issues/7757). [#7761](https://github.com/ClickHouse/ClickHouse/pull/7761) ([Amos Bird](https://github.com/amosbird)) +* Support `Array(Decimal)` in `if` function. [#7721](https://github.com/ClickHouse/ClickHouse/pull/7721) ([Artem Zuikov](https://github.com/4ertus2)) +* Support Decimals in `arrayDifference`, `arrayCumSum` and `arrayCumSumNegative` functions. [#7724](https://github.com/ClickHouse/ClickHouse/pull/7724) ([Artem Zuikov](https://github.com/4ertus2)) +* Added `lifetime` column to `system.dictionaries` table. [#6820](https://github.com/ClickHouse/ClickHouse/issues/6820) [#7727](https://github.com/ClickHouse/ClickHouse/pull/7727) ([kekekekule](https://github.com/kekekekule)) +* Improved check for existing parts on different disks for `*MergeTree` table engines. Addresses [#7660](https://github.com/ClickHouse/ClickHouse/issues/7660). [#8440](https://github.com/ClickHouse/ClickHouse/pull/8440) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Integration with `AWS SDK` for `S3` interactions which allows to use all S3 features out of the box. [#8011](https://github.com/ClickHouse/ClickHouse/pull/8011) ([Pavel Kovalenko](https://github.com/Jokser)) +* Added support for subqueries in `Live View` tables. [#7792](https://github.com/ClickHouse/ClickHouse/pull/7792) ([vzakaznikov](https://github.com/vzakaznikov)) +* Check for using `Date` or `DateTime` column from `TTL` expressions was removed. [#7920](https://github.com/ClickHouse/ClickHouse/pull/7920) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Information about disk was added to `system.detached_parts` table. [#7833](https://github.com/ClickHouse/ClickHouse/pull/7833) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Now settings `max_(table|partition)_size_to_drop` can be changed without a restart. [#7779](https://github.com/ClickHouse/ClickHouse/pull/7779) ([Grigory Pervakov](https://github.com/GrigoryPervakov)) +* Slightly better usability of error messages. Ask user not to remove the lines below `Stack trace:`. [#7897](https://github.com/ClickHouse/ClickHouse/pull/7897) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Better reading messages from `Kafka` engine in various formats after [#7935](https://github.com/ClickHouse/ClickHouse/issues/7935). [#8035](https://github.com/ClickHouse/ClickHouse/pull/8035) ([Ivan](https://github.com/abyss7)) +* Better compatibility with MySQL clients which don't support `sha2_password` auth plugin. [#8036](https://github.com/ClickHouse/ClickHouse/pull/8036) ([Yuriy Baranov](https://github.com/yurriy)) +* Support more column types in MySQL compatibility server. [#7975](https://github.com/ClickHouse/ClickHouse/pull/7975) ([Yuriy Baranov](https://github.com/yurriy)) +* Implement `ORDER BY` optimization for `Merge`, `Buffer` and `Materilized View` storages with underlying `MergeTree` tables. [#8130](https://github.com/ClickHouse/ClickHouse/pull/8130) ([Anton Popov](https://github.com/CurtizJ)) +* Now we always use POSIX implementation of `getrandom` to have better compatibility with old kernels (< 3.17). [#7940](https://github.com/ClickHouse/ClickHouse/pull/7940) ([Amos Bird](https://github.com/amosbird)) +* Better check for valid destination in a move TTL rule. [#8410](https://github.com/ClickHouse/ClickHouse/pull/8410) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Better checks for broken insert batches for `Distributed` table engine. [#7933](https://github.com/ClickHouse/ClickHouse/pull/7933) ([Azat Khuzhin](https://github.com/azat)) +* Add column with array of parts name which mutations must process in future to `system.mutations` table. [#8179](https://github.com/ClickHouse/ClickHouse/pull/8179) ([alesapin](https://github.com/alesapin)) +* Parallel merge sort optimization for processors. [#8552](https://github.com/ClickHouse/ClickHouse/pull/8552) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* The settings `mark_cache_min_lifetime` is now obsolete and does nothing. In previous versions, mark cache can grow in memory larger than `mark_cache_size` to accomodate data within `mark_cache_min_lifetime` seconds. That was leading to confusion and higher memory usage than expected, that is especially bad on memory constrained systems. If you will see performance degradation after installing this release, you should increase the `mark_cache_size`. [#8484](https://github.com/ClickHouse/ClickHouse/pull/8484) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Preparation to use `tid` everywhere. This is needed for [#7477](https://github.com/ClickHouse/ClickHouse/issues/7477). [#8276](https://github.com/ClickHouse/ClickHouse/pull/8276) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +#### Performance Improvement +* Performance optimizations in processors pipeline. [#7988](https://github.com/ClickHouse/ClickHouse/pull/7988) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Non-blocking updates of expired keys in cache dictionaries (with permission to read old ones). [#8303](https://github.com/ClickHouse/ClickHouse/pull/8303) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Compile ClickHouse without `-fno-omit-frame-pointer` globally to spare one more register. [#8097](https://github.com/ClickHouse/ClickHouse/pull/8097) ([Amos Bird](https://github.com/amosbird)) +* Speedup `greatCircleDistance` function and add performance tests for it. [#7307](https://github.com/ClickHouse/ClickHouse/pull/7307) ([Olga Khvostikova](https://github.com/stavrolia)) +* Improved performance of function `roundDown`. [#8465](https://github.com/ClickHouse/ClickHouse/pull/8465) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Improved performance of `max`, `min`, `argMin`, `argMax` for `DateTime64` data type. [#8199](https://github.com/ClickHouse/ClickHouse/pull/8199) ([Vasily Nemkov](https://github.com/Enmk)) +* Improved performance of sorting without a limit or with big limit and external sorting. [#8545](https://github.com/ClickHouse/ClickHouse/pull/8545) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Improved performance of formatting floating point numbers up to 6 times. [#8542](https://github.com/ClickHouse/ClickHouse/pull/8542) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Improved performance of `modulo` function. [#7750](https://github.com/ClickHouse/ClickHouse/pull/7750) ([Amos Bird](https://github.com/amosbird)) +* Optimized `ORDER BY` and merging with single column key. [#8335](https://github.com/ClickHouse/ClickHouse/pull/8335) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Better implementation for `arrayReduce`, `-Array` and `-State` combinators. [#7710](https://github.com/ClickHouse/ClickHouse/pull/7710) ([Amos Bird](https://github.com/amosbird)) +* Now `PREWHERE` should be optimized to be at least as efficient as `WHERE`. [#7769](https://github.com/ClickHouse/ClickHouse/pull/7769) ([Amos Bird](https://github.com/amosbird)) +* Improve the way `round` and `roundBankers` handling negative numbers. [#8229](https://github.com/ClickHouse/ClickHouse/pull/8229) ([hcz](https://github.com/hczhcz)) +* Improved decoding performance of `DoubleDelta` and `Gorilla` codecs by roughly 30-40%. This fixes [#7082](https://github.com/ClickHouse/ClickHouse/issues/7082). [#8019](https://github.com/ClickHouse/ClickHouse/pull/8019) ([Vasily Nemkov](https://github.com/Enmk)) +* Improved performance of `base64` related functions. [#8444](https://github.com/ClickHouse/ClickHouse/pull/8444) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Added a function `geoDistance`. It is similar to `greatCircleDistance` but uses approximation to WGS-84 ellipsoid model. The performance of both functions are near the same. [#8086](https://github.com/ClickHouse/ClickHouse/pull/8086) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Faster `min` and `max` aggregation functions for `Decimal` data type. [#8144](https://github.com/ClickHouse/ClickHouse/pull/8144) ([Artem Zuikov](https://github.com/4ertus2)) +* Vectorize processing `arrayReduce`. [#7608](https://github.com/ClickHouse/ClickHouse/pull/7608) ([Amos Bird](https://github.com/amosbird)) +* `if` chains are now optimized as `multiIf`. [#8355](https://github.com/ClickHouse/ClickHouse/pull/8355) ([kamalov-ruslan](https://github.com/kamalov-ruslan)) +* Fix performance regression of `Kafka` table engine introduced in 19.15. This fixes [#7261](https://github.com/ClickHouse/ClickHouse/issues/7261). [#7935](https://github.com/ClickHouse/ClickHouse/pull/7935) ([filimonov](https://github.com/filimonov)) +* Removed "pie" code generation that `gcc` from Debian packages occasionally brings by default. [#8483](https://github.com/ClickHouse/ClickHouse/pull/8483) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Parallel parsing data formats [#6553](https://github.com/ClickHouse/ClickHouse/pull/6553) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Enable optimized parser of `Values` with expressions by default (`input_format_values_deduce_templates_of_expressions=1`). [#8231](https://github.com/ClickHouse/ClickHouse/pull/8231) ([tavplubix](https://github.com/tavplubix)) + +#### Build/Testing/Packaging Improvement +* Build fixes for `ARM` and in minimal mode. [#8304](https://github.com/ClickHouse/ClickHouse/pull/8304) ([proller](https://github.com/proller)) +* Add coverage file flush for `clickhouse-server` when std::atexit is not called. Also slightly improved logging in stateless tests with coverage. [#8267](https://github.com/ClickHouse/ClickHouse/pull/8267) ([alesapin](https://github.com/alesapin)) +* Update LLVM library in contrib. Avoid using LLVM from OS packages. [#8258](https://github.com/ClickHouse/ClickHouse/pull/8258) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Make bundled `curl` build fully quiet. [#8232](https://github.com/ClickHouse/ClickHouse/pull/8232) [#8203](https://github.com/ClickHouse/ClickHouse/pull/8203) ([Pavel Kovalenko](https://github.com/Jokser)) +* Fix some `MemorySanitizer` warnings. [#8235](https://github.com/ClickHouse/ClickHouse/pull/8235) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Use `add_warning` and `no_warning` macros in `CMakeLists.txt`. [#8604](https://github.com/ClickHouse/ClickHouse/pull/8604) ([Ivan](https://github.com/abyss7)) +* Add support of Minio S3 Compatible object (https://min.io/) for better integration tests. [#7863](https://github.com/ClickHouse/ClickHouse/pull/7863) [#7875](https://github.com/ClickHouse/ClickHouse/pull/7875) ([Pavel Kovalenko](https://github.com/Jokser)) +* Imported `libc` headers to contrib. It allows to make builds more consistent across various systems (only for `x86_64-linux-gnu`). [#5773](https://github.com/ClickHouse/ClickHouse/pull/5773) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Remove `-fPIC` from some libraries. [#8464](https://github.com/ClickHouse/ClickHouse/pull/8464) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Clean `CMakeLists.txt` for curl. See https://github.com/ClickHouse/ClickHouse/pull/8011#issuecomment-569478910 [#8459](https://github.com/ClickHouse/ClickHouse/pull/8459) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Silent warnings in `CapNProto` library. [#8220](https://github.com/ClickHouse/ClickHouse/pull/8220) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Add performance tests for short string optimized hash tables. [#7679](https://github.com/ClickHouse/ClickHouse/pull/7679) ([Amos Bird](https://github.com/amosbird)) +* Now ClickHouse will build on `AArch64` even if `MADV_FREE` is not available. This fixes [#8027](https://github.com/ClickHouse/ClickHouse/issues/8027). [#8243](https://github.com/ClickHouse/ClickHouse/pull/8243) ([Amos Bird](https://github.com/amosbird)) +* Update `zlib-ng` to fix memory sanitizer problems. [#7182](https://github.com/ClickHouse/ClickHouse/pull/7182) [#8206](https://github.com/ClickHouse/ClickHouse/pull/8206) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Enable internal MySQL library on non-Linux system, because usage of OS packages is very fragile and usually doesn't work at all. This fixes [#5765](https://github.com/ClickHouse/ClickHouse/issues/5765). [#8426](https://github.com/ClickHouse/ClickHouse/pull/8426) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed build on some systems after enabling `libc++`. This supersedes [#8374](https://github.com/ClickHouse/ClickHouse/issues/8374). [#8380](https://github.com/ClickHouse/ClickHouse/pull/8380) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Make `Field` methods more type-safe to find more errors. [#7386](https://github.com/ClickHouse/ClickHouse/pull/7386) [#8209](https://github.com/ClickHouse/ClickHouse/pull/8209) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Added missing files to the `libc-headers` submodule. [#8507](https://github.com/ClickHouse/ClickHouse/pull/8507) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix wrong `JSON` quoting in performance test output. [#8497](https://github.com/ClickHouse/ClickHouse/pull/8497) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Now stack trace is displayed for `std::exception` and `Poco::Exception`. In previous versions it was available only for `DB::Exception`. This improves diagnostics. [#8501](https://github.com/ClickHouse/ClickHouse/pull/8501) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Porting `clock_gettime` and `clock_nanosleep` for fresh glibc versions. [#8054](https://github.com/ClickHouse/ClickHouse/pull/8054) ([Amos Bird](https://github.com/amosbird)) +* Enable `part_log` in example config for developers. [#8609](https://github.com/ClickHouse/ClickHouse/pull/8609) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix async nature of reload in `01036_no_superfluous_dict_reload_on_create_database*`. [#8111](https://github.com/ClickHouse/ClickHouse/pull/8111) ([Azat Khuzhin](https://github.com/azat)) +* Fixed codec performance tests. [#8615](https://github.com/ClickHouse/ClickHouse/pull/8615) ([Vasily Nemkov](https://github.com/Enmk)) +* Add install scripts for `.tgz` build and documentation for them. [#8612](https://github.com/ClickHouse/ClickHouse/pull/8612) [#8591](https://github.com/ClickHouse/ClickHouse/pull/8591) ([alesapin](https://github.com/alesapin)) +* Removed old `ZSTD` test (it was created in year 2016 to reproduce the bug that pre 1.0 version of ZSTD has had). This fixes [#8618](https://github.com/ClickHouse/ClickHouse/issues/8618). [#8619](https://github.com/ClickHouse/ClickHouse/pull/8619) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed build on Mac OS Catalina. [#8600](https://github.com/ClickHouse/ClickHouse/pull/8600) ([meo](https://github.com/meob)) +* Increased number of rows in codec performance tests to make results noticeable. [#8574](https://github.com/ClickHouse/ClickHouse/pull/8574) ([Vasily Nemkov](https://github.com/Enmk)) +* In debug builds, treat `LOGICAL_ERROR` exceptions as assertion failures, so that they are easier to notice. [#8475](https://github.com/ClickHouse/ClickHouse/pull/8475) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Make formats-related performance test more deterministic. [#8477](https://github.com/ClickHouse/ClickHouse/pull/8477) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Update `lz4` to fix a MemorySanitizer failure. [#8181](https://github.com/ClickHouse/ClickHouse/pull/8181) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Suppress a known MemorySanitizer false positive in exception handling. [#8182](https://github.com/ClickHouse/ClickHouse/pull/8182) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Update `gcc` and `g++` to version 9 in `build/docker/build.sh` [#7766](https://github.com/ClickHouse/ClickHouse/pull/7766) ([TLightSky](https://github.com/tlightsky)) +* Add performance test case to test that `PREWHERE` is worse than `WHERE`. [#7768](https://github.com/ClickHouse/ClickHouse/pull/7768) ([Amos Bird](https://github.com/amosbird)) +* Progress towards fixing one flacky test. [#8621](https://github.com/ClickHouse/ClickHouse/pull/8621) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Avoid MemorySanitizer report for data from `libunwind`. [#8539](https://github.com/ClickHouse/ClickHouse/pull/8539) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Updated `libc++` to the latest version. [#8324](https://github.com/ClickHouse/ClickHouse/pull/8324) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Build ICU library from sources. This fixes [#6460](https://github.com/ClickHouse/ClickHouse/issues/6460). [#8219](https://github.com/ClickHouse/ClickHouse/pull/8219) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Switched from `libressl` to `openssl`. ClickHouse should support TLS 1.3 and SNI after this change. This fixes [#8171](https://github.com/ClickHouse/ClickHouse/issues/8171). [#8218](https://github.com/ClickHouse/ClickHouse/pull/8218) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed UBSan report when using `chacha20_poly1305` from SSL (happens on connect to https://yandex.ru/). [#8214](https://github.com/ClickHouse/ClickHouse/pull/8214) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix mode of default password file for `.deb` linux distros. [#8075](https://github.com/ClickHouse/ClickHouse/pull/8075) ([proller](https://github.com/proller)) +* Improved expression for getting `clickhouse-server` PID in `clickhouse-test`. [#8063](https://github.com/ClickHouse/ClickHouse/pull/8063) ([Alexander Kazakov](https://github.com/Akazz)) +* Updated contrib/googletest to v1.10.0. [#8587](https://github.com/ClickHouse/ClickHouse/pull/8587) ([Alexander Burmak](https://github.com/Alex-Burmak)) +* Fixed ThreadSaninitizer report in `base64` library. Also updated this library to the latest version, but it doesn't matter. This fixes [#8397](https://github.com/ClickHouse/ClickHouse/issues/8397). [#8403](https://github.com/ClickHouse/ClickHouse/pull/8403) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix `00600_replace_running_query` for processors. [#8272](https://github.com/ClickHouse/ClickHouse/pull/8272) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Remove support for `tcmalloc` to make `CMakeLists.txt` simpler. [#8310](https://github.com/ClickHouse/ClickHouse/pull/8310) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Release gcc builds now use `libc++` instead of `libstdc++`. Recently `libc++` was used only with clang. This will improve consistency of build configurations and portability. [#8311](https://github.com/ClickHouse/ClickHouse/pull/8311) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Enable ICU library for build with MemorySanitizer. [#8222](https://github.com/ClickHouse/ClickHouse/pull/8222) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Suppress warnings from `CapNProto` library. [#8224](https://github.com/ClickHouse/ClickHouse/pull/8224) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Removed special cases of code for `tcmalloc`, because it's no longer supported. [#8225](https://github.com/ClickHouse/ClickHouse/pull/8225) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* In CI coverage task, kill the server gracefully to allow it to save the coverage report. This fixes incomplete coverage reports we've been seeing lately. [#8142](https://github.com/ClickHouse/ClickHouse/pull/8142) ([alesapin](https://github.com/alesapin)) +* Performance tests for all codecs against `Float64` and `UInt64` values. [#8349](https://github.com/ClickHouse/ClickHouse/pull/8349) ([Vasily Nemkov](https://github.com/Enmk)) +* `termcap` is very much deprecated and lead to various problems (f.g. missing "up" cap and echoing `^J` instead of multi line) . Favor `terminfo` or bundled `ncurses`. [#7737](https://github.com/ClickHouse/ClickHouse/pull/7737) ([Amos Bird](https://github.com/amosbird)) +* Fix `test_storage_s3` integration test. [#7734](https://github.com/ClickHouse/ClickHouse/pull/7734) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Support `StorageFile(, null) ` to insert block into given format file without actually write to disk. This is required for performance tests. [#8455](https://github.com/ClickHouse/ClickHouse/pull/8455) ([Amos Bird](https://github.com/amosbird)) +* Added argument `--print-time` to functional tests which prints execution time per test. [#8001](https://github.com/ClickHouse/ClickHouse/pull/8001) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Added asserts to `KeyCondition` while evaluating RPN. This will fix warning from gcc-9. [#8279](https://github.com/ClickHouse/ClickHouse/pull/8279) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Dump cmake options in CI builds. [#8273](https://github.com/ClickHouse/ClickHouse/pull/8273) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Don't generate debug info for some fat libraries. [#8271](https://github.com/ClickHouse/ClickHouse/pull/8271) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Make `log_to_console.xml` always log to stderr, regardless of is it interactive or not. [#8395](https://github.com/ClickHouse/ClickHouse/pull/8395) ([Alexander Kuzmenkov](https://github.com/akuzm)) +* Removed some unused features from `clickhouse-performance-test` tool. [#8555](https://github.com/ClickHouse/ClickHouse/pull/8555) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Now we will also search for `lld-X` with corresponding `clang-X` version. [#8092](https://github.com/ClickHouse/ClickHouse/pull/8092) ([alesapin](https://github.com/alesapin)) +* Parquet build improvement. [#8421](https://github.com/ClickHouse/ClickHouse/pull/8421) ([maxulan](https://github.com/maxulan)) +* More GCC warnings [#8221](https://github.com/ClickHouse/ClickHouse/pull/8221) ([kreuzerkrieg](https://github.com/kreuzerkrieg)) +* Package for Arch Linux now allows to run ClickHouse server, and not only client. [#8534](https://github.com/ClickHouse/ClickHouse/pull/8534) ([Vladimir Chebotarev](https://github.com/excitoon)) +* Fix test with processors. Tiny performance fixes. [#7672](https://github.com/ClickHouse/ClickHouse/pull/7672) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Update contrib/protobuf. [#8256](https://github.com/ClickHouse/ClickHouse/pull/8256) ([Matwey V. Kornilov](https://github.com/matwey)) +* In preparation of switching to c++20 as a new year celebration. "May the C++ force be with ClickHouse." [#8447](https://github.com/ClickHouse/ClickHouse/pull/8447) ([Amos Bird](https://github.com/amosbird)) + +#### Experimental Feature +* Added experimental setting `min_bytes_to_use_mmap_io`. It allows to read big files without copying data from kernel to userspace. The setting is disabled by default. Recommended threshold is about 64 MB, because mmap/munmap is slow. [#8520](https://github.com/ClickHouse/ClickHouse/pull/8520) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Reworked quotas as a part of access control system. Added new table `system.quotas`, new functions `currentQuota`, `currentQuotaKey`, new SQL syntax `CREATE QUOTA`, `ALTER QUOTA`, `DROP QUOTA`, `SHOW QUOTA`. [#7257](https://github.com/ClickHouse/ClickHouse/pull/7257) ([Vitaly Baranov](https://github.com/vitlibar)) +* Allow skipping unknown settings with warnings instead of throwing exceptions. [#7653](https://github.com/ClickHouse/ClickHouse/pull/7653) ([Vitaly Baranov](https://github.com/vitlibar)) +* Reworked row policies as a part of access control system. Added new table `system.row_policies`, new function `currentRowPolicies()`, new SQL syntax `CREATE POLICY`, `ALTER POLICY`, `DROP POLICY`, `SHOW CREATE POLICY`, `SHOW POLICIES`. [#7808](https://github.com/ClickHouse/ClickHouse/pull/7808) ([Vitaly Baranov](https://github.com/vitlibar)) + +#### Security Fix +* Fixed the possibility of reading directories structure in tables with `File` table engine. This fixes [#8536](https://github.com/ClickHouse/ClickHouse/issues/8536). [#8537](https://github.com/ClickHouse/ClickHouse/pull/8537) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +## [Changelog for 2019](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/whats-new/changelog/2019.md) diff --git a/docs/en/whats-new/changelog/index.md b/docs/en/whats-new/changelog/index.md index c13441a8bd5..08fe7e3b8dc 100644 --- a/docs/en/whats-new/changelog/index.md +++ b/docs/en/whats-new/changelog/index.md @@ -1,7 +1,7 @@ --- toc_folder_title: Changelog toc_priority: 74 -toc_title: '2020' +toc_title: '2021' --- {% include "content/changelog.md" %} From b75dd0baddf3066bfde0cc6a5fccea370bdcdeb4 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 18 Jan 2021 19:00:46 +0300 Subject: [PATCH 262/264] DOCSUP-5604: Edit and translate to Russian (#18929) Co-authored-by: Alexander Kazakov --- .../external-dicts-dict-sources.md | 16 +++++----- .../external-dicts-dict-sources.md | 29 ++++++++++++------- 2 files changed, 26 insertions(+), 19 deletions(-) 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 b86f0d6bc4e..7cd26a9dffb 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 @@ -93,7 +93,7 @@ Setting fields: - `path` – The absolute path to the file. - `format` – The file format. All the formats described in “[Formats](../../../interfaces/formats.md#formats)” are supported. -When dictionary with FILE source is created via DDL command (`CREATE DICTIONARY ...`), source of the dictionary have to be located in `user_files` directory, to prevent DB users accessing arbitrary file on clickhouse node. +When dictionary with source `FILE` is created via DDL command (`CREATE DICTIONARY ...`), the source file needs to be located in `user_files` directory, to prevent DB users accessing arbitrary file on ClickHouse node. ## Executable File {#dicts-external_dicts_dict_sources-executable} @@ -115,7 +115,7 @@ Setting fields: - `command` – The absolute path to the executable file, or the file name (if the program directory is written to `PATH`). - `format` – The file format. All the formats described in “[Formats](../../../interfaces/formats.md#formats)” are supported. -That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on clickhouse node. +That dictionary source can be configured only via XML configuration. Creating dictionaries with executable source via DDL is disabled, otherwise, the DB user would be able to execute arbitrary binary on ClickHouse node. ## Http(s) {#dicts-external_dicts_dict_sources-http} @@ -160,14 +160,14 @@ Setting fields: - `url` – The source URL. - `format` – The file format. All the formats described in “[Formats](../../../interfaces/formats.md#formats)” are supported. - `credentials` – Basic HTTP authentication. Optional parameter. - - `user` – Username required for the authentication. - - `password` – Password required for the authentication. +- `user` – Username required for the authentication. +- `password` – Password required for the authentication. - `headers` – All custom HTTP headers entries used for the HTTP request. Optional parameter. - - `header` – Single HTTP header entry. - - `name` – Identifiant name used for the header send on the request. - - `value` – Value set for a specific identifiant name. +- `header` – Single HTTP header entry. +- `name` – Identifiant name used for the header send on the request. +- `value` – Value set for a specific identifiant name. -When creating a dictionary using the DDL command (`CREATE DICTIONARY ...`) remote hosts for HTTP dictionaries checked with the `remote_url_allow_hosts` section from config to prevent database users to access arbitrary HTTP server. +When creating a dictionary using the DDL command (`CREATE DICTIONARY ...`) remote hosts for HTTP dictionaries are checked against the contents of `remote_url_allow_hosts` section from config to prevent database users to access arbitrary HTTP server. ## ODBC {#dicts-external_dicts_dict_sources-odbc} diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 1367216e401..3bb11b638b2 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -90,8 +90,10 @@ SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) Поля настройки: -- `path` — Абсолютный путь к файлу. -- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе «[Форматы](../../../interfaces/formats.md#formats)». +- `path` — абсолютный путь к файлу. +- `format` — формат файла. Поддерживаются все форматы, описанные в разделе «[Форматы](../../../interfaces/formats.md#formats)». + +Если словарь с источником `FILE` создается с помощью DDL-команды (`CREATE DICTIONARY ...`), источник словаря должен быть расположен в каталоге `user_files`. Иначе пользователи базы данных будут иметь доступ к произвольному файлу на узле ClickHouse. ## Исполняемый файл {#dicts-external_dicts_dict_sources-executable} @@ -108,16 +110,12 @@ SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) ``` -или - -``` sql -SOURCE(EXECUTABLE(command 'cat /opt/dictionaries/os.tsv' format 'TabSeparated')) -``` - Поля настройки: -- `command` — Абсолютный путь к исполняемому файлу или имя файла (если каталог программы прописан в `PATH`). -- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе «[Форматы](../../../interfaces/formats.md#formats)». +- `command` — абсолютный путь к исполняемому файлу или имя файла (если каталог программы прописан в `PATH`). +- `format` — формат файла. Поддерживаются все форматы, описанные в разделе «[Форматы](../../../interfaces/formats.md#formats)». + +Этот источник словаря может быть настроен только с помощью XML-конфигурации. Создание словарей с исполняемым источником с помощью DDL отключено. Иначе пользователь базы данных сможет выполнить произвольный бинарный файл на узле ClickHouse. ## HTTP(s) {#dicts-external_dicts_dict_sources-http} @@ -160,7 +158,16 @@ SOURCE(HTTP( Поля настройки: - `url` — URL источника. -- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе «[Форматы](../../../interfaces/formats.md#formats)». +- `format` — формат файла. Поддерживаются все форматы, описанные в разделе «[Форматы](../../../interfaces/formats.md#formats)». +- `credentials` – базовая HTTP-аутентификация. Необязательный параметр. +- `user` – имя пользователя, необходимое для аутентификации. +- `password` – пароль, необходимый для аутентификации. +- `headers` – все пользовательские записи HTTP-заголовков, используемые для HTTP-запроса. Необязательный параметр. +- `header` – одна запись HTTP-заголовка. +- `name` – идентифицирующее имя, используемое для отправки заголовка запроса. +- `value` – значение, заданное для конкретного идентифицирующего имени. + +При создании словаря с помощью DDL-команды (`CREATE DICTIONARY ...`) удаленные хосты для HTTP-словарей проверяются в разделе `remote_url_allow_hosts` из конфигурации сервера. Иначе пользователи базы данных будут иметь доступ к произвольному HTTP-серверу. ## ODBC {#dicts-external_dicts_dict_sources-odbc} From cf3a89cd7b722d3ba00af7deec52aebac14a1b0d Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Mon, 18 Jan 2021 19:01:07 +0300 Subject: [PATCH 263/264] DOCSUP-4710: Added support numeric parameters in number and string data types (#18696) Co-authored-by: Alexander Kazakov --- docs/en/sql-reference/data-types/float.md | 17 +++++++--- docs/en/sql-reference/data-types/int-uint.md | 33 +++++++++++++------- docs/en/sql-reference/data-types/string.md | 2 ++ docs/ru/sql-reference/data-types/float.md | 19 +++++++---- docs/ru/sql-reference/data-types/int-uint.md | 33 +++++++++++++------- docs/ru/sql-reference/data-types/string.md | 2 ++ 6 files changed, 71 insertions(+), 35 deletions(-) diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 767001c51d2..1e3486cdae7 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -9,11 +9,18 @@ toc_title: Float32, Float64 Types are equivalent to types of C: -- `Float32` - `float` -- `Float64` - `double` +- `Float32` — `float`. +- `Float64` — `double`. We recommend that you store data in integer form whenever possible. For example, convert fixed precision numbers to integer values, such as monetary amounts or page load times in milliseconds. +Aliases: + +- `Float32` — `FLOAT`. +- `Float64` — `DOUBLE`. + +When creating tables, numeric parameters for floating point numbers can be set (e.g. `FLOAT(12)`, `FLOAT(15, 22)`, `DOUBLE(12)`, `DOUBLE(4, 18)`), but ClickHouse ignores them. + ## Using Floating-point Numbers {#using-floating-point-numbers} - Computations with floating-point numbers might produce a rounding error. @@ -52,7 +59,7 @@ SELECT 0.5 / 0 └────────────────┘ ``` -- `-Inf` – Negative infinity. +- `-Inf` — Negative infinity. @@ -66,7 +73,7 @@ SELECT -0.5 / 0 └─────────────────┘ ``` -- `NaN` – Not a number. +- `NaN` — Not a number. @@ -80,6 +87,6 @@ SELECT 0 / 0 └──────────────┘ ``` - See the rules for `NaN` sorting in the section [ORDER BY clause](../sql_reference/statements/select/order-by.md). +See the rules for `NaN` sorting in the section [ORDER BY clause](../../sql-reference/statements/select/order-by.md). [Original article](https://clickhouse.tech/docs/en/data_types/float/) diff --git a/docs/en/sql-reference/data-types/int-uint.md b/docs/en/sql-reference/data-types/int-uint.md index 2af855a340d..f0a706b0a37 100644 --- a/docs/en/sql-reference/data-types/int-uint.md +++ b/docs/en/sql-reference/data-types/int-uint.md @@ -7,23 +7,32 @@ toc_title: UInt8, UInt16, UInt32, UInt64, UInt256, Int8, Int16, Int32, Int64, In Fixed-length integers, with or without a sign. +When creating tables, numeric parameters for integer numbers can be set (e.g. `TINYINT(8)`, `SMALLINT(16)`, `INT(32)`, `BIGINT(64)`), but ClickHouse ignores them. + ## Int Ranges {#int-ranges} -- Int8 - \[-128 : 127\] -- Int16 - \[-32768 : 32767\] -- Int32 - \[-2147483648 : 2147483647\] -- Int64 - \[-9223372036854775808 : 9223372036854775807\] -- Int128 - \[-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727\] -- Int256 - \[-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967\] +- `Int8` — \[-128 : 127\] +- `Int16` — \[-32768 : 32767\] +- `Int32` — \[-2147483648 : 2147483647\] +- `Int64` — \[-9223372036854775808 : 9223372036854775807\] +- `Int128` — \[-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727\] +- `Int256` — \[-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967\] + +Aliases: + +- `Int8` — `TINYINT`, `BOOL`, `BOOLEAN`, `INT1`. +- `Int16` — `SMALLINT`, `INT2`. +- `Int32` — `INT`, `INT4`, `INTEGER`. +- `Int64` — `BIGINT`. ## Uint Ranges {#uint-ranges} -- UInt8 - \[0 : 255\] -- UInt16 - \[0 : 65535\] -- UInt32 - \[0 : 4294967295\] -- UInt64 - \[0 : 18446744073709551615\] -- UInt256 - \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\] +- `UInt8` — \[0 : 255\] +- `UInt16` — \[0 : 65535\] +- `UInt32` — \[0 : 4294967295\] +- `UInt64` — \[0 : 18446744073709551615\] +- `UInt256` — \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\] -UInt128 is not supported yet. +`UInt128` is not supported yet. [Original article](https://clickhouse.tech/docs/en/data_types/int_uint/) diff --git a/docs/en/sql-reference/data-types/string.md b/docs/en/sql-reference/data-types/string.md index a990bd0742b..42d8798c8a3 100644 --- a/docs/en/sql-reference/data-types/string.md +++ b/docs/en/sql-reference/data-types/string.md @@ -8,6 +8,8 @@ toc_title: String Strings of an arbitrary length. The length is not limited. The value can contain an arbitrary set of bytes, including null bytes. The String type replaces the types VARCHAR, BLOB, CLOB, and others from other DBMSs. +When creating tables, numeric parameters for string fields can be set (e.g. `VARCHAR(255)`), but ClickHouse ignores them. + ## Encodings {#encodings} ClickHouse doesn’t have the concept of encodings. Strings can contain an arbitrary set of bytes, which are stored and output as-is. diff --git a/docs/ru/sql-reference/data-types/float.md b/docs/ru/sql-reference/data-types/float.md index f2e85f35041..0e861f170b7 100644 --- a/docs/ru/sql-reference/data-types/float.md +++ b/docs/ru/sql-reference/data-types/float.md @@ -9,8 +9,15 @@ toc_title: Float32, Float64 Типы эквивалентны типам языка С: -- `Float32` - `float`; -- `Float64` - `double`. +- `Float32` — `float`. +- `Float64` — `double`. + +Синонимы: + +- `Float32` — `FLOAT`. +- `Float64` — `DOUBLE`. + +При создании таблиц для чисел с плавающей запятой можно указывать числовые параметры (например, `FLOAT(12)`, `FLOAT(15, 22)`, `DOUBLE(12)`, `DOUBLE(4, 18)`), но ClickHouse их проигнорирует. Рекомендуется хранить данные в целочисленном виде всегда, когда это возможно. Например, переводите в целочисленные значения числа с фиксированной точностью, такие как денежные суммы или времена загрузки страниц в миллисекундах. @@ -38,7 +45,7 @@ SELECT 1 - 0.9 В отличие от стандартного SQL, ClickHouse поддерживает следующие категории чисел с плавающей запятой: -- `Inf` - бесконечность. +- `Inf` — бесконечность. @@ -52,7 +59,7 @@ SELECT 0.5 / 0 └────────────────┘ ``` -- `-Inf` - отрицательная бесконечность; +- `-Inf` — отрицательная бесконечность. @@ -66,7 +73,7 @@ SELECT -0.5 / 0 └─────────────────┘ ``` -- `NaN` - не число. +- `NaN` — не число. @@ -80,6 +87,6 @@ SELECT 0 / 0 └──────────────┘ ``` - Смотрите правила сортировки `NaN` в разделе [Секция ORDER BY](../sql_reference/data_types/float.md). +Смотрите правила сортировки `NaN` в разделе [Секция ORDER BY ](../../sql-reference/statements/select/order-by.md). [Оригинальная статья](https://clickhouse.tech/docs/ru/data_types/float/) diff --git a/docs/ru/sql-reference/data-types/int-uint.md b/docs/ru/sql-reference/data-types/int-uint.md index c45c639aace..d3c342e467a 100644 --- a/docs/ru/sql-reference/data-types/int-uint.md +++ b/docs/ru/sql-reference/data-types/int-uint.md @@ -7,23 +7,32 @@ toc_title: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64 Целые числа фиксированной длины, без знака или со знаком. +При создании таблиц для целых чисел можно указывать числовые параметры (например `TINYINT(8)`, `SMALLINT(16)`, `INT(32)`, `BIGINT(64)`), но ClickHouse их проигнорирует. + ## Диапазоны Int {#int-ranges} -- Int8 - \[-128 : 127\] -- Int16 - \[-32768 : 32767\] -- Int32 - \[-2147483648 : 2147483647\] -- Int64 - \[-9223372036854775808 : 9223372036854775807\] -- Int128 - \[-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727\] -- Int256 - \[-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967\] +- `Int8` — \[-128 : 127\] +- `Int16` — \[-32768 : 32767\] +- `Int32` — \[-2147483648 : 2147483647\] +- `Int64` — \[-9223372036854775808 : 9223372036854775807\] +- `Int128` — \[-170141183460469231731687303715884105728 : 170141183460469231731687303715884105727\] +- `Int256` — \[-57896044618658097711785492504343953926634992332820282019728792003956564819968 : 57896044618658097711785492504343953926634992332820282019728792003956564819967\] + +Синонимы: + +- `Int8` — `TINYINT`, `BOOL`, `BOOLEAN`, `INT1`. +- `Int16` — `SMALLINT`, `INT2`. +- `Int32` — `INT`, `INT4`, `INTEGER`. +- `Int64` — `BIGINT`. ## Диапазоны Uint {#uint-ranges} -- UInt8 - \[0 : 255\] -- UInt16 - \[0 : 65535\] -- UInt32 - \[0 : 4294967295\] -- UInt64 - \[0 : 18446744073709551615\] -- UInt256 - \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\] +- `UInt8` — \[0 : 255\] +- `UInt16` — \[0 : 65535\] +- `UInt32` — \[0 : 4294967295\] +- `UInt64` — \[0 : 18446744073709551615\] +- `UInt256` — \[0 : 115792089237316195423570985008687907853269984665640564039457584007913129639935\] -UInt128 пока не реализован. +`UInt128` пока не реализован. [Оригинальная статья](https://clickhouse.tech/docs/ru/data_types/int_uint/) diff --git a/docs/ru/sql-reference/data-types/string.md b/docs/ru/sql-reference/data-types/string.md index 798caec4d62..6a07f7e51de 100644 --- a/docs/ru/sql-reference/data-types/string.md +++ b/docs/ru/sql-reference/data-types/string.md @@ -8,6 +8,8 @@ toc_title: String Строки произвольной длины. Длина не ограничена. Значение может содержать произвольный набор байт, включая нулевые байты. Таким образом, тип String заменяет типы VARCHAR, BLOB, CLOB и т. п. из других СУБД. +При создании таблиц для строк можно указывать числовые параметры (например `VARCHAR(255)`), но СlickHouse их проигнорирует. + ## Кодировки {#kodirovki} В ClickHouse нет понятия кодировок. Строки могут содержать произвольный набор байт, который хранится и выводится, как есть. From c7e93a9c74c8c53f42fcbea509ceb166c82a95ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 18 Jan 2021 20:58:31 +0300 Subject: [PATCH 264/264] Prepare changelog --- CHANGELOG.md | 418 ++++++++++------------ utils/simple-backport/format-changelog.py | 2 +- 2 files changed, 197 insertions(+), 223 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index fbaf92ce659..8da4919c975 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,313 +1,287 @@ -622 PRs added between v20.12.5.14-stable and v21.1.2.15-stable. -curl: (22) The requested URL returned error: 404 Not Found -Failed to download 'https://api.github.com/repos/ClickHouse/ClickHouse/pulls/18701' to 'pr18701.json'. Contents: ''. -curl: (28) Failed to connect to api.github.com port 443: Connection timed out -Failed to download 'https://api.github.com/repos/ClickHouse/ClickHouse/pulls/17832' to 'pr17832.json'. Contents: ''. -/home/milovidov/.local/lib/python3.8/site-packages/fuzzywuzzy/fuzz.py:11: UserWarning: Using slow pure-python SequenceMatcher. Install python-Levenshtein to remove this warning - warnings.warn('Using slow pure-python SequenceMatcher. Install python-Levenshtein to remove this warning') -### ClickHouse release v21.1.2.15-stable FIXME as compared to v20.12.5.14-stable +## ClickHouse release 21.1 + +### ClickHouse release v21.1.2.15-stable 2021-01-18 #### Backward Incompatible Change -* Remove `sumburConsistentHash` function. This closes [#18120](https://github.com/ClickHouse/ClickHouse/issues/18120). [#18656](https://github.com/ClickHouse/ClickHouse/pull/18656) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* The setting `input_format_null_as_default` is enabled by default. [#17525](https://github.com/ClickHouse/ClickHouse/pull/17525) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Check settings constraints for profile settings from config. Server will fail to start if users.xml contain settings that do not meet corresponding constraints. [#18486](https://github.com/ClickHouse/ClickHouse/pull/18486) ([tavplubix](https://github.com/tavplubix)). * Restrict `ALTER MODIFY SETTING` from changing storage settings that affects data parts (`write_final_mark` and `enable_mixed_granularity_parts`). [#18306](https://github.com/ClickHouse/ClickHouse/pull/18306) ([Amos Bird](https://github.com/amosbird)). * Set `insert_quorum_parallel` to 1 by default. It is significantly more convenient to use than "sequential" quorum inserts. But if you rely to sequential consistency, you should set the setting back to zero. [#17567](https://github.com/ClickHouse/ClickHouse/pull/17567) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* The setting `input_format_null_as_default` is enabled by default. [#17525](https://github.com/ClickHouse/ClickHouse/pull/17525) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Remove `sumburConsistentHash` function. This closes [#18120](https://github.com/ClickHouse/ClickHouse/issues/18120). [#18656](https://github.com/ClickHouse/ClickHouse/pull/18656) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Removed aggregate functions `timeSeriesGroupSum`, `timeSeriesGroupRateSum` because a friend of mine said they never worked. This fixes [#16869](https://github.com/ClickHouse/ClickHouse/issues/16869). If you have luck using these functions, write a email to clickhouse-feedback@yandex-team.com. [#17423](https://github.com/ClickHouse/ClickHouse/pull/17423) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Prohibit toUnixTimestamp(Date()) (before it just returns UInt16 representation of Date). [#17376](https://github.com/ClickHouse/ClickHouse/pull/17376) ([Azat Khuzhin](https://github.com/azat)). * Allow using extended integer types (`Int128`, `Int256`, `UInt256`) in `avg` and `avgWeighted` functions. Also allow using different types (integer, decimal, floating point) for value and for weight in `avgWeighted` function. This is a backward-incompatible change: now the `avg` and `avgWeighted` functions always return `Float64` (as documented). Before this change the return type for `Decimal` arguments was also `Decimal`. [#15419](https://github.com/ClickHouse/ClickHouse/pull/15419) ([Mike](https://github.com/myrrc)). #### New Feature -* Function `position` now supports `position(needle in haystack)` synax for SQL compatibility. This closes [#18701](https://github.com/ClickHouse/ClickHouse/issues/18701). ... [#18779](https://github.com/ClickHouse/ClickHouse/pull/18779) ([Jianmei Zhang](https://github.com/zhangjmruc)). -* Now we have a new storage setting `max_partitions_to_read` for tables in the MergeTree family. It limits the max number of partitions that can be accessed in one query. A user setting `force_max_partition_limit` is also added to enforce this constraint. [#18712](https://github.com/ClickHouse/ClickHouse/pull/18712) ([Amos Bird](https://github.com/amosbird)). -* Add `query_id` column to `system.part_log` for inserted parts. closes [#10097](https://github.com/ClickHouse/ClickHouse/issues/10097). [#18644](https://github.com/ClickHouse/ClickHouse/pull/18644) ([flynn](https://github.com/ucasFL)). +* Implement gRPC protocol in ClickHouse. [#15111](https://github.com/ClickHouse/ClickHouse/pull/15111) ([Vitaly Baranov](https://github.com/vitlibar)). +* Allow to use multiple zookeeper clusters. [#17070](https://github.com/ClickHouse/ClickHouse/pull/17070) ([fastio](https://github.com/fastio)). * Implemented `REPLACE TABLE` and `CREATE OR REPLACE TABLE` queries. [#18521](https://github.com/ClickHouse/ClickHouse/pull/18521) ([tavplubix](https://github.com/tavplubix)). -* - IP Dictionary supports key fetching. Resolves [#18241](https://github.com/ClickHouse/ClickHouse/issues/18241). [#18480](https://github.com/ClickHouse/ClickHouse/pull/18480) ([vdimir](https://github.com/vdimir)). -* Allow create table as select with columns specification. Example `CREATE TABLE t1 (x String) ENGINE = Memory AS SELECT 1;`. [#18060](https://github.com/ClickHouse/ClickHouse/pull/18060) ([Maksim Kita](https://github.com/kitaisreal)). +* Implement `UNION DISTINCT` and treat the plain `UNION` clause as `UNION DISTINCT` by default. Add a setting `union_default_mode` that allows to treat it as `UNION ALL` or require explicit mode specification. [#16338](https://github.com/ClickHouse/ClickHouse/pull/16338) ([flynn](https://github.com/ucasFL)). +* Added function `accurateCastOrNull`. This closes [#10290](https://github.com/ClickHouse/ClickHouse/issues/10290). Add type conversions in `x IN (subquery)` expressions. This closes [#10266](https://github.com/ClickHouse/ClickHouse/issues/10266). [#16724](https://github.com/ClickHouse/ClickHouse/pull/16724) ([Maksim Kita](https://github.com/kitaisreal)). +* IP Dictionary supports `IPv4` / `IPv6` types directly. [#17571](https://github.com/ClickHouse/ClickHouse/pull/17571) ([vdimir](https://github.com/vdimir)). +* IP Dictionary supports key fetching. Resolves [#18241](https://github.com/ClickHouse/ClickHouse/issues/18241). [#18480](https://github.com/ClickHouse/ClickHouse/pull/18480) ([vdimir](https://github.com/vdimir)). +* Add `*.zst` compression/decompression support for data import and export. It enables using `*.zst` in `file()` function and `Content-encoding: zstd` in HTTP client. This closes [#16791 ](https://github.com/ClickHouse/ClickHouse/issues/16791). [#17144](https://github.com/ClickHouse/ClickHouse/pull/17144) ([Abi Palagashvili](https://github.com/fibersel)). +* Added `mannWitneyUTest`, `studentTTest` and `welchTTest` aggregate functions. Refactored `rankCorr` a bit. [#16883](https://github.com/ClickHouse/ClickHouse/pull/16883) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add functions `countMatches`/`countMatchesCaseInsensitive`. [#17459](https://github.com/ClickHouse/ClickHouse/pull/17459) ([Azat Khuzhin](https://github.com/azat)). +* Implement `countSubstrings()`/`countSubstringsCaseInsensitive()`/`countSubstringsCaseInsensitiveUTF8()` (Count the number of substring occurrences). [#17347](https://github.com/ClickHouse/ClickHouse/pull/17347) ([Azat Khuzhin](https://github.com/azat)). +* Add information about used databases, tables and columns in system.query_log. Add `query_kind` and `normalized_query_hash` fields. [#17726](https://github.com/ClickHouse/ClickHouse/pull/17726) ([Amos Bird](https://github.com/amosbird)). +* Add a setting `optimize_on_insert`. When enabled, do the same transformation for INSERTed block of data as if merge was done on this block (e.g. Replacing, Collapsing, Aggregating...). This setting is enabled by default. This can influence Materialized View and MaterializeMySQL behaviour (see detailed description). This closes [#10683](https://github.com/ClickHouse/ClickHouse/issues/10683). [#16954](https://github.com/ClickHouse/ClickHouse/pull/16954) ([Kruglov Pavel](https://github.com/Avogar)). +* Kerberos Authenticaiton for HDFS. [#16621](https://github.com/ClickHouse/ClickHouse/pull/16621) ([Ilya Golshtein](https://github.com/ilejn)). * Support `SHOW SETTINGS` statement to show parameters in system.settings. `SHOW CHANGED SETTINGS` and `LIKE/ILIKE` clause are also supported. [#18056](https://github.com/ClickHouse/ClickHouse/pull/18056) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* Function `position` now supports `POSITION(needle IN haystack)` synax for SQL compatibility. This closes [#18701](https://github.com/ClickHouse/ClickHouse/issues/18701). ... [#18779](https://github.com/ClickHouse/ClickHouse/pull/18779) ([Jianmei Zhang](https://github.com/zhangjmruc)). +* Now we have a new storage setting `max_partitions_to_read` for tables in the MergeTree family. It limits the max number of partitions that can be accessed in one query. A user setting `force_max_partition_limit` is also added to enforce this constraint. [#18712](https://github.com/ClickHouse/ClickHouse/pull/18712) ([Amos Bird](https://github.com/amosbird)). +* Add `query_id` column to `system.part_log` for inserted parts. Closes [#10097](https://github.com/ClickHouse/ClickHouse/issues/10097). [#18644](https://github.com/ClickHouse/ClickHouse/pull/18644) ([flynn](https://github.com/ucasFL)). +* Allow create table as select with columns specification. Example `CREATE TABLE t1 (x String) ENGINE = Memory AS SELECT 1;`. [#18060](https://github.com/ClickHouse/ClickHouse/pull/18060) ([Maksim Kita](https://github.com/kitaisreal)). * Added `arrayMin`, `arrayMax`, `arrayAvg` aggregation functions. [#18032](https://github.com/ClickHouse/ClickHouse/pull/18032) ([Maksim Kita](https://github.com/kitaisreal)). * Implemented `ATTACH TABLE name FROM 'path/to/data/' (col1 Type1, ...` query. It creates new table with provided structure and attaches table data from provided directory in `user_files`. [#17903](https://github.com/ClickHouse/ClickHouse/pull/17903) ([tavplubix](https://github.com/tavplubix)). +* Add mutation support for StorageMemory. This closes [#9117](https://github.com/ClickHouse/ClickHouse/issues/9117). [#15127](https://github.com/ClickHouse/ClickHouse/pull/15127) ([flynn](https://github.com/ucasFL)). +* Support syntax `EXISTS DATABASE name`. [#18458](https://github.com/ClickHouse/ClickHouse/pull/18458) ([Du Chuan](https://github.com/spongedu)). +* Support builtin function `isIPv4String` && `isIPv6String` like [MySQL](https://github.com/ClickHouse/ClickHouse/compare/master...spongedu:support_is_ipv4?expand=1). [#18349](https://github.com/ClickHouse/ClickHouse/pull/18349) ([Du Chuan](https://github.com/spongedu)). +* Add a new setting `insert_distributed_one_random_shard = 1` to allow insertion into multi-sharded distributed table without any distributed key. [#18294](https://github.com/ClickHouse/ClickHouse/pull/18294) ([Amos Bird](https://github.com/amosbird)). * Add settings `min_compress_block_size` and `max_compress_block_size` to MergeTreeSettings, which have higher priority than the global settings and take effect when they are set. close [13890](https://github.com/ClickHouse/ClickHouse/issues/13890). [#17867](https://github.com/ClickHouse/ClickHouse/pull/17867) ([flynn](https://github.com/ucasFL)). -* Add bitmap64 feature. [#17858](https://github.com/ClickHouse/ClickHouse/pull/17858) ([Andy Yang](https://github.com/andyyzh)). +* Add support for 64bit roaring bitmaps. [#17858](https://github.com/ClickHouse/ClickHouse/pull/17858) ([Andy Yang](https://github.com/andyyzh)). * Extended `OPTIMIZE ... DEDUPLICATE` syntax to allow explicit (or implicit with asterisk/column transformers) list of columns to check for duplicates on. ... [#17846](https://github.com/ClickHouse/ClickHouse/pull/17846) ([Vasily Nemkov](https://github.com/Enmk)). -* Added functions `toMJD`, `fromMJD`, `toMJDOrNull`, and `fromMJDOrNull`. These functions convert between Proleptic Gregorian calendar date and Modified Julian Day number. [#17750](https://github.com/ClickHouse/ClickHouse/pull/17750) ([PHO](https://github.com/depressed-pho)). -* Add ability to use custom TLD list in functions `firstSignificantSubdomainCustom`, `cutToFirstSignificantSubdomainCustom`. [#17748](https://github.com/ClickHouse/ClickHouse/pull/17748) ([Azat Khuzhin](https://github.com/azat)). -* Add support for PROXYv1 protocol to wrap native TCP interface. Allow quotas to be keyed by proxy-forwarded IP address (applied for PROXYv1 address and for X-Forwarded-For from HTTP interface). This is useful when you provide access to ClickHouse only via trusted proxy (e.g. CloudFlare) but want to account user resources by their original IP addresses. This fixes [#17268](https://github.com/ClickHouse/ClickHouse/issues/17268). [#17707](https://github.com/ClickHouse/ClickHouse/pull/17707) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now clickhouse-client supports opening EDITOR to edit commands. `Alt-Shift-E`. [#17665](https://github.com/ClickHouse/ClickHouse/pull/17665) ([Amos Bird](https://github.com/amosbird)). +* Added functions `toModifiedJulianDay`, `fromModifiedJulianDay`, `toModifiedJulianDayOrNull`, and `fromModifiedJulianDayOrNull`. These functions convert between Proleptic Gregorian calendar date and Modified Julian Day number. [#17750](https://github.com/ClickHouse/ClickHouse/pull/17750) ([PHO](https://github.com/depressed-pho)). +* Add ability to use custom TLD list: added functions `firstSignificantSubdomainCustom`, `cutToFirstSignificantSubdomainCustom`. [#17748](https://github.com/ClickHouse/ClickHouse/pull/17748) ([Azat Khuzhin](https://github.com/azat)). +* Add support for `PROXYv1` protocol to wrap native TCP interface. Allow quotas to be keyed by proxy-forwarded IP address (applied for `PROXYv1` address and for `X-Forwarded-For` from HTTP interface). This is useful when you provide access to ClickHouse only via trusted proxy (e.g. CloudFlare) but want to account user resources by their original IP addresses. This fixes [#17268](https://github.com/ClickHouse/ClickHouse/issues/17268). [#17707](https://github.com/ClickHouse/ClickHouse/pull/17707) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now clickhouse-client supports opening `EDITOR` to edit commands. `Alt-Shift-E`. [#17665](https://github.com/ClickHouse/ClickHouse/pull/17665) ([Amos Bird](https://github.com/amosbird)). * Add function `encodeXMLComponent` to escape characters to place string into XML text node or attribute. [#17659](https://github.com/ClickHouse/ClickHouse/pull/17659) ([nauta](https://github.com/nautaa)). -* Introduce `DETACH TABLE/VIEW ... PERMANENTLY` syntax, so that after restarting the table does not reappear back automatically (only by explicit request). The table can still be attached back using the short syntax ATTACH TABLE. Implements [#5555](https://github.com/ClickHouse/ClickHouse/issues/5555). Fixes [#13850](https://github.com/ClickHouse/ClickHouse/issues/13850). [#17642](https://github.com/ClickHouse/ClickHouse/pull/17642) ([filimonov](https://github.com/filimonov)). +* Introduce `DETACH TABLE/VIEW ... PERMANENTLY` syntax, so that after restarting the table does not reappear back automatically on restart (only by explicit request). The table can still be attached back using the short syntax ATTACH TABLE. Implements [#5555](https://github.com/ClickHouse/ClickHouse/issues/5555). Fixes [#13850](https://github.com/ClickHouse/ClickHouse/issues/13850). [#17642](https://github.com/ClickHouse/ClickHouse/pull/17642) ([filimonov](https://github.com/filimonov)). * Add asynchronous metrics on total amount of rows, bytes and parts in MergeTree tables. This fix [#11714](https://github.com/ClickHouse/ClickHouse/issues/11714). [#17639](https://github.com/ClickHouse/ClickHouse/pull/17639) ([flynn](https://github.com/ucasFL)). -* related: [#16176](https://github.com/ClickHouse/ClickHouse/issues/16176) Usage: ``` set limit = 10; set offset = 20; ``` this two settings will affect SELECT query as if it is added like ``` select * from ($your_original_select_query) tmp limit xxx offset xxx; ```. [#17633](https://github.com/ClickHouse/ClickHouse/pull/17633) ([hexiaoting](https://github.com/hexiaoting)). -* * IP Dictionary supports `IPv4` / `IPv6` types directly. [#17571](https://github.com/ClickHouse/ClickHouse/pull/17571) ([vdimir](https://github.com/vdimir)). -* add ```*.zst``` compression/decompression support.It enables using ```*.zst``` in ```file()``` function and ```Content-encoding: zstd``` in http client.This closes [#16791 ](https://github.com/ClickHouse/ClickHouse/issues/16791). [#17144](https://github.com/ClickHouse/ClickHouse/pull/17144) ([Abi Palagashvili](https://github.com/fibersel)). -* Add a setting optimize_on_insert. When enabled, do the same transformation for INSERTed block of data as if merge was done on this block (e.g. Replacing, Collapsing, Aggregating...). This setting will be enabled as default. This can influence Materialized View and MaterializeMySQL behaviour (see detailed description). This closes [#10683](https://github.com/ClickHouse/ClickHouse/issues/10683). [#16954](https://github.com/ClickHouse/ClickHouse/pull/16954) ([Kruglov Pavel](https://github.com/Avogar)). -* Added `mannWitneyUTest`, `studentTTest` and `welchTTest` aggregate functions. Refactored RankCorr a bit. [#16883](https://github.com/ClickHouse/ClickHouse/pull/16883) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Provide a new aggregator combinator : `-SimpleState` to build SimpleAggregateFunction types via query. It's useful for defining MaterializedView of AggregatingMergeTree engine, and will benefit projections too. [#16853](https://github.com/ClickHouse/ClickHouse/pull/16853) ([Amos Bird](https://github.com/amosbird)). -* ... [#16819](https://github.com/ClickHouse/ClickHouse/pull/16819) ([Aleksandrov Vladimir](https://github.com/invis87)). -* Use https://github.com/lemire/fast_float to parse floating point numbers. [#16787](https://github.com/ClickHouse/ClickHouse/pull/16787) ([Maksim Kita](https://github.com/kitaisreal)). -* Added function `accurateCastOrNull`. This closes [#10290](https://github.com/ClickHouse/ClickHouse/issues/10290). Add type conversions in `x IN (subquery)` expressions. This closes [#10266](https://github.com/ClickHouse/ClickHouse/issues/10266). [#16724](https://github.com/ClickHouse/ClickHouse/pull/16724) ([Maksim Kita](https://github.com/kitaisreal)). -* Kerberos Authenticaiton for HDFS. [#16621](https://github.com/ClickHouse/ClickHouse/pull/16621) ([Ilya Golshtein](https://github.com/ilejn)). -* Implement `UNION DISTINCT` and treat the plain `UNION` clause as `UNION DISTINCT` by default. Add a setting `union_default_mode` that allows to treat it as `UNION ALL` or require explicit mode specification. [#16338](https://github.com/ClickHouse/ClickHouse/pull/16338) ([flynn](https://github.com/ucasFL)). -* Added `queries-file` parameter for clickhouse-client and clickhouse-local. [#15930](https://github.com/ClickHouse/ClickHouse/pull/15930) ([Maksim Kita](https://github.com/kitaisreal)). -* Add new datatype Map for supporting storage k:v . related to issue: [#1841](https://github.com/ClickHouse/ClickHouse/issues/1841) 1st version for Map only support String type of key and value. Later I will implement Int and other type for key and value. [#15806](https://github.com/ClickHouse/ClickHouse/pull/15806) ([hexiaoting](https://github.com/hexiaoting)). -* Implement gRPC protocol in ClickHouse. [#15111](https://github.com/ClickHouse/ClickHouse/pull/15111) ([Vitaly Baranov](https://github.com/vitlibar)). -* - parallel parsing was rewritten to processors - parallel formatting was implemented. [#11617](https://github.com/ClickHouse/ClickHouse/pull/11617) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Added functions `Simhash`, `Minhash`, `bitHammingDistance`, `tupleHammingDistance`. [#7649](https://github.com/ClickHouse/ClickHouse/pull/7649) ([flynn](https://github.com/ucasFL)). +* Add settings `limit` and `offset` for out-of-SQL pagination: [#16176](https://github.com/ClickHouse/ClickHouse/issues/16176) They are useful for building APIs. These two settings will affect SELECT query as if it is added like `select * from (your_original_select_query) t limit xxx offset xxx;`. [#17633](https://github.com/ClickHouse/ClickHouse/pull/17633) ([hexiaoting](https://github.com/hexiaoting)). +* Provide a new aggregator combinator : `-SimpleState` to build `SimpleAggregateFunction` types via query. It's useful for defining MaterializedView of AggregatingMergeTree engine, and will benefit projections too. [#16853](https://github.com/ClickHouse/ClickHouse/pull/16853) ([Amos Bird](https://github.com/amosbird)). +* Added `queries-file` parameter for `clickhouse-client` and `clickhouse-local`. [#15930](https://github.com/ClickHouse/ClickHouse/pull/15930) ([Maksim Kita](https://github.com/kitaisreal)). +* Added `query` parameter for `clickhouse-benchmark`. [#17832](https://github.com/ClickHouse/ClickHouse/pull/17832) ([Maksim Kita](https://github.com/kitaisreal)). +* `EXPLAIN AST` now support queries other then `SELECT`. [#18136](https://github.com/ClickHouse/ClickHouse/pull/18136) ([taiyang-li](https://github.com/taiyang-li)). + + +#### Experimental Feature + +* Added functions for calculation of minHash and simHash of text n-grams and shingles. They are intended for semi-duplicate search. Also functions `bitHammingDistance` and `tupleHammingDistance` are added. [#7649](https://github.com/ClickHouse/ClickHouse/pull/7649) ([flynn](https://github.com/ucasFL)). +* Add new data type `Map`. See [#1841](https://github.com/ClickHouse/ClickHouse/issues/1841). First version for Map only supports `String` type of key and value. [#15806](https://github.com/ClickHouse/ClickHouse/pull/15806) ([hexiaoting](https://github.com/hexiaoting)). +* Implement alternative SQL parser based on ANTLR4 runtime and generated from EBNF grammar. [#11298](https://github.com/ClickHouse/ClickHouse/pull/11298) ([Ivan](https://github.com/abyss7)). + + +#### Performance Improvement + +* New IP Dictionary implementation with lower memory consumption, improved performance for some cases, and fixed bugs. [#16804](https://github.com/ClickHouse/ClickHouse/pull/16804) ([vdimir](https://github.com/vdimir)). +* Parallel formatting for data export. [#11617](https://github.com/ClickHouse/ClickHouse/pull/11617) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* LDAP integration: Added `verification_cooldown` parameter in LDAP server connection configuration to allow caching of successful "bind" attempts for configurable period of time. [#15988](https://github.com/ClickHouse/ClickHouse/pull/15988) ([Denis Glazachev](https://github.com/traceon)). +* Add `--no-system-table` option for `clickhouse-local` to run without system tables. This avoids initialization of `DateLUT` that may take noticeable amount of time (tens of milliseconds) at startup. [#18899](https://github.com/ClickHouse/ClickHouse/pull/18899) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Replace `PODArray` with `PODArrayWithStackMemory` in `AggregateFunctionWindowFunnelData` to improve `windowFunnel` function performance. [#18817](https://github.com/ClickHouse/ClickHouse/pull/18817) ([flynn](https://github.com/ucasFL)). +* Don't send empty blocks to shards on synchronous INSERT into Distributed table. This closes [#14571](https://github.com/ClickHouse/ClickHouse/issues/14571). [#18775](https://github.com/ClickHouse/ClickHouse/pull/18775) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Optimized read for StorageMemory. [#18052](https://github.com/ClickHouse/ClickHouse/pull/18052) ([Maksim Kita](https://github.com/kitaisreal)). +* Using Dragonbox algorithm for float to string conversion instead of ryu. This improves performance of float to string conversion significantly. [#17831](https://github.com/ClickHouse/ClickHouse/pull/17831) ([Maksim Kita](https://github.com/kitaisreal)). +* Speedup `IPv6CIDRToRange` implementation. [#17569](https://github.com/ClickHouse/ClickHouse/pull/17569) ([vdimir](https://github.com/vdimir)). +* Add `remerge_sort_lowered_memory_bytes_ratio` setting (If memory usage after remerge does not reduced by this ratio, remerge will be disabled). [#17539](https://github.com/ClickHouse/ClickHouse/pull/17539) ([Azat Khuzhin](https://github.com/azat)). +* Improve performance of AggregatingMergeTree with SimpleAggregateFunction(String) in PK. [#17109](https://github.com/ClickHouse/ClickHouse/pull/17109) ([Azat Khuzhin](https://github.com/azat)). +* Now the `-If` combinator is devirtualized, and `count` is properly vectorized. It is for [this PR](https://github.com/ClickHouse/ClickHouse/pull/17041). [#17043](https://github.com/ClickHouse/ClickHouse/pull/17043) ([Amos Bird](https://github.com/amosbird)). +* Fix performance of reading from `Merge` tables over huge number of `MergeTree` tables. Fixes [#7748](https://github.com/ClickHouse/ClickHouse/issues/7748). [#16988](https://github.com/ClickHouse/ClickHouse/pull/16988) ([Anton Popov](https://github.com/CurtizJ)). +* Improved performance of function `repeat`. [#16937](https://github.com/ClickHouse/ClickHouse/pull/16937) ([satanson](https://github.com/satanson)). +* Slightly improved performance of float parsing. [#16809](https://github.com/ClickHouse/ClickHouse/pull/16809) ([Maksim Kita](https://github.com/kitaisreal)). +* Add possibility to skip merged partitions for `OPTIMIZE TABLE ... FINAL`. [#15939](https://github.com/ClickHouse/ClickHouse/pull/15939) ([Kruglov Pavel](https://github.com/Avogar)). +* Integrate with [fast_float from Daniel Lemire](https://github.com/lemire/fast_float) to parse floating point numbers. [#16787](https://github.com/ClickHouse/ClickHouse/pull/16787) ([Maksim Kita](https://github.com/kitaisreal)). It is not enabled, because performance its performance is still lower than rough float parser in ClickHouse. +* Fix max_distributed_connections (affects `prefer_localhost_replica = 1` and `max_threads != max_distributed_connections`). [#17848](https://github.com/ClickHouse/ClickHouse/pull/17848) ([Azat Khuzhin](https://github.com/azat)). +* Adaptive choice of single/multi part upload when sending data to S3. Single part upload is controlled by a new setting `max_single_part_upload_size`. [#17934](https://github.com/ClickHouse/ClickHouse/pull/17934) ([Pavel Kovalenko](https://github.com/Jokser)). +* Support for async tasks in `PipelineExecutor`. Initial support of async sockets for remote queries. [#17868](https://github.com/ClickHouse/ClickHouse/pull/17868) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Allow to use `optimize_move_to_prewhere` optimization with compact parts, when sizes of columns are unknown. [#17330](https://github.com/ClickHouse/ClickHouse/pull/17330) ([Anton Popov](https://github.com/CurtizJ)). + + +#### Improvement + +* Avoid deadlock when executing INSERT SELECT into itself from a table with `TinyLog` or `Log` table engines. This closes [#6802](https://github.com/ClickHouse/ClickHouse/issues/6802). This closes [#18691](https://github.com/ClickHouse/ClickHouse/issues/18691). This closes [#16812](https://github.com/ClickHouse/ClickHouse/issues/16812). This closes [#14570](https://github.com/ClickHouse/ClickHouse/issues/14570). [#15260](https://github.com/ClickHouse/ClickHouse/pull/15260) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Support `SHOW CREATE VIEW name` syntax like [MySQL](https://dev.mysql.com/doc/refman/5.7/en/show-create-view.html). [#18095](https://github.com/ClickHouse/ClickHouse/pull/18095) ([Du Chuan](https://github.com/spongedu)). +* All queries of type `Decimal * Float` or vice versa are allowed, including aggregate ones (e.g. `SELECT sum(decimal_field * 1.1)` or `SELECT dec_col * float_col`), the result type is Float32 or Float64. [#18145](https://github.com/ClickHouse/ClickHouse/pull/18145) ([Mike](https://github.com/myrrc)). +* Improved minimal Web UI: add history; add sharing support; avoid race condition of different requests; add request in-flight and ready indicators; add favicon; detect Ctrl+Enter if textarea is not in focus. [#17293](https://github.com/ClickHouse/ClickHouse/pull/17293) [#17770](https://github.com/ClickHouse/ClickHouse/pull/17770) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* clickhouse-server didn't send `close` request to ZooKeeper server. [#16837](https://github.com/ClickHouse/ClickHouse/pull/16837) ([alesapin](https://github.com/alesapin)). +* Avoid server abnormal termination in case of too low memory limits (`max_memory_usage = 1` / `max_untracked_memory = 1`). [#17453](https://github.com/ClickHouse/ClickHouse/pull/17453) ([Azat Khuzhin](https://github.com/azat)). +* Fix non-deterministic result of `windowFunnel` function in case of same timestamp for different events. [#18884](https://github.com/ClickHouse/ClickHouse/pull/18884) ([Fuwang Hu](https://github.com/fuwhu)). +* Docker: Explicitly set uid / gid of clickhouse user & group to the fixed values (101) in clickhouse-server Docker images. [#19096](https://github.com/ClickHouse/ClickHouse/pull/19096) ([filimonov](https://github.com/filimonov)). +* Asynchronous INSERTs to `Distributed` tables: Two new settings (by analogy with MergeTree family) has been added: - `fsync_after_insert` - Do fsync for every inserted. Will decreases performance of inserts. - `fsync_directories` - Do fsync for temporary directory (that is used for async INSERT only) after all operations (writes, renames, etc.). [#18864](https://github.com/ClickHouse/ClickHouse/pull/18864) ([Azat Khuzhin](https://github.com/azat)). +* `SYSTEM KILL` command started to work in Docker. This closes [#18847](https://github.com/ClickHouse/ClickHouse/issues/18847). [#18848](https://github.com/ClickHouse/ClickHouse/pull/18848) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Expand macros in the zk path when executing `FETCH PARTITION`. [#18839](https://github.com/ClickHouse/ClickHouse/pull/18839) ([fastio](https://github.com/fastio)). +* Apply `ALTER TABLE ON CLUSTER MODIFY SETTING ...` to all replicas. Because we don't replicate such alter commands. [#18789](https://github.com/ClickHouse/ClickHouse/pull/18789) ([Amos Bird](https://github.com/amosbird)). +* Allow column transformer `EXCEPT` to accept a string as regular expression matcher. This resolves [#18685](https://github.com/ClickHouse/ClickHouse/issues/18685) . [#18699](https://github.com/ClickHouse/ClickHouse/pull/18699) ([Amos Bird](https://github.com/amosbird)). +* Fix SimpleAggregateFunction in SummingMergeTree. Now it works like AggregateFunction. In previous versions values were summed together regardless to the aggregate function. This fixes [#18564](https://github.com/ClickHouse/ClickHouse/issues/18564) . [#8052](https://github.com/ClickHouse/ClickHouse/issues/8052). [#18637](https://github.com/ClickHouse/ClickHouse/pull/18637) ([Amos Bird](https://github.com/amosbird)). Another fix of using `SimpleAggregateFunction` in `SummingMergeTree`. This fixes [#18676](https://github.com/ClickHouse/ClickHouse/issues/18676) . [#18677](https://github.com/ClickHouse/ClickHouse/pull/18677) ([Amos Bird](https://github.com/amosbird)). +* Fixed assertion error inside allocator in case when last argument of function bar is NaN. Now simple ClickHouse's exception is being thrown. This fixes [#17876](https://github.com/ClickHouse/ClickHouse/issues/17876). [#18520](https://github.com/ClickHouse/ClickHouse/pull/18520) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix usability issue: no newline after exception message in some tools. [#18444](https://github.com/ClickHouse/ClickHouse/pull/18444) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Add ability to modify primary and partition key column type from `LowCardinality(Type)` to `Type` and vice versa. Also add an ability to modify primary key column type from `EnumX ` to `IntX` type. Fixes [#5604](https://github.com/ClickHouse/ClickHouse/issues/5604). [#18362](https://github.com/ClickHouse/ClickHouse/pull/18362) ([alesapin](https://github.com/alesapin)). +* Implement `untuple` field access. [#18133](https://github.com/ClickHouse/ClickHouse/issues/18133). [#18309](https://github.com/ClickHouse/ClickHouse/pull/18309) ([hexiaoting](https://github.com/hexiaoting)). +* Allow to parse Array fields from CSV if it is represented as a string containing array that was serialized as nested CSV. Example: `"[""Hello"", ""world"", ""42"""" TV""]"` will parse as `['Hello', 'world', '42" TV']`. Allow to parse array in CSV in a string without enclosing braces. Example: `"'Hello', 'world', '42"" TV'"` will parse as `['Hello', 'world', '42" TV']`. [#18271](https://github.com/ClickHouse/ClickHouse/pull/18271) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make better adaptive granularity calculation for merge tree wide parts. [#18223](https://github.com/ClickHouse/ClickHouse/pull/18223) ([alesapin](https://github.com/alesapin)). +* Now `clickhouse install` could work on Mac. The problem was that there is no procfs on this platform. [#18201](https://github.com/ClickHouse/ClickHouse/pull/18201) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Better hints for `SHOW ...` query syntax. [#18183](https://github.com/ClickHouse/ClickHouse/pull/18183) ([Du Chuan](https://github.com/spongedu)). +* Array aggregation `arrayMin`, `arrayMax`, `arraySum`, `arrayAvg` support for `Int128`, `Int256`, `UInt256`. [#18147](https://github.com/ClickHouse/ClickHouse/pull/18147) ([Maksim Kita](https://github.com/kitaisreal)). +* Add `disk` to Set and Join storage settings. [#18112](https://github.com/ClickHouse/ClickHouse/pull/18112) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). +* Access control: Now table function `merge()` requires current user to have `SELECT` privilege on each table it receives data from. This PR fixes [#16964](https://github.com/ClickHouse/ClickHouse/issues/16964). [#18104](https://github.com/ClickHouse/ClickHouse/pull/18104) [#17983](https://github.com/ClickHouse/ClickHouse/pull/17983) ([Vitaly Baranov](https://github.com/vitlibar)). +* Temporary tables are visible in the system tables `system.tables` and `system.columns` now only in those session where they have been created. The internal database `_temporary_and_external_tables` is now hidden in those system tables; temporary tables are shown as tables with empty database with the `is_temporary` flag set instead. [#18014](https://github.com/ClickHouse/ClickHouse/pull/18014) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix clickhouse-client rendering issue when the size of terminal window changes. [#18009](https://github.com/ClickHouse/ClickHouse/pull/18009) ([Amos Bird](https://github.com/amosbird)). +* Decrease log verbosity of the events when the client drops the connection from Warning to Information. [#18005](https://github.com/ClickHouse/ClickHouse/pull/18005) ([filimonov](https://github.com/filimonov)). +* Forcibly removing empty or bad metadata files from filesystem for DiskS3. S3 is an experimental feature. [#17935](https://github.com/ClickHouse/ClickHouse/pull/17935) ([Pavel Kovalenko](https://github.com/Jokser)). +* Access control: `allow_introspection_functions=0` prohibits usage of introspection functions but doesn't prohibit giving grants for them anymore (the grantee will need to set `allow_introspection_functions=1` for himself to be able to use that grant). Similarly `allow_ddl=0` prohibits usage of DDL commands but doesn't prohibit giving grants for them anymore. [#17908](https://github.com/ClickHouse/ClickHouse/pull/17908) ([Vitaly Baranov](https://github.com/vitlibar)). +* Usability improvement: hints for column names. [#17112](https://github.com/ClickHouse/ClickHouse/issues/17112). [#17857](https://github.com/ClickHouse/ClickHouse/pull/17857) ([fastio](https://github.com/fastio)). +* Add diagnostic information when two merge tables try to read each other's data. [#17854](https://github.com/ClickHouse/ClickHouse/pull/17854) ([徐炘](https://github.com/weeds085490)). +* Let the possibility to override timeout value for running script using the ClickHouse docker image. [#17818](https://github.com/ClickHouse/ClickHouse/pull/17818) ([Guillaume Tassery](https://github.com/YiuRULE)). +* Check system log tables' engine definition grammar to prevent some configuration errors. Notes that this grammar check is not semantical, that means such mistakes as non-existent columns / expression functions would be not found out util the table is created. [#17739](https://github.com/ClickHouse/ClickHouse/pull/17739) ([Du Chuan](https://github.com/spongedu)). +* Removed exception throwing at `RabbitMQ` table initialization if there was no connection (it will be reconnecting in the background). [#17709](https://github.com/ClickHouse/ClickHouse/pull/17709) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Do not ignore server memory limits during Buffer flush. [#17646](https://github.com/ClickHouse/ClickHouse/pull/17646) ([Azat Khuzhin](https://github.com/azat)). +* Switch to patched version of RocksDB (from ClickHouse-Extras) to fix use-after-free error. [#17643](https://github.com/ClickHouse/ClickHouse/pull/17643) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Added an offset to exception message for parallel parsing. This fixes [#17457](https://github.com/ClickHouse/ClickHouse/issues/17457). [#17641](https://github.com/ClickHouse/ClickHouse/pull/17641) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Don't throw "Too many parts" error in the middle of INSERT query. [#17566](https://github.com/ClickHouse/ClickHouse/pull/17566) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow query parameters in UPDATE statement of ALTER query. Fixes [#10976](https://github.com/ClickHouse/ClickHouse/issues/10976). [#17563](https://github.com/ClickHouse/ClickHouse/pull/17563) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Query obfuscator: avoid usage of some SQL keywords for identifier names. [#17526](https://github.com/ClickHouse/ClickHouse/pull/17526) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Export current max ddl entry executed by DDLWorker via server metric. It's useful to check if DDLWorker hangs somewhere. [#17464](https://github.com/ClickHouse/ClickHouse/pull/17464) ([Amos Bird](https://github.com/amosbird)). +* Export asynchronous metrics of all servers current threads. It's useful to track down issues like [this](https://github.com/ClickHouse-Extras/poco/pull/28). [#17463](https://github.com/ClickHouse/ClickHouse/pull/17463) ([Amos Bird](https://github.com/amosbird)). +* Include dynamic columns like MATERIALIZED / ALIAS for wildcard query when settings `asterisk_include_materialized_columns` and `asterisk_include_alias_columns` are turned on. [#17462](https://github.com/ClickHouse/ClickHouse/pull/17462) ([Ken Chen](https://github.com/chenziliang)). +* Allow specifying [TTL](https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl) to remove old entries from [system log tables](https://clickhouse.tech/docs/en/operations/system-tables/), using the `` attribute in `config.xml`. [#17438](https://github.com/ClickHouse/ClickHouse/pull/17438) ([Du Chuan](https://github.com/spongedu)). +* Now queries coming to the server via MySQL and PostgreSQL protocols have distinctive interface types (which can be seen in the `interface` column of the table`system.query_log`): `4` for MySQL, and `5` for PostgreSQL, instead of formerly used `1` which is now used for the native protocol only. [#17437](https://github.com/ClickHouse/ClickHouse/pull/17437) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix parsing of SETTINGS clause of the `INSERT ... SELECT ... SETTINGS` query. [#17414](https://github.com/ClickHouse/ClickHouse/pull/17414) ([Azat Khuzhin](https://github.com/azat)). +* Correctly account memory in RadixSort. [#17412](https://github.com/ClickHouse/ClickHouse/pull/17412) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Add eof check in `receiveHello` in server to prevent getting `Attempt to read after eof` exception. [#17365](https://github.com/ClickHouse/ClickHouse/pull/17365) ([Kruglov Pavel](https://github.com/Avogar)). +* Avoid possible stack overflow in bigint conversion. Big integers are experimental. [#17269](https://github.com/ClickHouse/ClickHouse/pull/17269) ([flynn](https://github.com/ucasFL)). +* Now `set` indices will work with `GLOBAL IN`. This fixes [#17232](https://github.com/ClickHouse/ClickHouse/issues/17232) , [#5576](https://github.com/ClickHouse/ClickHouse/issues/5576) . [#17253](https://github.com/ClickHouse/ClickHouse/pull/17253) ([Amos Bird](https://github.com/amosbird)). +* Add limit for http redirects in request to S3 storage (`s3_max_redirects`). [#17220](https://github.com/ClickHouse/ClickHouse/pull/17220) ([ianton-ru](https://github.com/ianton-ru)). +* When `-OrNull` combinator combined `-If`, `-Merge`, `-MergeState`, `-State` combinators, we should put `-OrNull` in front. [#16935](https://github.com/ClickHouse/ClickHouse/pull/16935) ([flynn](https://github.com/ucasFL)). +* Support HTTP proxy and HTTPS S3 endpoint configuration. [#16861](https://github.com/ClickHouse/ClickHouse/pull/16861) ([Pavel Kovalenko](https://github.com/Jokser)). +* Added proper authentication using environment, `~/.aws` and `AssumeRole` for S3 client. [#16856](https://github.com/ClickHouse/ClickHouse/pull/16856) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Add more OpenTelemetry spans. Add an example of how to export the span data to Zipkin. [#16535](https://github.com/ClickHouse/ClickHouse/pull/16535) ([Alexander Kuzmenkov](https://github.com/akuzm)). +* Cache dictionaries: Completely eliminate callbacks and locks for acquiring them. Keys are not divided into "not found" and "expired", but stored in the same map during query. [#14958](https://github.com/ClickHouse/ClickHouse/pull/14958) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix never worked `fsync_part_directory`/`fsync_after_insert`/`in_memory_parts_insert_sync` (experimental feature). [#18845](https://github.com/ClickHouse/ClickHouse/pull/18845) ([Azat Khuzhin](https://github.com/azat)). +* Allow using `Atomic` engine for nested database of `MaterializeMySQL` engine. [#14849](https://github.com/ClickHouse/ClickHouse/pull/14849) ([tavplubix](https://github.com/tavplubix)). + #### Bug Fix -* - Split RemoteQueryExecutorReadContext into module part - Fix leaking of pipe fd for `async_socket_for_remote`. [#19153](https://github.com/ClickHouse/ClickHouse/pull/19153) ([Azat Khuzhin](https://github.com/azat)). +* Fix the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) (Amos Bird, [alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix index analysis of binary functions with constant argument which leads to wrong query results. This fixes [#18364](https://github.com/ClickHouse/ClickHouse/issues/18364). [#18373](https://github.com/ClickHouse/ClickHouse/pull/18373) ([Amos Bird](https://github.com/amosbird)). +* Fix possible wrong index analysis when the types of the index comparison are different. This fixes [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). +* Disable write with AIO during merges because it can lead to extremely rare data corruption of primary key columns during merge. [#18481](https://github.com/ClickHouse/ClickHouse/pull/18481) ([alesapin](https://github.com/alesapin)). +* Restrict merges from wide to compact parts. In case of vertical merge it led to broken result part. [#18381](https://github.com/ClickHouse/ClickHouse/pull/18381) ([Anton Popov](https://github.com/CurtizJ)). +* Fix possible incomplete query result while reading from `MergeTree*` in case of read backoff (message ` MergeTreeReadPool: Will lower number of threads` in logs). Was introduced in [#16423](https://github.com/ClickHouse/ClickHouse/issues/16423). Fixes [#18137](https://github.com/ClickHouse/ClickHouse/issues/18137). [#18216](https://github.com/ClickHouse/ClickHouse/pull/18216) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix use after free bug in `rocksdb` library. [#18862](https://github.com/ClickHouse/ClickHouse/pull/18862) ([sundyli](https://github.com/sundy-li)). * Fix infinite reading from file in `ORC` format (was introduced in [#10580](https://github.com/ClickHouse/ClickHouse/issues/10580)). Fixes [#19095](https://github.com/ClickHouse/ClickHouse/issues/19095). [#19134](https://github.com/ClickHouse/ClickHouse/pull/19134) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix bug in merge tree data writer which can lead to marks with bigger size than fixed granularity size. Fixes [#18913](https://github.com/ClickHouse/ClickHouse/issues/18913). [#19123](https://github.com/ClickHouse/ClickHouse/pull/19123) ([alesapin](https://github.com/alesapin)). * Fix startup bug when clickhouse was not able to read compression codec from `LowCardinality(Nullable(...))` and throws exception `Attempt to read after EOF`. Fixes [#18340](https://github.com/ClickHouse/ClickHouse/issues/18340). [#19101](https://github.com/ClickHouse/ClickHouse/pull/19101) ([alesapin](https://github.com/alesapin)). -* Simplify the implementation of `tupleHammingDistance`. Support for tuples of any equal length. Fixes [#19029](https://github.com/ClickHouse/ClickHouse/issues/19029). [#19084](https://github.com/ClickHouse/ClickHouse/pull/19084) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Restrict `MODIFY TTL` queries for `MergeTree` tables created in old syntax. Previously the query succeeded, but actually it had no effect. [#19064](https://github.com/ClickHouse/ClickHouse/pull/19064) ([Anton Popov](https://github.com/CurtizJ)). * Make sure `groupUniqArray` returns correct type for argument of Enum type. This closes [#17875](https://github.com/ClickHouse/ClickHouse/issues/17875). [#19019](https://github.com/ClickHouse/ClickHouse/pull/19019) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix possible error `Expected single dictionary argument for function` if use function `ignore` with `LowCardinality` argument. Fixes [#14275](https://github.com/ClickHouse/ClickHouse/issues/14275). [#19016](https://github.com/ClickHouse/ClickHouse/pull/19016) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix inserting of `LowCardinality` column to table with `TinyLog` engine. Fixes [#18629](https://github.com/ClickHouse/ClickHouse/issues/18629). [#19010](https://github.com/ClickHouse/ClickHouse/pull/19010) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Join tries to materialize const columns, but our code waits for them in other places. [#18982](https://github.com/ClickHouse/ClickHouse/pull/18982) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Join tries to materialize const columns, but our code wants them in other places. [#18982](https://github.com/ClickHouse/ClickHouse/pull/18982) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Disable `optimize_move_functions_out_of_any` because optimization is not always correct. This closes [#18051](https://github.com/ClickHouse/ClickHouse/issues/18051). This closes [#18973](https://github.com/ClickHouse/ClickHouse/issues/18973). [#18981](https://github.com/ClickHouse/ClickHouse/pull/18981) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix possible exception `QueryPipeline stream: different number of columns` caused by merging of query plan's `Expression` steps. Fixes [#18190](https://github.com/ClickHouse/ClickHouse/issues/18190). [#18980](https://github.com/ClickHouse/ClickHouse/pull/18980) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fixed very rare deadlock at shutdown. [#18977](https://github.com/ClickHouse/ClickHouse/pull/18977) ([tavplubix](https://github.com/tavplubix)). * Fix incorrect behavior when `ALTER TABLE ... DROP PART 'part_name'` query removes all deduplication blocks for the whole partition. Fixes [#18874](https://github.com/ClickHouse/ClickHouse/issues/18874). [#18969](https://github.com/ClickHouse/ClickHouse/pull/18969) ([alesapin](https://github.com/alesapin)). -* Fix error `Task was not found in task queue` (possible only for remote queries, with `async_socket_for_remote = 1`). [#18964](https://github.com/ClickHouse/ClickHouse/pull/18964) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix bug when mutation with some escaped text (like `ALTER ... UPDATE e = CAST('foo', 'Enum8(\'foo\' = 1')` serialized incorrectly. Fixes [#18878](https://github.com/ClickHouse/ClickHouse/issues/18878). [#18944](https://github.com/ClickHouse/ClickHouse/pull/18944) ([alesapin](https://github.com/alesapin)). * Attach partition should reset the mutation. [#18804](https://github.com/ClickHouse/ClickHouse/issues/18804). [#18935](https://github.com/ClickHouse/ClickHouse/pull/18935) ([fastio](https://github.com/fastio)). * Fix issue with `bitmapOrCardinality` that may lead to nullptr dereference. This closes [#18911](https://github.com/ClickHouse/ClickHouse/issues/18911). [#18912](https://github.com/ClickHouse/ClickHouse/pull/18912) ([sundyli](https://github.com/sundy-li)). -* Fix possible hang at shutdown in clickhouse-local. This fixes [#18891](https://github.com/ClickHouse/ClickHouse/issues/18891). [#18893](https://github.com/ClickHouse/ClickHouse/pull/18893) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* change the sorting key of events_list from timestamp to (timestamp, event_index). [#18884](https://github.com/ClickHouse/ClickHouse/pull/18884) ([Fuwang Hu](https://github.com/fuwhu)). +* Fix possible hang at shutdown in `clickhouse-local`. This fixes [#18891](https://github.com/ClickHouse/ClickHouse/issues/18891). [#18893](https://github.com/ClickHouse/ClickHouse/pull/18893) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Queries for external databases (MySQL, ODBC, JDBC) were incorrectly rewritten if there was an expression in form of `x IN table`. This fixes [#9756](https://github.com/ClickHouse/ClickHouse/issues/9756). [#18876](https://github.com/ClickHouse/ClickHouse/pull/18876) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix use after free bug in rocksdb. [#18862](https://github.com/ClickHouse/ClickHouse/pull/18862) ([sundyli](https://github.com/sundy-li)). -* - Fix never worked `fsync_part_directory`/`fsync_after_insert`/`in_memory_parts_insert_sync`. [#18845](https://github.com/ClickHouse/ClickHouse/pull/18845) ([Azat Khuzhin](https://github.com/azat)). * Fix *If combinator with unary function and Nullable types. [#18806](https://github.com/ClickHouse/ClickHouse/pull/18806) ([Azat Khuzhin](https://github.com/azat)). -* Asynchronous distributed INSERTs can be rejected by the server if the setting `network_compression_method` is globally set to non-default value. This fixes [#18741](https://github.com/ClickHouse/ClickHouse/issues/18741). [#18776](https://github.com/ClickHouse/ClickHouse/pull/18776) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix the issue that asynchronous distributed INSERTs can be rejected by the server if the setting `network_compression_method` is globally set to non-default value. This fixes [#18741](https://github.com/ClickHouse/ClickHouse/issues/18741). [#18776](https://github.com/ClickHouse/ClickHouse/pull/18776) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fixed `Attempt to read after eof` error when trying to `CAST` `NULL` from `Nullable(String)` to `Nullable(Decimal(P, S))`. Now function `CAST` returns `NULL` when it cannot parse decimal from nullable string. Fixes [#7690](https://github.com/ClickHouse/ClickHouse/issues/7690). [#18718](https://github.com/ClickHouse/ClickHouse/pull/18718) ([Winter Zhang](https://github.com/zhang2014)). -* Fix Logger with unmatched arg size. [#18717](https://github.com/ClickHouse/ClickHouse/pull/18717) ([sundyli](https://github.com/sundy-li)). +* Fix minor issue with logging. [#18717](https://github.com/ClickHouse/ClickHouse/pull/18717) ([sundyli](https://github.com/sundy-li)). * Fix removing of empty parts in `ReplicatedMergeTree` tables, created with old syntax. Fixes [#18582](https://github.com/ClickHouse/ClickHouse/issues/18582). [#18614](https://github.com/ClickHouse/ClickHouse/pull/18614) ([Anton Popov](https://github.com/CurtizJ)). * Fix previous bug when date overflow with different values. Strict Date value limit to "2106-02-07", cast date > "2106-02-07" to value 0. [#18565](https://github.com/ClickHouse/ClickHouse/pull/18565) ([hexiaoting](https://github.com/hexiaoting)). -* Add FixedString Data type support. I'll get this exception "Code: 50, e.displayText() = DB::Exception: Unsupported type FixedString(1)" when replicating data from MySQL to ClickHouse. This patch fixes bug [#18450](https://github.com/ClickHouse/ClickHouse/issues/18450) Also fixes [#6556](https://github.com/ClickHouse/ClickHouse/issues/6556). [#18553](https://github.com/ClickHouse/ClickHouse/pull/18553) ([awesomeleo](https://github.com/awesomeleo)). +* Add FixedString data type support for replication from MySQL. Replication from MySQL is an experimental feature. This patch fixes [#18450](https://github.com/ClickHouse/ClickHouse/issues/18450) Also fixes [#6556](https://github.com/ClickHouse/ClickHouse/issues/6556). [#18553](https://github.com/ClickHouse/ClickHouse/pull/18553) ([awesomeleo](https://github.com/awesomeleo)). * Fix possible `Pipeline stuck` error while using `ORDER BY` after subquery with `RIGHT` or `FULL` join. [#18550](https://github.com/ClickHouse/ClickHouse/pull/18550) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix bug which may lead to `ALTER` queries hung after corresponding mutation kill. Found by thread fuzzer. [#18518](https://github.com/ClickHouse/ClickHouse/pull/18518) ([alesapin](https://github.com/alesapin)). -* Disable write with AIO during merges because it can lead to extremely rare data corruption of primary key columns during merge. [#18481](https://github.com/ClickHouse/ClickHouse/pull/18481) ([alesapin](https://github.com/alesapin)). * Proper support for 12AM in `parseDateTimeBestEffort` function. This fixes [#18402](https://github.com/ClickHouse/ClickHouse/issues/18402). [#18449](https://github.com/ClickHouse/ClickHouse/pull/18449) ([vladimir-golovchenko](https://github.com/vladimir-golovchenko)). * Fixed `value is too short` error when executing `toType(...)` functions (`toDate`, `toUInt32`, etc) with argument of type `Nullable(String)`. Now such functions return `NULL` on parsing errors instead of throwing exception. Fixes [#7673](https://github.com/ClickHouse/ClickHouse/issues/7673). [#18445](https://github.com/ClickHouse/ClickHouse/pull/18445) ([tavplubix](https://github.com/tavplubix)). * Fix the unexpected behaviour of `SHOW TABLES`. [#18431](https://github.com/ClickHouse/ClickHouse/pull/18431) ([fastio](https://github.com/fastio)). * Fix -SimpleState combinator generates incompatible arugment type and return type. [#18404](https://github.com/ClickHouse/ClickHouse/pull/18404) ([Amos Bird](https://github.com/amosbird)). * Fix possible race condition in concurrent usage of `Set` or `Join` tables and selects from `system.tables`. [#18385](https://github.com/ClickHouse/ClickHouse/pull/18385) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Restrict merges from wide to compact parts. In case of vertical merge it led to broken result part. [#18381](https://github.com/ClickHouse/ClickHouse/pull/18381) ([Anton Popov](https://github.com/CurtizJ)). * Fix filling table `system.settings_profile_elements`. This PR fixes [#18231](https://github.com/ClickHouse/ClickHouse/issues/18231). [#18379](https://github.com/ClickHouse/ClickHouse/pull/18379) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix index analysis of binary functions with constant argument which leads to wrong query results. This fixes [#18364](https://github.com/ClickHouse/ClickHouse/issues/18364). [#18373](https://github.com/ClickHouse/ClickHouse/pull/18373) ([Amos Bird](https://github.com/amosbird)). * Fix possible crashes in aggregate functions with combinator `Distinct`, while using two-level aggregation. Fixes [#17682](https://github.com/ClickHouse/ClickHouse/issues/17682). [#18365](https://github.com/ClickHouse/ClickHouse/pull/18365) ([Anton Popov](https://github.com/CurtizJ)). -* - Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual IPv4/IPv6 stack; - Fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes; Possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). -* `SELECT count() FROM table` now can be executed if only one any column can be selected from the `table`. This PR fixes [#10639](https://github.com/ClickHouse/ClickHouse/issues/10639). [#18233](https://github.com/ClickHouse/ClickHouse/pull/18233) ([Vitaly Baranov](https://github.com/vitlibar)). -* `SELECT JOIN` now requires the `SELECT` privilege on each of the joined tables. This PR fixes [#17654](https://github.com/ClickHouse/ClickHouse/issues/17654). [#18232](https://github.com/ClickHouse/ClickHouse/pull/18232) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix possible incomplete query result while reading from `MergeTree*` in case of read backoff (message ` MergeTreeReadPool: Will lower number of threads` in logs). Was introduced in [#16423](https://github.com/ClickHouse/ClickHouse/issues/16423). Fixes [#18137](https://github.com/ClickHouse/ClickHouse/issues/18137). [#18216](https://github.com/ClickHouse/ClickHouse/pull/18216) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed issue when `clickhouse-odbc-bridge` process is unreachable by server on machines with dual IPv4/IPv6 stack; Fixed issue when ODBC dictionary updates are performed using malformed queries and/or cause crashes of the odbc-bridge process; Possibly closes [#14489](https://github.com/ClickHouse/ClickHouse/issues/14489). [#18278](https://github.com/ClickHouse/ClickHouse/pull/18278) ([Denis Glazachev](https://github.com/traceon)). +* Access control: `SELECT count() FROM table` now can be executed if the user has access to at least single column from a table. This PR fixes [#10639](https://github.com/ClickHouse/ClickHouse/issues/10639). [#18233](https://github.com/ClickHouse/ClickHouse/pull/18233) ([Vitaly Baranov](https://github.com/vitlibar)). +* Access control: `SELECT JOIN` now requires the `SELECT` privilege on each of the joined tables. This PR fixes [#17654](https://github.com/ClickHouse/ClickHouse/issues/17654). [#18232](https://github.com/ClickHouse/ClickHouse/pull/18232) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix key comparison between Enum and Int types. This fixes [#17989](https://github.com/ClickHouse/ClickHouse/issues/17989). [#18214](https://github.com/ClickHouse/ClickHouse/pull/18214) ([Amos Bird](https://github.com/amosbird)). -* fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) fix unique key convert crash in MaterializeMySQL database engine. [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). -* Related to [#17466](https://github.com/ClickHouse/ClickHouse/issues/17466). [#18188](https://github.com/ClickHouse/ClickHouse/pull/18188) ([hexiaoting](https://github.com/hexiaoting)). +* Replication from MySQL (experimental feature). Fixes [#18186](https://github.com/ClickHouse/ClickHouse/issues/18186) Fixes [#16372](https://github.com/ClickHouse/ClickHouse/issues/16372) Fix unique key convert issue in MaterializeMySQL database engine. [#18211](https://github.com/ClickHouse/ClickHouse/pull/18211) ([Winter Zhang](https://github.com/zhang2014)). +* Fix inconsistency for queries with both `WITH FILL` and `WITH TIES` [#17466](https://github.com/ClickHouse/ClickHouse/issues/17466). [#18188](https://github.com/ClickHouse/ClickHouse/pull/18188) ([hexiaoting](https://github.com/hexiaoting)). * Fix inserting a row with default value in case of parsing error in the last column. Fixes [#17712](https://github.com/ClickHouse/ClickHouse/issues/17712). [#18182](https://github.com/ClickHouse/ClickHouse/pull/18182) ([Jianmei Zhang](https://github.com/zhangjmruc)). * Fix `Unknown setting profile` error on attempt to set settings profile. [#18167](https://github.com/ClickHouse/ClickHouse/pull/18167) ([tavplubix](https://github.com/tavplubix)). * Fix error when query `MODIFY COLUMN ... REMOVE TTL` doesn't actually remove column TTL. [#18130](https://github.com/ClickHouse/ClickHouse/pull/18130) ([alesapin](https://github.com/alesapin)). * Fixed `std::out_of_range: basic_string` in S3 URL parsing. [#18059](https://github.com/ClickHouse/ClickHouse/pull/18059) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fix corruption in librdkafka snappy decompression (was a problem only for gcc10 builds, but official builds uses clang already, so at least recent official releases are not affected). [#18053](https://github.com/ClickHouse/ClickHouse/pull/18053) ([Azat Khuzhin](https://github.com/azat)). * Fix comparison of `DateTime64` and `Date`. Fixes [#13804](https://github.com/ClickHouse/ClickHouse/issues/13804) and [#11222](https://github.com/ClickHouse/ClickHouse/issues/11222). ... [#18050](https://github.com/ClickHouse/ClickHouse/pull/18050) ([Vasily Nemkov](https://github.com/Enmk)). -* fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) support convert MySQL prefix index for MaterializeMySQL CC: @tavplubix. [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). +* Replication from MySQL (experimental feature): Fixes [#15187](https://github.com/ClickHouse/ClickHouse/issues/15187) Fixes [#17912](https://github.com/ClickHouse/ClickHouse/issues/17912) support convert MySQL prefix index for MaterializeMySQL. [#17944](https://github.com/ClickHouse/ClickHouse/pull/17944) ([Winter Zhang](https://github.com/zhang2014)). * When server log rotation was configured using `logger.size` parameter with numeric value larger than 2^32, the logs were not rotated properly. This is fixed. [#17905](https://github.com/ClickHouse/ClickHouse/pull/17905) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* Fix comparison of `DateTime64` and `Date`. Fixes [#13804](https://github.com/ClickHouse/ClickHouse/issues/13804) and [#11222](https://github.com/ClickHouse/ClickHouse/issues/11222). ... [#17895](https://github.com/ClickHouse/ClickHouse/pull/17895) ([Vasily Nemkov](https://github.com/Enmk)). * Trivial query optimization was producing wrong result if query contains ARRAY JOIN (so query is actually non trivial). [#17887](https://github.com/ClickHouse/ClickHouse/pull/17887) ([sundyli](https://github.com/sundy-li)). -* Fix max_distributed_connections (affects `prefer_localhost_replica=1` and `max_threads!=max_distributed_connections`). [#17848](https://github.com/ClickHouse/ClickHouse/pull/17848) ([Azat Khuzhin](https://github.com/azat)). * Fix possible segfault in `topK` aggregate function. This closes [#17404](https://github.com/ClickHouse/ClickHouse/issues/17404). [#17845](https://github.com/ClickHouse/ClickHouse/pull/17845) ([Maksim Kita](https://github.com/kitaisreal)). -* fix incorrect initialize `max_compress_block_size` of MergeTreeWriterSettings with `min_compress_block_size`. [#17833](https://github.com/ClickHouse/ClickHouse/pull/17833) ([flynn](https://github.com/ucasFL)). -* Do not restore parts from WAL if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). +* WAL (experimental feature): Do not restore parts from WAL if `in_memory_parts_enable_wal` is disabled. [#17802](https://github.com/ClickHouse/ClickHouse/pull/17802) ([detailyang](https://github.com/detailyang)). * Exception message about max table size to drop was displayed incorrectly. [#17764](https://github.com/ClickHouse/ClickHouse/pull/17764) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed segfault when there is not enough space when inserting into `Distributed` table. [#17737](https://github.com/ClickHouse/ClickHouse/pull/17737) ([tavplubix](https://github.com/tavplubix)). +* Fixed possible segfault when there is not enough space when inserting into `Distributed` table. [#17737](https://github.com/ClickHouse/ClickHouse/pull/17737) ([tavplubix](https://github.com/tavplubix)). * Fixed problem when ClickHouse fails to resume connection to MySQL servers. [#17681](https://github.com/ClickHouse/ClickHouse/pull/17681) ([Alexander Kazakov](https://github.com/Akazz)). -* Fixed `Function not implemented` error when executing `RENAME` query in `Atomic` database with ClickHouse running on Windows Subsystem for Linux. Fixes [#17661](https://github.com/ClickHouse/ClickHouse/issues/17661). [#17664](https://github.com/ClickHouse/ClickHouse/pull/17664) ([tavplubix](https://github.com/tavplubix)). +* Windows: Fixed `Function not implemented` error when executing `RENAME` query in `Atomic` database with ClickHouse running on Windows Subsystem for Linux. Fixes [#17661](https://github.com/ClickHouse/ClickHouse/issues/17661). [#17664](https://github.com/ClickHouse/ClickHouse/pull/17664) ([tavplubix](https://github.com/tavplubix)). * In might be determined incorrectly if cluster is circular- (cross-) replicated or not when executing `ON CLUSTER` query due to race condition when `pool_size` > 1. It's fixed. [#17640](https://github.com/ClickHouse/ClickHouse/pull/17640) ([tavplubix](https://github.com/tavplubix)). * Fix empty `system.stack_trace` table when server is running in daemon mode. [#17630](https://github.com/ClickHouse/ClickHouse/pull/17630) ([Amos Bird](https://github.com/amosbird)). * Exception `fmt::v7::format_error` can be logged in background for MergeTree tables. This fixes [#17613](https://github.com/ClickHouse/ClickHouse/issues/17613). [#17615](https://github.com/ClickHouse/ClickHouse/pull/17615) ([alexey-milovidov](https://github.com/alexey-milovidov)). * When clickhouse-client is used in interactive mode with multiline queries, single line comment was erronously extended till the end of query. This fixes [#13654](https://github.com/ClickHouse/ClickHouse/issues/13654). [#17565](https://github.com/ClickHouse/ClickHouse/pull/17565) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix the issue when server can stop accepting connections in very rare cases. [#17542](https://github.com/ClickHouse/ClickHouse/pull/17542) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Fix alter query hang when the corresponding mutation was killed on the different replica. Fixes [#16953](https://github.com/ClickHouse/ClickHouse/issues/16953). [#17499](https://github.com/ClickHouse/ClickHouse/pull/17499) ([alesapin](https://github.com/alesapin)). -* Fix bug when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). +* Fix issue with memory accounting when mark cache size was underestimated by clickhouse. It may happen when there are a lot of tiny files with marks. [#17496](https://github.com/ClickHouse/ClickHouse/pull/17496) ([alesapin](https://github.com/alesapin)). * Fix `ORDER BY` with enabled setting `optimize_redundant_functions_in_order_by`. [#17471](https://github.com/ClickHouse/ClickHouse/pull/17471) ([Anton Popov](https://github.com/CurtizJ)). -* Avoid server abnormal termination in case of too low memory limits (`max_memory_usage=1`/`max_untracked_memory=1`). [#17453](https://github.com/ClickHouse/ClickHouse/pull/17453) ([Azat Khuzhin](https://github.com/azat)). * Fix duplicates after `DISTINCT` which were possible because of incorrect optimization. Fixes [#17294](https://github.com/ClickHouse/ClickHouse/issues/17294). [#17296](https://github.com/ClickHouse/ClickHouse/pull/17296) ([li chengxiang](https://github.com/chengxianglibra)). [#17439](https://github.com/ClickHouse/ClickHouse/pull/17439) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fixed high CPU usage in background tasks of *MergeTree tables. [#17416](https://github.com/ClickHouse/ClickHouse/pull/17416) ([tavplubix](https://github.com/tavplubix)). -* Fix crash while reading from `JOIN` table with `LowCardinality` types. Fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* fixes [#16835](https://github.com/ClickHouse/ClickHouse/issues/16835) try fix miss match header with MySQL SHOW statement. [#17366](https://github.com/ClickHouse/ClickHouse/pull/17366) ([Winter Zhang](https://github.com/zhang2014)). -* Fix indeterministic functions with predicate optimizer. This fixes [#17244](https://github.com/ClickHouse/ClickHouse/issues/17244). [#17273](https://github.com/ClickHouse/ClickHouse/pull/17273) ([Winter Zhang](https://github.com/zhang2014)). +* Fix possible crash while reading from `JOIN` table with `LowCardinality` types. Fixes [#17228](https://github.com/ClickHouse/ClickHouse/issues/17228). [#17397](https://github.com/ClickHouse/ClickHouse/pull/17397) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Replication from MySQL (experimental feature): Fixes [#16835](https://github.com/ClickHouse/ClickHouse/issues/16835) try fix miss match header with MySQL SHOW statement. [#17366](https://github.com/ClickHouse/ClickHouse/pull/17366) ([Winter Zhang](https://github.com/zhang2014)). +* Fix nondeterministic functions with predicate optimizer. This fixes [#17244](https://github.com/ClickHouse/ClickHouse/issues/17244). [#17273](https://github.com/ClickHouse/ClickHouse/pull/17273) ([Winter Zhang](https://github.com/zhang2014)). * Fix possible `Unexpected packet Data received from client` error for Distributed queries with `LIMIT`. [#17254](https://github.com/ClickHouse/ClickHouse/pull/17254) ([Azat Khuzhin](https://github.com/azat)). -* Fix set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246) . [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). -* Fix [#15235](https://github.com/ClickHouse/ClickHouse/issues/15235). When clickhouse-copier handle non-partitioned table, throws segfault error. [#17248](https://github.com/ClickHouse/ClickHouse/pull/17248) ([Qi Chen](https://github.com/kaka11chen)). -* Fixed possible not-working mutations for parts stored on S3 disk. [#17227](https://github.com/ClickHouse/ClickHouse/pull/17227) ([Pavel Kovalenko](https://github.com/Jokser)). -* Fix possible wrong index analysis when the types of the index comparison are different. This fixes [#17122](https://github.com/ClickHouse/ClickHouse/issues/17122). [#17145](https://github.com/ClickHouse/ClickHouse/pull/17145) ([Amos Bird](https://github.com/amosbird)). -* Bug fix for funciton fuzzBits, related issue: [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). -* - Fix optimize_distributed_group_by_sharding_key for query with OFFSET only. [#16996](https://github.com/ClickHouse/ClickHouse/pull/16996) ([Azat Khuzhin](https://github.com/azat)). -* Fix Merge(Distributed()) with JOIN. [#16993](https://github.com/ClickHouse/ClickHouse/pull/16993) ([Azat Khuzhin](https://github.com/azat)). -* Fix order by optimization with monotonous functions. Fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). +* Fix set index invalidation when there are const columns in the subquery. This fixes [#17246](https://github.com/ClickHouse/ClickHouse/issues/17246). [#17249](https://github.com/ClickHouse/ClickHouse/pull/17249) ([Amos Bird](https://github.com/amosbird)). +* clickhouse-copier: Fix for non-partitioned tables [#15235](https://github.com/ClickHouse/ClickHouse/issues/15235). [#17248](https://github.com/ClickHouse/ClickHouse/pull/17248) ([Qi Chen](https://github.com/kaka11chen)). +* Fixed possible not-working mutations for parts stored on S3 disk (experimental feature). [#17227](https://github.com/ClickHouse/ClickHouse/pull/17227) ([Pavel Kovalenko](https://github.com/Jokser)). +* Bug fix for funciton `fuzzBits`, related issue: [#16980](https://github.com/ClickHouse/ClickHouse/issues/16980). [#17051](https://github.com/ClickHouse/ClickHouse/pull/17051) ([hexiaoting](https://github.com/hexiaoting)). +* Fix `optimize_distributed_group_by_sharding_key` for query with OFFSET only. [#16996](https://github.com/ClickHouse/ClickHouse/pull/16996) ([Azat Khuzhin](https://github.com/azat)). +* Fix queries from `Merge` tables over `Distributed` tables with JOINs. [#16993](https://github.com/ClickHouse/ClickHouse/pull/16993) ([Azat Khuzhin](https://github.com/azat)). +* Fix order by optimization with monotonic functions. Fixes [#16107](https://github.com/ClickHouse/ClickHouse/issues/16107). [#16956](https://github.com/ClickHouse/ClickHouse/pull/16956) ([Anton Popov](https://github.com/CurtizJ)). * Fix incorrect comparison of types `DateTime64` with different scales. Fixes [#16655](https://github.com/ClickHouse/ClickHouse/issues/16655) ... [#16952](https://github.com/ClickHouse/ClickHouse/pull/16952) ([Vasily Nemkov](https://github.com/Enmk)). * Fix optimization of group by with enabled setting `optimize_aggregators_of_group_by_keys` and joins. Fixes [#12604](https://github.com/ClickHouse/ClickHouse/issues/12604). [#16951](https://github.com/ClickHouse/ClickHouse/pull/16951) ([Anton Popov](https://github.com/CurtizJ)). -* TODO. [#16866](https://github.com/ClickHouse/ClickHouse/pull/16866) ([tavplubix](https://github.com/tavplubix)). -* Fix bug when clickhouse-server doesn't send `close` request to ZooKeeper server. [#16837](https://github.com/ClickHouse/ClickHouse/pull/16837) ([alesapin](https://github.com/alesapin)). -* Fix optimize_trivial_count_query with partition predicate. [#16767](https://github.com/ClickHouse/ClickHouse/pull/16767) ([Azat Khuzhin](https://github.com/azat)). -* Return number of affected rows for INSERT queries via MySQL protocol. Previously ClickHouse used to always return 0, it's fixed. Fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). -* Fix inconsistent behavior caused by `select_sequential_consistency` for optimized trivial count query and system.tables. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). -* Throw error when use ColumnTransformer replace non exist column. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). -* Fix crash in case of not equi-join ON expression in RIGH|FULL JOIN. [#15162](https://github.com/ClickHouse/ClickHouse/pull/15162) ([Artem Zuikov](https://github.com/4ertus2)). +* Minor fix in SHOW ACCESS query. [#16866](https://github.com/ClickHouse/ClickHouse/pull/16866) ([tavplubix](https://github.com/tavplubix)). +* Fix the behaviour with enabled `optimize_trivial_count_query` setting with partition predicate. [#16767](https://github.com/ClickHouse/ClickHouse/pull/16767) ([Azat Khuzhin](https://github.com/azat)). +* Return number of affected rows for INSERT queries via MySQL wire protocol. Previously ClickHouse used to always return 0, it's fixed. Fixes [#16605](https://github.com/ClickHouse/ClickHouse/issues/16605). [#16715](https://github.com/ClickHouse/ClickHouse/pull/16715) ([Winter Zhang](https://github.com/zhang2014)). +* Fix inconsistent behavior caused by `select_sequential_consistency` for optimized trivial count query and system tables. [#16309](https://github.com/ClickHouse/ClickHouse/pull/16309) ([Hao Chen](https://github.com/haoch)). +* Throw error when `REPLACE` column transformer operates on non existing column. [#16183](https://github.com/ClickHouse/ClickHouse/pull/16183) ([hexiaoting](https://github.com/hexiaoting)). +* Throw exception in case of not equi-join ON expression in RIGH|FULL JOIN. [#15162](https://github.com/ClickHouse/ClickHouse/pull/15162) ([Artem Zuikov](https://github.com/4ertus2)). -#### Improvement - -* Explicitly set uid / gid of clickhouse user & group to the fixed values (101) in clickhouse-server images. [#19096](https://github.com/ClickHouse/ClickHouse/pull/19096) ([filimonov](https://github.com/filimonov)). -* Two new settings (by analogy with MergeTree family) has been added: - `fsync_after_insert` - Do fsync for every inserted. Will decreases performance of inserts. - `fsync_directories` - Do fsync for temporary directory (that is used for async INSERT only) after all operations (writes, renames, etc.). [#18864](https://github.com/ClickHouse/ClickHouse/pull/18864) ([Azat Khuzhin](https://github.com/azat)). -* `SYSTEM KILL` command started to work in Docker. This closes [#18847](https://github.com/ClickHouse/ClickHouse/issues/18847). [#18848](https://github.com/ClickHouse/ClickHouse/pull/18848) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Expand macros in the zk path when executing fetchPartition. [#18839](https://github.com/ClickHouse/ClickHouse/pull/18839) ([fastio](https://github.com/fastio)). -* Apply `ALTER TABLE ON CLUSTER MODIFY SETTING ...` to all replicas. Because we don't replicate such alter commands. [#18789](https://github.com/ClickHouse/ClickHouse/pull/18789) ([Amos Bird](https://github.com/amosbird)). -* Allow column transformer `EXCEPT` to accept a string as regular expression matcher. This resolves [#18685](https://github.com/ClickHouse/ClickHouse/issues/18685) . [#18699](https://github.com/ClickHouse/ClickHouse/pull/18699) ([Amos Bird](https://github.com/amosbird)). -* Another fix of using SimpleAggregateFunction in SummingMergeTree. This fixes [#18676](https://github.com/ClickHouse/ClickHouse/issues/18676) . [#18677](https://github.com/ClickHouse/ClickHouse/pull/18677) ([Amos Bird](https://github.com/amosbird)). -* Fix SimpleAggregateFunction in SummingMergeTree. Now it works like AggregateFunction. In previous versions values were summed together regardless to the aggregate function. This fixes [#18564](https://github.com/ClickHouse/ClickHouse/issues/18564) . [#8052](https://github.com/ClickHouse/ClickHouse/issues/8052). [#18637](https://github.com/ClickHouse/ClickHouse/pull/18637) ([Amos Bird](https://github.com/amosbird)). -* PODArray: Avoid call to memcpy with (nullptr, 0) arguments (Fix UBSan report). This fixes [#18525](https://github.com/ClickHouse/ClickHouse/issues/18525). [#18526](https://github.com/ClickHouse/ClickHouse/pull/18526) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed assertion error inside allocator in case when last argument of function bar is NaN. Now simple ClickHouse's exception is being thrown. This fixes [#17876](https://github.com/ClickHouse/ClickHouse/issues/17876). [#18520](https://github.com/ClickHouse/ClickHouse/pull/18520) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* support syntax `EXISTS DATABASE name`. [#18458](https://github.com/ClickHouse/ClickHouse/pull/18458) ([Du Chuan](https://github.com/spongedu)). -* Fix bug: no newline after exception message in some tools. [#18444](https://github.com/ClickHouse/ClickHouse/pull/18444) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Add ability to modify primary and partition key column type from `LowCardinality(Type)` to `Type` and vice versa. Also add an ability to modify primary key column type from `EnumX ` to `IntX` type. Fixes [#5604](https://github.com/ClickHouse/ClickHouse/issues/5604). [#18362](https://github.com/ClickHouse/ClickHouse/pull/18362) ([alesapin](https://github.com/alesapin)). -* Support builtin function `isIPv4String` && `isIPv6String` like [MySQL](https://github.com/ClickHouse/ClickHouse/compare/master...spongedu:support_is_ipv4?expand=1). [#18349](https://github.com/ClickHouse/ClickHouse/pull/18349) ([Du Chuan](https://github.com/spongedu)). -* Fix potential server crash during Buffer rollback (that is impossible in current ClickHouse version). [#18329](https://github.com/ClickHouse/ClickHouse/pull/18329) ([Azat Khuzhin](https://github.com/azat)). -* related to [#18133](https://github.com/ClickHouse/ClickHouse/issues/18133). [#18309](https://github.com/ClickHouse/ClickHouse/pull/18309) ([hexiaoting](https://github.com/hexiaoting)). -* Add a new setting `insert_distributed_one_random_shard = 1` to allow insertion into multi-sharded distributed table without any distributed key. [#18294](https://github.com/ClickHouse/ClickHouse/pull/18294) ([Amos Bird](https://github.com/amosbird)). -* Allow to parse Array fields from CSV if it is represented as a string containing array that was serialized as nested CSV. Example: `"[""Hello"", ""world"", ""42"""" TV""]"` will parse as `['Hello', 'world', '42" TV']`. Allow to parse array in CSV in a string without enclosing braces. Example: `"'Hello', 'world', '42"" TV'"` will parse as `['Hello', 'world', '42" TV']`. [#18271](https://github.com/ClickHouse/ClickHouse/pull/18271) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make better adaptive granularity calculation for merge tree wide parts. [#18223](https://github.com/ClickHouse/ClickHouse/pull/18223) ([alesapin](https://github.com/alesapin)). -* Now clickhouse-install could work on Mac. The problem was that there is no procfs on this platform. [#18201](https://github.com/ClickHouse/ClickHouse/pull/18201) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Better hints for `SHOW ...` query syntax. [#18183](https://github.com/ClickHouse/ClickHouse/pull/18183) ([Du Chuan](https://github.com/spongedu)). -* Array aggregation `arrayMin`, `arrayMax`, `arraySum`, `arrayAvg` support for `Int128`, `Int256`, `UInt256`. [#18147](https://github.com/ClickHouse/ClickHouse/pull/18147) ([Maksim Kita](https://github.com/kitaisreal)). -* All queries of type `Decimal * Float` or vice versa are allowed, including aggregate ones (e.g. `SELECT sum(decimal_field * 1.1)` or `SELECT dec_col * float_col`), the result type is Float32 or Float64. [#18145](https://github.com/ClickHouse/ClickHouse/pull/18145) ([Mike](https://github.com/myrrc)). -* `EXPLAIN AST` now support queries other then `SELECT`. [#18136](https://github.com/ClickHouse/ClickHouse/pull/18136) ([taiyang-li](https://github.com/taiyang-li)). -* Add `disk` to Set and Join storage settings. [#18112](https://github.com/ClickHouse/ClickHouse/pull/18112) ([Grigory Pervakov](https://github.com/GrigoryPervakov)). -* Now the table function `merge()` requires the current user to have the `SELECT` privilege on each table it receives data from. This PR fixes [#16964](https://github.com/ClickHouse/ClickHouse/issues/16964). [#18104](https://github.com/ClickHouse/ClickHouse/pull/18104) ([Vitaly Baranov](https://github.com/vitlibar)). -* Support `SHOW CREATE VIEW name` syntax like [MySQL](https://dev.mysql.com/doc/refman/5.7/en/show-create-view.html). [#18095](https://github.com/ClickHouse/ClickHouse/pull/18095) ([Du Chuan](https://github.com/spongedu)). -* Fix dead list watches removal for TestKeeperStorage. [#18065](https://github.com/ClickHouse/ClickHouse/pull/18065) ([alesapin](https://github.com/alesapin)). -* Temporary tables are visible in the system tables `system.tables` and `system.columns` now only in those session where they have been created. The internal database `_temporary_and_external_tables` is now hidden in those system tables; temporary tables are shown as tables with empty database with the `is_temporary` flag set instead. [#18014](https://github.com/ClickHouse/ClickHouse/pull/18014) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix clickhouse-client rendering issue when the size of terminal window changes. [#18009](https://github.com/ClickHouse/ClickHouse/pull/18009) ([Amos Bird](https://github.com/amosbird)). -* Decrease log verbosity of the events when the client drops the connection from WARNING to INFORMATION. [#18005](https://github.com/ClickHouse/ClickHouse/pull/18005) ([filimonov](https://github.com/filimonov)). -* Now the table function `merge()` requires the current user to have the `SELECT` privilege on each table it receives data from. This PR fixes [#16964](https://github.com/ClickHouse/ClickHouse/issues/16964). [#17983](https://github.com/ClickHouse/ClickHouse/pull/17983) ([Vitaly Baranov](https://github.com/vitlibar)). -* Forcibly removing empty or bad metadata files from filesystem for DiskS3. S3 is an experimental feature. [#17935](https://github.com/ClickHouse/ClickHouse/pull/17935) ([Pavel Kovalenko](https://github.com/Jokser)). -* Adaptive choose of single/multi part upload in WriteBufferFromS3. Single part upload is controlled by a new setting 'max_single_part_upload_size'. [#17934](https://github.com/ClickHouse/ClickHouse/pull/17934) ([Pavel Kovalenko](https://github.com/Jokser)). -* Ability to set custom metadata when putting S3 object. [#17909](https://github.com/ClickHouse/ClickHouse/pull/17909) ([Pavel Kovalenko](https://github.com/Jokser)). -* `allow_introspection_functions=0` prohibits usage of introspection functions but doesn't prohibit giving grants for them anymore (the grantee will need to set `allow_introspection_functions=1` for himself to be able to use that grant). Similarly `allow_ddl=0` prohibits usage of DDL commands but doesn't prohibit giving grants for them anymore. [#17908](https://github.com/ClickHouse/ClickHouse/pull/17908) ([Vitaly Baranov](https://github.com/vitlibar)). -* Support for async tasks in `PipelineExecutor`. Initial support of async sockets for remote queries. [#17868](https://github.com/ClickHouse/ClickHouse/pull/17868) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Hints for column names. [#17112](https://github.com/ClickHouse/ClickHouse/issues/17112). [#17857](https://github.com/ClickHouse/ClickHouse/pull/17857) ([fastio](https://github.com/fastio)). -* Add diagnostic information when two merge tables try to read each other's data. [#17854](https://github.com/ClickHouse/ClickHouse/pull/17854) ([徐炘](https://github.com/weeds085490)). -* Add metrics(Parts, PartsActive, PartsInactive) for part number in MergeTree in clickhouse. [#17838](https://github.com/ClickHouse/ClickHouse/pull/17838) ([徐炘](https://github.com/weeds085490)). -* - Let the possibility to override timeout value for running script using the ClickHouse docker image. [#17818](https://github.com/ClickHouse/ClickHouse/pull/17818) ([Guillaume Tassery](https://github.com/YiuRULE)). -* Improvement of Web UI: do not add empty query to browser history. [#17770](https://github.com/ClickHouse/ClickHouse/pull/17770) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improves the path concatenation of zookeeper paths inside DDLWorker. [#17767](https://github.com/ClickHouse/ClickHouse/pull/17767) ([Bharat Nallan](https://github.com/bharatnc)). -* Check system log tables' engine definition grammatically to prevent some configuration errors. Notes that this grammar check is not semantical, that means such mistakes as non-existent columns / expression functions would be not found out util the table is created. [#17739](https://github.com/ClickHouse/ClickHouse/pull/17739) ([Du Chuan](https://github.com/spongedu)). -* system.query_log now has extensive information to achieve better query analysis. [#17726](https://github.com/ClickHouse/ClickHouse/pull/17726) ([Amos Bird](https://github.com/amosbird)). -* Removed exception throwing at table initialization if there was no connection (it will be reconnecting in the background). [#17709](https://github.com/ClickHouse/ClickHouse/pull/17709) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Do not ignore server memory limits during Buffer flush. [#17646](https://github.com/ClickHouse/ClickHouse/pull/17646) ([Azat Khuzhin](https://github.com/azat)). -* Switch to patched version of RocksDB (from ClickHouse-Extras). [#17643](https://github.com/ClickHouse/ClickHouse/pull/17643) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* This fixes [#17457](https://github.com/ClickHouse/ClickHouse/issues/17457). [#17641](https://github.com/ClickHouse/ClickHouse/pull/17641) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Allow to reload symbols from debug file. This PR also fixes a build-id issue. [#17637](https://github.com/ClickHouse/ClickHouse/pull/17637) ([Amos Bird](https://github.com/amosbird)). -* Don't throw "Too many parts" error in the middle of INSERT query. [#17566](https://github.com/ClickHouse/ClickHouse/pull/17566) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow query parameters in UPDATE statement of ALTER query. Fixes [#10976](https://github.com/ClickHouse/ClickHouse/issues/10976). [#17563](https://github.com/ClickHouse/ClickHouse/pull/17563) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Query obfuscator: avoid usage of some SQL keywords for identifier names. [#17526](https://github.com/ClickHouse/ClickHouse/pull/17526) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Export current max ddl entry executed by DDLWorker. It's useful to check if DDLWorker hangs somewhere. [#17464](https://github.com/ClickHouse/ClickHouse/pull/17464) ([Amos Bird](https://github.com/amosbird)). -* Export asynchronous metrics of all servers current threads. It's useful to track down issues like https://github.com/ClickHouse-Extras/poco/pull/28. [#17463](https://github.com/ClickHouse/ClickHouse/pull/17463) ([Amos Bird](https://github.com/amosbird)). -* Return dynamic columns like MATERIALIZED / ALIAS for wildcard query when switches `asterisk_include_materialized_columns` and `asterisk_include_alias_columns` are turned on. [#17462](https://github.com/ClickHouse/ClickHouse/pull/17462) ([Ken Chen](https://github.com/chenziliang)). -* Add functions countMatches/countMatchesCaseInsensitive. [#17459](https://github.com/ClickHouse/ClickHouse/pull/17459) ([Azat Khuzhin](https://github.com/azat)). -* Allow specifying [TTL](https://clickhouse.tech/docs/en/engines/table-engines/mergetree-family/mergetree/#mergetree-table-ttl) to remove old entries from [system log tables](https://clickhouse.tech/docs/en/operations/system-tables/), using the `` attribute in `config.xml`. [#17438](https://github.com/ClickHouse/ClickHouse/pull/17438) ([Du Chuan](https://github.com/spongedu)). -* Now queries coming to the server via MySQL and PostgreSQL protocols have distinctive interface types (which can be seen in the `interface` column of the table`system.query_log`): `4` for MySQL, and `5` for PostgreSQL, instead of formerly used `1` which is now used for the native protocol only. [#17437](https://github.com/ClickHouse/ClickHouse/pull/17437) ([Vitaly Baranov](https://github.com/vitlibar)). -* Simplify Sys/V init script. It was not working on Ubuntu 12.04. [#17428](https://github.com/ClickHouse/ClickHouse/pull/17428) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Multiple improvements in `./clickhouse install` script. [#17421](https://github.com/ClickHouse/ClickHouse/pull/17421) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix parsing of SETTINGS clause of the INSERT ... SELECT ... SETTINGS query. [#17414](https://github.com/ClickHouse/ClickHouse/pull/17414) ([Azat Khuzhin](https://github.com/azat)). -* Replaced `malloc` with `new`, so that the `MemoryTracker` takes this memory into account. [#17412](https://github.com/ClickHouse/ClickHouse/pull/17412) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Add eof check in receiveHello to prevent getting `Attempt to read after eof` exception. [#17365](https://github.com/ClickHouse/ClickHouse/pull/17365) ([Kruglov Pavel](https://github.com/Avogar)). -* Implement `countSubstrings()`/`countSubstringsCaseInsensitive()`/`countSubstringsCaseInsensitiveUTF8()` (Count the number of substring occurrences). [#17347](https://github.com/ClickHouse/ClickHouse/pull/17347) ([Azat Khuzhin](https://github.com/azat)). -* Allow to use `optimize_move_to_prewhere` optimization with compact parts, when sizes of columns are unknown. [#17330](https://github.com/ClickHouse/ClickHouse/pull/17330) ([Anton Popov](https://github.com/CurtizJ)). -* Improved minimal Web UI: add history; add sharing support; avoid race condition of different requests; add request in-flight and ready indicators; add favicon; detect Ctrl+Enter if textarea is not in focus. [#17293](https://github.com/ClickHouse/ClickHouse/pull/17293) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Avoid possible stack overflow in bigint conversion. Big integers are experimental. [#17269](https://github.com/ClickHouse/ClickHouse/pull/17269) ([flynn](https://github.com/ucasFL)). -* Now set indices will work with `GLOBAL IN`. This fixes [#17232](https://github.com/ClickHouse/ClickHouse/issues/17232) , [#5576](https://github.com/ClickHouse/ClickHouse/issues/5576) . [#17253](https://github.com/ClickHouse/ClickHouse/pull/17253) ([Amos Bird](https://github.com/amosbird)). -* - Add limit for http redirects in request to S3 storage ('s3_max_redirects'). [#17220](https://github.com/ClickHouse/ClickHouse/pull/17220) ([ianton-ru](https://github.com/ianton-ru)). -* - Add configuration for multi zookeeper clusters. [#17070](https://github.com/ClickHouse/ClickHouse/pull/17070) ([fastio](https://github.com/fastio)). -* When `-OrNull` combinator combined `-If`, `-Merge`, `-MergeState`, `-State` combinators, we should put `-OrNull` in front. [#16935](https://github.com/ClickHouse/ClickHouse/pull/16935) ([flynn](https://github.com/ucasFL)). -* Support HTTP proxy and HTTPS S3 endpoint configuration. [#16861](https://github.com/ClickHouse/ClickHouse/pull/16861) ([Pavel Kovalenko](https://github.com/Jokser)). -* Added proper authentication using environment, `~/.aws` and `AssumeRole` for S3 client. [#16856](https://github.com/ClickHouse/ClickHouse/pull/16856) ([Vladimir Chebotarev](https://github.com/excitoon)). -* - New IP Dictionary implementation with lower memory consumption, improved performance for some cases, and fixed bugs. [#16804](https://github.com/ClickHouse/ClickHouse/pull/16804) ([vdimir](https://github.com/vdimir)). -* Add more OpenTelemetry spans. Add an example of how to export the span data to Zipkin. [#16535](https://github.com/ClickHouse/ClickHouse/pull/16535) ([Alexander Kuzmenkov](https://github.com/akuzm)). -* - Added `verification_cooldown` parameter in LDAP server connection configuration to allow caching of successful "bind" attempts for configurable period of time. [#15988](https://github.com/ClickHouse/ClickHouse/pull/15988) ([Denis Glazachev](https://github.com/traceon)). -* Avoid deadlock when executing INSERT SELECT into itself from a table with `TinyLog` or `Log` table engines. This closes [#6802](https://github.com/ClickHouse/ClickHouse/issues/6802). This closes [#18691](https://github.com/ClickHouse/ClickHouse/issues/18691). This closes [#16812](https://github.com/ClickHouse/ClickHouse/issues/16812). This closes [#14570](https://github.com/ClickHouse/ClickHouse/issues/14570). [#15260](https://github.com/ClickHouse/ClickHouse/pull/15260) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* fix [#9117](https://github.com/ClickHouse/ClickHouse/issues/9117). [#15127](https://github.com/ClickHouse/ClickHouse/pull/15127) ([flynn](https://github.com/ucasFL)). -* * Completely eliminate callbacks and locks for acquiring them. * Keys are not divided into "not found" and "expired", but stored in the same map during query. [#14958](https://github.com/ClickHouse/ClickHouse/pull/14958) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Allow using `Atomic` engine for nested database of `MaterializeMySQL` engine. [#14849](https://github.com/ClickHouse/ClickHouse/pull/14849) ([tavplubix](https://github.com/tavplubix)). -* Implement new parser based on ANTLR4 runtime and generated from EBNF grammar. [#11298](https://github.com/ClickHouse/ClickHouse/pull/11298) ([Ivan](https://github.com/abyss7)). - -#### Performance Improvement - -* Add `--no-system-table` option for `clickhouse-local` to run without system tables. This avoids initialization of `DateLUT` that may take noticeable amount of time (tens of milliseconds) at startup. [#18899](https://github.com/ClickHouse/ClickHouse/pull/18899) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Replace `PODArray` with `PODArrayWithStackMemory` in `AggregateFunctionWindowFunnelData` to improvement `windowFunnel` function performance. [#18817](https://github.com/ClickHouse/ClickHouse/pull/18817) ([flynn](https://github.com/ucasFL)). -* Don't send empty blocks to shards on synchronous INSERT into Distributed table. This closes [#14571](https://github.com/ClickHouse/ClickHouse/issues/14571). [#18775](https://github.com/ClickHouse/ClickHouse/pull/18775) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Optimized read for StorageMemory. [#18052](https://github.com/ClickHouse/ClickHouse/pull/18052) ([Maksim Kita](https://github.com/kitaisreal)). -* Using dragonbox algorithm for float to string conversion instead of ryu. [#17831](https://github.com/ClickHouse/ClickHouse/pull/17831) ([Maksim Kita](https://github.com/kitaisreal)). -* Speedup `IPv6CIDRToRange` implementation. [#17569](https://github.com/ClickHouse/ClickHouse/pull/17569) ([vdimir](https://github.com/vdimir)). -* Add `remerge_sort_lowered_memory_bytes_ratio` setting (If memory usage after remerge does not reduced by this ratio, remerge will be disabled). [#17539](https://github.com/ClickHouse/ClickHouse/pull/17539) ([Azat Khuzhin](https://github.com/azat)). -* Improve performance of AggregatingMergeTree w/ SimpleAggregateFunction(String) in PK. [#17109](https://github.com/ClickHouse/ClickHouse/pull/17109) ([Azat Khuzhin](https://github.com/azat)). -* Now the `-If` combinator is devirtualized, and `count` is properly vectorized. This is for https://github.com/ClickHouse/ClickHouse/pull/17041. [#17043](https://github.com/ClickHouse/ClickHouse/pull/17043) ([Amos Bird](https://github.com/amosbird)). -* Fix performance of reading from `Merge` tables over huge number of `MergeTree` tables. Fixes [#7748](https://github.com/ClickHouse/ClickHouse/issues/7748). [#16988](https://github.com/ClickHouse/ClickHouse/pull/16988) ([Anton Popov](https://github.com/CurtizJ)). -* Improved performance of function `repeat`. [#16937](https://github.com/ClickHouse/ClickHouse/pull/16937) ([satanson](https://github.com/satanson)). -* Slightly improved performance of float parsing. [#16809](https://github.com/ClickHouse/ClickHouse/pull/16809) ([Maksim Kita](https://github.com/kitaisreal)). -* Add possibility to skip merged partitions for OPTIMIZE TABLE ... FINAL. [#15939](https://github.com/ClickHouse/ClickHouse/pull/15939) ([Kruglov Pavel](https://github.com/Avogar)). #### Build/Testing/Packaging Improvement -* Add `SYSTEM SUSPEND` command for fault injection. It can be used to faciliate failover tests. This closes [#15979](https://github.com/ClickHouse/ClickHouse/issues/15979). [#18850](https://github.com/ClickHouse/ClickHouse/pull/18850) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Removed the -finline-hint-functions flag not present in GCC. [#18846](https://github.com/ClickHouse/ClickHouse/pull/18846) ([Mike](https://github.com/myrrc)). * Add simple integrity check for ClickHouse binary. It allows to detect corruption due to faulty hardware (bit rot on storage media or bit flips in RAM). [#18811](https://github.com/ClickHouse/ClickHouse/pull/18811) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Change `OpenSSL` to `BoringSSL`. It allows to avoid issues with sanitizers. This fixes [#12490](https://github.com/ClickHouse/ClickHouse/issues/12490). This fixes [#17502](https://github.com/ClickHouse/ClickHouse/issues/17502). This fixes [#12952](https://github.com/ClickHouse/ClickHouse/issues/12952). [#18129](https://github.com/ClickHouse/ClickHouse/pull/18129) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Simplify `Sys/V` init script. It was not working on Ubuntu 12.04 or older. [#17428](https://github.com/ClickHouse/ClickHouse/pull/17428) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Multiple improvements in `./clickhouse install` script. [#17421](https://github.com/ClickHouse/ClickHouse/pull/17421) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now ClickHouse can pretend to be a fake ZooKeeper. Currently, storage implementation is just stored in-memory hash-table, and server partially support ZooKeeper protocol. [#16877](https://github.com/ClickHouse/ClickHouse/pull/16877) ([alesapin](https://github.com/alesapin)). +* Fix dead list watches removal for TestKeeperStorage (a mock for ZooKeeper). [#18065](https://github.com/ClickHouse/ClickHouse/pull/18065) ([alesapin](https://github.com/alesapin)). +* Add `SYSTEM SUSPEND` command for fault injection. It can be used to faciliate failover tests. This closes [#15979](https://github.com/ClickHouse/ClickHouse/issues/15979). [#18850](https://github.com/ClickHouse/ClickHouse/pull/18850) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Generate build id when ClickHouse is linked with `lld`. It's appeared that `lld` does not generate it by default on my machine. Build id is used for crash reports and introspection. [#18808](https://github.com/ClickHouse/ClickHouse/pull/18808) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* TestFlows: fixes to LDAP tests that fail due to slow test execution. [#18790](https://github.com/ClickHouse/ClickHouse/pull/18790) ([vzakaznikov](https://github.com/vzakaznikov)). * Fix shellcheck errors in style check. [#18566](https://github.com/ClickHouse/ClickHouse/pull/18566) ([Ilya Yatsishin](https://github.com/qoega)). * Update timezones info to 2020e. [#18531](https://github.com/ClickHouse/ClickHouse/pull/18531) ([alesapin](https://github.com/alesapin)). * Fix codespell warnings. Split style checks into separate parts. Update style checks docker image. [#18463](https://github.com/ClickHouse/ClickHouse/pull/18463) ([Ilya Yatsishin](https://github.com/qoega)). -* Check for leftovers of conflict markers in docs. [#18332](https://github.com/ClickHouse/ClickHouse/pull/18332) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Automated check for leftovers of conflict markers in docs. [#18332](https://github.com/ClickHouse/ClickHouse/pull/18332) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Enable Thread Fuzzer for stateless tests flaky check. [#18299](https://github.com/ClickHouse/ClickHouse/pull/18299) ([alesapin](https://github.com/alesapin)). -* Merging requirements for AES encryption functions. Updating aes_encryption tests to use new requirements. Updating TestFlows version to 1.6.72. [#18221](https://github.com/ClickHouse/ClickHouse/pull/18221) ([vzakaznikov](https://github.com/vzakaznikov)). -* - Updating TestFlows version to the latest 1.6.72 - Re-generating requirements.py. [#18208](https://github.com/ClickHouse/ClickHouse/pull/18208) ([vzakaznikov](https://github.com/vzakaznikov)). * Do not use non thread-safe function `strerror`. [#18204](https://github.com/ClickHouse/ClickHouse/pull/18204) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Update `anchore/scan-action@main` workflow action (was moved from master). [#18192](https://github.com/ClickHouse/ClickHouse/pull/18192) ([Stig Bakken](https://github.com/stigsb)). -* Fix usage of uninitialized value in function toModifiedJulianDayOrNull, reported by MSan. Was discovered [here](https://github.com/ClickHouse/ClickHouse/pull/17726#issuecomment-744050500). [#18172](https://github.com/ClickHouse/ClickHouse/pull/18172) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Change OpenSSL to BoringSSL. It allows to avoid issues with sanitizers. This fixes [#12490](https://github.com/ClickHouse/ClickHouse/issues/12490). This fixes [#17502](https://github.com/ClickHouse/ClickHouse/issues/17502). This fixes [#12952](https://github.com/ClickHouse/ClickHouse/issues/12952). [#18129](https://github.com/ClickHouse/ClickHouse/pull/18129) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now, `clickhouse-test` DROP/CREATE databases with a timeout. [#18098](https://github.com/ClickHouse/ClickHouse/pull/18098) ([alesapin](https://github.com/alesapin)). -* Adjusting timeouts a bit, in the good hope that it will prevent flakiness of the test. [#18000](https://github.com/ClickHouse/ClickHouse/pull/18000) ([filimonov](https://github.com/filimonov)). -* Enable Pytest framework for stateless tests. [#17902](https://github.com/ClickHouse/ClickHouse/pull/17902) ([Ivan](https://github.com/abyss7)). -* Add our own CMakeList for dragonbox which was added in [#17831](https://github.com/ClickHouse/ClickHouse/issues/17831). [#17869](https://github.com/ClickHouse/ClickHouse/pull/17869) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Updating TestFlows README.md to include "How To Debug Why Test Failed" section. [#17808](https://github.com/ClickHouse/ClickHouse/pull/17808) ([vzakaznikov](https://github.com/vzakaznikov)). -* - Testflows tests for RBAC [ACCESS MANAGEMENT](https://clickhouse.tech/docs/en/sql-reference/statements/grant/#grant-access-management) privileges. [#17804](https://github.com/ClickHouse/ClickHouse/pull/17804) ([MyroTk](https://github.com/MyroTk)). +* Update `anchore/scan-action@main` workflow action (was moved from `master` to `main`). [#18192](https://github.com/ClickHouse/ClickHouse/pull/18192) ([Stig Bakken](https://github.com/stigsb)). +* Now `clickhouse-test` does DROP/CREATE databases with a timeout. [#18098](https://github.com/ClickHouse/ClickHouse/pull/18098) ([alesapin](https://github.com/alesapin)). +* Enable experimental support for Pytest framework for stateless tests. [#17902](https://github.com/ClickHouse/ClickHouse/pull/17902) ([Ivan](https://github.com/abyss7)). * Now we use the fresh docker daemon version in integration tests. [#17671](https://github.com/ClickHouse/ClickHouse/pull/17671) ([alesapin](https://github.com/alesapin)). -* - RBAC testflows tests for SHOW, TRUNCATE, KILL, and OPTIMIZE. - Updates to old tests. - Resolved comments from #https://github.com/ClickHouse/ClickHouse/pull/16977. [#17657](https://github.com/ClickHouse/ClickHouse/pull/17657) ([MyroTk](https://github.com/MyroTk)). -* Add an integration test: ClickHouse killed while insert for MaterializeMySQL ENGINE. [#17622](https://github.com/ClickHouse/ClickHouse/pull/17622) ([TCeason](https://github.com/TCeason)). -* Add an integration test: MySQL server killed while insert for MaterializeMySQL Engine. [#17614](https://github.com/ClickHouse/ClickHouse/pull/17614) ([TCeason](https://github.com/TCeason)). * Send info about official build, memory, cpu and free disk space to Sentry if it is enabled. Sentry is opt-in feature to help ClickHouse developers. This closes [#17279](https://github.com/ClickHouse/ClickHouse/issues/17279). [#17543](https://github.com/ClickHouse/ClickHouse/pull/17543) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* There was an uninitialized variable in the code of Copier. [#17363](https://github.com/ClickHouse/ClickHouse/pull/17363) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* `PODArray` does not initialize "new" elements when resizing, unlike `std::vector`. This probably fixes [this failure](https://clickhouse-test-reports.s3.yandex.net/17309/065cd002578f2e8228f12a2744bd40c970065e0c/stress_test_(memory)/stderr.log) from [#17309](https://github.com/ClickHouse/ClickHouse/issues/17309). [#17344](https://github.com/ClickHouse/ClickHouse/pull/17344) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* * Added RBAC tests for `ATTACH`, `CREATE`, `DROP`, and `DETACH`. [#16977](https://github.com/ClickHouse/ClickHouse/pull/16977) ([MyroTk](https://github.com/MyroTk)). -* Now ClickHouse can pretend to be a fake ZooKeeper. Currently, storage implementation is just stored in-memory hash-table, and server partially support ZooKeeper protocol. [#16877](https://github.com/ClickHouse/ClickHouse/pull/16877) ([alesapin](https://github.com/alesapin)). -* Add some test for MaterializeMySQL. e.g. network partition, MySQL kill sync thread... [#16806](https://github.com/ClickHouse/ClickHouse/pull/16806) ([TCeason](https://github.com/TCeason)). -* ... Detailed description / Documentation draft: ClickHouse-Extras repo contains fix for the issue with ipv6 in Arrow Flight library. See https://github.com/ClickHouse/ClickHouse/pull/16243#issuecomment-720830294 for details. [#16664](https://github.com/ClickHouse/ClickHouse/pull/16664) ([Zhanna](https://github.com/FawnD2)). +* There was an uninitialized variable in the code of clickhouse-copier. [#17363](https://github.com/ClickHouse/ClickHouse/pull/17363) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix [one MSan report](https://clickhouse-test-reports.s3.yandex.net/17309/065cd002578f2e8228f12a2744bd40c970065e0c/stress_test_(memory)/stderr.log) from [#17309](https://github.com/ClickHouse/ClickHouse/issues/17309). [#17344](https://github.com/ClickHouse/ClickHouse/pull/17344) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fix for the issue with IPv6 in Arrow Flight library. See [the comments](https://github.com/ClickHouse/ClickHouse/pull/16243#issuecomment-720830294) for details. [#16664](https://github.com/ClickHouse/ClickHouse/pull/16664) ([Zhanna](https://github.com/FawnD2)). * Add a library that replaces some `libc` functions to traps that will terminate the process. [#16366](https://github.com/ClickHouse/ClickHouse/pull/16366) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Provide diagnostics in server logs in case of stack overflow, send error message to clickhouse-client. This closes [#14840](https://github.com/ClickHouse/ClickHouse/issues/14840). [#16346](https://github.com/ClickHouse/ClickHouse/pull/16346) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Now we can run almost all stateless functional tests in parallel. [#15236](https://github.com/ClickHouse/ClickHouse/pull/15236) ([alesapin](https://github.com/alesapin)). +* Fix corruption in `librdkafka` snappy decompression (was a problem only for gcc10 builds, but official builds uses clang already, so at least recent official releases are not affected). [#18053](https://github.com/ClickHouse/ClickHouse/pull/18053) ([Azat Khuzhin](https://github.com/azat)). * If server was terminated by OOM killer, print message in log. [#13516](https://github.com/ClickHouse/ClickHouse/pull/13516) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* PODArray: Avoid call to memcpy with (nullptr, 0) arguments (Fix UBSan report). This fixes [#18525](https://github.com/ClickHouse/ClickHouse/issues/18525). [#18526](https://github.com/ClickHouse/ClickHouse/pull/18526) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Minor improvement for path concatenation of zookeeper paths inside DDLWorker. [#17767](https://github.com/ClickHouse/ClickHouse/pull/17767) ([Bharat Nallan](https://github.com/bharatnc)). +* Allow to reload symbols from debug file. This PR also fixes a build-id issue. [#17637](https://github.com/ClickHouse/ClickHouse/pull/17637) ([Amos Bird](https://github.com/amosbird)). +* TestFlows: fixes to LDAP tests that fail due to slow test execution. [#18790](https://github.com/ClickHouse/ClickHouse/pull/18790) ([vzakaznikov](https://github.com/vzakaznikov)). +* TestFlows: Merging requirements for AES encryption functions. Updating aes_encryption tests to use new requirements. Updating TestFlows version to 1.6.72. [#18221](https://github.com/ClickHouse/ClickHouse/pull/18221) ([vzakaznikov](https://github.com/vzakaznikov)). +* TestFlows: Updating TestFlows version to the latest 1.6.72. Re-generating requirements.py. [#18208](https://github.com/ClickHouse/ClickHouse/pull/18208) ([vzakaznikov](https://github.com/vzakaznikov)). +* TestFlows: Updating TestFlows README.md to include "How To Debug Why Test Failed" section. [#17808](https://github.com/ClickHouse/ClickHouse/pull/17808) ([vzakaznikov](https://github.com/vzakaznikov)). +* TestFlows: tests for RBAC [ACCESS MANAGEMENT](https://clickhouse.tech/docs/en/sql-reference/statements/grant/#grant-access-management) privileges. [#17804](https://github.com/ClickHouse/ClickHouse/pull/17804) ([MyroTk](https://github.com/MyroTk)). +* TestFlows: RBAC tests for SHOW, TRUNCATE, KILL, and OPTIMIZE. - Updates to old tests. - Resolved comments from #https://github.com/ClickHouse/ClickHouse/pull/16977. [#17657](https://github.com/ClickHouse/ClickHouse/pull/17657) ([MyroTk](https://github.com/MyroTk)). +* TestFlows: Added RBAC tests for `ATTACH`, `CREATE`, `DROP`, and `DETACH`. [#16977](https://github.com/ClickHouse/ClickHouse/pull/16977) ([MyroTk](https://github.com/MyroTk)). -#### NO CL ENTRY -* NO CL ENTRY: 'Revert "Add metrics for part number in MergeTree in ClickHouse"'. [#18834](https://github.com/ClickHouse/ClickHouse/pull/18834) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Fix typo in array functions' documentation'. [#18792](https://github.com/ClickHouse/ClickHouse/pull/18792) ([Bertrand Junqua](https://github.com/Bertrand31)). -* NO CL ENTRY: 'Revert "Add some extra tests to copier"'. [#18636](https://github.com/ClickHouse/ClickHouse/pull/18636) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* NO CL ENTRY: 'Revert "Fix access rights required for the merge() table function."'. [#18103](https://github.com/ClickHouse/ClickHouse/pull/18103) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Исправил опечатку в названии ОС RedHad->RedHat'. [#18028](https://github.com/ClickHouse/ClickHouse/pull/18028) ([Erixonich](https://github.com/Erixonich)). -* NO CL ENTRY: 'Revert "Date vs DateTime64 comparison"'. [#17985](https://github.com/ClickHouse/ClickHouse/pull/17985) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "Fix index granularity calculation on block borders"'. [#17918](https://github.com/ClickHouse/ClickHouse/pull/17918) ([alesapin](https://github.com/alesapin)). -* NO CL ENTRY: 'Update README.md'. [#17596](https://github.com/ClickHouse/ClickHouse/pull/17596) ([Robert Hodges](https://github.com/hodgesrm)). -* NO CL ENTRY: 'Revert "Bump mkdocs-macros-plugin from 0.4.20 to 0.5.0 in /docs/tools"'. [#17405](https://github.com/ClickHouse/ClickHouse/pull/17405) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* NO CL ENTRY: 'Revert "Attempt to fix Stress test (MSan)"'. [#17372](https://github.com/ClickHouse/ClickHouse/pull/17372) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +## [Changelog for 2020](https://github.com/ClickHouse/ClickHouse/blob/master/docs/en/whats-new/changelog/2020.md) diff --git a/utils/simple-backport/format-changelog.py b/utils/simple-backport/format-changelog.py index 861faafdcfd..56fe973eb6f 100755 --- a/utils/simple-backport/format-changelog.py +++ b/utils/simple-backport/format-changelog.py @@ -78,7 +78,7 @@ def parse_one_pull_request(item): # This array gives the preferred category order, and is also used to # normalize category names. categories_preferred_order = ['Backward Incompatible Change', - 'New Feature', 'Bug Fix', 'Improvement', 'Performance Improvement', + 'New Feature', 'Performance Improvement', 'Improvement', 'Bug Fix', 'Build/Testing/Packaging Improvement', 'Other'] category_to_pr = collections.defaultdict(lambda: [])