From f7c7c5a9c7b2960c4e41f8c1cbfeb570c44f7e9d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 9 Mar 2021 20:25:23 +0300 Subject: [PATCH 001/358] Revert "refactoring of serializations" This reverts commit df6663dcb6943ca53f3c8d862dd536c8d526a594. --- programs/server/config.xml | 2 + src/Columns/ColumnArray.cpp | 18 ++ src/Columns/ColumnArray.h | 4 + src/Columns/ColumnString.cpp | 16 ++ src/Columns/ColumnString.h | 3 + src/Columns/ColumnVector.h | 17 ++ src/Columns/IColumn.h | 3 + src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp | 6 + src/DataTypes/DataTypeCustom_fwd.h | 18 ++ src/DataTypes/DataTypeFixedString.cpp | 1 + src/DataTypes/DataTypeTuple.cpp | 23 +++ src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/IDataType.cpp | 51 +++++- src/DataTypes/IDataType.h | 27 ++- src/DataTypes/Serializations/ISerialization.h | 7 + .../Serializations/SerializationIP.cpp | 1 - .../Serializations/SerializationInfo.cpp | 122 ++++++++++++++ .../Serializations/SerializationInfo.h | 32 ++++ .../Serializations/SerializationNothing.h | 2 - .../Serializations/SerializationSparse.cpp | 155 ++++++++++++++++++ .../Serializations/SerializationSparse.h | 42 +++++ .../Serializations/SerializationUUID.h | 2 - .../Serializations/SerializationWrapper.h | 2 - src/DataTypes/ya.make | 5 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 + .../MergeTree/MergeTreeDataPartWide.cpp | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 8 +- .../MergeTreeDataPartWriterOnDisk.cpp | 3 - .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 3 - .../MergeTree/MergeTreeDataPartWriterWide.cpp | 9 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 3 + .../MergeTree/MergeTreeDataWriter.cpp | 1 + .../MergeTree/MergedBlockOutputStream.cpp | 12 ++ .../MergeTree/MergedBlockOutputStream.h | 1 + 35 files changed, 566 insertions(+), 41 deletions(-) create mode 100644 src/DataTypes/DataTypeCustom_fwd.h create mode 100644 src/DataTypes/Serializations/SerializationInfo.cpp create mode 100644 src/DataTypes/Serializations/SerializationInfo.h create mode 100644 src/DataTypes/Serializations/SerializationSparse.cpp create mode 100644 src/DataTypes/Serializations/SerializationSparse.h diff --git a/programs/server/config.xml b/programs/server/config.xml index b72cf53ca03..0745f16fa76 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -117,6 +117,7 @@ This setting could be used to switch replication to another network interface (the server may be connected to multiple networks via multiple addresses) --> + @@ -142,6 +143,7 @@ --> + diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 2ab784febc6..9bea4c4bedc 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1206,4 +1206,22 @@ void ColumnArray::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } +void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices) const +{ + const auto & offsets_data = getOffsets(); + for (size_t i = 0; i < offsets_data.size(); ++i) + if (offsets_data[i] != offsets_data[i - 1]) + indices.push_back(i); +} + +size_t ColumnArray::getNumberOfNonDefaultValues() const +{ + const auto & offsets_data = getOffsets(); + size_t res = 0; + for (size_t i = 0; i < offsets_data.size(); ++i) + res += (offsets_data[i] != offsets_data[i - 1]); + + return res; +} + } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 7d01d04735b..19614365b2a 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -139,6 +139,10 @@ public: return false; } + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices) const override; + + size_t getNumberOfNonDefaultValues() const override; + bool isCollationSupported() const override { return getData().isCollationSupported(); } private: diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 2316fbaa375..61276909a55 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -530,6 +530,22 @@ void ColumnString::getExtremes(Field & min, Field & max) const get(max_idx, max); } +void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices) const +{ + for (size_t i = 0; i < offsets.size(); ++i) + if (offsets[i] - offsets[i - 1] > 1) + indices.push_back(i); +} + +size_t ColumnString::getNumberOfNonDefaultValues() const +{ + size_t res = 0; + for (size_t i = 0; i < offsets.size(); ++i) + res += (offsets[i] - offsets[i - 1] > 1); + + return res; +} + ColumnPtr ColumnString::compress() const { size_t source_chars_size = chars.size(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index e6aed113501..fe09abda7b5 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -277,6 +277,9 @@ public: return typeid(rhs) == typeid(ColumnString); } + void getIndicesOfNonDefaultValues(Offsets & indices) const override; + size_t getNumberOfNonDefaultValues() const override; + Chars & getChars() { return chars; } const Chars & getChars() const { return chars; } diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 5af5ef20310..a7ee420c2cc 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -303,6 +303,23 @@ public: return typeid(rhs) == typeid(ColumnVector); } + void getIndicesOfNonDefaultValues(IColumn::Offsets & offsets) const override + { + offsets.reserve(data.size()); + for (size_t i = 0; i < data.size(); ++i) + if (data[i] != T{}) + offsets.push_back(i); + } + + size_t getNumberOfNonDefaultValues() const override + { + size_t res = 0; + for (size_t i = 0; i < data.size(); ++i) + res += (data[i] != T{}); + + return res; + } + ColumnPtr compress() const override; /// Replace elements that match the filter with zeroes. If inverted replaces not matched elements. diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 2b8a5eb9b9f..44b7e280118 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -363,6 +363,9 @@ public: throw Exception("Method structureEquals is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + virtual void getIndicesOfNonDefaultValues(Offsets & /* offsets */) const {} + virtual size_t getNumberOfNonDefaultValues() const { return 0; } + /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. virtual Ptr compress() const diff --git a/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp b/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp index 808aa43528e..f0eeaa21be8 100644 --- a/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp +++ b/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp @@ -5,6 +5,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; +} + void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory) { factory.registerSimpleDataTypeCustom("IPv4", [] diff --git a/src/DataTypes/DataTypeCustom_fwd.h b/src/DataTypes/DataTypeCustom_fwd.h new file mode 100644 index 00000000000..39c82bc4366 --- /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/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp index a40592ba023..4b6042febab 100644 --- a/src/DataTypes/DataTypeFixedString.cpp +++ b/src/DataTypes/DataTypeFixedString.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int UNEXPECTED_AST_STRUCTURE; + extern const int TOO_LARGE_STRING_SIZE; } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index b30efb163ab..371a50a32cf 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -329,6 +330,28 @@ SerializationPtr DataTypeTuple::getSerialization(const String & column_name, con return std::make_shared(std::move(serializations), have_explicit_names); } +SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const SerializationInfo & info) const +{ + SerializationTuple::ElementSerializations serializations(elems.size()); + for (size_t i = 0; i < elems.size(); ++i) + { + auto subcolumn_name = Nested::concatenateName(column_name, names[i]); + + ISerialization::Settings settings = + { + .num_rows = info.getNumberOfRows(), + .num_non_default_rows = info.getNumberOfNonDefaultValues(subcolumn_name), + .min_ratio_for_dense_serialization = 10 + }; + + auto serializaion = elems[i]->getSerialization(settings); + serializations[i] = std::make_shared(serializaion, names[i]); + } + + return std::make_shared(std::move(serializations), have_explicit_names); +} + + static DataTypePtr create(const ASTPtr & arguments) { if (!arguments || arguments->children.empty()) diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index e572b23f987..8b301674e61 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -56,6 +56,7 @@ public: ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const override; + SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const override; SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index c0679557ec9..2b09bc12054 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -11,6 +11,8 @@ #include #include #include +#include +#include #include @@ -146,17 +148,55 @@ SerializationPtr IDataType::getDefaultSerialization() const return doGetDefaultSerialization(); } +SerializationPtr IDataType::getSparseSerialization() const +{ + return std::make_shared(getDefaultSerialization()); +} + SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_name, const BaseSerializationGetter &) const { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } +SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const +{ + ISerialization::Settings settings = + { + .num_rows = info.getNumberOfRows(), + .num_non_default_rows = info.getNumberOfNonDefaultValues(column_name), + .min_ratio_for_dense_serialization = 10 + }; + + return getSerialization(settings); +} + +SerializationPtr IDataType::getSerialization(const IColumn & column) const +{ + ISerialization::Settings settings = + { + .num_rows = column.size(), + .num_non_default_rows = column.getNumberOfNonDefaultValues(), + .min_ratio_for_dense_serialization = 10 + }; + + return getSerialization(settings); +} + +SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const +{ + // if (settings.num_non_default_rows * settings.min_ratio_for_dense_serialization < settings.num_rows) + // return getSparseSerialization(); + + UNUSED(settings); + + return getDefaultSerialization(); +} + // static SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, const IDataType::StreamExistenceCallback & callback) { if (column.isSubcolumn()) { - /// Wrap to custom serialization deepest subcolumn, which is represented in non-complex type. auto base_serialization_getter = [&](const IDataType & subcolumn_type) { return subcolumn_type.getSerialization(column.name, callback); @@ -169,8 +209,15 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, con return column.type->getSerialization(column.name, callback); } -SerializationPtr IDataType::getSerialization(const String &, const StreamExistenceCallback &) const +SerializationPtr IDataType::getSerialization(const String & column_name, const StreamExistenceCallback & callback) const { + auto sparse_idx_name = escapeForFileName(column_name) + ".sparse.idx"; + if (callback(sparse_idx_name)) + return getSparseSerialization(); + + UNUSED(column_name); + UNUSED(callback); + return getDefaultSerialization(); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 9b762cfa4c5..9e60c0b3b1d 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -31,12 +31,7 @@ class SerializationInfo; /** Properties of data type. - * - * Contains methods for getting serialization instances. - * One data type may have different serializations, which can be chosen - * dynamically before reading or writing, according to information about - * column content (see `getSerialization` methods). - * + * Contains methods for serialization/deserialization. * Implementations of this interface represent a data type (example: UInt8) * or parametric family of data types (example: Array(...)). * @@ -69,27 +64,29 @@ public: virtual ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const; Names getSubcolumnNames() const; - /// Returns default serialization of data type. - SerializationPtr getDefaultSerialization() const; - /// Asks whether the stream with given name exists in table. + SerializationPtr getDefaultSerialization() const; + SerializationPtr getSparseSerialization() const; + + /// Asks wether the stream with given name exists in table. /// If callback returned true for all streams, which are required for /// one of serialization types, that serialization will be chosen for reading. /// If callback always returned false, the default serialization will be chosen. using StreamExistenceCallback = std::function; using BaseSerializationGetter = std::function; - /// Chooses serialization for reading of one column or subcolumns by - /// checking existence of substreams using callback. + virtual SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const; + virtual SerializationPtr getSubcolumnSerialization( + const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; + static SerializationPtr getSerialization( const NameAndTypePair & column, const StreamExistenceCallback & callback = [](const String &) { return false; }); - virtual SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const; + virtual SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; - /// Returns serialization wrapper for reading one particular subcolumn of data type. - virtual SerializationPtr getSubcolumnSerialization( - const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; + SerializationPtr getSerialization(const ISerialization::Settings & settings) const; + SerializationPtr getSerialization(const IColumn & column) const; using StreamCallbackWithType = std::function; diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index bc1ecd44e93..9c0a6e683a5 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -140,6 +140,13 @@ public: double avg_value_size_hint = 0; }; + struct Settings + { + size_t num_rows; + size_t num_non_default_rows; + size_t min_ratio_for_dense_serialization; + }; + /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. virtual void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & /*settings*/, diff --git a/src/DataTypes/Serializations/SerializationIP.cpp b/src/DataTypes/Serializations/SerializationIP.cpp index ec49f960c77..d14122671d9 100644 --- a/src/DataTypes/Serializations/SerializationIP.cpp +++ b/src/DataTypes/Serializations/SerializationIP.cpp @@ -10,7 +10,6 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; - extern const int ILLEGAL_COLUMN; } SerializationIPv4::SerializationIPv4(const SerializationPtr & nested_) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp new file mode 100644 index 00000000000..cb86bac7514 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -0,0 +1,122 @@ +#include +#include +#include +#include + +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +void SerializationInfo::add(const Block & block) +{ + number_of_rows += block.rows(); + for (const auto & elem : block) + { + non_default_values[elem.name] = elem.column->getNumberOfNonDefaultValues(); + for (const auto & subname : elem.type->getSubcolumnNames()) + { + auto subcolumn = elem.type->getSubcolumn(subname, *elem.column); + auto full_name = Nested::concatenateName(elem.name, subname); + non_default_values[full_name] += subcolumn->getNumberOfNonDefaultValues(); + } + } +} + +void SerializationInfo::add(const SerializationInfo & other) +{ + number_of_rows += other.number_of_rows; + for (const auto & [name, num] : other.non_default_values) + non_default_values[name] += num; +} + +size_t SerializationInfo::getNumberOfNonDefaultValues(const String & column_name) const +{ + auto it = non_default_values.find(column_name); + if (it == non_default_values.end()) + return 0; + return it->second; +} + +namespace +{ + +constexpr auto KEY_NUMBER_OF_ROWS = "number_of_rows"; +constexpr auto KEY_NUMBER_OF_NON_DEFAULT_VALUES = "number_of_non_default_values"; +constexpr auto KEY_NUMBER = "number"; +constexpr auto KEY_NAME = "name"; +constexpr auto KEY_VERSION = "version"; + +} + +void SerializationInfo::fromJSON(const String & json_str) +{ + Poco::JSON::Parser parser; + auto object = parser.parse(json_str).extract(); + + if (object->has(KEY_NUMBER_OF_ROWS)) + number_of_rows = object->getValue(KEY_NUMBER_OF_ROWS); + + if (object->has(KEY_NUMBER_OF_NON_DEFAULT_VALUES)) + { + auto array = object->getArray(KEY_NUMBER_OF_NON_DEFAULT_VALUES); + for (const auto & elem : *array) + { + auto elem_object = elem.extract(); + if (!elem_object->has(KEY_NUMBER) || !elem_object->has(KEY_NAME)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Missed field 'name' or 'number' in SerializationInfo of columns"); + + auto name = elem_object->getValue(KEY_NAME); + auto number = elem_object->getValue(KEY_NUMBER); + non_default_values[name] = number; + } + } +} + +String SerializationInfo::toJSON() const +{ + Poco::JSON::Object info; + info.set(KEY_VERSION, version); + info.set(KEY_NUMBER_OF_ROWS, number_of_rows); + + Poco::JSON::Array column_infos; + for (const auto & [name, num] : non_default_values) + { + Poco::JSON::Object column_info; + column_info.set(KEY_NAME, name); + column_info.set(KEY_NUMBER, num); + column_infos.add(std::move(column_info)); + } + + info.set(KEY_NUMBER_OF_NON_DEFAULT_VALUES, std::move(column_infos)); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(info, oss); + + return oss.str(); +} + +void SerializationInfo::read(ReadBuffer & in) +{ + String json_str; + readString(json_str, in); + fromJSON(json_str); +} + +void SerializationInfo::write(WriteBuffer & out) const +{ + writeString(toJSON(), out); +} + +} diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h new file mode 100644 index 00000000000..ceee705cf57 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -0,0 +1,32 @@ +#pragma once + +#include + +namespace DB +{ + +class SerializationInfo +{ +public: + static constexpr auto version = 1; + + using NameToNumber = std::unordered_map; + + void add(const Block & block); + void add(const SerializationInfo & other); + + size_t getNumberOfNonDefaultValues(const String & column_name) const; + size_t getNumberOfRows() const { return number_of_rows; } + + void read(ReadBuffer & in); + void write(WriteBuffer & out) const; + +private: + void fromJSON(const String & json_str); + String toJSON() const; + + size_t number_of_rows = 0; + NameToNumber non_default_values; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationNothing.h b/src/DataTypes/Serializations/SerializationNothing.h index a7b26c117bc..e9d25c62ee0 100644 --- a/src/DataTypes/Serializations/SerializationNothing.h +++ b/src/DataTypes/Serializations/SerializationNothing.h @@ -1,5 +1,3 @@ -#pragma once - #include #include diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp new file mode 100644 index 00000000000..19dddf84464 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -0,0 +1,155 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +void serializeOffsetsPositionIndependent(const IColumn::Offsets & offsets, WriteBuffer & ostr) +{ + size_t size = offsets.size(); + IColumn::Offset prev_offset = 0; + for (size_t i = 0; i < size; ++i) + { + IColumn::Offset current_offset = offsets[i]; + writeIntBinary(current_offset - prev_offset, ostr); + prev_offset = current_offset; + } +} + +void deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, ReadBuffer & istr) +{ + IColumn::Offset current_offset = 0; + while (!istr.eof()) + { + IColumn::Offset current_size = 0; + readIntBinary(current_size, istr); + current_offset += current_size; + offsets.push_back(current_offset); + } +} + +} + +SerializationSparse::SerializationSparse(const SerializationPtr & nested_serialization_) + : SerializationWrapper(nested_serialization_) +{ +} + +void SerializationSparse::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +{ + path.push_back(Substream::SparseOffsets); + callback(path); + path.back() = Substream::SparseElements; + nested_serialization->enumerateStreams(callback, path); + path.pop_back(); +} + +void SerializationSparse::serializeBinaryBulkStatePrefix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::SparseElements); + nested_serialization->serializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); +} + +void SerializationSparse::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::SparseElements); + nested_serialization->serializeBinaryBulkStateSuffix(settings, state); + settings.path.pop_back(); +} + +void SerializationSparse::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + settings.path.push_back(Substream::SparseElements); + nested_serialization->deserializeBinaryBulkStatePrefix(settings, state); + settings.path.pop_back(); +} + +void SerializationSparse::serializeBinaryBulkWithMultipleStreams( + const IColumn & column, + size_t offset, + size_t limit, + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + UNUSED(limit); + UNUSED(offset); + + /// TODO: inefficient. + /// TODO: use limit and offset + size_t size = column.size(); + + auto offsets_column = DataTypeNumber().createColumn(); + auto & offsets_data = assert_cast &>(*offsets_column).getData(); + + column.getIndicesOfNonDefaultValues(offsets_data); + auto values = column.index(*offsets_column, 0); + offsets_data.push_back(size); + + settings.path.push_back(Substream::SparseOffsets); + if (auto * stream = settings.getter(settings.path)) + serializeOffsetsPositionIndependent(offsets_data, *stream); + + settings.path.back() = Substream::SparseElements; + nested_serialization->serializeBinaryBulkWithMultipleStreams(*values, 0, 0, settings, state); + + settings.path.pop_back(); +} + +void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const +{ + settings.path.push_back(Substream::SparseOffsets); + + auto offsets_column = DataTypeNumber().createColumn(); + auto & offsets_data = assert_cast &>(*offsets_column).getData(); + + if (auto * stream = settings.getter(settings.path)) + deserializeOffsetsPositionIndependent(offsets_data, *stream); + + settings.path.back() = Substream::SparseElements; + + ColumnPtr values = column->cloneEmpty(); + nested_serialization->deserializeBinaryBulkWithMultipleStreams(values, limit, settings, state, cache); + + auto mutable_column = column->assumeMutable(); + size_t size = values->size(); + ssize_t prev_offset = -1; + + for (size_t i = 0; i < size; ++i) + { + size_t offsets_diff = static_cast(offsets_data[i]) - prev_offset; + + if (offsets_diff > 1) + mutable_column->insertManyDefaults(offsets_diff - 1); + + mutable_column->insertFrom(*values, i); + prev_offset = offsets_data[i]; + } + + size_t offsets_diff = offsets_data[size] - prev_offset; + if (offsets_diff > 1) + mutable_column->insertManyDefaults(offsets_diff - 1); + + settings.path.pop_back(); +} + +} diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h new file mode 100644 index 00000000000..a5f8c7547c3 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +namespace DB +{ + +class SerializationSparse final : public SerializationWrapper +{ +public: + SerializationSparse(const SerializationPtr & nested_); + + 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( + ColumnPtr & column, + size_t limit, + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state, + SubstreamsCache * cache) const override; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationUUID.h b/src/DataTypes/Serializations/SerializationUUID.h index 93bf166bbd9..fee6cd31853 100644 --- a/src/DataTypes/Serializations/SerializationUUID.h +++ b/src/DataTypes/Serializations/SerializationUUID.h @@ -1,5 +1,3 @@ -#pragma once - #include namespace DB diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 399d3b198b3..7fa6e3a0907 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -6,8 +6,6 @@ namespace DB { -/// Wrapper for serialization, which calls methods, which are not overridden, from nested serialization. -/// You can inherit this class, when you need to override bunch of methods, to avoid boilerplate code. class SerializationWrapper : public ISerialization { protected: diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index e7294c298e5..bd57e1a5a47 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -15,6 +15,7 @@ SRCS( DataTypeCustomGeo.cpp DataTypeCustomIPv4AndIPv6.cpp DataTypeCustomSimpleAggregateFunction.cpp + DataTypeCustomSimpleTextSerialization.cpp DataTypeDate.cpp DataTypeDateTime.cpp DataTypeDateTime64.cpp @@ -43,7 +44,6 @@ SRCS( Serializations/ISerialization.cpp Serializations/SerializationAggregateFunction.cpp Serializations/SerializationArray.cpp - Serializations/SerializationCustomSimpleText.cpp Serializations/SerializationDate.cpp Serializations/SerializationDateTime.cpp Serializations/SerializationDateTime64.cpp @@ -51,12 +51,13 @@ SRCS( Serializations/SerializationDecimalBase.cpp Serializations/SerializationEnum.cpp Serializations/SerializationFixedString.cpp - Serializations/SerializationIP.cpp + Serializations/SerializationInfo.cpp Serializations/SerializationLowCardinality.cpp Serializations/SerializationMap.cpp Serializations/SerializationNothing.cpp Serializations/SerializationNullable.cpp Serializations/SerializationNumber.cpp + Serializations/SerializationSparse.cpp Serializations/SerializationString.cpp Serializations/SerializationTuple.cpp Serializations/SerializationTupleElement.cpp diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 3da2a52e2f3..4098077d8de 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -16,6 +16,7 @@ #include #include #include +#include #include @@ -221,6 +222,8 @@ public: TTLInfos ttl_infos; + SerializationInfo serialization_info; + /// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex void setState(State new_state) const; State getState() const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 0d59e1d3e1a..eeb1c72788b 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -725,6 +725,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor need_remove_expired_values = true; force_ttl = true; } + + new_data_part->serialization_info.add(part->serialization_info); } const auto & part_min_ttl = new_data_part->ttl_infos.part_min_ttl; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 1da115efa70..1cbc9859fe9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -238,7 +238,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const { String filename; - auto serialization = column.type->getDefaultSerialization(); + auto serialization = column.type->getSerialization(column.name, serialization_info); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (filename.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 2efda206cf9..c7b5051ebd9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -63,7 +63,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, compressed_streams.emplace(stream_name, stream); }; - column.type->enumerateStreams(serializations[column.name], callback); + column.type->enumerateStreams(column.type->getDefaultSerialization(), callback); } namespace @@ -105,7 +105,6 @@ Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, /// Write single granule of one column (rows between 2 marks) void writeColumnSingleGranule( const ColumnWithTypeAndName & column, - const SerializationPtr & serialization, ISerialization::OutputStreamGetter stream_getter, size_t from_row, size_t number_of_rows) @@ -117,6 +116,7 @@ void writeColumnSingleGranule( serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; + auto serialization = column.type->getDefaultSerialization(); serialization->serializeBinaryBulkStatePrefix(serialize_settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, state); serialization->serializeBinaryBulkStateSuffix(serialize_settings, state); @@ -203,9 +203,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeIntBinary(plain_hashing.count(), marks); writeIntBinary(UInt64(0), marks); - writeColumnSingleGranule( - block.getByName(name_and_type->name), serializations[name_and_type->name], - stream_getter, granule.start_row, granule.rows_to_write); + writeColumnSingleGranule(block.getByName(name_and_type->name), stream_getter, granule.start_row, granule.rows_to_write); /// Each type always have at least one substream prev_stream->hashing_buf.next(); //-V522 diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 9902add9847..2ea35969a4e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -90,9 +90,6 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( if (!disk->exists(part_path)) disk->createDirectories(part_path); - for (const auto & column : columns_list) - serializations.emplace(column.name, column.type->getDefaultSerialization()); - if (settings.rewrite_primary_key) initPrimaryIndex(); initSkipIndices(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index d952950e461..704b38ba6d5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -132,9 +132,6 @@ protected: MergeTreeIndexAggregators skip_indices_aggregators; std::vector skip_index_accumulated_marks; - using SerializationsMap = std::unordered_map; - SerializationsMap serializations; - std::unique_ptr index_file_stream; std::unique_ptr index_stream; DataTypes index_types; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 74338dfe5e2..04179fbb781 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -112,7 +112,9 @@ void MergeTreeDataPartWriterWide::addStreams( settings.max_compress_block_size); }; - column.type->enumerateStreams(serializations[column.name], callback); + auto serialization = column.type->getSerialization(column.name, data_part->serialization_info); + column.type->enumerateStreams(serialization, callback); + serializations.emplace(column.name, std::move(serialization)); } @@ -299,7 +301,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( ISerialization::SerializeBinaryBulkSettings & serialize_settings, const Granule & granule) { - const auto & serialization = serializations[name_and_type.name]; + auto serialization = serializations[name_and_type.name]; serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, 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. @@ -404,7 +406,8 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, size_t mark_num; - const auto & serialization = serializations[name]; + auto serialization = type.getDefaultSerialization(); + for (mark_num = 0; !mrk_in.eof(); ++mark_num) { if (mark_num > index_granularity.getMarksCount()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 5eaaa0c1bbe..227c95edfb2 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -110,6 +110,9 @@ private: using ColumnStreams = std::map; ColumnStreams column_streams; + using Serializations = std::map; + Serializations serializations; + /// Non written marks to disk (for each column). Waiting until all rows for /// this marks will be written to disk. using MarksForColumns = std::unordered_map; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 7f9c5df2754..46c918d4591 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -388,6 +388,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); new_data_part->ttl_infos.update(move_ttl_infos); + new_data_part->serialization_info.add(block); /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index ef79bfae0e2..aefbe28b45b 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -83,6 +83,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( new_part->setColumns(part_columns); new_part->rows_count = rows_count; + new_part->serialization_info = serialization_info; new_part->modification_time = time(nullptr); new_part->index = writer->releaseIndexColumns(); new_part->checksums = checksums; @@ -155,6 +156,15 @@ void MergedBlockOutputStream::finalizePartOnDisk( out->sync(); } + // if (serialization_info.getNumberOfRows() > 0) + // { + // auto out = volume->getDisk()->writeFile(part_path + "serialization.txt", 4096); + // serialization_info.write(*out); + // out->finalize(); + // if (sync) + // out->sync(); + // } + if (default_codec != nullptr) { auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096); @@ -185,6 +195,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm return; writer->write(block, permutation); + serialization_info.add(block); + rows_count += rows; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index d04df598218..288d38baf4f 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -57,6 +57,7 @@ private: IMergeTreeDataPart::MinMaxIndex minmax_idx; size_t rows_count = 0; CompressionCodecPtr default_codec; + SerializationInfo serialization_info; }; } From 577d5713001b4078037e3501c3eb920eb51432a8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 12 Mar 2021 19:33:41 +0300 Subject: [PATCH 002/358] ColumnSparse: initial implementation --- src/AggregateFunctions/AggregateFunctionSum.h | 2 + src/AggregateFunctions/IAggregateFunction.h | 55 ++ src/Columns/ColumnArray.cpp | 18 +- src/Columns/ColumnArray.h | 4 +- src/Columns/ColumnSparse.cpp | 486 ++++++++++++++++++ src/Columns/ColumnSparse.h | 161 ++++++ src/Columns/ColumnString.cpp | 16 +- src/Columns/ColumnString.h | 4 +- src/Columns/ColumnVector.h | 25 +- src/Columns/IColumn.cpp | 11 + src/Columns/IColumn.h | 12 +- src/Common/SparseArray.h | 0 src/DataStreams/NativeBlockInputStream.cpp | 14 +- src/DataStreams/NativeBlockOutputStream.cpp | 16 +- src/DataStreams/NativeBlockOutputStream.h | 2 +- src/DataTypes/DataTypeTuple.cpp | 4 +- src/DataTypes/IDataType.cpp | 28 +- src/DataTypes/IDataType.h | 5 +- src/DataTypes/Serializations/ISerialization.h | 58 ++- .../Serializations/SerializationInfo.cpp | 36 +- .../Serializations/SerializationInfo.h | 12 +- .../SerializationLowCardinality.cpp | 42 +- .../Serializations/SerializationSparse.cpp | 256 ++++++--- .../Serializations/SerializationSparse.h | 5 + .../Serializations/SerializationTuple.cpp | 39 +- src/Interpreters/Aggregator.cpp | 22 +- src/Interpreters/Aggregator.h | 1 + .../Formats/Impl/PrettyBlockOutputFormat.cpp | 4 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 32 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 11 +- .../MergeTree/MergeTreeDataPartWide.cpp | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 12 +- .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 3 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 22 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 3 - .../MergeTree/MergeTreeReaderWide.cpp | 8 +- src/Storages/MergeTree/MergeTreeSettings.h | 1 + .../MergeTree/MergedBlockOutputStream.cpp | 22 +- 40 files changed, 1206 insertions(+), 252 deletions(-) create mode 100644 src/Columns/ColumnSparse.cpp create mode 100644 src/Columns/ColumnSparse.h create mode 100644 src/Common/SparseArray.h diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index bd1f9fc302e..343686a6f6d 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -361,6 +361,8 @@ public: } } + void addManyDefaults(size_t /* length */) const override {} + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).merge(this->data(rhs)); diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index d15ff4e8a78..42ffa0cd4eb 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -153,11 +154,20 @@ public: Arena * arena, ssize_t if_argument_pos = -1) const = 0; + virtual void addBatchSparse( + AggregateDataPtr * places, + size_t place_offset, + const IColumn ** columns, + Arena * arena) const = 0; + /** The same for single place. */ virtual void addBatchSinglePlace( size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const = 0; + virtual void addBatchSparseSinglePlace( + AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0; + /** The same for single place when need to aggregate only filtered data. */ virtual void addBatchSinglePlaceNotNull( @@ -213,6 +223,13 @@ public: */ virtual AggregateFunctionPtr getNestedFunction() const { return {}; } + virtual bool supportsSparseArguments() const { return false; } + + virtual void addManyDefaults(size_t /* length */) const + { + throw Exception("Method addManyDefaults is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + const DataTypes & getArgumentTypes() const { return argument_types; } const Array & getParameters() const { return parameters; } @@ -278,6 +295,32 @@ public: } } + void addBatchSparse( + AggregateDataPtr * places, + size_t place_offset, + const IColumn ** columns, + Arena * arena) const override + { + const auto & column_sparse = assert_cast(*columns[0]); + const auto * values = &column_sparse.getValuesColumn(); + const auto & offsets_data = column_sparse.getOffsetsData(); + + size_t offset_pos = 0; + size_t offsets_size = offsets_data.size(); + for (size_t i = 0; i < column_sparse.size(); ++i) + { + if (offset_pos < offsets_size && i == offsets_data[offset_pos]) + { + static_cast(this)->add(places[i] + place_offset, &values, offset_pos + 1, arena); + ++offset_pos; + } + else + { + static_cast(this)->add(places[i] + place_offset, &values, 0, arena); + } + } + } + void addBatchSinglePlace( size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const override { @@ -297,6 +340,18 @@ public: } } + void addBatchSparseSinglePlace( + AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override + { + const auto & column_sparse = assert_cast(*columns[0]); + const auto * values = &column_sparse.getValuesColumn(); + + for (size_t i = 1; i < values->size(); ++i) + static_cast(this)->add(place, &values, i, arena); + + static_cast(this)->addManyDefaults(column_sparse.getNumberOfDefaults()); + } + void addBatchSinglePlaceNotNull( size_t batch_size, AggregateDataPtr place, diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 9bea4c4bedc..e8e60cd941e 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1206,7 +1206,7 @@ void ColumnArray::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } -void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices) const +void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t, size_t) const { const auto & offsets_data = getOffsets(); for (size_t i = 0; i < offsets_data.size(); ++i) @@ -1214,14 +1214,14 @@ void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices) const indices.push_back(i); } -size_t ColumnArray::getNumberOfNonDefaultValues() const -{ - const auto & offsets_data = getOffsets(); - size_t res = 0; - for (size_t i = 0; i < offsets_data.size(); ++i) - res += (offsets_data[i] != offsets_data[i - 1]); +// size_t ColumnArray::getNumberOfDefaultRows() const +// { +// const auto & offsets_data = getOffsets(); +// size_t res = 0; +// for (size_t i = 0; i < offsets_data.size(); ++i) +// res += (offsets_data[i] != offsets_data[i - 1]); - return res; -} +// return res; +// } } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 19614365b2a..8e285eb3dff 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -139,9 +139,9 @@ public: return false; } - void getIndicesOfNonDefaultValues(IColumn::Offsets & indices) const override; + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t offset, size_t limit) const override; - size_t getNumberOfNonDefaultValues() const override; + // size_t getNumberOfDefaultRows() const override; bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp new file mode 100644 index 00000000000..e67cb45847d --- /dev/null +++ b/src/Columns/ColumnSparse.cpp @@ -0,0 +1,486 @@ +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ColumnSparse::ColumnSparse(MutableColumnPtr && values_) + : values(std::move(values_)), _size(0) +{ + if (!values->empty()) + throw Exception("Not empty values passed to ColumnSparse, but no offsets passed", ErrorCodes::LOGICAL_ERROR); + + values->insertDefault(); + offsets = ColumnUInt64::create(); +} + +ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offsets_, size_t size_) + : values(std::move(values_)), offsets(std::move(offsets_)), _size(size_) +{ + const ColumnUInt64 * offsets_concrete = typeid_cast(offsets.get()); + + if (!offsets_concrete) + throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::LOGICAL_ERROR); + + if (offsets->size() + 1 != values->size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Values size is inconsistent with offsets size. Expected: {}, got {}", offsets->size() + 1, values->size()); +} + +MutableColumnPtr ColumnSparse::cloneResized(size_t new_size) const +{ + if (new_size == 0) + return ColumnSparse::create(values->cloneEmpty()); + + if (new_size >= _size) + return ColumnSparse::create(IColumn::mutate(values), IColumn::mutate(offsets), new_size); + + auto res = ColumnSparse::create(values->cloneEmpty()); + res->insertRangeFrom(*this, 0, new_size); + return res; +} + +bool ColumnSparse::isNullAt(size_t n) const +{ + return values->isNullAt(getValueIndex(n)); +} + +Field ColumnSparse::operator[](size_t n) const +{ + return (*values)[getValueIndex(n)]; +} + +void ColumnSparse::get(size_t n, Field & res) const +{ + values->get(n, res); +} + +bool ColumnSparse::getBool(size_t n) const +{ + return values->getBool(getValueIndex(n)); +} + +UInt64 ColumnSparse::get64(size_t n) const +{ + return values->get64(getValueIndex(n)); +} + +StringRef ColumnSparse::getDataAt(size_t n) const +{ + return values->getDataAt(getValueIndex(n)); +} + +ColumnPtr ColumnSparse::convertToFullColumnIfSparse() const +{ + auto res = values->cloneEmpty(); + const auto & offsets_data = getOffsetsData(); + size_t current_offset = 0; + for (size_t i = 0; i < offsets_data.size(); ++i) + { + size_t offsets_diff = offsets_data[i] - current_offset; + current_offset = offsets_data[i]; + if (offsets_diff > 1) + res->insertManyDefaults(offsets_diff - 1); + res->insertFrom(*values, i + 1); + } + + res->insertManyDefaults(_size - current_offset); + return res; +} + + +void ColumnSparse::insertData(const char * pos, size_t length) +{ + _size += length; + return values->insertData(pos, length); +} + +StringRef ColumnSparse::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const +{ + return values->serializeValueIntoArena(getValueIndex(n), arena, begin); +} + +const char * ColumnSparse::deserializeAndInsertFromArena(const char * pos) +{ + UNUSED(pos); + throwMustBeDense(); +} + +void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t length) +{ + size_t end = start + length; + auto & offsets_data = getOffsetsData(); + + if (const auto * src_sparse = typeid_cast(&src)) + { + const auto & src_offsets = src_sparse->getOffsetsData(); + const auto & src_values = src_sparse->getValuesColumn(); + + size_t offset_start = std::lower_bound(src_offsets.begin(), src_offsets.end(), start) - src_offsets.begin(); + size_t offset_end = std::lower_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); + + insertManyDefaults(offset_start - start); + offsets_data.push_back(_size); + + for (size_t i = offset_start + 1; i < offset_end; ++i) + { + size_t current_diff = src_offsets[i] - src_offsets[i - 1]; + insertManyDefaults(current_diff - 1); + offsets_data.push_back(_size); + ++_size; + } + + insertManyDefaults(end - offset_end); + values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start); + } + else + { + for (size_t i = start; i < end; ++i) + { + offsets_data.push_back(_size); + ++_size; + } + + values->insertRangeFrom(src, start, length); + } +} + +void ColumnSparse::insert(const Field & x) +{ + getOffsetsData().push_back(_size); + values->insert(x); + ++_size; +} + +void ColumnSparse::insertFrom(const IColumn & src, size_t n) +{ + + if (const auto * src_sparse = typeid_cast(&src)) + { + if (size_t value_index = src_sparse->getValueIndex(n)) + { + getOffsetsData().push_back(_size); + values->insertFrom(src_sparse->getValuesColumn(), value_index); + } + } + else + { + getOffsetsData().push_back(_size); + values->insertFrom(src, n); + } + + ++_size; +} + +void ColumnSparse::insertDefault() +{ + ++_size; +} + +void ColumnSparse::insertManyDefaults(size_t length) +{ + _size += length; +} + +void ColumnSparse::popBack(size_t n) +{ + assert(n < _size); + + auto & offsets_data = getOffsetsData(); + size_t new_size = _size - n; + + size_t removed_values = 0; + while(!offsets_data.empty() && offsets_data.back() >= new_size) + { + offsets_data.pop_back(); + ++removed_values; + } + + if (removed_values) + values->popBack(removed_values); + + _size = new_size; +} + +ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const +{ + if (_size != filt.size()) + throw Exception("Size of filter doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + if (offsets->empty()) + { + auto res = cloneEmpty(); + res->insertManyDefaults(countBytesInFilter(filt)); + return res; + } + + const auto & offsets_data = getOffsetsData(); + + auto res_offsets = offsets->cloneEmpty(); + auto & res_offsets_data = assert_cast(*res_offsets).getData(); + + Filter values_filter; + values_filter.reserve(values->size()); + values_filter.push_back(1); + size_t values_result_size_hint = 1; + + size_t offset_pos = 0; + size_t res_offset = 0; + + for (size_t i = 0; i < _size; ++i) + { + if (offset_pos < offsets_data.size() && i == offsets_data[offset_pos]) + { + if (filt[i]) + { + res_offsets_data.push_back(res_offset); + values_filter.push_back(1); + ++res_offset; + ++values_result_size_hint; + } + else + { + values_filter.push_back(0); + } + + ++offset_pos; + } + else + { + res_offset += filt[i] != 0; + } + } + + auto res_values = values->filter(values_filter, values_result_size_hint); + return this->create(std::move(res_values), std::move(res_offsets), res_offset); +} + +ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const +{ + UNUSED(perm); + UNUSED(limit); + + throwMustBeDense(); +} + +ColumnPtr ColumnSparse::index(const IColumn & indexes, size_t limit) const +{ + UNUSED(indexes); + UNUSED(limit); + + throwMustBeDense(); +} + +int ColumnSparse::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const +{ + UNUSED(n); + UNUSED(m); + UNUSED(rhs_); + UNUSED(null_direction_hint); + + std::cerr << "rhs: " << rhs_.dumpStructure() << "\n"; + + throwMustBeDense(); +} + +void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, + PaddedPODArray * row_indexes, PaddedPODArray & compare_results, + int direction, int nan_direction_hint) const +{ + UNUSED(rhs); + UNUSED(rhs_row_num); + UNUSED(row_indexes); + UNUSED(compare_results); + UNUSED(direction); + UNUSED(nan_direction_hint); + + throwMustBeDense(); +} + +int ColumnSparse::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator &) const +{ + UNUSED(n); + UNUSED(m); + UNUSED(rhs); + UNUSED(null_direction_hint); + + throwMustBeDense(); +} + +bool ColumnSparse::hasEqualValues() const +{ + return offsets->size() == 0; +} + +void ColumnSparse::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +{ + UNUSED(reverse); + UNUSED(limit); + UNUSED(null_direction_hint); + UNUSED(res); + + throwMustBeDense(); +} + +void ColumnSparse::updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const +{ + UNUSED(reverse); + UNUSED(null_direction_hint); + UNUSED(limit); + UNUSED(res); + UNUSED(equal_range); + + throwMustBeDense(); +} + +void ColumnSparse::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +{ + UNUSED(collator); + UNUSED(reverse); + UNUSED(limit); + UNUSED(null_direction_hint); + UNUSED(res); + + throwMustBeDense(); +} + +void ColumnSparse::updatePermutationWithCollation( + const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const +{ + UNUSED(collator); + UNUSED(reverse); + UNUSED(limit); + UNUSED(null_direction_hint); + UNUSED(res); + UNUSED(equal_range); + + throwMustBeDense(); +} + +void ColumnSparse::reserve(size_t) +{ +} + +size_t ColumnSparse::byteSize() const +{ + return values->byteSize() + offsets->byteSize(); +} + +size_t ColumnSparse::byteSizeAt(size_t n) const +{ + size_t index = getValueIndex(n); + size_t res = values->byteSizeAt(index); + if (index) + res += sizeof(UInt64); + + return res; +} + +size_t ColumnSparse::allocatedBytes() const +{ + return values->allocatedBytes() + offsets->allocatedBytes(); +} + +void ColumnSparse::protect() +{ + throwMustBeDense(); +} + +ColumnPtr ColumnSparse::replicate(const Offsets & replicate_offsets) const +{ + UNUSED(replicate_offsets); + throwMustBeDense(); +} + +void ColumnSparse::updateHashWithValue(size_t n, SipHash & hash) const +{ + UNUSED(n); + UNUSED(hash); + throwMustBeDense(); +} + +void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const +{ + UNUSED(hash); + throwMustBeDense(); +} + +void ColumnSparse::updateHashFast(SipHash & hash) const +{ + UNUSED(hash); + throwMustBeDense(); +} + +void ColumnSparse::getExtremes(Field & min, Field & max) const +{ + UNUSED(min); + UNUSED(max); + throwMustBeDense(); +} + +void ColumnSparse::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const +{ + const auto & offsets_data = getOffsetsData(); + auto start = from ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from) : offsets_data.begin(); + auto end = limit ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from + limit) : offsets_data.end(); + + indices.assign(start, end); +} + +size_t ColumnSparse::getNumberOfDefaultRows(size_t step) const +{ + return (_size - offsets->size()) / step; +} + +MutableColumns ColumnSparse::scatter(ColumnIndex num_columns, const Selector & selector) const +{ + UNUSED(num_columns); + UNUSED(selector); + throwMustBeDense(); +} + +void ColumnSparse::gather(ColumnGathererStream & gatherer_stream) +{ + UNUSED(gatherer_stream); + throwMustBeDense(); +} + +ColumnPtr ColumnSparse::compress() const +{ + throwMustBeDense(); +} + +bool ColumnSparse::structureEquals(const IColumn & rhs) const +{ + UNUSED(rhs); + throwMustBeDense(); +} + +const IColumn::Offsets & ColumnSparse::getOffsetsData() const +{ + return assert_cast(*offsets).getData(); +} + +IColumn::Offsets & ColumnSparse::getOffsetsData() +{ + return assert_cast(*offsets).getData(); +} + +size_t ColumnSparse::getValueIndex(size_t n) const +{ + assert(n < _size); + + const auto & offsets_data = getOffsetsData(); + auto it = std::lower_bound(offsets_data.begin(), offsets_data.end(), n); + if (it == offsets_data.end() || *it != n) + return 0; + + return it - offsets_data.begin() + 1; +} + +} diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h new file mode 100644 index 00000000000..c14c64c010e --- /dev/null +++ b/src/Columns/ColumnSparse.h @@ -0,0 +1,161 @@ +#pragma once + +#include +#include +#include +#include +#include + +class Collator; + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class ColumnSparse final : public COWHelper +{ +private: + friend class COWHelper; + + explicit ColumnSparse(MutableColumnPtr && values_); + ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offsets_, size_t size_); + ColumnSparse(const ColumnSparse &) = default; + +public: + /** Create immutable column using immutable arguments. This arguments may be shared with other columns. + * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. + */ + using Base = COWHelper; + static Ptr create(const ColumnPtr & values_, const ColumnPtr & offsets_, size_t size_) + { + return Base::create(values_->assumeMutable(), offsets_->assumeMutable(), size_); + } + + static MutablePtr create(MutableColumnPtr && values_, MutableColumnPtr && offsets_, size_t size_) + { + return Base::create(std::move(values_), std::move(offsets_), size_); + } + + static Ptr create(const ColumnPtr & values_) + { + return Base::create(values_->assumeMutable()); + } + + template >> + static MutablePtr create(Arg && arg) + { + return Base::create(std::forward(arg)); + } + + const char * getFamilyName() const override { return "Sparse"; } + std::string getName() const override { return "Sparse(" + values->getName() + ")"; } + TypeIndex getDataType() const override { return values->getDataType(); } + MutableColumnPtr cloneResized(size_t new_size) const override; + size_t size() const override { return _size; } + bool isNullAt(size_t n) const override; + Field operator[](size_t n) const override; + void get(size_t n, Field & res) const override; + bool getBool(size_t n) const override; + UInt64 get64(size_t n) const override; + StringRef getDataAt(size_t n) const override; + + ColumnPtr convertToFullColumnIfSparse() const override; + + /// Will insert null value if pos=nullptr + void insertData(const char * pos, size_t length) override; + StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; + const char * deserializeAndInsertFromArena(const char * pos) override; + void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; + void insert(const Field & x) override; + void insertFrom(const IColumn & src, size_t n) override; + void insertDefault() override; + void insertManyDefaults(size_t length) override; + + void popBack(size_t n) override; + ColumnPtr filter(const Filter & filt, ssize_t) const override; + ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; + int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; + void compareColumn(const IColumn & rhs, size_t rhs_row_num, + PaddedPODArray * row_indexes, PaddedPODArray & compare_results, + int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator &) const override; + bool hasEqualValues() const override; + void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; + void updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const override; + void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; + void updatePermutationWithCollation( + const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const override; + void reserve(size_t n) override; + size_t byteSize() const override; + size_t byteSizeAt(size_t n) const override; + size_t allocatedBytes() const override; + void protect() override; + ColumnPtr replicate(const Offsets & replicate_offsets) const override; + void updateHashWithValue(size_t n, SipHash & hash) const override; + void updateWeakHash32(WeakHash32 & hash) const override; + void updateHashFast(SipHash & hash) const override; + void getExtremes(Field & min, Field & max) const override; + + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + size_t getNumberOfDefaultRows(size_t step) const override; + + MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; + + void gather(ColumnGathererStream & gatherer_stream) override; + + ColumnPtr compress() const override; + + void forEachSubcolumn(ColumnCallback callback) override + { + callback(values); + callback(offsets); + } + + bool structureEquals(const IColumn & rhs) const override; + + bool isNullable() const override { return values->isNullable(); } + bool isFixedAndContiguous() const override { return false; } + bool valuesHaveFixedSize() const override { return values->valuesHaveFixedSize(); } + size_t sizeOfValueIfFixed() const override { return values->sizeOfValueIfFixed() + values->sizeOfValueIfFixed(); } + bool isCollationSupported() const override { return values->isCollationSupported(); } + + size_t getNumberOfDefaults() const { return _size - offsets->size(); } + size_t getNumberOfTrailingDefaults() const + { + return offsets->empty() ? _size : _size - getOffsetsData().back() - 1; + } + + size_t getValueIndex(size_t n) const; + + const IColumn & getValuesColumn() const { return *values; } + IColumn & getValuesColumn() { return *values; } + + const ColumnPtr & getValuesPtr() const { return values; } + ColumnPtr & getValuesPtr() { return values; } + + const IColumn::Offsets & getOffsetsData() const; + IColumn::Offsets & getOffsetsData(); + + const ColumnPtr & getOffsetsPtr() const { return offsets; } + ColumnPtr & getOffsetsPtr() { return offsets; } + + const IColumn & getOffsetsColumn() const { return *offsets; } + IColumn & getOffsetsColumn() { return *offsets; } + +private: + [[noreturn]] void throwMustBeDense() const + { + throw Exception("Not implemented for ColumnSparse", ErrorCodes::LOGICAL_ERROR); + } + + WrappedPtr values; + WrappedPtr offsets; + size_t _size; +}; + +} diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 61276909a55..cab848c8b07 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -530,21 +530,21 @@ void ColumnString::getExtremes(Field & min, Field & max) const get(max_idx, max); } -void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices) const +void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices, size_t, size_t) const { for (size_t i = 0; i < offsets.size(); ++i) if (offsets[i] - offsets[i - 1] > 1) indices.push_back(i); } -size_t ColumnString::getNumberOfNonDefaultValues() const -{ - size_t res = 0; - for (size_t i = 0; i < offsets.size(); ++i) - res += (offsets[i] - offsets[i - 1] > 1); +// size_t ColumnString::getNumberOfDefaultRows() const +// { +// size_t res = 0; +// for (size_t i = 0; i < offsets.size(); ++i) +// res += (offsets[i] - offsets[i - 1] > 1); - return res; -} +// return res; +// } ColumnPtr ColumnString::compress() const { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index fe09abda7b5..ef9f51b9cfe 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -277,8 +277,8 @@ public: return typeid(rhs) == typeid(ColumnString); } - void getIndicesOfNonDefaultValues(Offsets & indices) const override; - size_t getNumberOfNonDefaultValues() const override; + void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; + // size_t getNumberOfDefaultRows() const override; Chars & getChars() { return chars; } const Chars & getChars() const { return chars; } diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index a7ee420c2cc..c2620a369de 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -303,19 +303,34 @@ public: return typeid(rhs) == typeid(ColumnVector); } - void getIndicesOfNonDefaultValues(IColumn::Offsets & offsets) const override + void getIndicesOfNonDefaultValues(IColumn::Offsets & offsets, size_t from, size_t limit) const override { offsets.reserve(data.size()); - for (size_t i = 0; i < data.size(); ++i) + size_t to = limit && from + limit < size() ? from + limit : size(); + for (size_t i = from; i < to; ++i) if (data[i] != T{}) offsets.push_back(i); } - size_t getNumberOfNonDefaultValues() const override + void insertAtOffsetsFrom(const IColumn::Offsets & offsets, const IColumn & values, size_t total_rows_hint) override + { + const auto & values_data = assert_cast(values).getData(); + + ssize_t position = static_cast(data.size()) - 1; + data.resize_fill(data.size() + total_rows_hint); + + for (size_t i = 0; i < offsets.size(); ++i) + { + position += offsets[i] + 1; + data[position] = values_data[i]; + } + } + + size_t getNumberOfDefaultRows(size_t step) const override { size_t res = 0; - for (size_t i = 0; i < data.size(); ++i) - res += (data[i] != T{}); + for (size_t i = 0; i < data.size(); i += step) + res += (data[i] == T{}); return res; } diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index a3ed0885651..88bc8fa3ead 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -30,6 +30,17 @@ void IColumn::insertFrom(const IColumn & src, size_t n) insert(src[n]); } +void IColumn::insertAtOffsetsFrom(const Offsets & offsets, const IColumn & values, size_t) +{ + assert(offsets.size() == values.size()); + for (size_t i = 0; i < offsets.size(); ++i) + { + if (offsets[i]) + insertManyDefaults(offsets[i]); + insertFrom(values, i); + } +} + bool isColumnNullable(const IColumn & column) { return checkColumn(column); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 44b7e280118..902353618f6 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -67,6 +67,8 @@ public: /// If column is ColumnLowCardinality, transforms is to full column. virtual Ptr convertToFullColumnIfLowCardinality() const { return getPtr(); } + virtual Ptr convertToFullColumnIfSparse() const { return getPtr(); } + /// Creates empty column with the same type. virtual MutablePtr cloneEmpty() const { return cloneResized(0); } @@ -363,8 +365,14 @@ public: throw Exception("Method structureEquals is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - virtual void getIndicesOfNonDefaultValues(Offsets & /* offsets */) const {} - virtual size_t getNumberOfNonDefaultValues() const { return 0; } + virtual void getIndicesOfNonDefaultValues(Offsets & /* offsets */, size_t, size_t) const {} + + virtual void insertAtOffsetsFrom(const Offsets & offsets, const IColumn & values, size_t total_rows_hint); + + static constexpr auto DEFAULT_ROWS_SEARCH_STEP = 8; + static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; + + virtual size_t getNumberOfDefaultRows(size_t /* step */) const { return {}; } /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. diff --git a/src/Common/SparseArray.h b/src/Common/SparseArray.h new file mode 100644 index 00000000000..e69de29bb2d diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index 9e39b4b2b28..9f0b0031fde 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -11,6 +11,8 @@ #include #include +#include + namespace DB { @@ -79,7 +81,7 @@ void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column settings.position_independent_encoding = false; ISerialization::DeserializeBinaryBulkStatePtr state; - auto serialization = type.getDefaultSerialization(); + auto serialization = type.getSerialization(*column); serialization->deserializeBinaryBulkStatePrefix(settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state); @@ -150,6 +152,10 @@ Block NativeBlockInputStream::readImpl() readBinary(type_name, istr); column.type = data_type_factory.get(type_name); + /// TODO: check revision. + SerializationKind serialization_kind; + readIntBinary(serialization_kind, istr); + if (use_index) { /// Index allows to do more checks. @@ -161,13 +167,19 @@ Block NativeBlockInputStream::readImpl() /// Data ColumnPtr read_column = column.type->createColumn(); + if (serialization_kind == SerializationKind::SPARSE) + read_column = ColumnSparse::create(read_column); 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); + /// TODO: maybe remove. + read_column = read_column->convertToFullColumnIfSparse(); column.column = std::move(read_column); + // std::cerr << "column.column: " << column.column->dumpStructure() << "\n"; + if (header) { /// Support insert from old clients without low cardinality type. diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index da68376201f..bee93efcc9f 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -41,7 +41,7 @@ void NativeBlockOutputStream::flush() } -void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) +void NativeBlockOutputStream::writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) @@ -53,12 +53,10 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr settings.position_independent_encoding = false; settings.low_cardinality_max_dictionary_size = 0; - auto serialization = type.getDefaultSerialization(); - ISerialization::SerializeBinaryBulkStatePtr state; - serialization->serializeBinaryBulkStatePrefix(settings, state); - serialization->serializeBinaryBulkWithMultipleStreams(*full_column, offset, limit, settings, state); - serialization->serializeBinaryBulkStateSuffix(settings, state); + serialization.serializeBinaryBulkStatePrefix(settings, state); + serialization.serializeBinaryBulkWithMultipleStreams(*full_column, offset, limit, settings, state); + serialization.serializeBinaryBulkStateSuffix(settings, state); } @@ -121,9 +119,13 @@ void NativeBlockOutputStream::write(const Block & block) writeStringBinary(type_name, ostr); + /// TODO: add revision + auto serialization = column.type->getSerialization(*column.column); + writeIntBinary(serialization->getKind(), ostr); + /// Data if (rows) /// Zero items of data is always represented as zero number of bytes. - writeData(*column.type, column.column, ostr, 0, 0); + writeData(*serialization, column.column, ostr, 0, 0); if (index_ostr) { diff --git a/src/DataStreams/NativeBlockOutputStream.h b/src/DataStreams/NativeBlockOutputStream.h index 64ccd267634..25ad4e1b470 100644 --- a/src/DataStreams/NativeBlockOutputStream.h +++ b/src/DataStreams/NativeBlockOutputStream.h @@ -30,7 +30,7 @@ public: void write(const Block & block) override; void flush() override; - static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit); + static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit); String getContentType() const override { return "application/octet-stream"; } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 371a50a32cf..435cdcbbe6e 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -340,8 +340,8 @@ SerializationPtr DataTypeTuple::getSerialization(const String & column_name, con ISerialization::Settings settings = { .num_rows = info.getNumberOfRows(), - .num_non_default_rows = info.getNumberOfNonDefaultValues(subcolumn_name), - .min_ratio_for_dense_serialization = 10 + .num_default_rows = info.getNumberOfDefaultRows(subcolumn_name), + .ratio_for_sparse_serialization = info.getRatioForSparseSerialization() }; auto serializaion = elems[i]->getSerialization(settings); diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 2b09bc12054..999dfada206 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -158,13 +159,14 @@ SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_n throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } + SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const { ISerialization::Settings settings = { .num_rows = info.getNumberOfRows(), - .num_non_default_rows = info.getNumberOfNonDefaultValues(column_name), - .min_ratio_for_dense_serialization = 10 + .num_default_rows = info.getNumberOfDefaultRows(column_name), + .ratio_for_sparse_serialization = info.getRatioForSparseSerialization() }; return getSerialization(settings); @@ -172,11 +174,14 @@ SerializationPtr IDataType::getSerialization(const String & column_name, const S SerializationPtr IDataType::getSerialization(const IColumn & column) const { + if (typeid_cast(&column)) + return getSparseSerialization(); + ISerialization::Settings settings = { .num_rows = column.size(), - .num_non_default_rows = column.getNumberOfNonDefaultValues(), - .min_ratio_for_dense_serialization = 10 + .num_default_rows = column.getNumberOfDefaultRows(IColumn::DEFAULT_ROWS_SEARCH_STEP), + .ratio_for_sparse_serialization = 10 }; return getSerialization(settings); @@ -184,10 +189,9 @@ SerializationPtr IDataType::getSerialization(const IColumn & column) const SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const { - // if (settings.num_non_default_rows * settings.min_ratio_for_dense_serialization < settings.num_rows) - // return getSparseSerialization(); - - UNUSED(settings); + double ratio = settings.num_rows ? std::min(static_cast(settings.num_default_rows) / settings.num_rows, 1.0) : 0.0; + if (ratio > settings.ratio_for_sparse_serialization) + return getSparseSerialization(); return getDefaultSerialization(); } @@ -215,9 +219,6 @@ SerializationPtr IDataType::getSerialization(const String & column_name, const S if (callback(sparse_idx_name)) return getSparseSerialization(); - UNUSED(column_name); - UNUSED(callback); - return getDefaultSerialization(); } @@ -238,4 +239,9 @@ void IDataType::enumerateStreams(const SerializationPtr & serialization, const S }, path); } +bool isSparseSerializaion(const SerializationPtr & serialization) +{ + return typeid_cast(serialization.get()); +} + } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 9e60c0b3b1d..865c081d859 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -75,6 +75,7 @@ public: using StreamExistenceCallback = std::function; using BaseSerializationGetter = std::function; + virtual SerializationPtr getSerialization(const IColumn & column) const; virtual SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const; virtual SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; @@ -86,7 +87,7 @@ public: virtual SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; SerializationPtr getSerialization(const ISerialization::Settings & settings) const; - SerializationPtr getSerialization(const IColumn & column) const; + // SerializationPtr getSerialization(const IColumn & column) const; using StreamCallbackWithType = std::function; @@ -503,4 +504,6 @@ template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; +bool isSparseSerializaion(const SerializationPtr & serialization); + } diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 9c0a6e683a5..b72feb72b6f 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -2,6 +2,8 @@ #include #include +#include +#include #include #include @@ -9,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + class IDataType; class ReadBuffer; @@ -25,6 +32,12 @@ class Field; struct FormatSettings; struct NameAndTypePair; +enum class SerializationKind : UInt8 +{ + DEFAULT = 0, + SPARSE = 1 +}; + class ISerialization { public: @@ -90,6 +103,8 @@ public: String toString() const; }; + virtual SerializationKind getKind() const { return SerializationKind::DEFAULT; } + /// 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; @@ -143,8 +158,8 @@ public: struct Settings { size_t num_rows; - size_t num_non_default_rows; - size_t min_ratio_for_dense_serialization; + size_t num_default_rows; + double ratio_for_sparse_serialization; }; /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. @@ -258,9 +273,48 @@ public: static ColumnPtr getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path); static bool isSpecialCompressionAllowed(const SubstreamPath & path); + + template + static State * checkAndGetSerializeState(SerializeBinaryBulkStatePtr & state, const Serialization &); + + template + static State * checkAndGetDeserializeState(DeserializeBinaryBulkStatePtr & state, const Serialization &); }; using SerializationPtr = std::shared_ptr; using Serializations = std::vector; +template +static State * checkAndGetState(StatePtr & state) +{ + if (!state) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Got empty state for {}", demangle(typeid(Serialization).name())); + + auto * state_concrete = typeid_cast(state.get()); + if (!state_concrete) + { + auto & state_ref = *state; + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Invalid State for {}. Expected: {}, got {}", + demangle(typeid(Serialization).name()), + demangle(typeid(State).name()), + demangle(typeid(state_ref).name())); + } + + return state_concrete; +} + +template +State * ISerialization::checkAndGetSerializeState(SerializeBinaryBulkStatePtr & state, const Serialization &) +{ + return checkAndGetState(state); +} + +template +State * ISerialization::checkAndGetDeserializeState(DeserializeBinaryBulkStatePtr & state, const Serialization &) +{ + return checkAndGetState(state); +} + } diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index cb86bac7514..124ee30729e 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -17,17 +17,25 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +SerializationInfo::SerializationInfo( + double ratio_for_sparse_serialization_, + size_t default_rows_search_step_) + : ratio_for_sparse_serialization(ratio_for_sparse_serialization_) + , default_rows_search_step(default_rows_search_step_) +{ +} + void SerializationInfo::add(const Block & block) { number_of_rows += block.rows(); for (const auto & elem : block) { - non_default_values[elem.name] = elem.column->getNumberOfNonDefaultValues(); + default_rows[elem.name] += elem.column->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; for (const auto & subname : elem.type->getSubcolumnNames()) { auto subcolumn = elem.type->getSubcolumn(subname, *elem.column); auto full_name = Nested::concatenateName(elem.name, subname); - non_default_values[full_name] += subcolumn->getNumberOfNonDefaultValues(); + default_rows[full_name] += subcolumn->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; } } } @@ -35,14 +43,14 @@ void SerializationInfo::add(const Block & block) void SerializationInfo::add(const SerializationInfo & other) { number_of_rows += other.number_of_rows; - for (const auto & [name, num] : other.non_default_values) - non_default_values[name] += num; + for (const auto & [name, num] : other.default_rows) + default_rows[name] += num; } -size_t SerializationInfo::getNumberOfNonDefaultValues(const String & column_name) const +size_t SerializationInfo::getNumberOfDefaultRows(const String & column_name) const { - auto it = non_default_values.find(column_name); - if (it == non_default_values.end()) + auto it = default_rows.find(column_name); + if (it == default_rows.end()) return 0; return it->second; } @@ -51,13 +59,15 @@ namespace { constexpr auto KEY_NUMBER_OF_ROWS = "number_of_rows"; -constexpr auto KEY_NUMBER_OF_NON_DEFAULT_VALUES = "number_of_non_default_values"; +constexpr auto KEY_NUMBER_OF_default_rows = "number_of_default_rows"; constexpr auto KEY_NUMBER = "number"; constexpr auto KEY_NAME = "name"; constexpr auto KEY_VERSION = "version"; } +/// TODO: add all fields. + void SerializationInfo::fromJSON(const String & json_str) { Poco::JSON::Parser parser; @@ -66,9 +76,9 @@ void SerializationInfo::fromJSON(const String & json_str) if (object->has(KEY_NUMBER_OF_ROWS)) number_of_rows = object->getValue(KEY_NUMBER_OF_ROWS); - if (object->has(KEY_NUMBER_OF_NON_DEFAULT_VALUES)) + if (object->has(KEY_NUMBER_OF_default_rows)) { - auto array = object->getArray(KEY_NUMBER_OF_NON_DEFAULT_VALUES); + auto array = object->getArray(KEY_NUMBER_OF_default_rows); for (const auto & elem : *array) { auto elem_object = elem.extract(); @@ -78,7 +88,7 @@ void SerializationInfo::fromJSON(const String & json_str) auto name = elem_object->getValue(KEY_NAME); auto number = elem_object->getValue(KEY_NUMBER); - non_default_values[name] = number; + default_rows[name] = number; } } } @@ -90,7 +100,7 @@ String SerializationInfo::toJSON() const info.set(KEY_NUMBER_OF_ROWS, number_of_rows); Poco::JSON::Array column_infos; - for (const auto & [name, num] : non_default_values) + for (const auto & [name, num] : default_rows) { Poco::JSON::Object column_info; column_info.set(KEY_NAME, name); @@ -98,7 +108,7 @@ String SerializationInfo::toJSON() const column_infos.add(std::move(column_info)); } - info.set(KEY_NUMBER_OF_NON_DEFAULT_VALUES, std::move(column_infos)); + info.set(KEY_NUMBER_OF_default_rows, std::move(column_infos)); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index ceee705cf57..6205c3c7e0c 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -12,11 +12,16 @@ public: using NameToNumber = std::unordered_map; + SerializationInfo( + double ratio_for_sparse_serialization_, + size_t default_rows_search_step_ = IColumn::DEFAULT_ROWS_SEARCH_STEP); + void add(const Block & block); void add(const SerializationInfo & other); - size_t getNumberOfNonDefaultValues(const String & column_name) const; + size_t getNumberOfDefaultRows(const String & column_name) const; size_t getNumberOfRows() const { return number_of_rows; } + double getRatioForSparseSerialization() const { return ratio_for_sparse_serialization; } void read(ReadBuffer & in); void write(WriteBuffer & out) const; @@ -25,8 +30,11 @@ private: void fromJSON(const String & json_str); String toJSON() const; + double ratio_for_sparse_serialization; + size_t default_rows_search_step; + size_t number_of_rows = 0; - NameToNumber non_default_values; + NameToNumber default_rows; }; } diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 31058cb6e57..71216435a7c 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -196,42 +196,6 @@ struct DeserializeStateLowCardinality : public ISerialization::DeserializeBinary explicit DeserializeStateLowCardinality(UInt64 key_version_) : key_version(key_version_) {} }; -static SerializeStateLowCardinality * checkAndGetLowCardinalitySerializeState( - ISerialization::SerializeBinaryBulkStatePtr & state) -{ - if (!state) - throw Exception("Got empty state for SerializationLowCardinality.", ErrorCodes::LOGICAL_ERROR); - - auto * low_cardinality_state = typeid_cast(state.get()); - if (!low_cardinality_state) - { - auto & state_ref = *state; - throw Exception("Invalid SerializeBinaryBulkState for SerializationLowCardinality. Expected: " - + demangle(typeid(SerializeStateLowCardinality).name()) + ", got " - + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); - } - - return low_cardinality_state; -} - -static DeserializeStateLowCardinality * checkAndGetLowCardinalityDeserializeState( - ISerialization::DeserializeBinaryBulkStatePtr & state) -{ - if (!state) - throw Exception("Got empty state for SerializationLowCardinality.", ErrorCodes::LOGICAL_ERROR); - - auto * low_cardinality_state = typeid_cast(state.get()); - if (!low_cardinality_state) - { - auto & state_ref = *state; - throw Exception("Invalid DeserializeBinaryBulkState for SerializationLowCardinality. Expected: " - + demangle(typeid(DeserializeStateLowCardinality).name()) + ", got " - + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); - } - - return low_cardinality_state; -} - void SerializationLowCardinality::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const @@ -256,7 +220,7 @@ void SerializationLowCardinality::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * low_cardinality_state = checkAndGetLowCardinalitySerializeState(state); + auto * low_cardinality_state = checkAndGetSerializeState(state, *this); KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); if (low_cardinality_state->shared_dictionary && settings.low_cardinality_max_dictionary_size) @@ -495,7 +459,7 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams( const ColumnLowCardinality & low_cardinality_column = typeid_cast(column); - auto * low_cardinality_state = checkAndGetLowCardinalitySerializeState(state); + auto * low_cardinality_state = checkAndGetSerializeState(state, *this); auto & global_dictionary = low_cardinality_state->shared_dictionary; KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); @@ -594,7 +558,7 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( if (!indexes_stream) throw Exception("Got empty stream for SerializationLowCardinality indexes.", ErrorCodes::LOGICAL_ERROR); - auto * low_cardinality_state = checkAndGetLowCardinalityDeserializeState(state); + auto * low_cardinality_state = checkAndGetDeserializeState(state, *this); KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); auto read_dictionary = [this, low_cardinality_state, keys_stream]() diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 19dddf84464..291d98d3135 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -12,28 +13,118 @@ namespace DB namespace { -void serializeOffsetsPositionIndependent(const IColumn::Offsets & offsets, WriteBuffer & ostr) +static constexpr auto END_OF_GRANULE_FLAG = 1ULL << 63; + +struct SerializeStateSparse : public ISerialization::SerializeBinaryBulkState { + size_t num_trailing_default_values = 0; + ISerialization::SerializeBinaryBulkStatePtr nested; +}; + +struct DeserializeStateSparse : public ISerialization::DeserializeBinaryBulkState +{ + size_t num_trailing_defaults = 0; + bool has_value_after_defaults = false; + ISerialization::DeserializeBinaryBulkStatePtr nested; +}; + +void serializeOffsetsPositionIndependent(const IColumn::Offsets & offsets, WriteBuffer & ostr, size_t start, size_t end) +{ + // std::cerr << "writing start: " << start << ", end: " << end << "\n"; + // std::cerr << "offsets: "; + // for (const auto & x : offsets) + // std::cerr << x << " "; + // std::cerr << "\n"; + size_t size = offsets.size(); - IColumn::Offset prev_offset = 0; for (size_t i = 0; i < size; ++i) { - IColumn::Offset current_offset = offsets[i]; - writeIntBinary(current_offset - prev_offset, ostr); - prev_offset = current_offset; + size_t group_size = offsets[i] - start; + + // std::cerr << "writing group_size: " << group_size << "\n"; + + writeIntBinary(group_size, ostr); + start += group_size + 1; } + + // std::cerr << "writing start: " << start << ", end: " << end << "\n"; + size_t group_size = start < end ? end - start : 0; + // std::cerr << "writing end group_size: " << group_size << "\n"; + group_size |= END_OF_GRANULE_FLAG; + writeIntBinary(group_size, ostr); } -void deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, ReadBuffer & istr) +// struct DeserializedRows +// { +// size_t total = 0; +// size_t trailing_defaults = 0; +// }; + +size_t deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, + ReadBuffer & istr, size_t limit, DeserializeStateSparse & state) { - IColumn::Offset current_offset = 0; + // std::cerr << "limit: " << limit << ", num_trailing: " << state.num_trailing_defaults + // << ", has_value_after_defaults: " << state.has_value_after_defaults << "\n"; + + if (limit && state.num_trailing_defaults >= limit) + { + state.num_trailing_defaults -= limit; + return limit; + } + + size_t total_rows = state.num_trailing_defaults; + if (state.has_value_after_defaults) + { + size_t start_of_group = offsets.empty() ? 0 : offsets.back() + 1; + offsets.push_back(start_of_group + state.num_trailing_defaults); + + state.has_value_after_defaults = false; + state.num_trailing_defaults = 0; + ++total_rows; + } + + size_t group_size; while (!istr.eof()) { - IColumn::Offset current_size = 0; - readIntBinary(current_size, istr); - current_offset += current_size; - offsets.push_back(current_offset); + readIntBinary(group_size, istr); + + bool end_of_granule = group_size & END_OF_GRANULE_FLAG; + group_size &= ~END_OF_GRANULE_FLAG; + + // std::cerr << "read group_size: " << group_size << ", end_of_granule: " << end_of_granule << "\n"; + size_t next_total_rows = total_rows + group_size; + group_size += state.num_trailing_defaults; + + + // std::cerr << "group_size: " << group_size << ", end_of_granule: " << end_of_granule << "\n"; + // std::cerr << "next_total_rows: " << next_total_rows << "\n"; + + if (limit && next_total_rows >= limit) + { + state.num_trailing_defaults = next_total_rows - limit; + state.has_value_after_defaults = !end_of_granule; + return limit; + } + + if (end_of_granule) + { + state.has_value_after_defaults = false; + state.num_trailing_defaults = group_size; + } + else + { + size_t start_of_group = offsets.empty() ? 0 : offsets.back() + 1; + offsets.push_back(start_of_group + group_size); + + state.num_trailing_defaults = 0; + state.has_value_after_defaults = false; + ++next_total_rows; + } + + total_rows = next_total_rows; } + + return total_rows; } } @@ -56,27 +147,13 @@ void SerializationSparse::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - settings.path.push_back(Substream::SparseElements); - nested_serialization->serializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); -} + auto state_sparse = std::make_shared(); -void SerializationSparse::serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ settings.path.push_back(Substream::SparseElements); - nested_serialization->serializeBinaryBulkStateSuffix(settings, state); + nested_serialization->serializeBinaryBulkStatePrefix(settings, state_sparse->nested); settings.path.pop_back(); -} -void SerializationSparse::deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const -{ - settings.path.push_back(Substream::SparseElements); - nested_serialization->deserializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); + state = std::move(state_sparse); } void SerializationSparse::serializeBinaryBulkWithMultipleStreams( @@ -86,30 +163,67 @@ void SerializationSparse::serializeBinaryBulkWithMultipleStreams( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - UNUSED(limit); - UNUSED(offset); - - /// TODO: inefficient. - /// TODO: use limit and offset size_t size = column.size(); + auto * state_sparse = checkAndGetSerializeState(state, *this); + + // std::cerr << "writing column: " << column.dumpStructure() << "\n"; auto offsets_column = DataTypeNumber().createColumn(); auto & offsets_data = assert_cast &>(*offsets_column).getData(); - - column.getIndicesOfNonDefaultValues(offsets_data); - auto values = column.index(*offsets_column, 0); - offsets_data.push_back(size); + column.getIndicesOfNonDefaultValues(offsets_data, offset, limit); settings.path.push_back(Substream::SparseOffsets); if (auto * stream = settings.getter(settings.path)) - serializeOffsetsPositionIndependent(offsets_data, *stream); + { + size_t end = limit && offset + limit < size ? offset + limit : size; + serializeOffsetsPositionIndependent(offsets_data, *stream, offset, end); + } - settings.path.back() = Substream::SparseElements; - nested_serialization->serializeBinaryBulkWithMultipleStreams(*values, 0, 0, settings, state); + if (!offsets_data.empty()) + { + settings.path.back() = Substream::SparseElements; + if (const auto * column_sparse = typeid_cast(&column)) + { + const auto & values = column_sparse->getValuesColumn(); + size_t begin = column_sparse->getValueIndex(offsets_data[0]); + size_t end = column_sparse->getValueIndex(offsets_data.back()); + // std::cerr << "begin: " << begin << ", end: " << end << "\n"; + nested_serialization->serializeBinaryBulkWithMultipleStreams(values, begin, end - begin + 1, settings, state_sparse->nested); + } + else + { + auto values = column.index(*offsets_column, 0); + nested_serialization->serializeBinaryBulkWithMultipleStreams(*values, 0, values->size(), settings, state_sparse->nested); + } + } settings.path.pop_back(); } +void SerializationSparse::serializeBinaryBulkStateSuffix( + SerializeBinaryBulkSettings & settings, + SerializeBinaryBulkStatePtr & state) const +{ + auto * state_sparse = checkAndGetSerializeState(state, *this); + + settings.path.push_back(Substream::SparseElements); + nested_serialization->serializeBinaryBulkStateSuffix(settings, state_sparse->nested); + settings.path.pop_back(); +} + +void SerializationSparse::deserializeBinaryBulkStatePrefix( + DeserializeBinaryBulkSettings & settings, + DeserializeBinaryBulkStatePtr & state) const +{ + auto state_sparse = std::make_shared(); + + settings.path.push_back(Substream::SparseElements); + nested_serialization->deserializeBinaryBulkStatePrefix(settings, state_sparse->nested); + settings.path.pop_back(); + + state = std::move(state_sparse); +} + void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( ColumnPtr & column, size_t limit, @@ -118,38 +232,50 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( SubstreamsCache * cache) const { settings.path.push_back(Substream::SparseOffsets); - - auto offsets_column = DataTypeNumber().createColumn(); - auto & offsets_data = assert_cast &>(*offsets_column).getData(); - - if (auto * stream = settings.getter(settings.path)) - deserializeOffsetsPositionIndependent(offsets_data, *stream); - - settings.path.back() = Substream::SparseElements; - - ColumnPtr values = column->cloneEmpty(); - nested_serialization->deserializeBinaryBulkWithMultipleStreams(values, limit, settings, state, cache); + auto * state_sparse = checkAndGetDeserializeState(state, *this); auto mutable_column = column->assumeMutable(); - size_t size = values->size(); - ssize_t prev_offset = -1; + auto & column_sparse = assert_cast(*mutable_column); + auto & offsets_data = column_sparse.getOffsetsData(); - for (size_t i = 0; i < size; ++i) - { - size_t offsets_diff = static_cast(offsets_data[i]) - prev_offset; + size_t old_size = offsets_data.size(); - if (offsets_diff > 1) - mutable_column->insertManyDefaults(offsets_diff - 1); + size_t read_rows = 0; + if (auto * stream = settings.getter(settings.path)) + read_rows = deserializeOffsetsPositionIndependent(offsets_data, *stream, limit, *state_sparse); - mutable_column->insertFrom(*values, i); - prev_offset = offsets_data[i]; - } - - size_t offsets_diff = offsets_data[size] - prev_offset; - if (offsets_diff > 1) - mutable_column->insertManyDefaults(offsets_diff - 1); + auto & values_column = column_sparse.getValuesPtr(); + size_t values_limit = offsets_data.size() - old_size; + settings.path.back() = Substream::SparseElements; + nested_serialization->deserializeBinaryBulkWithMultipleStreams(values_column, values_limit, settings, state_sparse->nested, cache); settings.path.pop_back(); + + if (offsets_data.size() + 1 != values_column->size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent sizes of values and offsets in SerializationSparse." + " Offsets size: {}, values size: {}", offsets_data.size(), values_column->size()); + + column_sparse.insertManyDefaults(read_rows); + + // std::cerr << "column_sparse: " << column_sparse.dumpStructure() << "\n"; + // std::cerr << "offsets: "; + // for (const auto & x : column_sparse.getOffsetsData()) + // std::cerr << x << " "; + // std::cerr << "\n"; + + // std::cerr << "values: "; + // for (size_t i = 0; i < column_sparse.getValuesColumn().size(); ++i) + // std::cerr << toString(column_sparse.getValuesColumn()[i]) << " "; + // std::cerr << "\n"; + + column = std::move(mutable_column); } +// void SerializationSparse::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +// { +// const auto & column_sparse = assert_cast(column); +// const auto & values_column = column_sparse.getValuesColumn(); +// nested_serialization->serializeText(values_column, column_sparse.getValueIndex(row_num), ostr, settings); +// } + } diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index a5f8c7547c3..73daf801dd2 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -10,6 +10,8 @@ class SerializationSparse final : public SerializationWrapper public: SerializationSparse(const SerializationPtr & nested_); + SerializationKind getKind() const override { return SerializationKind::SPARSE; } + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; void serializeBinaryBulkStatePrefix( @@ -37,6 +39,9 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; + + // void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + }; } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index bdeea80477e..0a0d1a8ec32 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -293,39 +293,6 @@ struct DeserializeBinaryBulkStateTuple : public ISerialization::DeserializeBinar std::vector states; }; -static SerializeBinaryBulkStateTuple * checkAndGetTupleSerializeState(ISerialization::SerializeBinaryBulkStatePtr & state) -{ - if (!state) - throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR); - - auto * tuple_state = typeid_cast(state.get()); - if (!tuple_state) - { - auto & state_ref = *state; - throw Exception("Invalid SerializeBinaryBulkState for DataTypeTuple. Expected: " - + demangle(typeid(SerializeBinaryBulkStateTuple).name()) + ", got " - + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); - } - - return tuple_state; -} - -static DeserializeBinaryBulkStateTuple * checkAndGetTupleDeserializeState(ISerialization::DeserializeBinaryBulkStatePtr & state) -{ - if (!state) - throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR); - - auto * tuple_state = typeid_cast(state.get()); - if (!tuple_state) - { - auto & state_ref = *state; - throw Exception("Invalid DeserializeBinaryBulkState for DataTypeTuple. Expected: " - + demangle(typeid(DeserializeBinaryBulkStateTuple).name()) + ", got " - + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); - } - - return tuple_state; -} void SerializationTuple::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, @@ -344,7 +311,7 @@ void SerializationTuple::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * tuple_state = checkAndGetTupleSerializeState(state); + auto * tuple_state = checkAndGetSerializeState(state, *this); for (size_t i = 0; i < elems.size(); ++i) elems[i]->serializeBinaryBulkStateSuffix(settings, tuple_state->states[i]); @@ -370,7 +337,7 @@ void SerializationTuple::serializeBinaryBulkWithMultipleStreams( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * tuple_state = checkAndGetTupleSerializeState(state); + auto * tuple_state = checkAndGetSerializeState(state, *this); for (const auto i : ext::range(0, ext::size(elems))) { @@ -386,7 +353,7 @@ void SerializationTuple::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { - auto * tuple_state = checkAndGetTupleDeserializeState(state); + auto * tuple_state = checkAndGetDeserializeState(state, *this); auto mutable_column = column->assumeMutable(); auto & column_tuple = assert_cast(*mutable_column); diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index abff6f21acf..2bc75dc9533 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -590,6 +591,8 @@ void NO_INLINE Aggregator::executeImplBatch( { if (inst->offsets) inst->batch_that->addBatchArray(rows, places.get(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool); + else if (inst->has_sparse_arguments) + inst->batch_that->addBatchSparse(places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); else inst->batch_that->addBatch(rows, places.get(), inst->state_offset, inst->batch_arguments, aggregates_pool); } @@ -608,6 +611,8 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( if (inst->offsets) inst->batch_that->addBatchSinglePlace( inst->offsets[static_cast(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena); + else if(inst->has_sparse_arguments) + inst->batch_that->addBatchSparseSinglePlace(res + inst->state_offset, inst->batch_arguments, arena); else inst->batch_that->addBatchSinglePlace(rows, res + inst->state_offset, inst->batch_arguments, arena); } @@ -643,19 +648,30 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns for (size_t i = 0; i < params.aggregates_size; ++i) { + bool allow_sparse_arguments = aggregate_columns[i].size() == 1; + bool has_sparse_arguments = false; + for (size_t j = 0; j < aggregate_columns[i].size(); ++j) { materialized_columns.push_back(columns.at(params.aggregates[i].arguments[j])->convertToFullColumnIfConst()); aggregate_columns[i][j] = materialized_columns.back().get(); - auto column_no_lc = recursiveRemoveLowCardinality(aggregate_columns[i][j]->getPtr()); - if (column_no_lc.get() != aggregate_columns[i][j]) + auto full_column = allow_sparse_arguments + ? aggregate_columns[i][j]->getPtr() + : aggregate_columns[i][j]->convertToFullColumnIfSparse(); + + full_column = recursiveRemoveLowCardinality(full_column); + if (full_column.get() != aggregate_columns[i][j]) { - materialized_columns.emplace_back(std::move(column_no_lc)); + materialized_columns.emplace_back(std::move(full_column)); aggregate_columns[i][j] = materialized_columns.back().get(); } + + if (typeid_cast(aggregate_columns[i][j])) + has_sparse_arguments = true; } + aggregate_functions_instructions[i].has_sparse_arguments = has_sparse_arguments; aggregate_functions_instructions[i].arguments = aggregate_columns[i].data(); aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; auto * that = aggregate_functions[i]; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index c5bcc1eb27f..e541733ee61 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1038,6 +1038,7 @@ protected: const IAggregateFunction * batch_that; const IColumn ** batch_arguments; const UInt64 * offsets = nullptr; + bool has_sparse_arguments = false; }; using AggregateFunctionInstructions = std::vector; diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 0825d9f329e..d962fa74ce5 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -157,7 +157,7 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) Serializations serializations(num_columns); for (size_t i = 0; i < num_columns; ++i) - serializations[i] = header.getByPosition(i).type->getDefaultSerialization(); + serializations[i] = header.getByPosition(i).type->getSerialization(*columns[i]); WidthsPerColumn widths; Widths max_widths; @@ -291,6 +291,8 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) writeCString(grid_symbols.bar, out); + std::cerr << "current row: " << toString((*columns[0])[i]) << "\n"; + for (size_t j = 0; j < num_columns; ++j) { if (j != 0) diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index fa987c6b949..9d9269341e5 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -26,7 +26,7 @@ void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind Serializations serializations(num_columns); for (size_t i = 0; i < num_columns; ++i) - serializations[i] = header.getByPosition(i).type->getDefaultSerialization(); + serializations[i] = header.getByPosition(i).type->getSerialization(*columns[i]); WidthsPerColumn widths; Widths max_widths; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 8175a648f64..23b104abd92 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -252,17 +252,16 @@ static void decrementTypeMetric(MergeTreeDataPartType type) IMergeTreeDataPart::IMergeTreeDataPart( - MergeTreeData & storage_, const String & name_, const VolumePtr & volume_, const std::optional & relative_path_, Type part_type_) - : storage(storage_) - , name(name_) - , info(MergeTreePartInfo::fromPartName(name_, storage.format_version)) - , volume(volume_) - , relative_path(relative_path_.value_or(name_)) - , index_granularity_info(storage_, part_type_) - , part_type(part_type_) + const MergeTreeData & storage_, + const String & name_, + const VolumePtr & volume_, + const std::optional & relative_path_, + Type part_type_) + : IMergeTreeDataPart( + storage_, name_, + MergeTreePartInfo::fromPartName(name_, storage_.format_version), + volume_, relative_path_, part_type_) { - incrementStateMetric(state); - incrementTypeMetric(part_type); } IMergeTreeDataPart::IMergeTreeDataPart( @@ -278,6 +277,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , volume(volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) + , serialization_info(storage_.getSettings()->ratio_for_sparse_serialization) , part_type(part_type_) { incrementStateMetric(state); @@ -563,7 +563,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks if (check_consistency) checkConsistency(require_columns_checksums); loadDefaultCompressionCodec(); - + loadSerializationInfo(); } void IMergeTreeDataPart::loadIndexGranularity() @@ -930,6 +930,16 @@ void IMergeTreeDataPart::loadUUID() } } +void IMergeTreeDataPart::loadSerializationInfo() +{ + String path = getFullRelativePath() + SERIALIZATION_FILE_NAME; + if (volume->getDisk()->exists(path)) + { + auto in = openForReading(volume->getDisk(), path); + serialization_info.read(*in); + } +} + void IMergeTreeDataPart::loadColumns(bool require) { String path = getFullRelativePath() + "columns.txt"; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 4098077d8de..7f85849688d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -72,7 +72,7 @@ public: Type part_type_); IMergeTreeDataPart( - MergeTreeData & storage_, + const MergeTreeData & storage_, const String & name_, const VolumePtr & volume, const std::optional & relative_path, @@ -178,6 +178,9 @@ public: mutable String relative_path; MergeTreeIndexGranularityInfo index_granularity_info; + /// TODO: add comment + SerializationInfo serialization_info; + size_t rows_count = 0; @@ -222,8 +225,6 @@ public: TTLInfos ttl_infos; - SerializationInfo serialization_info; - /// Current state of the part. If the part is in working set already, it should be accessed via data_parts mutex void setState(State new_state) const; State getState() const; @@ -360,6 +361,8 @@ public: static inline constexpr auto UUID_FILE_NAME = "uuid.txt"; + static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.txt"; + /// Checks that all TTLs (table min/max, column ttls, so on) for part /// calculated. Part without calculated TTL may exist if TTL was added after /// part creation (using alter query with materialize_ttl setting). @@ -421,6 +424,8 @@ private: /// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros void loadTTLInfos(); + void loadSerializationInfo(); + void loadPartitionAndMinMaxIndex(); /// Load default compression codec from file default_compression_codec.txt diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 1cbc9859fe9..d5f34e379a3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -238,7 +238,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const { String filename; - auto serialization = column.type->getSerialization(column.name, serialization_info); + auto serialization = getSerializationForColumn(column); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (filename.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index c7b5051ebd9..2aa517a9833 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -33,8 +33,12 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( , marks(*marks_file) { const auto & storage_columns = metadata_snapshot->getColumns(); + serializations.reserve(columns_list.size()); for (const auto & column : columns_list) + { + serializations.emplace(column.name, column.type->getDefaultSerialization()); addStreams(column, storage_columns.getCodecDescOrDefault(column.name, default_codec)); + } } void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc) @@ -63,7 +67,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, compressed_streams.emplace(stream_name, stream); }; - column.type->enumerateStreams(column.type->getDefaultSerialization(), callback); + column.type->enumerateStreams(serializations[column.name], callback); } namespace @@ -105,6 +109,7 @@ Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, /// Write single granule of one column (rows between 2 marks) void writeColumnSingleGranule( const ColumnWithTypeAndName & column, + const SerializationPtr & serialization, ISerialization::OutputStreamGetter stream_getter, size_t from_row, size_t number_of_rows) @@ -116,7 +121,6 @@ void writeColumnSingleGranule( serialize_settings.position_independent_encoding = true; serialize_settings.low_cardinality_max_dictionary_size = 0; - auto serialization = column.type->getDefaultSerialization(); serialization->serializeBinaryBulkStatePrefix(serialize_settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, state); serialization->serializeBinaryBulkStateSuffix(serialize_settings, state); @@ -203,7 +207,9 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeIntBinary(plain_hashing.count(), marks); writeIntBinary(UInt64(0), marks); - writeColumnSingleGranule(block.getByName(name_and_type->name), stream_getter, granule.start_row, granule.rows_to_write); + writeColumnSingleGranule( + block.getByName(name_and_type->name), serializations[name_and_type->name], + stream_getter, granule.start_row, granule.rows_to_write); /// Each type always have at least one substream prev_stream->hashing_buf.next(); //-V522 diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 2ea35969a4e..2494195223a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -121,7 +121,7 @@ static size_t computeIndexGranularityImpl( } else { - size_t size_of_row_in_bytes = block_size_in_memory / rows_in_block; + size_t size_of_row_in_bytes = std::max(block_size_in_memory / rows_in_block, 1UL); index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes; } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 704b38ba6d5..d952950e461 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -132,6 +132,9 @@ protected: MergeTreeIndexAggregators skip_indices_aggregators; std::vector skip_index_accumulated_marks; + using SerializationsMap = std::unordered_map; + SerializationsMap serializations; + std::unique_ptr index_file_stream; std::unique_ptr index_stream; DataTypes index_types; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 04179fbb781..b5b36ab6f44 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -81,7 +81,10 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) + { + serializations.emplace(it.name, it.type->getSerialization(it.name, data_part->serialization_info)); addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec)); + } } @@ -112,9 +115,7 @@ void MergeTreeDataPartWriterWide::addStreams( settings.max_compress_block_size); }; - auto serialization = column.type->getSerialization(column.name, data_part->serialization_info); - column.type->enumerateStreams(serialization, callback); - serializations.emplace(column.name, std::move(serialization)); + column.type->enumerateStreams(serializations[column.name], callback); } @@ -193,7 +194,14 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm fillIndexGranularity(index_granularity_for_block, block.rows()); } - auto granules_to_write = getGranulesToWrite(index_granularity, block.rows(), getCurrentMark(), rows_written_in_last_mark); + Block block_to_write = block; + for (auto & col : block_to_write) + { + if (serializations[col.name]->getKind() != SerializationKind::SPARSE) + col.column = col.column->convertToFullColumnIfSparse(); + } + + auto granules_to_write = getGranulesToWrite(index_granularity, block_to_write.rows(), getCurrentMark(), rows_written_in_last_mark); auto offset_columns = written_offset_columns ? *written_offset_columns : WrittenOffsetColumns{}; Block primary_key_block; @@ -205,7 +213,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm auto it = columns_list.begin(); for (size_t i = 0; i < columns_list.size(); ++i, ++it) { - const ColumnWithTypeAndName & column = block.getByName(it->name); + const ColumnWithTypeAndName & column = block_to_write.getByName(it->name); if (permutation) { @@ -301,7 +309,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( ISerialization::SerializeBinaryBulkSettings & serialize_settings, const Granule & granule) { - auto serialization = serializations[name_and_type.name]; + const auto & serialization = serializations[name_and_type.name]; serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, 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. @@ -406,7 +414,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, size_t mark_num; - auto serialization = type.getDefaultSerialization(); + const auto & serialization = serializations[name]; for (mark_num = 0; !mrk_in.eof(); ++mark_num) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 227c95edfb2..5eaaa0c1bbe 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -110,9 +110,6 @@ private: using ColumnStreams = std::map; ColumnStreams column_streams; - using Serializations = std::map; - Serializations serializations; - /// Non written marks to disk (for each column). Waiting until all rows for /// this marks will be written to disk. using MarksForColumns = std::unordered_map; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 0da2f643eb0..efa9e429259 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -83,7 +84,12 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si /// 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(); + { + if (isSparseSerializaion(serializations[name])) + res_columns[pos] = ColumnSparse::create(type->createColumn()); + else + res_columns[pos] = type->createColumn(); + } auto & column = res_columns[pos]; try diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 16657b4083d..7cd02fdf85e 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -32,6 +32,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ + M(Float, ratio_for_sparse_serialization, 1.1, "", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index aefbe28b45b..1301eef0421 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -24,6 +24,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( : IMergedBlockOutputStream(data_part, metadata_snapshot_) , columns_list(columns_list_) , default_codec(default_codec_) + , serialization_info(storage.getSettings()->ratio_for_sparse_serialization) { MergeTreeWriterSettings writer_settings( storage.global_context.getSettings(), @@ -147,6 +148,18 @@ void MergedBlockOutputStream::finalizePartOnDisk( removeEmptyColumnsFromPart(new_part, part_columns, checksums); + if (serialization_info.getNumberOfRows() > 0) + { + auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); + HashingWriteBuffer out_hashing(*out); + serialization_info.write(out_hashing); + checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); + checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash(); + out->finalize(); + if (sync) + out->sync(); + } + { /// Write a file with a description of columns. auto out = volume->getDisk()->writeFile(part_path + "columns.txt", 4096); @@ -156,15 +169,6 @@ void MergedBlockOutputStream::finalizePartOnDisk( out->sync(); } - // if (serialization_info.getNumberOfRows() > 0) - // { - // auto out = volume->getDisk()->writeFile(part_path + "serialization.txt", 4096); - // serialization_info.write(*out); - // out->finalize(); - // if (sync) - // out->sync(); - // } - if (default_codec != nullptr) { auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME, 4096); From 17071471f22bea83ff63f4c0419461637e2042a6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 31 Mar 2021 04:08:27 +0300 Subject: [PATCH 003/358] ColumnSparse: support of functions --- src/Columns/ColumnSparse.cpp | 22 ++++--- src/Columns/ColumnSparse.h | 2 + src/Columns/FilterDescription.cpp | 3 + src/Columns/IColumn.h | 2 + src/DataStreams/NativeBlockInputStream.cpp | 4 +- src/DataStreams/NativeBlockOutputStream.cpp | 4 ++ .../Serializations/ISerialization.cpp | 11 ++++ src/DataTypes/Serializations/ISerialization.h | 17 ++--- .../Serializations/SerializationSparse.h | 2 +- src/Functions/IFunction.cpp | 62 ++++++++++++++++++- src/Functions/IFunctionAdaptors.h | 3 + src/Functions/IFunctionImpl.h | 2 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- .../System/StorageSystemPartsColumns.cpp | 9 ++- 14 files changed, 122 insertions(+), 23 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index e67cb45847d..c8edf23b9df 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -85,11 +85,14 @@ ColumnPtr ColumnSparse::convertToFullColumnIfSparse() const size_t offsets_diff = offsets_data[i] - current_offset; current_offset = offsets_data[i]; if (offsets_diff > 1) - res->insertManyDefaults(offsets_diff - 1); + res->insertManyFrom(*values, 0, offsets_diff - 1); res->insertFrom(*values, i + 1); } - res->insertManyDefaults(_size - current_offset); + size_t offsets_diff = _size - current_offset; + if(offsets_diff > 1) + res->insertManyFrom(*values, 0, offsets_diff - 1); + return res; } @@ -111,6 +114,11 @@ const char * ColumnSparse::deserializeAndInsertFromArena(const char * pos) throwMustBeDense(); } +const char * ColumnSparse::skipSerializedInArena(const char *) const +{ + throwMustBeDense(); +} + void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t length) { size_t end = start + length; @@ -278,14 +286,10 @@ ColumnPtr ColumnSparse::index(const IColumn & indexes, size_t limit) const int ColumnSparse::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const { - UNUSED(n); - UNUSED(m); - UNUSED(rhs_); - UNUSED(null_direction_hint); + if (const auto * rhs_sparse = typeid_cast(&rhs_)) + return values->compareAt(getValueIndex(n), rhs_sparse->getValueIndex(m), rhs_sparse->getValuesColumn(), null_direction_hint); - std::cerr << "rhs: " << rhs_.dumpStructure() << "\n"; - - throwMustBeDense(); + return values->compareAt(getValueIndex(n), m, rhs_, null_direction_hint); } void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index c14c64c010e..226d85e81b8 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -51,6 +51,7 @@ public: return Base::create(std::forward(arg)); } + bool isSparse() const override { return true; } const char * getFamilyName() const override { return "Sparse"; } std::string getName() const override { return "Sparse(" + values->getName() + ")"; } TypeIndex getDataType() const override { return values->getDataType(); } @@ -69,6 +70,7 @@ public: void insertData(const char * pos, size_t length) override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; const char * deserializeAndInsertFromArena(const char * pos) override; + const char * skipSerializedInArena(const char *) const override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; void insert(const Field & x) override; void insertFrom(const IColumn & src, size_t n) override; diff --git a/src/Columns/FilterDescription.cpp b/src/Columns/FilterDescription.cpp index d216094eaab..7e9cac68f00 100644 --- a/src/Columns/FilterDescription.cpp +++ b/src/Columns/FilterDescription.cpp @@ -50,6 +50,9 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) FilterDescription::FilterDescription(const IColumn & column_) { + if (column_.isSparse()) + data_holder = column_.convertToFullColumnIfSparse(); + if (column_.lowCardinality()) data_holder = column_.convertToFullColumnIfLowCardinality(); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 253926fe579..253ed176fc7 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -460,6 +460,8 @@ public: virtual bool lowCardinality() const { return false; } + virtual bool isSparse() const { return false; } + virtual bool isCollationSupported() const { return false; } virtual ~IColumn() = default; diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index f8b0a07d9c9..7560cd5006c 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -153,7 +153,7 @@ Block NativeBlockInputStream::readImpl() column.type = data_type_factory.get(type_name); /// TODO: check revision. - SerializationKind serialization_kind; + ISerialization::Kind serialization_kind; readIntBinary(serialization_kind, istr); if (use_index) @@ -167,7 +167,7 @@ Block NativeBlockInputStream::readImpl() /// Data ColumnPtr read_column = column.type->createColumn(); - if (serialization_kind == SerializationKind::SPARSE) + if (serialization_kind == ISerialization::Kind::SPARSE) read_column = ColumnSparse::create(read_column); double avg_value_size_hint = avg_value_size_hints.empty() ? 0 : avg_value_size_hints[i]; diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index bee93efcc9f..29d552613d2 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -119,6 +119,10 @@ void NativeBlockOutputStream::write(const Block & block) writeStringBinary(type_name, ostr); + std::cerr << "column before: " << column.column->dumpStructure() << "\n"; + column.column = column.column->convertToFullColumnIfSparse(); + std::cerr << "column after: " << column.column->dumpStructure() << "\n"; + /// TODO: add revision auto serialization = column.type->getSerialization(*column.column); writeIntBinary(serialization->getKind(), ostr); diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index ab2e8e1958b..a37397b321b 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -14,6 +14,17 @@ namespace ErrorCodes extern const int MULTIPLE_STREAMS_REQUIRED; } +String ISerialization::kindToString(Kind kind) +{ + switch (kind) + { + case Kind::DEFAULT: + return "Default"; + case Kind::SPARSE: + return "Sparse"; + } +} + String ISerialization::Substream::toString() const { switch (type) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index a9093f919b3..a6b780f3780 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -32,18 +32,21 @@ class Field; struct FormatSettings; struct NameAndTypePair; -enum class SerializationKind : UInt8 -{ - DEFAULT = 0, - SPARSE = 1 -}; - class ISerialization { public: ISerialization() = default; virtual ~ISerialization() = default; + enum class Kind : UInt8 + { + DEFAULT = 0, + SPARSE = 1 + }; + + virtual Kind getKind() const { return Kind::DEFAULT; } + static String kindToString(Kind kind); + /** Binary serialization for range of values in column - for writing to disk/network, etc. * * Some data types are represented in multiple streams while being serialized. @@ -103,8 +106,6 @@ public: String toString() const; }; - virtual SerializationKind getKind() const { return SerializationKind::DEFAULT; } - /// 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; diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index 73daf801dd2..24a306de8ee 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -10,7 +10,7 @@ class SerializationSparse final : public SerializationWrapper public: SerializationSparse(const SerializationPtr & nested_); - SerializationKind getKind() const override { return SerializationKind::SPARSE; } + Kind getKind() const override { return Kind::SPARSE; } void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index e4a1adb8525..1ed0282f8ff 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -379,7 +380,13 @@ static void convertLowCardinalityColumnsToFull(ColumnsWithTypeAndName & args) } } -ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const +static void convertSparseColumnsToFull(ColumnsWithTypeAndName & args) +{ + for (auto & column : args) + column.column = column.column->convertToFullColumnIfSparse(); +} + +ColumnPtr ExecutableFunctionAdaptor::executeWithoutSparseColumns(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const { if (impl->useDefaultImplementationForLowCardinalityColumns()) { @@ -452,6 +459,59 @@ ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & argu return executeWithoutLowCardinalityColumns(arguments, result_type, input_rows_count, dry_run); } +ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const +{ + if (impl->useDefaultImplementationForSparseColumns()) + { + size_t num_sparse_columns = 0; + size_t num_full_columns = 0; + size_t sparse_column_position = 0; + + for (size_t i = 0; i < arguments.size(); ++i) + { + if (typeid_cast(arguments[i].column.get())) + { + sparse_column_position = i; + ++num_sparse_columns; + } + else if (!isColumnConst(*arguments[i].column)) + { + ++num_full_columns; + } + } + + auto columns_without_sparse = arguments; + if (num_sparse_columns == 1 && num_full_columns == 0) + { + auto & arg_with_sparse = columns_without_sparse[sparse_column_position]; + ColumnPtr sparse_offsets; + { + /// New scope to avoid possible mistakes on dangling reference. + const auto & column_sparse = assert_cast(*arg_with_sparse.column); + sparse_offsets = column_sparse.getOffsetsPtr(); + arg_with_sparse.column = column_sparse.getValuesPtr(); + } + + size_t values_size = arg_with_sparse.column->size(); + for (size_t i = 0; i < columns_without_sparse.size(); ++i) + { + if (i == sparse_column_position) + continue; + + columns_without_sparse[i].column = columns_without_sparse[i].column->cloneResized(values_size); + } + + auto res = executeWithoutSparseColumns(columns_without_sparse, result_type, input_rows_count, dry_run); + return ColumnSparse::create(res, sparse_offsets, input_rows_count); + } + + convertSparseColumnsToFull(columns_without_sparse); + return executeWithoutSparseColumns(columns_without_sparse, result_type, input_rows_count, dry_run); + } + + return executeWithoutSparseColumns(arguments, result_type, input_rows_count, dry_run); +} + void FunctionOverloadResolverAdaptor::checkNumberOfArguments(size_t number_of_arguments) const { if (isVariadic()) diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index cbb71c99b1b..6a0de199ff7 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -32,6 +32,9 @@ private: ColumnPtr executeWithoutLowCardinalityColumns( const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const; + + ColumnPtr executeWithoutSparseColumns( + const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const; }; class FunctionBaseAdaptor final : public IFunctionBase diff --git a/src/Functions/IFunctionImpl.h b/src/Functions/IFunctionImpl.h index a9ec6926c63..67a8df64379 100644 --- a/src/Functions/IFunctionImpl.h +++ b/src/Functions/IFunctionImpl.h @@ -62,6 +62,8 @@ public: */ virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; } + virtual bool useDefaultImplementationForSparseColumns() const { return true; } + /** Some arguments could remain constant during this implementation. */ virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 6cba32c6234..95cf3c921ca 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -197,7 +197,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm Block block_to_write = block; for (auto & col : block_to_write) { - if (serializations[col.name]->getKind() != SerializationKind::SPARSE) + if (serializations[col.name]->getKind() != ISerialization::Kind::SPARSE) col.column = col.column->convertToFullColumnIfSparse(); } diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 703de70d17f..05999e3dda4 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -56,7 +56,8 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_ {"column_bytes_on_disk", std::make_shared()}, {"column_data_compressed_bytes", std::make_shared()}, {"column_data_uncompressed_bytes", std::make_shared()}, - {"column_marks_bytes", std::make_shared()} + {"column_marks_bytes", std::make_shared()}, + {"serialization_kind", std::make_shared()} } ) { @@ -212,6 +213,12 @@ void StorageSystemPartsColumns::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.marks); + if (columns_mask[src_index++]) + { + auto kind = part->getSerializationForColumn(column)->getKind(); + columns[res_index++]->insert(ISerialization::kindToString(kind)); + } + if (has_state_column) columns[res_index++]->insert(part->stateString()); } From 2a7a77e0a2310aef9fe0a520232bd9d04071a33a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 1 Apr 2021 21:18:28 +0300 Subject: [PATCH 004/358] ColumnSparse: support of functions --- src/AggregateFunctions/IAggregateFunction.h | 3 ++- src/Columns/ColumnSparse.cpp | 19 +-------------- src/Columns/ColumnString.cpp | 19 +++++++-------- src/Columns/ColumnString.h | 2 +- src/Columns/ColumnVector.cpp | 13 +++++++++++ src/Columns/ColumnVector.h | 14 +---------- src/Columns/FilterDescription.cpp | 1 + src/Columns/IColumn.cpp | 23 +++++++++++++++---- src/Columns/IColumn.h | 8 +++---- src/DataStreams/NativeBlockOutputStream.cpp | 2 -- .../Serializations/SerializationSparse.cpp | 3 +++ src/Functions/IFunction.cpp | 10 +++++++- 12 files changed, 63 insertions(+), 54 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index ff232b97999..1ebdd0b87cb 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -350,7 +350,8 @@ public: for (size_t i = 1; i < values->size(); ++i) static_cast(this)->add(place, &values, i, arena); - static_cast(this)->addManyDefaults(column_sparse.getNumberOfDefaults()); + for (size_t i = 0; i < column_sparse.getNumberOfDefaults(); ++i) + static_cast(this)->add(place, &values, 0, arena); } void addBatchSinglePlaceNotNull( diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index c8edf23b9df..a8a78957586 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -77,26 +77,9 @@ StringRef ColumnSparse::getDataAt(size_t n) const ColumnPtr ColumnSparse::convertToFullColumnIfSparse() const { - auto res = values->cloneEmpty(); - const auto & offsets_data = getOffsetsData(); - size_t current_offset = 0; - for (size_t i = 0; i < offsets_data.size(); ++i) - { - size_t offsets_diff = offsets_data[i] - current_offset; - current_offset = offsets_data[i]; - if (offsets_diff > 1) - res->insertManyFrom(*values, 0, offsets_diff - 1); - res->insertFrom(*values, i + 1); - } - - size_t offsets_diff = _size - current_offset; - if(offsets_diff > 1) - res->insertManyFrom(*values, 0, offsets_diff - 1); - - return res; + return values->createWithOffsets(getOffsetsData(), _size); } - void ColumnSparse::insertData(const char * pos, size_t length) { _size += length; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 7a6d33b041e..78722fa6fda 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -536,21 +536,22 @@ void ColumnString::getExtremes(Field & min, Field & max) const get(max_idx, max); } -void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices, size_t, size_t) const +void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const { - for (size_t i = 0; i < offsets.size(); ++i) + size_t to = limit && from + limit < size() ? from + limit : size(); + for (size_t i = from; i < to; ++i) if (offsets[i] - offsets[i - 1] > 1) indices.push_back(i); } -// size_t ColumnString::getNumberOfDefaultRows() const -// { -// size_t res = 0; -// for (size_t i = 0; i < offsets.size(); ++i) -// res += (offsets[i] - offsets[i - 1] > 1); +size_t ColumnString::getNumberOfDefaultRows(size_t step) const +{ + size_t res = 0; + for (size_t i = 0; i < offsets.size(); i += step) + res += (offsets[i] == offsets[i - 1] + 1); -// return res; -// } + return res; +} ColumnPtr ColumnString::compress() const { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index e745c719e5b..14edde5d63b 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -280,7 +280,7 @@ public: } void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; - // size_t getNumberOfDefaultRows() const override; + size_t getNumberOfDefaultRows(size_t step) const override; Chars & getChars() { return chars; } const Chars & getChars() const { return chars; } diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index a64906ba257..49b76eb0fd9 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -554,6 +554,19 @@ ColumnPtr ColumnVector::compress() const }); } +template +ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const +{ + auto res = this->create(); + auto & res_data = res->getData(); + + res_data.resize_fill(total_rows, data[0]); + for (size_t i = 0; i < offsets.size(); ++i) + res_data[offsets[i]] = data[i + 1]; + + return res; +} + /// Explicit template instantiations - to avoid code bloat in headers. template class ColumnVector; template class ColumnVector; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index b451f8d5b2d..f7540f75164 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -314,19 +314,7 @@ public: offsets.push_back(i); } - void insertAtOffsetsFrom(const IColumn::Offsets & offsets, const IColumn & values, size_t total_rows_hint) override - { - const auto & values_data = assert_cast(values).getData(); - - ssize_t position = static_cast(data.size()) - 1; - data.resize_fill(data.size() + total_rows_hint); - - for (size_t i = 0; i < offsets.size(); ++i) - { - position += offsets[i] + 1; - data[position] = values_data[i]; - } - } + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const override; size_t getNumberOfDefaultRows(size_t step) const override { diff --git a/src/Columns/FilterDescription.cpp b/src/Columns/FilterDescription.cpp index 7e9cac68f00..8e9836729a6 100644 --- a/src/Columns/FilterDescription.cpp +++ b/src/Columns/FilterDescription.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 88bc8fa3ead..866daf448c5 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -30,15 +30,28 @@ void IColumn::insertFrom(const IColumn & src, size_t n) insert(src[n]); } -void IColumn::insertAtOffsetsFrom(const Offsets & offsets, const IColumn & values, size_t) +ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows) const { - assert(offsets.size() == values.size()); + auto res = cloneEmpty(); + res->reserve(total_rows); + + size_t current_offset = 0; for (size_t i = 0; i < offsets.size(); ++i) { - if (offsets[i]) - insertManyDefaults(offsets[i]); - insertFrom(values, i); + size_t offsets_diff = offsets[i] - current_offset; + current_offset = offsets[i]; + + if (offsets_diff > 1) + res->insertManyFrom(*this, 0, offsets_diff - 1); + + res->insertFrom(*this, i + 1); } + + size_t offsets_diff = total_rows - current_offset; + if(offsets_diff > 1) + res->insertManyFrom(*this, 0, offsets_diff - 1); + + return res; } bool isColumnNullable(const IColumn & column) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 253ed176fc7..5c67e7c0ac1 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -369,15 +369,15 @@ public: throw Exception("Method structureEquals is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - virtual void getIndicesOfNonDefaultValues(Offsets & /* offsets */, size_t, size_t) const {} - - virtual void insertAtOffsetsFrom(const Offsets & offsets, const IColumn & values, size_t total_rows_hint); - static constexpr auto DEFAULT_ROWS_SEARCH_STEP = 8; static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; virtual size_t getNumberOfDefaultRows(size_t /* step */) const { return {}; } + virtual void getIndicesOfNonDefaultValues(Offsets & /* offsets */, size_t, size_t) const {} + + virtual Ptr createWithOffsets(const Offsets & offsets, size_t total_rows) const; + /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. virtual Ptr compress() const diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index 29d552613d2..bf660664f1f 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -119,9 +119,7 @@ void NativeBlockOutputStream::write(const Block & block) writeStringBinary(type_name, ostr); - std::cerr << "column before: " << column.column->dumpStructure() << "\n"; column.column = column.column->convertToFullColumnIfSparse(); - std::cerr << "column after: " << column.column->dumpStructure() << "\n"; /// TODO: add revision auto serialization = column.type->getSerialization(*column.column); diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 291d98d3135..8e3005f2f6d 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -72,6 +72,9 @@ size_t deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, return limit; } + /// TODO: + offsets.reserve(limit / 10); /// TODO + size_t total_rows = state.num_trailing_defaults; if (state.has_value_after_defaults) { diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 1ed0282f8ff..701c676cc5a 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -501,7 +501,15 @@ ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & argu columns_without_sparse[i].column = columns_without_sparse[i].column->cloneResized(values_size); } - auto res = executeWithoutSparseColumns(columns_without_sparse, result_type, input_rows_count, dry_run); + auto res = executeWithoutSparseColumns(columns_without_sparse, result_type, values_size, dry_run); + + /// If default of sparse column was changed after execution of function, convert to full column. + if (res->compareAt(0, 0, *arg_with_sparse.column, 0) != 0) + { + const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); + return res->createWithOffsets(offsets_data, input_rows_count); + } + return ColumnSparse::create(res, sparse_offsets, input_rows_count); } From a78f531597602793fe277d6821b91eb5dc24ae36 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 3 Apr 2021 03:04:48 +0300 Subject: [PATCH 005/358] ColumnSparse: fixes --- src/Columns/ColumnArray.cpp | 25 ++-- src/Columns/ColumnArray.h | 5 +- src/Columns/ColumnDecimal.cpp | 20 +++ src/Columns/ColumnDecimal.h | 3 + src/Columns/ColumnNullable.cpp | 10 ++ src/Columns/ColumnNullable.h | 3 + src/Columns/ColumnSparse.cpp | 63 ++++++--- src/Columns/ColumnString.cpp | 1 + src/Columns/ColumnVector.cpp | 20 +++ src/Columns/ColumnVector.h | 20 +-- src/Columns/IColumn.cpp | 14 +- src/Columns/IColumn.h | 4 +- .../Serializations/SerializationSparse.cpp | 78 ++---------- src/Interpreters/Aggregator.cpp | 12 +- .../0_stateless/01780_column_sparse.reference | 120 ++++++++++++++++++ .../0_stateless/01780_column_sparse.sql | 34 +++++ 16 files changed, 308 insertions(+), 124 deletions(-) create mode 100644 tests/queries/0_stateless/01780_column_sparse.reference create mode 100644 tests/queries/0_stateless/01780_column_sparse.sql diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 327d204494f..6e10a15d920 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1216,22 +1216,23 @@ void ColumnArray::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } -void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t, size_t) const +size_t ColumnArray::getNumberOfDefaultRows(size_t step) const { const auto & offsets_data = getOffsets(); - for (size_t i = 0; i < offsets_data.size(); ++i) + size_t res = 0; + for (size_t i = 0; i < offsets_data.size(); i += step) + res += (offsets_data[i] != offsets_data[i - 1]); + + return res; +} + +void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const +{ + const auto & offsets_data = getOffsets(); + size_t to = limit && from + limit < size() ? from + limit : size(); + for (size_t i = from; i < to; ++i) if (offsets_data[i] != offsets_data[i - 1]) indices.push_back(i); } -// size_t ColumnArray::getNumberOfDefaultRows() const -// { -// const auto & offsets_data = getOffsets(); -// size_t res = 0; -// for (size_t i = 0; i < offsets_data.size(); ++i) -// res += (offsets_data[i] != offsets_data[i - 1]); - -// return res; -// } - } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 77c32bdcf8e..fab6573f159 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -140,9 +140,8 @@ public: return false; } - void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t offset, size_t limit) const override; - - // size_t getNumberOfDefaultRows() const override; + size_t getNumberOfDefaultRows(size_t step) const override; + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index ec08db274b3..3f3dffeb5b2 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -383,6 +383,26 @@ ColumnPtr ColumnDecimal::compress() const }); } +template +size_t ColumnDecimal::getNumberOfDefaultRows(size_t step) const +{ + size_t res = 0; + for (size_t i = 0; i < data.size(); i += step) + res += (data[i] == T{}); + + return res; +} + +template +void ColumnDecimal::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const +{ + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + for (size_t i = from; i < to; ++i) + if (data[i] != T{}) + indices.push_back(i); +} + template void ColumnDecimal::getExtremes(Field & min, Field & max) const { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index a4fa15c2d95..a58f6c23258 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -176,6 +176,9 @@ public: ColumnPtr compress() const override; + size_t getNumberOfDefaultRows(size_t step) const override; + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + void insertValue(const T value) { data.push_back(value); } Container & getData() { return data; } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 1e529751437..d7cc7f8a36a 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -686,6 +686,16 @@ void ColumnNullable::checkConsistency() const ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT); } +size_t ColumnNullable::getNumberOfDefaultRows(size_t step) const +{ + return null_map->getNumberOfDefaultRows(step); +} + +void ColumnNullable::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const +{ + return null_map->getIndicesOfNonDefaultValues(indices, from, limit); +} + ColumnPtr makeNullable(const ColumnPtr & column) { if (isColumnNullable(*column)) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 963b3e1e8fa..6f6f678b76b 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -171,6 +171,9 @@ public: /// Check that size of null map equals to size of nested column. void checkConsistency() const; + size_t getNumberOfDefaultRows(size_t step) const override; + void getIndicesOfNonDefaultValues(Offsets & offsets, size_t from, size_t limit) const override; + private: WrappedPtr nested_column; WrappedPtr null_map; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index a8a78957586..27d677d4342 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -104,30 +104,47 @@ const char * ColumnSparse::skipSerializedInArena(const char *) const void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - size_t end = start + length; + if (length == 0) + return; + + if (start + length >= src.size()) + throw Exception("Parameter out of bound in IColumnString::insertRangeFrom method.", + ErrorCodes::LOGICAL_ERROR); + auto & offsets_data = getOffsetsData(); + size_t end = start + length; if (const auto * src_sparse = typeid_cast(&src)) { const auto & src_offsets = src_sparse->getOffsetsData(); const auto & src_values = src_sparse->getValuesColumn(); - size_t offset_start = std::lower_bound(src_offsets.begin(), src_offsets.end(), start) - src_offsets.begin(); - size_t offset_end = std::lower_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); - - insertManyDefaults(offset_start - start); - offsets_data.push_back(_size); - - for (size_t i = offset_start + 1; i < offset_end; ++i) + if (!src_offsets.empty()) { - size_t current_diff = src_offsets[i] - src_offsets[i - 1]; - insertManyDefaults(current_diff - 1); + size_t offset_start = std::lower_bound(src_offsets.begin(), src_offsets.end(), start) - src_offsets.begin(); + size_t offset_end = std::upper_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); + if (offset_end != 0) + --offset_end; + + insertManyDefaults(src_offsets[offset_start] - start); offsets_data.push_back(_size); ++_size; - } - insertManyDefaults(end - offset_end); - values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start); + for (size_t i = offset_start + 1; i < offset_end; ++i) + { + size_t current_diff = src_offsets[i] - src_offsets[i - 1]; + insertManyDefaults(current_diff - 1); + offsets_data.push_back(_size); + ++_size; + } + + insertManyDefaults(end - src_offsets[offset_end]); + values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start + 1); + } + else + { + insertManyDefaults(length); + } } else { @@ -253,10 +270,24 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const { - UNUSED(perm); - UNUSED(limit); + limit = limit ? std::min(limit, _size) : _size; - throwMustBeDense(); + auto res_values = values->cloneEmpty(); + auto res_offsets = offsets->cloneEmpty(); + auto & res_offsets_data = assert_cast(*res_offsets).getData(); + res_values->insertDefault(); + + for (size_t i = 0; i < limit; ++i) + { + size_t index = getValueIndex(perm[i]); + if (index != 0) + { + res_values->insertFrom(*values, index); + res_offsets_data.push_back(i); + } + } + + return ColumnSparse::create(std::move(res_values), std::move(res_offsets), limit); } ColumnPtr ColumnSparse::index(const IColumn & indexes, size_t limit) const diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 78722fa6fda..437e7e4bd7a 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -539,6 +539,7 @@ void ColumnString::getExtremes(Field & min, Field & max) const void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const { size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); for (size_t i = from; i < to; ++i) if (offsets[i] - offsets[i - 1] > 1) indices.push_back(i); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 49b76eb0fd9..8849e8ff222 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -554,6 +554,26 @@ ColumnPtr ColumnVector::compress() const }); } +template +size_t ColumnVector::getNumberOfDefaultRows(size_t step) const +{ + size_t res = 0; + for (size_t i = 0; i < data.size(); i += step) + res += (data[i] == T{}); + + return res; +} + +template +void ColumnVector::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const +{ + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + for (size_t i = from; i < to; ++i) + if (data[i] != T{}) + indices.push_back(i); +} + template ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const { diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index f7540f75164..7e997079b2a 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -305,26 +305,10 @@ public: return typeid(rhs) == typeid(ColumnVector); } - void getIndicesOfNonDefaultValues(IColumn::Offsets & offsets, size_t from, size_t limit) const override - { - offsets.reserve(data.size()); - size_t to = limit && from + limit < size() ? from + limit : size(); - for (size_t i = from; i < to; ++i) - if (data[i] != T{}) - offsets.push_back(i); - } - + size_t getNumberOfDefaultRows(size_t step) const override; + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const override; - size_t getNumberOfDefaultRows(size_t step) const override - { - size_t res = 0; - for (size_t i = 0; i < data.size(); i += step) - res += (data[i] == T{}); - - return res; - } - ColumnPtr compress() const override; /// Replace elements that match the filter with zeroes. If inverted replaces not matched elements. diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 866daf448c5..e3626fd2c82 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -30,15 +30,23 @@ void IColumn::insertFrom(const IColumn & src, size_t n) insert(src[n]); } +void IColumn::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const +{ + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + for (size_t i = from; i < to; ++i) + indices.push_back(i); +} + ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows) const { auto res = cloneEmpty(); res->reserve(total_rows); - size_t current_offset = 0; + ssize_t current_offset = -1; for (size_t i = 0; i < offsets.size(); ++i) { - size_t offsets_diff = offsets[i] - current_offset; + ssize_t offsets_diff = static_cast(offsets[i]) - current_offset; current_offset = offsets[i]; if (offsets_diff > 1) @@ -47,7 +55,7 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows) res->insertFrom(*this, i + 1); } - size_t offsets_diff = total_rows - current_offset; + ssize_t offsets_diff = static_cast(total_rows) - current_offset; if(offsets_diff > 1) res->insertManyFrom(*this, 0, offsets_diff - 1); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 5c67e7c0ac1..d3162794b1f 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -372,9 +372,9 @@ public: static constexpr auto DEFAULT_ROWS_SEARCH_STEP = 8; static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; - virtual size_t getNumberOfDefaultRows(size_t /* step */) const { return {}; } + virtual size_t getNumberOfDefaultRows(size_t /* step */) const { return 0; } - virtual void getIndicesOfNonDefaultValues(Offsets & /* offsets */, size_t, size_t) const {} + virtual void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const; virtual Ptr createWithOffsets(const Offsets & offsets, size_t total_rows) const; diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 8e3005f2f6d..50858392c3b 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -15,12 +15,6 @@ namespace static constexpr auto END_OF_GRANULE_FLAG = 1ULL << 63; -struct SerializeStateSparse : public ISerialization::SerializeBinaryBulkState -{ - size_t num_trailing_default_values = 0; - ISerialization::SerializeBinaryBulkStatePtr nested; -}; - struct DeserializeStateSparse : public ISerialization::DeserializeBinaryBulkState { size_t num_trailing_defaults = 0; @@ -28,44 +22,24 @@ struct DeserializeStateSparse : public ISerialization::DeserializeBinaryBulkStat ISerialization::DeserializeBinaryBulkStatePtr nested; }; -void serializeOffsetsPositionIndependent(const IColumn::Offsets & offsets, WriteBuffer & ostr, size_t start, size_t end) +void serializeOffsets(const IColumn::Offsets & offsets, WriteBuffer & ostr, size_t start, size_t end) { - // std::cerr << "writing start: " << start << ", end: " << end << "\n"; - // std::cerr << "offsets: "; - // for (const auto & x : offsets) - // std::cerr << x << " "; - // std::cerr << "\n"; - size_t size = offsets.size(); for (size_t i = 0; i < size; ++i) { size_t group_size = offsets[i] - start; - - // std::cerr << "writing group_size: " << group_size << "\n"; - writeIntBinary(group_size, ostr); start += group_size + 1; } - // std::cerr << "writing start: " << start << ", end: " << end << "\n"; size_t group_size = start < end ? end - start : 0; - // std::cerr << "writing end group_size: " << group_size << "\n"; group_size |= END_OF_GRANULE_FLAG; writeIntBinary(group_size, ostr); } -// struct DeserializedRows -// { -// size_t total = 0; -// size_t trailing_defaults = 0; -// }; - -size_t deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, +size_t deserializeOffsets(IColumn::Offsets & offsets, ReadBuffer & istr, size_t limit, DeserializeStateSparse & state) { - // std::cerr << "limit: " << limit << ", num_trailing: " << state.num_trailing_defaults - // << ", has_value_after_defaults: " << state.has_value_after_defaults << "\n"; - if (limit && state.num_trailing_defaults >= limit) { state.num_trailing_defaults -= limit; @@ -73,7 +47,7 @@ size_t deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, } /// TODO: - offsets.reserve(limit / 10); /// TODO + offsets.reserve(limit / 10); size_t total_rows = state.num_trailing_defaults; if (state.has_value_after_defaults) @@ -94,14 +68,9 @@ size_t deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, bool end_of_granule = group_size & END_OF_GRANULE_FLAG; group_size &= ~END_OF_GRANULE_FLAG; - // std::cerr << "read group_size: " << group_size << ", end_of_granule: " << end_of_granule << "\n"; size_t next_total_rows = total_rows + group_size; group_size += state.num_trailing_defaults; - - // std::cerr << "group_size: " << group_size << ", end_of_granule: " << end_of_granule << "\n"; - // std::cerr << "next_total_rows: " << next_total_rows << "\n"; - if (limit && next_total_rows >= limit) { state.num_trailing_defaults = next_total_rows - limit; @@ -150,13 +119,9 @@ void SerializationSparse::serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto state_sparse = std::make_shared(); - settings.path.push_back(Substream::SparseElements); - nested_serialization->serializeBinaryBulkStatePrefix(settings, state_sparse->nested); + nested_serialization->serializeBinaryBulkStatePrefix(settings, state); settings.path.pop_back(); - - state = std::move(state_sparse); } void SerializationSparse::serializeBinaryBulkWithMultipleStreams( @@ -167,9 +132,6 @@ void SerializationSparse::serializeBinaryBulkWithMultipleStreams( SerializeBinaryBulkStatePtr & state) const { size_t size = column.size(); - auto * state_sparse = checkAndGetSerializeState(state, *this); - - // std::cerr << "writing column: " << column.dumpStructure() << "\n"; auto offsets_column = DataTypeNumber().createColumn(); auto & offsets_data = assert_cast &>(*offsets_column).getData(); @@ -179,7 +141,7 @@ void SerializationSparse::serializeBinaryBulkWithMultipleStreams( if (auto * stream = settings.getter(settings.path)) { size_t end = limit && offset + limit < size ? offset + limit : size; - serializeOffsetsPositionIndependent(offsets_data, *stream, offset, end); + serializeOffsets(offsets_data, *stream, offset, end); } if (!offsets_data.empty()) @@ -190,13 +152,12 @@ void SerializationSparse::serializeBinaryBulkWithMultipleStreams( const auto & values = column_sparse->getValuesColumn(); size_t begin = column_sparse->getValueIndex(offsets_data[0]); size_t end = column_sparse->getValueIndex(offsets_data.back()); - // std::cerr << "begin: " << begin << ", end: " << end << "\n"; - nested_serialization->serializeBinaryBulkWithMultipleStreams(values, begin, end - begin + 1, settings, state_sparse->nested); + nested_serialization->serializeBinaryBulkWithMultipleStreams(values, begin, end - begin + 1, settings, state); } else { auto values = column.index(*offsets_column, 0); - nested_serialization->serializeBinaryBulkWithMultipleStreams(*values, 0, values->size(), settings, state_sparse->nested); + nested_serialization->serializeBinaryBulkWithMultipleStreams(*values, 0, values->size(), settings, state); } } @@ -207,10 +168,8 @@ void SerializationSparse::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * state_sparse = checkAndGetSerializeState(state, *this); - settings.path.push_back(Substream::SparseElements); - nested_serialization->serializeBinaryBulkStateSuffix(settings, state_sparse->nested); + nested_serialization->serializeBinaryBulkStateSuffix(settings, state); settings.path.pop_back(); } @@ -245,7 +204,7 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( size_t read_rows = 0; if (auto * stream = settings.getter(settings.path)) - read_rows = deserializeOffsetsPositionIndependent(offsets_data, *stream, limit, *state_sparse); + read_rows = deserializeOffsets(offsets_data, *stream, limit, *state_sparse); auto & values_column = column_sparse.getValuesPtr(); size_t values_limit = offsets_data.size() - old_size; @@ -259,26 +218,7 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( " Offsets size: {}, values size: {}", offsets_data.size(), values_column->size()); column_sparse.insertManyDefaults(read_rows); - - // std::cerr << "column_sparse: " << column_sparse.dumpStructure() << "\n"; - // std::cerr << "offsets: "; - // for (const auto & x : column_sparse.getOffsetsData()) - // std::cerr << x << " "; - // std::cerr << "\n"; - - // std::cerr << "values: "; - // for (size_t i = 0; i < column_sparse.getValuesColumn().size(); ++i) - // std::cerr << toString(column_sparse.getValuesColumn()[i]) << " "; - // std::cerr << "\n"; - column = std::move(mutable_column); } -// void SerializationSparse::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const -// { -// const auto & column_sparse = assert_cast(column); -// const auto & values_column = column_sparse.getValuesColumn(); -// nested_serialization->serializeText(values_column, column_sparse.getValueIndex(row_num), ostr, settings); -// } - } diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 6d06ec91b01..9c9def2c8c0 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -483,7 +483,17 @@ void NO_INLINE Aggregator::executeImplBatch( } } - if (!has_arrays) + bool has_sparse = false; + for (auto * inst = aggregate_instructions; inst->that; ++inst) + { + if (inst->has_sparse_arguments) + { + has_sparse = true; + break; + } + } + + if (!has_arrays && !has_sparse) { for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) { diff --git a/tests/queries/0_stateless/01780_column_sparse.reference b/tests/queries/0_stateless/01780_column_sparse.reference new file mode 100644 index 00000000000..abe04f9fda8 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse.reference @@ -0,0 +1,120 @@ +arr1 Sparse +arr2 Sparse +id Default +s Sparse +u Sparse +10 10 10 [] [] +20 20 20 [] [] +30 30 30 [] [] +40 40 40 [] [] +50 50 50 [] [] +60 60 60 [] [] +70 70 70 [] [] +80 80 80 [] [] +90 90 90 [] [] +100 100 100 [] [] +110 110 110 [] [] +120 120 120 [] [] +130 130 130 [] [] +140 140 140 [] [] +150 150 150 [] [] +160 160 160 [] [] +170 170 170 [] [] +180 180 180 [] [] +190 190 190 [] [] +=========== +0 0 0 [] [] +5 0 5 [] [] +10 10 10 [] [] +15 0 15 [] [] +20 20 20 [] [] +25 0 25 [] [] +30 30 30 [] [] +35 0 35 ['0','1','2','3','4'] [] +40 40 40 [] [] +45 0 45 [] [] +50 50 50 [] [] +55 0 55 [] [] +60 60 60 [] [] +65 0 65 [] [] +70 70 70 [] [] +75 0 75 [] [] +80 80 80 [] [] +85 0 85 [] [] +90 90 90 [] [] +95 0 95 [] [] +100 100 100 [] [] +105 0 105 ['0','1','2','3','4'] [] +110 110 110 [] [] +115 0 115 [] [] +120 120 120 [] [] +125 0 125 [] [] +130 130 130 [] [] +135 0 135 [] [] +140 140 140 [] [] +145 0 145 [] [] +150 150 150 [] [] +155 0 155 [] [] +160 160 160 [] [] +165 0 165 [] [] +170 170 170 [] [] +175 0 175 ['0','1','2','3','4'] [] +180 180 180 [] [] +185 0 185 [] [] +190 190 190 [] [] +195 0 195 [] [] +=========== +7 0 ['0','1','2','3','4','5','6'] [] +14 0 ['0','1','2','3'] [] +21 0 ['0'] [] +28 0 ['0','1','2','3','4','5','6','7'] [] +35 0 35 ['0','1','2','3','4'] [] +42 0 ['0','1'] [] +49 0 ['0','1','2','3','4','5','6','7','8'] [] +56 0 ['0','1','2','3','4','5'] [] +63 0 ['0','1','2'] [] +77 0 ['0','1','2','3','4','5','6'] [] +84 0 ['0','1','2','3'] [0,1,2,3] +91 0 ['0'] [] +98 0 ['0','1','2','3','4','5','6','7'] [] +105 0 105 ['0','1','2','3','4'] [] +112 0 ['0','1'] [] +119 0 ['0','1','2','3','4','5','6','7','8'] [] +126 0 ['0','1','2','3','4','5'] [] +133 0 ['0','1','2'] [] +147 0 ['0','1','2','3','4','5','6'] [] +154 0 ['0','1','2','3'] [] +161 0 ['0'] [] +168 0 ['0','1','2','3','4','5','6','7'] [0,1,2,3,4,5,6,7] +175 0 175 ['0','1','2','3','4'] [] +182 0 ['0','1'] [] +189 0 ['0','1','2','3','4','5','6','7','8'] [] +196 0 ['0','1','2','3','4','5'] [] +=========== +12 0 [] [0,1] +24 0 [] [0,1,2,3] +36 0 [] [0,1,2,3,4,5] +48 0 [] [0,1,2,3,4,5,6,7] +72 0 [] [0,1] +84 0 ['0','1','2','3'] [0,1,2,3] +96 0 [] [0,1,2,3,4,5] +108 0 [] [0,1,2,3,4,5,6,7] +132 0 [] [0,1] +144 0 [] [0,1,2,3] +156 0 [] [0,1,2,3,4,5] +168 0 ['0','1','2','3','4','5','6','7'] [0,1,2,3,4,5,6,7] +192 0 [] [0,1] +1900 +210 +360 +300 +240 +190 +330 +270 +=========== +[1] +[1,3] +[1,3,5] +[1,3,5,7] +[1] diff --git a/tests/queries/0_stateless/01780_column_sparse.sql b/tests/queries/0_stateless/01780_column_sparse.sql new file mode 100644 index 00000000000..8297fc2ee71 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse.sql @@ -0,0 +1,34 @@ +DROP TABLE IF EXISTS t_sparse; + +CREATE TABLE t_sparse (id UInt64, u UInt64, s String, arr1 Array(String), arr2 Array(UInt64)) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.1; + +INSERT INTO t_sparse SELECT + number, + if (number % 10 = 0, number, 0), + if (number % 5 = 0, toString(number), ''), + if (number % 7 = 0, arrayMap(x -> toString(x), range(number % 10)), []), + if (number % 12 = 0, range(number % 10), []) +FROM numbers (200); + +SELECT column, serialization_kind FROM system.parts_columns +WHERE table = 't_sparse' AND database = currentDatabase() +ORDER BY column; + +SELECT * FROM t_sparse WHERE u != 0 ORDER BY id; +SELECT '==========='; +SELECT * FROM t_sparse WHERE s != '' ORDER BY id; +SELECT '==========='; +SELECT * FROM t_sparse WHERE arr1 != [] ORDER BY id; +SELECT '==========='; +SELECT * FROM t_sparse WHERE arr2 != [] ORDER BY id; + +SELECT sum(u) FROM t_sparse; +SELECT sum(u) FROM t_sparse GROUP BY id % 7; + +SELECT '==========='; + +SELECT arrayFilter(x -> x % 2 = 1, arr2) FROM t_sparse WHERE arr2 != [] LIMIT 5; + +DROP TABLE t_sparse; From 0d1ac1fec3671ca73b3f3e7a34922d1999425c99 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 3 Apr 2021 05:40:30 +0300 Subject: [PATCH 006/358] ColumnSparse: sorting --- src/Columns/ColumnSparse.cpp | 126 +++++++++++++----- src/Columns/ColumnSparse.h | 7 +- .../0_stateless/01780_column_sparse.reference | 70 +++++++++- .../0_stateless/01780_column_sparse.sql | 20 ++- 4 files changed, 183 insertions(+), 40 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 27d677d4342..45045b42bda 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,5 +1,7 @@ #include #include +#include +#include namespace DB { @@ -7,6 +9,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; } ColumnSparse::ColumnSparse(MutableColumnPtr && values_) @@ -86,6 +89,7 @@ void ColumnSparse::insertData(const char * pos, size_t length) return values->insertData(pos, length); } +/// TODO: maybe need to reimplement it. StringRef ColumnSparse::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { return values->serializeValueIntoArena(getValueIndex(n), arena, begin); @@ -93,13 +97,13 @@ StringRef ColumnSparse::serializeValueIntoArena(size_t n, Arena & arena, char co const char * ColumnSparse::deserializeAndInsertFromArena(const char * pos) { - UNUSED(pos); - throwMustBeDense(); + ++_size; + return values->deserializeAndInsertFromArena(pos); } -const char * ColumnSparse::skipSerializedInArena(const char *) const +const char * ColumnSparse::skipSerializedInArena(const char * pos) const { - throwMustBeDense(); + return values->skipSerializedInArena(pos); } void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t length) @@ -272,9 +276,12 @@ ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const { limit = limit ? std::min(limit, _size) : _size; - auto res_values = values->cloneEmpty(); + if (perm.size() < limit) + throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + auto res_offsets = offsets->cloneEmpty(); auto & res_offsets_data = assert_cast(*res_offsets).getData(); + auto res_values = values->cloneEmpty(); res_values->insertDefault(); for (size_t i = 0; i < limit; ++i) @@ -292,10 +299,30 @@ ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const ColumnPtr ColumnSparse::index(const IColumn & indexes, size_t limit) const { - UNUSED(indexes); - UNUSED(limit); + return selectIndexImpl(*this, indexes, limit); +} - throwMustBeDense(); +template +ColumnPtr ColumnSparse::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + limit = limit ? std::min(limit, indexes.size()) : indexes.size(); + + auto res_offsets = offsets->cloneEmpty(); + auto & res_offsets_data = assert_cast(*res_offsets).getData(); + auto res_values = values->cloneEmpty(); + res_values->insertDefault(); + + for (size_t i = 0; i < limit; ++i) + { + size_t index = getValueIndex(indexes[i]); + if (index != 0) + { + res_values->insertFrom(*values, index); + res_offsets_data.push_back(i); + } + } + + return ColumnSparse::create(std::move(res_values), std::move(res_offsets), limit); } int ColumnSparse::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const @@ -320,14 +347,12 @@ void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, throwMustBeDense(); } -int ColumnSparse::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator &) const +int ColumnSparse::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator & collator) const { - UNUSED(n); - UNUSED(m); - UNUSED(rhs); - UNUSED(null_direction_hint); + if (const auto * rhs_sparse = typeid_cast(&rhs)) + return values->compareAtWithCollation(getValueIndex(n), rhs_sparse->getValueIndex(m), rhs_sparse->getValuesColumn(), null_direction_hint, collator); - throwMustBeDense(); + return values->compareAtWithCollation(getValueIndex(n), m, rhs, null_direction_hint, collator); } bool ColumnSparse::hasEqualValues() const @@ -337,12 +362,54 @@ bool ColumnSparse::hasEqualValues() const void ColumnSparse::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const { - UNUSED(reverse); - UNUSED(limit); - UNUSED(null_direction_hint); - UNUSED(res); + if (_size == 0) + return; - throwMustBeDense(); + res.resize(_size); + for (size_t i = 0; i < _size; ++i) + res[i] = i; + + if (offsets->empty()) + return; + + Permutation perm; + values->getPermutation(reverse, limit, null_direction_hint, perm); + + if (limit == 0 || limit > _size) + limit = _size; + + size_t num_of_defaults = getNumberOfDefaults(); + size_t row = 0; + size_t current_offset = 0; + size_t current_default_row = 0; + const auto & offsets_data = getOffsetsData(); + + for (size_t i = 0; i < perm.size() && row < limit; ++i) + { + if (perm[i] == 0) + { + if (!num_of_defaults) + continue; + + while (row < limit && current_default_row < _size) + { + while (current_offset < offsets_data.size() && current_default_row == offsets_data[current_offset]) + { + ++current_offset; + ++current_default_row; + } + + res[row++] = current_default_row++; + } + } + else + { + res[row] = offsets_data[perm[i] - 1]; + ++row; + } + } + + assert(row == limit); } void ColumnSparse::updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const @@ -406,7 +473,8 @@ size_t ColumnSparse::allocatedBytes() const void ColumnSparse::protect() { - throwMustBeDense(); + values->protect(); + offsets->protect(); } ColumnPtr ColumnSparse::replicate(const Offsets & replicate_offsets) const @@ -417,9 +485,7 @@ ColumnPtr ColumnSparse::replicate(const Offsets & replicate_offsets) const void ColumnSparse::updateHashWithValue(size_t n, SipHash & hash) const { - UNUSED(n); - UNUSED(hash); - throwMustBeDense(); + values->updateHashWithValue(getValueIndex(n), hash); } void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const @@ -430,15 +496,14 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const void ColumnSparse::updateHashFast(SipHash & hash) const { - UNUSED(hash); - throwMustBeDense(); + values->updateHashFast(hash); + offsets->updateHashFast(hash); + hash.update(_size); } void ColumnSparse::getExtremes(Field & min, Field & max) const { - UNUSED(min); - UNUSED(max); - throwMustBeDense(); + values->getExtremes(min, max); } void ColumnSparse::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const @@ -475,8 +540,9 @@ ColumnPtr ColumnSparse::compress() const bool ColumnSparse::structureEquals(const IColumn & rhs) const { - UNUSED(rhs); - throwMustBeDense(); + if (auto rhs_sparse = typeid_cast(&rhs)) + return values->structureEquals(*rhs_sparse->values); + return false; } const IColumn::Offsets & ColumnSparse::getOffsetsData() const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 226d85e81b8..a4ba2df91c0 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -80,12 +80,17 @@ public: void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; + ColumnPtr index(const IColumn & indexes, size_t limit) const override; + + template + ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; + int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; - int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator &) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator & collator) const override; bool hasEqualValues() const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const override; diff --git a/tests/queries/0_stateless/01780_column_sparse.reference b/tests/queries/0_stateless/01780_column_sparse.reference index abe04f9fda8..6823c57d410 100644 --- a/tests/queries/0_stateless/01780_column_sparse.reference +++ b/tests/queries/0_stateless/01780_column_sparse.reference @@ -1,8 +1,26 @@ +-- { echo } + +DROP TABLE IF EXISTS t_sparse; +DROP TABLE IF EXISTS t_sparse_1; +CREATE TABLE t_sparse (id UInt64, u UInt64, s String, arr1 Array(String), arr2 Array(UInt64)) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.1; +INSERT INTO t_sparse SELECT + number, + if (number % 10 = 0, number, 0), + if (number % 5 = 0, toString(number), ''), + if (number % 7 = 0, arrayMap(x -> toString(x), range(number % 10)), []), + if (number % 12 = 0, range(number % 10), []) +FROM numbers (200); +SELECT column, serialization_kind FROM system.parts_columns +WHERE table = 't_sparse' AND database = currentDatabase() +ORDER BY column; arr1 Sparse arr2 Sparse id Default s Sparse u Sparse +SELECT * FROM t_sparse WHERE u != 0 ORDER BY id; 10 10 10 [] [] 20 20 20 [] [] 30 30 30 [] [] @@ -22,7 +40,7 @@ u Sparse 170 170 170 [] [] 180 180 180 [] [] 190 190 190 [] [] -=========== +SELECT * FROM t_sparse WHERE s != '' ORDER BY id; 0 0 0 [] [] 5 0 5 [] [] 10 10 10 [] [] @@ -63,7 +81,7 @@ u Sparse 185 0 185 [] [] 190 190 190 [] [] 195 0 195 [] [] -=========== +SELECT * FROM t_sparse WHERE arr1 != [] ORDER BY id; 7 0 ['0','1','2','3','4','5','6'] [] 14 0 ['0','1','2','3'] [] 21 0 ['0'] [] @@ -90,7 +108,7 @@ u Sparse 182 0 ['0','1'] [] 189 0 ['0','1','2','3','4','5','6','7','8'] [] 196 0 ['0','1','2','3','4','5'] [] -=========== +SELECT * FROM t_sparse WHERE arr2 != [] ORDER BY id; 12 0 [] [0,1] 24 0 [] [0,1,2,3] 36 0 [] [0,1,2,3,4,5] @@ -104,7 +122,9 @@ u Sparse 156 0 [] [0,1,2,3,4,5] 168 0 ['0','1','2','3','4','5','6','7'] [0,1,2,3,4,5,6,7] 192 0 [] [0,1] +SELECT sum(u) FROM t_sparse; 1900 +SELECT sum(u) FROM t_sparse GROUP BY id % 7; 210 360 300 @@ -112,9 +132,51 @@ u Sparse 190 330 270 -=========== +SELECT arrayFilter(x -> x % 2 = 1, arr2) FROM t_sparse WHERE arr2 != [] LIMIT 5; [1] [1,3] [1,3,5] [1,3,5,7] [1] +CREATE TABLE t_sparse_1 (id UInt64, v Int64) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0; +INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0); +SELECT * FROM t_sparse_1 ORDER BY v; +7 -2 +4 -1 +3 0 +5 0 +6 0 +8 0 +9 0 +11 0 +2 1 +10 4 +1 6 +SELECT * FROM t_sparse_1 ORDER BY v DESC; +1 6 +10 4 +2 1 +3 0 +5 0 +6 0 +8 0 +9 0 +11 0 +4 -1 +7 -2 +SELECT * FROM t_sparse_1 ORDER BY v LIMIT 5; +7 -2 +4 -1 +3 0 +5 0 +6 0 +SELECT * FROM t_sparse_1 ORDER BY v DESC LIMIT 5; +1 6 +10 4 +2 1 +3 0 +5 0 +DROP TABLE t_sparse; +DROP TABLE t_sparse_1; diff --git a/tests/queries/0_stateless/01780_column_sparse.sql b/tests/queries/0_stateless/01780_column_sparse.sql index 8297fc2ee71..3efa263d2a8 100644 --- a/tests/queries/0_stateless/01780_column_sparse.sql +++ b/tests/queries/0_stateless/01780_column_sparse.sql @@ -1,4 +1,7 @@ +-- { echo } + DROP TABLE IF EXISTS t_sparse; +DROP TABLE IF EXISTS t_sparse_1; CREATE TABLE t_sparse (id UInt64, u UInt64, s String, arr1 Array(String), arr2 Array(UInt64)) ENGINE = MergeTree ORDER BY tuple() @@ -17,18 +20,25 @@ WHERE table = 't_sparse' AND database = currentDatabase() ORDER BY column; SELECT * FROM t_sparse WHERE u != 0 ORDER BY id; -SELECT '==========='; SELECT * FROM t_sparse WHERE s != '' ORDER BY id; -SELECT '==========='; SELECT * FROM t_sparse WHERE arr1 != [] ORDER BY id; -SELECT '==========='; SELECT * FROM t_sparse WHERE arr2 != [] ORDER BY id; SELECT sum(u) FROM t_sparse; SELECT sum(u) FROM t_sparse GROUP BY id % 7; -SELECT '==========='; - SELECT arrayFilter(x -> x % 2 = 1, arr2) FROM t_sparse WHERE arr2 != [] LIMIT 5; +CREATE TABLE t_sparse_1 (id UInt64, v Int64) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0; + +INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0); + +SELECT * FROM t_sparse_1 ORDER BY v; +SELECT * FROM t_sparse_1 ORDER BY v DESC; +SELECT * FROM t_sparse_1 ORDER BY v LIMIT 5; +SELECT * FROM t_sparse_1 ORDER BY v DESC LIMIT 5; + DROP TABLE t_sparse; +DROP TABLE t_sparse_1; From d7ca6ecc502bd15db9c1bb7739035d143f2db1b0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Apr 2021 03:02:48 +0300 Subject: [PATCH 007/358] ColumnSparse: sorting and dummy aggregation --- src/Columns/ColumnSparse.cpp | 109 ++++++++++++------ src/Columns/ColumnSparse.h | 5 + src/Interpreters/Aggregator.cpp | 2 +- .../Transforms/SortingTransform.cpp | 7 ++ 4 files changed, 88 insertions(+), 35 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 45045b42bda..bcb39a804c4 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -337,14 +338,25 @@ void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const { - UNUSED(rhs); - UNUSED(rhs_row_num); - UNUSED(row_indexes); - UNUSED(compare_results); - UNUSED(direction); - UNUSED(nan_direction_hint); + if (row_indexes) + { + /// TODO: implement without conversion to full column. + auto this_full = convertToFullColumnIfSparse(); + auto rhs_full = rhs.convertToFullColumnIfSparse(); + this_full->compareColumn(*rhs_full, rhs_row_num, row_indexes, compare_results, direction, nan_direction_hint); + } + else + { + const auto & rhs_sparse = assert_cast(rhs); + PaddedPODArray nested_result; + values->compareColumn(rhs_sparse.getValuesColumn(), rhs_sparse.getValueIndex(rhs_row_num), + nullptr, nested_result, direction, nan_direction_hint); - throwMustBeDense(); + const auto & offsets_data = getOffsetsData(); + compare_results.resize(_size, nested_result[0]); + for (size_t i = 0; i < offsets_data.size(); ++i) + compare_results[offsets_data[i]] = nested_result[i + 1]; + } } int ColumnSparse::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator & collator) const @@ -360,7 +372,7 @@ bool ColumnSparse::hasEqualValues() const return offsets->size() == 0; } -void ColumnSparse::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const { if (_size == 0) return; @@ -373,7 +385,10 @@ void ColumnSparse::getPermutation(bool reverse, size_t limit, int null_direction return; Permutation perm; - values->getPermutation(reverse, limit, null_direction_hint, perm); + if (collator) + values->getPermutationWithCollation(*collator, reverse, limit, null_direction_hint, perm); + else + values->getPermutation(reverse, limit, null_direction_hint, perm); if (limit == 0 || limit > _size) limit = _size; @@ -412,39 +427,27 @@ void ColumnSparse::getPermutation(bool reverse, size_t limit, int null_direction assert(row == limit); } +void ColumnSparse::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const +{ + return getPermutationImpl(reverse, limit, null_direction_hint, res, nullptr); +} + void ColumnSparse::updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const { - UNUSED(reverse); - UNUSED(null_direction_hint); - UNUSED(limit); - UNUSED(res); - UNUSED(equal_range); - - throwMustBeDense(); + auto this_full = convertToFullColumnIfSparse(); + this_full->updatePermutation(reverse, limit, null_direction_hint, res, equal_range); } void ColumnSparse::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const { - UNUSED(collator); - UNUSED(reverse); - UNUSED(limit); - UNUSED(null_direction_hint); - UNUSED(res); - - throwMustBeDense(); + return getPermutationImpl(reverse, limit, null_direction_hint, res, &collator); } void ColumnSparse::updatePermutationWithCollation( const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const { - UNUSED(collator); - UNUSED(reverse); - UNUSED(limit); - UNUSED(null_direction_hint); - UNUSED(res); - UNUSED(equal_range); - - throwMustBeDense(); + auto this_full = convertToFullColumnIfSparse(); + this_full->updatePermutationWithCollation(collator, reverse, limit, null_direction_hint, res, equal_range); } void ColumnSparse::reserve(size_t) @@ -479,8 +482,37 @@ void ColumnSparse::protect() ColumnPtr ColumnSparse::replicate(const Offsets & replicate_offsets) const { - UNUSED(replicate_offsets); - throwMustBeDense(); + /// TODO: implement specializations. + if (_size != replicate_offsets.size()) + throw Exception("Size of offsets doesn't match size of column.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + if (_size == 0) + return ColumnSparse::create(values->cloneEmpty()); + + const auto & offsets_data = getOffsetsData(); + auto res_offsets = offsets->cloneEmpty(); + auto & res_offsets_data = assert_cast(*res_offsets).getData(); + auto res_values = values->cloneEmpty(); + res_values->insertDefault(); + + size_t current_offset = 0; + for (size_t i = 0; i < _size; ++i) + { + if (current_offset < offsets_data.size() && i == offsets_data[current_offset]) + { + size_t replicate_size = replicate_offsets[i] - replicate_offsets[i - 1]; + res_offsets_data.reserve(res_offsets_data.size() + replicate_size); + for (size_t row = replicate_offsets[i - 1]; row < replicate_offsets[i]; ++row) + { + res_offsets_data.push_back(row); + res_values->insertFrom(*values, current_offset + 1); + } + + ++current_offset; + } + } + + return ColumnSparse::create(std::move(res_values), std::move(res_offsets), replicate_offsets.back()); } void ColumnSparse::updateHashWithValue(size_t n, SipHash & hash) const @@ -535,7 +567,16 @@ void ColumnSparse::gather(ColumnGathererStream & gatherer_stream) ColumnPtr ColumnSparse::compress() const { - throwMustBeDense(); + auto values_compressed = values->compress(); + auto offsets_compressed = offsets->compress(); + + size_t byte_size = values_compressed->byteSize() + offsets_compressed->byteSize(); + + return ColumnCompressed::create(size(), byte_size, + [values_compressed = std::move(values_compressed), offsets_compressed = std::move(offsets_compressed), size = size()] + { + return ColumnSparse::create(values_compressed->decompress(), offsets_compressed->decompress(), size); + }); } bool ColumnSparse::structureEquals(const IColumn & rhs) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index a4ba2df91c0..837c721b01f 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -90,13 +90,18 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator & collator) const override; bool hasEqualValues() const override; + + void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const; + void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const override; void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutationWithCollation( const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const override; + void reserve(size_t n) override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 10449ccf1ea..a92d08555fd 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -707,7 +707,7 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData /// Remember the columns we will work with for (size_t i = 0; i < params.keys_size; ++i) { - materialized_columns.push_back(columns.at(params.keys[i])->convertToFullColumnIfConst()); + materialized_columns.push_back(columns.at(params.keys[i])->convertToFullColumnIfSparse()->convertToFullColumnIfConst()); key_columns[i] = materialized_columns.back().get(); if (!result.isLowCardinality()) diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index 11f23530c9e..fb924651ffc 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -37,6 +37,13 @@ MergeSorter::MergeSorter(Chunks chunks_, SortDescription & description_, size_t if (chunk.getNumRows() == 0) continue; + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = column->convertToFullColumnIfSparse(); + + chunk.setColumns(std::move(columns), num_rows); + cursors.emplace_back(chunk.getColumns(), description); has_collation |= cursors.back().has_collation; From 4b8e90338d867899ccc055d240a6b7554f24d7a7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 6 Apr 2021 18:59:03 +0300 Subject: [PATCH 008/358] ColumnSparse: fixes --- src/Columns/ColumnSparse.cpp | 24 +++++++++++ src/Columns/ColumnSparse.h | 8 ++-- src/Columns/FilterDescription.cpp | 2 +- src/Core/NamesAndTypes.cpp | 12 ++++++ src/Core/NamesAndTypes.h | 2 + src/DataStreams/NativeBlockInputStream.cpp | 2 +- src/DataStreams/NativeBlockOutputStream.cpp | 3 +- src/DataTypes/DataTypeTuple.cpp | 14 +++++++ src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/IDataType.cpp | 16 +++++--- src/DataTypes/IDataType.h | 4 +- .../Serializations/ISerialization.cpp | 14 ++++++- .../Serializations/SerializationTuple.h | 2 + .../Serializations/SerializationWrapper.h | 2 + src/Functions/IFunction.cpp | 2 +- src/Interpreters/Aggregator.cpp | 4 +- src/Processors/Chunk.cpp | 11 +++++ src/Processors/Chunk.h | 2 + src/Processors/Merges/IMergingTransform.cpp | 1 + .../Transforms/SortingTransform.cpp | 7 +--- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 +++--- .../MergeTree/MergeTreeReaderWide.cpp | 7 +--- .../01780_column_sparse_tuple.reference | 40 +++++++++++++++++++ .../0_stateless/01780_column_sparse_tuple.sql | 21 ++++++++++ 24 files changed, 175 insertions(+), 38 deletions(-) create mode 100644 tests/queries/0_stateless/01780_column_sparse_tuple.reference create mode 100644 tests/queries/0_stateless/01780_column_sparse_tuple.sql diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index bcb39a804c4..7c7befd52e9 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -586,6 +587,12 @@ bool ColumnSparse::structureEquals(const IColumn & rhs) const return false; } +void ColumnSparse::forEachSubcolumn(ColumnCallback callback) +{ + callback(values); + callback(offsets); +} + const IColumn::Offsets & ColumnSparse::getOffsetsData() const { return assert_cast(*offsets).getData(); @@ -608,4 +615,21 @@ size_t ColumnSparse::getValueIndex(size_t n) const return it - offsets_data.begin() + 1; } +ColumnPtr recursiveRemoveSparse(const ColumnPtr & column) +{ + if (!column) + return column; + + if (const auto * column_tuple = typeid_cast(column.get())) + { + auto columns = column_tuple->getColumns(); + for (auto & element : columns) + element = recursiveRemoveSparse(element); + + return ColumnTuple::create(columns); + } + + return column->convertToFullColumnIfSparse(); +} + } diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 837c721b01f..f0beb4cdefb 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -122,11 +122,7 @@ public: ColumnPtr compress() const override; - void forEachSubcolumn(ColumnCallback callback) override - { - callback(values); - callback(offsets); - } + void forEachSubcolumn(ColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; @@ -170,4 +166,6 @@ private: size_t _size; }; +ColumnPtr recursiveRemoveSparse(const ColumnPtr & column); + } diff --git a/src/Columns/FilterDescription.cpp b/src/Columns/FilterDescription.cpp index 8e9836729a6..bebd0230d8d 100644 --- a/src/Columns/FilterDescription.cpp +++ b/src/Columns/FilterDescription.cpp @@ -52,7 +52,7 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) FilterDescription::FilterDescription(const IColumn & column_) { if (column_.isSparse()) - data_holder = column_.convertToFullColumnIfSparse(); + data_holder = recursiveRemoveSparse(column_.getPtr()); if (column_.lowCardinality()) data_holder = column_.convertToFullColumnIfLowCardinality(); diff --git a/src/Core/NamesAndTypes.cpp b/src/Core/NamesAndTypes.cpp index 57d29c96c53..f25b616f358 100644 --- a/src/Core/NamesAndTypes.cpp +++ b/src/Core/NamesAndTypes.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -42,6 +43,17 @@ String NameAndTypePair::getSubcolumnName() const return name.substr(*subcolumn_delimiter_position + 1, name.size() - *subcolumn_delimiter_position); } +String NameAndTypePair::dump() const +{ + WriteBufferFromOwnString out; + out << "name: " << name << "\n" + << "type: " << type->getName() << "\n" + << "name in storage: " << getNameInStorage() << "\n" + << "type in storage: " << getTypeInStorage()->getName(); + + return out.str(); +} + void NamesAndTypesList::readText(ReadBuffer & buf) { const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index fc86c7f6a1d..a3686e0e883 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -39,6 +39,8 @@ public: return name == rhs.name && type->equals(*rhs.type); } + String dump() const; + String name; DataTypePtr type; diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index 7560cd5006c..5780d160dc3 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -175,7 +175,7 @@ Block NativeBlockInputStream::readImpl() readData(*column.type, read_column, istr, rows, avg_value_size_hint); /// TODO: maybe remove. - read_column = read_column->convertToFullColumnIfSparse(); + read_column = recursiveRemoveSparse(read_column); column.column = std::move(read_column); // std::cerr << "column.column: " << column.column->dumpStructure() << "\n"; diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index 5badb11d338..b05e80354b4 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB { @@ -119,7 +120,7 @@ void NativeBlockOutputStream::write(const Block & block) writeStringBinary(type_name, ostr); - column.column = column.column->convertToFullColumnIfSparse(); + column.column = recursiveRemoveSparse(column.column); /// TODO: add revision auto serialization = column.type->getSerialization(*column.column); diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 435cdcbbe6e..0fbb98ff933 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -163,6 +163,20 @@ MutableColumnPtr DataTypeTuple::createColumn() const return ColumnTuple::create(std::move(tuple_columns)); } +MutableColumnPtr DataTypeTuple::createColumn(const ISerialization & serialization) const +{ + const auto & element_serializations = + assert_cast(serialization).getElementsSerializations(); + + size_t size = elems.size(); + assert(element_serializations.size() == size); + MutableColumns tuple_columns(size); + for (size_t i = 0; i < size; ++i) + tuple_columns[i] = elems[i]->createColumn(*element_serializations[i]); + + return ColumnTuple::create(std::move(tuple_columns)); +} + Field DataTypeTuple::getDefault() const { return Tuple(ext::map(elems, [] (const DataTypePtr & elem) { return elem->getDefault(); })); diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 8b301674e61..5e19bb3d77d 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -38,6 +38,7 @@ public: bool canBeInsideNullable() const override { return false; } MutableColumnPtr createColumn() const override; + MutableColumnPtr createColumn(const ISerialization & serialization) const override; Field getDefault() const override; void insertDefaultInto(IColumn & column) const override; diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 999dfada206..ea442e5ec68 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -62,6 +62,15 @@ void IDataType::updateAvgValueSizeHint(const IColumn & column, double & avg_valu } } +MutableColumnPtr IDataType::createColumn(const ISerialization & serialization) const +{ + auto column = createColumn(); + if (serialization.getKind() == ISerialization::Kind::SPARSE) + return ColumnSparse::create(std::move(column)); + + return column; +} + ColumnPtr IDataType::createColumnConst(size_t size, const Field & field) const { auto column = createColumn(); @@ -190,7 +199,7 @@ SerializationPtr IDataType::getSerialization(const IColumn & column) const SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const { double ratio = settings.num_rows ? std::min(static_cast(settings.num_default_rows) / settings.num_rows, 1.0) : 0.0; - if (ratio > settings.ratio_for_sparse_serialization) + if (ratio >= settings.ratio_for_sparse_serialization) return getSparseSerialization(); return getDefaultSerialization(); @@ -239,9 +248,4 @@ void IDataType::enumerateStreams(const SerializationPtr & serialization, const S }, path); } -bool isSparseSerializaion(const SerializationPtr & serialization) -{ - return typeid_cast(serialization.get()); -} - } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 865c081d859..2689b989268 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -106,6 +106,8 @@ public: */ virtual MutableColumnPtr createColumn() const = 0; + virtual MutableColumnPtr createColumn(const ISerialization & serialization) const; + /** Create ColumnConst for corresponding type, with specified size and value. */ ColumnPtr createColumnConst(size_t size, const Field & field) const; @@ -504,6 +506,4 @@ template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; template <> inline constexpr bool IsDataTypeDateOrDateTime = true; -bool isSparseSerializaion(const SerializationPtr & serialization); - } diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index a37397b321b..68316ef7650 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -153,11 +153,23 @@ String ISerialization::getFileNameForStream(const NameAndTypePair & column, cons return getFileNameForStream(column.getNameInStorage(), path); } +static size_t isOffsetsOfNested(const ISerialization::SubstreamPath & path) +{ + if (path.empty()) + return false; + + for (const auto & elem : path) + if (elem.type == ISerialization::Substream::ArrayElements) + return false; + + return path.back().type == ISerialization::Substream::ArraySizes; +} + String ISerialization::getFileNameForStream(const String & name_in_storage, const SubstreamPath & path) { String stream_name; auto nested_storage_name = Nested::extractTableName(name_in_storage); - if (name_in_storage != nested_storage_name && (path.size() == 1 && path[0].type == ISerialization::Substream::ArraySizes)) + if (name_in_storage != nested_storage_name && isOffsetsOfNested(path)) stream_name = escapeForFileName(nested_storage_name); else stream_name = escapeForFileName(name_in_storage); diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index 13668572fff..afe3ab3a38f 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -59,6 +59,8 @@ public: DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; + const ElementSerializations & getElementsSerializations() const { return elems; } + private: ElementSerializations elems; bool have_explicit_names; diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 7fa6e3a0907..6a887884293 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -14,6 +14,8 @@ protected: public: SerializationWrapper(const SerializationPtr & nested_serialization_) : nested_serialization(nested_serialization_) {} + Kind getKind() const override { return nested_serialization->getKind(); } + void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; void serializeBinaryBulkStatePrefix( diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 701c676cc5a..8b1a7ebd641 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -383,7 +383,7 @@ static void convertLowCardinalityColumnsToFull(ColumnsWithTypeAndName & args) static void convertSparseColumnsToFull(ColumnsWithTypeAndName & args) { for (auto & column : args) - column.column = column.column->convertToFullColumnIfSparse(); + column.column = recursiveRemoveSparse(column.column); } ColumnPtr ExecutableFunctionAdaptor::executeWithoutSparseColumns(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index a92d08555fd..81b938b171e 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -628,7 +628,7 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns auto full_column = allow_sparse_arguments ? aggregate_columns[i][j]->getPtr() - : aggregate_columns[i][j]->convertToFullColumnIfSparse(); + : recursiveRemoveSparse(aggregate_columns[i][j]->getPtr()); full_column = recursiveRemoveLowCardinality(full_column); if (full_column.get() != aggregate_columns[i][j]) @@ -707,7 +707,7 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData /// Remember the columns we will work with for (size_t i = 0; i < params.keys_size; ++i) { - materialized_columns.push_back(columns.at(params.keys[i])->convertToFullColumnIfSparse()->convertToFullColumnIfConst()); + materialized_columns.push_back(recursiveRemoveSparse(columns.at(params.keys[i]))->convertToFullColumnIfConst()); key_columns[i] = materialized_columns.back().get(); if (!result.isLowCardinality()) diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 4800bfca2ce..94b48d3ec32 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -167,4 +168,14 @@ const ChunkMissingValues::RowsBitMask & ChunkMissingValues::getDefaultsBitmask(s return none; } +void convertToFullIfSparse(Chunk & chunk) +{ + size_t num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); + for (auto & column : columns) + column = recursiveRemoveSparse(column); + + chunk.setColumns(std::move(columns), num_rows); +} + } diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index bb8266d6e27..d5d75038d2f 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -129,4 +129,6 @@ private: RowsMaskByColumnId rows_mask_by_column_id; }; +void convertToFullIfSparse(Chunk & chunk); + } diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index eff786b150f..eb1bd5110e9 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -87,6 +87,7 @@ IProcessor::Status IMergingTransformBase::prepareInitializeInputs() continue; } + convertToFullIfSparse(chunk); state.init_chunks[i].set(std::move(chunk)); input_states[i].is_initialized = true; } diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index fb924651ffc..59e59c524af 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -37,12 +37,7 @@ MergeSorter::MergeSorter(Chunks chunks_, SortDescription & description_, size_t if (chunk.getNumRows() == 0) continue; - size_t num_rows = chunk.getNumRows(); - auto columns = chunk.detachColumns(); - for (auto & column : columns) - column = column->convertToFullColumnIfSparse(); - - chunk.setColumns(std::move(columns), num_rows); + convertToFullIfSparse(chunk); cursors.emplace_back(chunk.getColumns(), description); has_collation |= cursors.back().has_collation; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 95cf3c921ca..a0d1078aea3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -95,6 +96,7 @@ void MergeTreeDataPartWriterWide::addStreams( IDataType::StreamCallbackWithType callback = [&] (const ISerialization::SubstreamPath & substream_path, const IDataType & substream_type) { String stream_name = ISerialization::getFileNameForStream(column, substream_path); + /// Shared offsets for Nested type. if (column_streams.count(stream_name)) return; @@ -195,11 +197,6 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm } Block block_to_write = block; - for (auto & col : block_to_write) - { - if (serializations[col.name]->getKind() != ISerialization::Kind::SPARSE) - col.column = col.column->convertToFullColumnIfSparse(); - } auto granules_to_write = getGranulesToWrite(index_granularity, block_to_write.rows(), getCurrentMark(), rows_written_in_last_mark); @@ -213,7 +210,10 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm auto it = columns_list.begin(); for (size_t i = 0; i < columns_list.size(); ++i, ++it) { - const ColumnWithTypeAndName & column = block_to_write.getByName(it->name); + auto & column = block_to_write.getByName(it->name); + + if (serializations[column.name]->getKind() != ISerialization::Kind::SPARSE) + column.column = recursiveRemoveSparse(column.column); if (permutation) { diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index efa9e429259..e8962a9332c 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -84,12 +84,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si /// 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) - { - if (isSparseSerializaion(serializations[name])) - res_columns[pos] = ColumnSparse::create(type->createColumn()); - else - res_columns[pos] = type->createColumn(); - } + res_columns[pos] = type->createColumn(*serializations[name]); auto & column = res_columns[pos]; try diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.reference b/tests/queries/0_stateless/01780_column_sparse_tuple.reference new file mode 100644 index 00000000000..7291d39ab2d --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.reference @@ -0,0 +1,40 @@ +(0,'a') +(0,'aa') +(0,'aaa') +(0,'aaaa') +(0,'aaaaa') +(20,'a') +(40,'a') +(60,'a') +(80,'a') +(100,'a') +(20,'a') +(40,'a') +(60,'a') +(80,'a') +(100,'a') +0 +0 +0 +0 +0 +20 +40 +60 +80 +100 +20 +40 +60 +80 +100 +a +aa +aaa +aaaa +aaaaa +a +a +a +a +a diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.sql b/tests/queries/0_stateless/01780_column_sparse_tuple.sql new file mode 100644 index 00000000000..1ecea77cc0e --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS sparse_tuple; + +CREATE TABLE sparse_tuple (id UInt64, t Tuple(a UInt64, s String)) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.5; + +INSERT INTO sparse_tuple SELECT number, (if (number % 20 = 0, number, 0), repeat('a', number % 10 + 1)) FROM numbers(1000); + +SELECT t FROM sparse_tuple ORDER BY id LIMIT 5; +SELECT t FROM sparse_tuple WHERE t.a != 0 ORDER BY id LIMIT 5; +SELECT t FROM sparse_tuple WHERE t.a != 0 ORDER BY t.a LIMIT 5; + +SELECT t.a FROM sparse_tuple ORDER BY id LIMIT 5; +SELECT t.a FROM sparse_tuple WHERE t.a != 0 ORDER BY id LIMIT 5; +SELECT t.a FROM sparse_tuple WHERE t.a != 0 ORDER BY t.a LIMIT 5; + +SELECT t.s FROM sparse_tuple ORDER BY id LIMIT 5; +SELECT t.s FROM sparse_tuple WHERE t.a != 0 ORDER BY id LIMIT 5; + + +DROP TABLE IF EXISTS sparse_tuple; From 1df3e26d15ecd7890d4c67e5e6d23cf2acbc6d2a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Apr 2021 05:08:57 +0300 Subject: [PATCH 009/358] temporarily make all columns sparse --- src/Storages/MergeTree/MergeTreeSettings.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index c036c5518aa..2f2b5dbdc30 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -26,13 +26,13 @@ struct Settings; M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ - M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(UInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ M(UInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_for_sparse_serialization, 1.1, "", 0) \ + M(Float, ratio_for_sparse_serialization, 0, "", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ From 298251e55df0e96a4586d9a3bc312422aff4e9b8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 12 Apr 2021 02:30:04 +0300 Subject: [PATCH 010/358] fix merges with sparse columns and disable sparse for some data types --- src/Columns/ColumnArray.cpp | 32 +++++++++---------- src/Columns/ColumnArray.h | 4 +-- src/Core/Block.cpp | 10 +++++- src/DataTypes/DataTypeAggregateFunction.h | 1 + src/DataTypes/DataTypeArray.h | 2 ++ src/DataTypes/IDataType.cpp | 21 +++++++----- src/DataTypes/IDataType.h | 1 + .../Serializations/SerializationSparse.cpp | 2 +- .../Merges/Algorithms/IMergingAlgorithm.h | 1 + src/Processors/Merges/IMergingTransform.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++-- .../MergeTree/MergeTreeDataPartWide.cpp | 5 ++- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 12 ++++--- 13 files changed, 62 insertions(+), 36 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 6e10a15d920..0b22a7fa7a0 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1216,23 +1216,23 @@ void ColumnArray::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } -size_t ColumnArray::getNumberOfDefaultRows(size_t step) const -{ - const auto & offsets_data = getOffsets(); - size_t res = 0; - for (size_t i = 0; i < offsets_data.size(); i += step) - res += (offsets_data[i] != offsets_data[i - 1]); +// size_t ColumnArray::getNumberOfDefaultRows(size_t step) const +// { +// const auto & offsets_data = getOffsets(); +// size_t res = 0; +// for (size_t i = 0; i < offsets_data.size(); i += step) +// res += (offsets_data[i] != offsets_data[i - 1]); - return res; -} +// return res; +// } -void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const -{ - const auto & offsets_data = getOffsets(); - size_t to = limit && from + limit < size() ? from + limit : size(); - for (size_t i = from; i < to; ++i) - if (offsets_data[i] != offsets_data[i - 1]) - indices.push_back(i); -} +// void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const +// { +// const auto & offsets_data = getOffsets(); +// size_t to = limit && from + limit < size() ? from + limit : size(); +// for (size_t i = from; i < to; ++i) +// if (offsets_data[i] != offsets_data[i - 1]) +// indices.push_back(i); +// } } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index fab6573f159..cc686855e37 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -140,8 +140,8 @@ public: return false; } - size_t getNumberOfDefaultRows(size_t step) const override; - void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + // size_t getNumberOfDefaultRows(size_t step) const override; + // void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 0c9a470dc1d..1f8915ded8a 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -9,6 +9,7 @@ #include #include +#include #include @@ -482,6 +483,13 @@ DataTypes Block::getDataTypes() const return res; } +static String getNameOfBaseColumn(const IColumn & column) +{ + if (const auto * column_sparse = checkAndGetColumn(&column)) + return column_sparse->getValuesColumn().getName(); + + return column.getName(); +} template static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description) @@ -515,7 +523,7 @@ static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, cons if (!actual.column || !expected.column) continue; - if (actual.column->getName() != expected.column->getName()) + if (getNameOfBaseColumn(*actual.column) != getNameOfBaseColumn(*expected.column)) return on_error("Block structure mismatch in " + context_description + " stream: different columns:\n" + lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR); diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index c3fea2ba727..3afe6b935a2 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -50,6 +50,7 @@ public: bool shouldAlignRightInPrettyFormats() const override { return false; } SerializationPtr doGetDefaultSerialization() const override; + bool supportsSparseSerialization() const override { return false; } }; diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index c720a15d798..8e29bef6830 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -54,6 +54,8 @@ public: return nested->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion(); } + /// TODO: support arrays with sparse serialization. + bool supportsSparseSerialization() const override { return false; } DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; SerializationPtr getSubcolumnSerialization( diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index ea442e5ec68..f2aa20514c4 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -168,7 +168,6 @@ SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_n throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } - SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const { ISerialization::Settings settings = @@ -183,7 +182,7 @@ SerializationPtr IDataType::getSerialization(const String & column_name, const S SerializationPtr IDataType::getSerialization(const IColumn & column) const { - if (typeid_cast(&column)) + if (column.isSparse()) return getSparseSerialization(); ISerialization::Settings settings = @@ -198,9 +197,12 @@ SerializationPtr IDataType::getSerialization(const IColumn & column) const SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const { - double ratio = settings.num_rows ? std::min(static_cast(settings.num_default_rows) / settings.num_rows, 1.0) : 0.0; - if (ratio >= settings.ratio_for_sparse_serialization) - return getSparseSerialization(); + if (supportsSparseSerialization()) + { + double ratio = settings.num_rows ? std::min(static_cast(settings.num_default_rows) / settings.num_rows, 1.0) : 0.0; + if (ratio >= settings.ratio_for_sparse_serialization) + return getSparseSerialization(); + } return getDefaultSerialization(); } @@ -224,9 +226,12 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, con SerializationPtr IDataType::getSerialization(const String & column_name, const StreamExistenceCallback & callback) const { - auto sparse_idx_name = escapeForFileName(column_name) + ".sparse.idx"; - if (callback(sparse_idx_name)) - return getSparseSerialization(); + if (supportsSparseSerialization()) + { + auto sparse_idx_name = escapeForFileName(column_name) + ".sparse.idx"; + if (callback(sparse_idx_name)) + return getSparseSerialization(); + } return getDefaultSerialization(); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 2689b989268..bb59bc8b3c6 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -64,6 +64,7 @@ public: virtual ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const; Names getSubcolumnNames() const; + virtual bool supportsSparseSerialization() const { return true; } SerializationPtr getDefaultSerialization() const; SerializationPtr getSparseSerialization() const; diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 50858392c3b..af4688de817 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -193,7 +193,6 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { - settings.path.push_back(Substream::SparseOffsets); auto * state_sparse = checkAndGetDeserializeState(state, *this); auto mutable_column = column->assumeMutable(); @@ -203,6 +202,7 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( size_t old_size = offsets_data.size(); size_t read_rows = 0; + settings.path.push_back(Substream::SparseOffsets); if (auto * stream = settings.getter(settings.path)) read_rows = deserializeOffsets(offsets_data, *stream, limit, *state_sparse); diff --git a/src/Processors/Merges/Algorithms/IMergingAlgorithm.h b/src/Processors/Merges/Algorithms/IMergingAlgorithm.h index 5c8d18875e7..c56324c7d9a 100644 --- a/src/Processors/Merges/Algorithms/IMergingAlgorithm.h +++ b/src/Processors/Merges/Algorithms/IMergingAlgorithm.h @@ -39,6 +39,7 @@ public: void set(Chunk chunk_) { + convertToFullIfSparse(chunk_); chunk = std::move(chunk_); skip_last_row = false; } diff --git a/src/Processors/Merges/IMergingTransform.cpp b/src/Processors/Merges/IMergingTransform.cpp index eb1bd5110e9..eff786b150f 100644 --- a/src/Processors/Merges/IMergingTransform.cpp +++ b/src/Processors/Merges/IMergingTransform.cpp @@ -87,7 +87,6 @@ IProcessor::Status IMergingTransformBase::prepareInitializeInputs() continue; } - convertToFullIfSparse(chunk); state.init_chunks[i].set(std::move(chunk)); input_states[i].is_initialized = true; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ab590612e04..2ac85eda268 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -829,7 +829,9 @@ void IMergeTreeDataPart::loadRowsCount() for (const auto & column : getColumns()) { /// Most trivial types - if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes()) + if (column.type->isValueRepresentedByNumber() + && !column.type->haveSubtypes() + && getSerializationForColumn(column)->getKind() == ISerialization::Kind::DEFAULT) { auto size = getColumnSize(column.name, *column.type); @@ -875,7 +877,7 @@ void IMergeTreeDataPart::loadRowsCount() { for (const NameAndTypePair & column : columns) { - ColumnPtr column_col = column.type->createColumn(); + ColumnPtr column_col = column.type->createColumn(*getSerializationForColumn(column)); if (!column_col->isFixedAndContiguous() || column_col->lowCardinality()) continue; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index d5f34e379a3..b029c3c6912 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -258,7 +258,10 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col #ifndef NDEBUG /// Most trivial types - if (rows_count != 0 && column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes()) + if (rows_count != 0 + && column.type->isValueRepresentedByNumber() + && !column.type->haveSubtypes() + && getSerializationForColumn(column)->getKind() == ISerialization::Kind::DEFAULT) { size_t rows_in_column = size.data_uncompressed / column.type->getSizeOfValueInMemory(); if (rows_in_column != rows_count) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index a0d1078aea3..7e7714d0bdd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -399,7 +399,9 @@ void MergeTreeDataPartWriterWide::writeColumn( void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, const IDataType & type) { - if (!type.isValueRepresentedByNumber() || type.haveSubtypes()) + const auto & serialization = serializations[name]; + + if (!type.isValueRepresentedByNumber() || type.haveSubtypes() || serialization->getKind() != ISerialization::Kind::SPARSE) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName()); auto disk = data_part->volume->getDisk(); @@ -414,8 +416,6 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, size_t mark_num; - const auto & serialization = serializations[name]; - for (mark_num = 0; !mrk_in.eof(); ++mark_num) { if (mark_num > index_granularity.getMarksCount()) @@ -559,8 +559,12 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch /// data according to marks. Otherwise throws LOGICAL_ERROR (equal to abort in debug mode) for (const auto & column : columns_list) { - if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes()) + if (column.type->isValueRepresentedByNumber() + && !column.type->haveSubtypes() + && serializations[column.name]->getKind() == ISerialization::Kind::DEFAULT) + { validateColumnOfFixedSize(column.name, *column.type); + } } #endif From fac54e9ccc0a241c82db1f82988369661c18d17f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 13 Apr 2021 15:08:24 +0300 Subject: [PATCH 011/358] ColumnSparse: unit tests and several fixes --- src/Columns/ColumnSparse.cpp | 51 +-- src/Columns/ColumnSparse.h | 9 +- src/Columns/tests/gtest_column_sparse.cpp | 341 ++++++++++++++++++ .../Serializations/SerializationSparse.cpp | 10 +- 4 files changed, 381 insertions(+), 30 deletions(-) create mode 100644 src/Columns/tests/gtest_column_sparse.cpp diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 7c7befd52e9..29abd09ac6d 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -34,7 +34,11 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs if (offsets->size() + 1 != values->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Values size is inconsistent with offsets size. Expected: {}, got {}", offsets->size() + 1, values->size()); + "Values size ({}) is inconsistent with offsets size ({})", values->size(), offsets->size()); + + if (_size < offsets->size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Size of sparse column ({}) cannot be lower than number of non-default values ({})", _size, offsets->size()); } MutableColumnPtr ColumnSparse::cloneResized(size_t new_size) const @@ -113,7 +117,7 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len if (length == 0) return; - if (start + length >= src.size()) + if (start + length > src.size()) throw Exception("Parameter out of bound in IColumnString::insertRangeFrom method.", ErrorCodes::LOGICAL_ERROR); @@ -125,13 +129,11 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len const auto & src_offsets = src_sparse->getOffsetsData(); const auto & src_values = src_sparse->getValuesColumn(); - if (!src_offsets.empty()) - { - size_t offset_start = std::lower_bound(src_offsets.begin(), src_offsets.end(), start) - src_offsets.begin(); - size_t offset_end = std::upper_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); - if (offset_end != 0) - --offset_end; + size_t offset_start = std::lower_bound(src_offsets.begin(), src_offsets.end(), start) - src_offsets.begin(); + size_t offset_end = std::upper_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); + if (offset_start != offset_end) + { insertManyDefaults(src_offsets[offset_start] - start); offsets_data.push_back(_size); ++_size; @@ -144,8 +146,8 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len ++_size; } - insertManyDefaults(end - src_offsets[offset_end]); - values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start + 1); + insertManyDefaults(end - src_offsets[offset_end - 1] - 1); + values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start); } else { @@ -354,7 +356,7 @@ void ColumnSparse::compareColumn(const IColumn & rhs, size_t rhs_row_num, nullptr, nested_result, direction, nan_direction_hint); const auto & offsets_data = getOffsetsData(); - compare_results.resize(_size, nested_result[0]); + compare_results.resize_fill(_size, nested_result[0]); for (size_t i = 0; i < offsets_data.size(); ++i) compare_results[offsets_data[i]] = nested_result[i + 1]; } @@ -379,21 +381,23 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc return; res.resize(_size); - for (size_t i = 0; i < _size; ++i) - res[i] = i; - if (offsets->empty()) + { + for (size_t i = 0; i < _size; ++i) + res[i] = i; return; - - Permutation perm; - if (collator) - values->getPermutationWithCollation(*collator, reverse, limit, null_direction_hint, perm); - else - values->getPermutation(reverse, limit, null_direction_hint, perm); + } if (limit == 0 || limit > _size) limit = _size; + Permutation perm; + /// limit + 1 for case when there is 0 default values + if (collator) + values->getPermutationWithCollation(*collator, reverse, limit + 1, null_direction_hint, perm); + else + values->getPermutation(reverse, limit + 1, null_direction_hint, perm); + size_t num_of_defaults = getNumberOfDefaults(); size_t row = 0; size_t current_offset = 0; @@ -407,7 +411,7 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc if (!num_of_defaults) continue; - while (row < limit && current_default_row < _size) + while (row < limit) { while (current_offset < offsets_data.size() && current_default_row == offsets_data[current_offset]) { @@ -415,6 +419,9 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc ++current_default_row; } + if (current_default_row == _size) + break; + res[row++] = current_default_row++; } } @@ -550,7 +557,7 @@ void ColumnSparse::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size size_t ColumnSparse::getNumberOfDefaultRows(size_t step) const { - return (_size - offsets->size()) / step; + return getNumberOfDefaults() / step; } MutableColumns ColumnSparse::scatter(ColumnIndex num_columns, const Selector & selector) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index f0beb4cdefb..87123ff0057 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -35,7 +35,8 @@ public: return Base::create(values_->assumeMutable(), offsets_->assumeMutable(), size_); } - static MutablePtr create(MutableColumnPtr && values_, MutableColumnPtr && offsets_, size_t size_) + template ::value>::type> + static MutablePtr create(TColumnPtr && values_, TColumnPtr && offsets_, size_t size_) { return Base::create(std::move(values_), std::move(offsets_), size_); } @@ -45,10 +46,10 @@ public: return Base::create(values_->assumeMutable()); } - template >> - static MutablePtr create(Arg && arg) + template ::value>::type> + static MutablePtr create(TColumnPtr && values_) { - return Base::create(std::forward(arg)); + return Base::create(std::forward(values_)); } bool isSparse() const override { return true; } diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp new file mode 100644 index 00000000000..b0b461894d0 --- /dev/null +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -0,0 +1,341 @@ +#include +#include + +#include +#include +#include + +#include +#include + +#include + +using namespace DB; +pcg64 rng(randomSeed()); + +std::pair createColumns(size_t n, size_t k) +{ + auto values = ColumnVector::create(); + auto offsets = ColumnVector::create(); + auto full = ColumnVector::create(); + + auto & values_data = values->getData(); + auto & offsets_data = offsets->getData(); + auto & full_data = full->getData(); + + values_data.push_back(0); + + for (size_t i = 0; i < n; ++i) + { + bool not_zero = rng() % k == 0; + size_t value = not_zero ? rng() % 1000000 : 0; + full_data.push_back(value); + + if (not_zero) + { + values_data.push_back(value); + offsets_data.push_back(i); + } + } + + auto sparse = ColumnSparse::create(std::move(values), std::move(offsets), n); + return std::make_pair(std::move(sparse), std::move(full)); +} + +bool checkEquals(const IColumn & lhs, const IColumn & rhs) +{ + if (lhs.size() != rhs.size()) + return false; + + for (size_t i = 0; i < lhs.size(); ++i) + if (lhs.compareAt(i, i, rhs, 0) != 0) + return false; + + return true; +} + +constexpr size_t T = 5000; +constexpr size_t MAX_ROWS = 10000; +constexpr size_t sparse_ratios[] = {1, 2, 5, 10, 32, 50, 64, 100, 256, 500, 1000, 5000, 10000}; +constexpr size_t K = sizeof(sparse_ratios) / sizeof(sparse_ratios[0]); + +#define DUMP_COLUMN(column) std::cerr << #column << ": " << (column)->dumpStructure() << "\n" +#define DUMP_NON_DEFAULTS(column) std::cerr << "non-default values in " #column ": " << (column)->size() - (column)->getNumberOfDefaultRows(1) << "\n" + +TEST(ColumnSparse, InsertRangeFrom) +{ + auto test_case = [&](size_t n1, size_t k1, size_t n2, size_t k2, size_t from, size_t len) + { + auto [sparse_dst, full_dst] = createColumns(n1, k1); + auto [sparse_src, full_src] = createColumns(n2, k2); + + sparse_dst->insertRangeFrom(*sparse_src, from, len); + full_dst->insertRangeFrom(*full_src, from, len); + + if (!checkEquals(*sparse_dst->convertToFullColumnIfSparse(), *full_dst)) + { + DUMP_COLUMN(sparse_src); + DUMP_COLUMN(full_src); + DUMP_COLUMN(sparse_dst); + DUMP_COLUMN(full_dst); + DUMP_NON_DEFAULTS(full_dst); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + } + }; + + try + { + for (size_t i = 0; i < T; ++i) + { + size_t n1 = rng() % MAX_ROWS + 1; + size_t k1 = sparse_ratios[rng() % K]; + + size_t n2 = rng() % MAX_ROWS + 1; + size_t k2 = sparse_ratios[rng() % K]; + + size_t from = rng() % n2; + size_t to = rng() % n2; + + if (from > to) + std::swap(from, to); + + test_case(n1, k1, n2, k2, from, to - from); + } + } + catch(const Exception & e) + { + FAIL() << e.displayText(); + } +} + +TEST(ColumnSparse, PopBack) +{ + auto test_case = [&](size_t n, size_t k, size_t m) + { + auto [sparse_dst, full_dst] = createColumns(n, k); + + sparse_dst->popBack(m); + full_dst->popBack(m); + + if (!checkEquals(*sparse_dst->convertToFullColumnIfSparse(), *full_dst)) + { + DUMP_COLUMN(sparse_dst); + DUMP_COLUMN(full_dst); + DUMP_NON_DEFAULTS(full_dst); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + } + }; + + try + { + for (size_t i = 0; i < T; ++i) + { + size_t n = rng() % MAX_ROWS + 1; + size_t k = sparse_ratios[rng() % K]; + size_t m = rng() % n; + + test_case(n, k, m); + } + } + catch(const Exception & e) + { + FAIL() << e.displayText(); + } +} + +TEST(ColumnSparse, Filter) +{ + auto test_case = [&](size_t n, size_t k, size_t m) + { + auto [sparse_src, full_src] = createColumns(n, k); + + PaddedPODArray filt(n); + for (size_t i = 0; i < n; ++i) + filt[i] = rng() % m == 0; + + auto sparse_dst = sparse_src->filter(filt, -1); + auto full_dst = full_src->filter(filt, -1); + + if (!checkEquals(*sparse_dst->convertToFullColumnIfSparse(), *full_dst)) + { + DUMP_COLUMN(sparse_src); + DUMP_COLUMN(full_src); + DUMP_COLUMN(sparse_dst); + DUMP_COLUMN(full_dst); + DUMP_NON_DEFAULTS(full_dst); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + } + }; + + try + { + for (size_t i = 0; i < T; ++i) + { + size_t n = rng() % MAX_ROWS + 1; + size_t k = sparse_ratios[rng() % K]; + size_t m = sparse_ratios[rng() % K]; + + test_case(n, k, m); + } + } + catch(const Exception & e) + { + FAIL() << e.displayText(); + } +} + +TEST(ColumnSparse, Permute) +{ + auto test_case = [&](size_t n, size_t k, size_t limit) + { + auto [sparse_src, full_src] = createColumns(n, k); + + PaddedPODArray perm(n); + std::iota(perm.begin(), perm.end(), 0); + std::shuffle(perm.begin(), perm.end(), rng); + + auto sparse_dst = sparse_src->permute(perm, limit); + auto full_dst = full_src->permute(perm, limit); + + if (limit) + { + sparse_dst = sparse_dst->cut(0, limit); + full_dst = full_dst->cut(0, limit); + } + + if (!checkEquals(*sparse_dst->convertToFullColumnIfSparse(), *full_dst)) + { + DUMP_COLUMN(sparse_src); + DUMP_COLUMN(full_src); + DUMP_COLUMN(sparse_dst); + DUMP_COLUMN(full_dst); + DUMP_NON_DEFAULTS(full_dst); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + } + }; + + try + { + for (size_t i = 0; i < T; ++i) + { + size_t n = rng() % MAX_ROWS + 1; + size_t k = sparse_ratios[rng() % K]; + size_t limit = rng() % 2 ? 0 : rng() % n; + + test_case(n, k, limit); + } + } + catch(const Exception & e) + { + FAIL() << e.displayText(); + } +} + +TEST(ColumnSparse, CompareColumn) +{ + auto test_case = [&](size_t n1, size_t k1, size_t n2, size_t k2, size_t row_num) + { + auto [sparse_src1, full_src1] = createColumns(n1, k1); + auto [sparse_src2, full_src2] = createColumns(n2, k2); + + PaddedPODArray comp_sparse; + PaddedPODArray comp_full; + + sparse_src1->compareColumn(*sparse_src2, row_num, nullptr, comp_sparse, 1, 1); + full_src1->compareColumn(*full_src2, row_num, nullptr, comp_full, 1, 1); + + if (comp_sparse != comp_full) + { + DUMP_COLUMN(sparse_src1); + DUMP_COLUMN(full_src1); + DUMP_COLUMN(sparse_src2); + DUMP_COLUMN(full_src2); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Compare results are unequal"); + } + }; + + try + { + for (size_t i = 0; i < T; ++i) + { + size_t n1 = rng() % MAX_ROWS + 1; + size_t k1 = sparse_ratios[rng() % K]; + + size_t n2 = rng() % MAX_ROWS + 1; + size_t k2 = sparse_ratios[rng() % K]; + + size_t row_num = rng() % n2; + + test_case(n1, k1, n2, k2, row_num); + } + } + catch(const Exception & e) + { + FAIL() << e.displayText(); + } +} + +TEST(ColumnSparse, GetPermutation) +{ + auto test_case = [&](size_t n, size_t k, size_t limit, bool reverse) + { + auto [sparse_src, full_src] = createColumns(n, k); + + PaddedPODArray perm_sparse; + PaddedPODArray perm_full; + + sparse_src->getPermutation(reverse, limit, 1, perm_sparse); + full_src->getPermutation(reverse, limit, 1, perm_full); + + auto sparse_sorted = sparse_src->permute(perm_sparse, limit); + auto full_sorted = full_src->permute(perm_full, limit); + + if (limit) + { + sparse_sorted = sparse_sorted->cut(0, limit); + full_sorted = full_sorted->cut(0, limit); + } + + if (!checkEquals(*sparse_sorted->convertToFullColumnIfSparse(), *full_sorted)) + { + DUMP_COLUMN(sparse_src); + DUMP_COLUMN(full_src); + DUMP_COLUMN(sparse_sorted); + DUMP_COLUMN(full_sorted); + DUMP_NON_DEFAULTS(full_sorted); + + std::cerr << "sparse: "; + for (size_t i = 0; i < sparse_sorted->size(); ++i) + std::cerr << applyVisitor(FieldVisitorToString(), (*sparse_sorted)[i]) << " "; + std::cerr << "\n"; + + std::cerr << "full: "; + for (size_t i = 0; i < full_sorted->size(); ++i) + std::cerr << applyVisitor(FieldVisitorToString(), (*full_sorted)[i]) << " "; + std::cerr << "\n"; + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Sorted columns are unequal"); + } + }; + + try + { + for (size_t i = 0; i < T; ++i) + { + size_t n = rng() % MAX_ROWS + 1; + size_t k = sparse_ratios[rng() % K]; + + size_t limit = rng() % 2 ? 0 : rng() % n; + bool reverse = rng() % 2; + + test_case(n, k, limit, reverse); + } + } + catch(const Exception & e) + { + FAIL() << e.displayText(); + } +} + +#undef DUMP_COLUMN +#undef DUMP_NON_DEFAULTS diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index af4688de817..eb4bcb0f1c2 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -7,6 +7,8 @@ #include #include +#include + namespace DB { @@ -38,7 +40,7 @@ void serializeOffsets(const IColumn::Offsets & offsets, WriteBuffer & ostr, size } size_t deserializeOffsets(IColumn::Offsets & offsets, - ReadBuffer & istr, size_t limit, DeserializeStateSparse & state) + ReadBuffer & istr, size_t start, size_t limit, DeserializeStateSparse & state) { if (limit && state.num_trailing_defaults >= limit) { @@ -52,7 +54,7 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, size_t total_rows = state.num_trailing_defaults; if (state.has_value_after_defaults) { - size_t start_of_group = offsets.empty() ? 0 : offsets.back() + 1; + size_t start_of_group = offsets.empty() ? start : offsets.back() + 1; offsets.push_back(start_of_group + state.num_trailing_defaults); state.has_value_after_defaults = false; @@ -85,7 +87,7 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } else { - size_t start_of_group = offsets.empty() ? 0 : offsets.back() + 1; + size_t start_of_group = offsets.empty() ? start : offsets.back() + 1; offsets.push_back(start_of_group + group_size); state.num_trailing_defaults = 0; @@ -204,7 +206,7 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( size_t read_rows = 0; settings.path.push_back(Substream::SparseOffsets); if (auto * stream = settings.getter(settings.path)) - read_rows = deserializeOffsets(offsets_data, *stream, limit, *state_sparse); + read_rows = deserializeOffsets(offsets_data, *stream, column_sparse.size(), limit, *state_sparse); auto & values_column = column_sparse.getValuesPtr(); size_t values_limit = offsets_data.size() - old_size; From 236ab68724c4b4388374e82097df9f4095212458 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 14 Apr 2021 16:20:05 +0300 Subject: [PATCH 012/358] ColumnSparse: several fixes --- src/Columns/ColumnSparse.cpp | 92 ++++++++++++------- src/Columns/ColumnSparse.h | 39 ++++++++ src/Columns/tests/gtest_column_sparse.cpp | 11 --- src/DataTypes/DataTypeTuple.cpp | 26 ++++-- .../Serializations/SerializationSparse.cpp | 22 ++++- .../Serializations/SerializationWrapper.h | 2 + src/Functions/IFunction.cpp | 7 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 2 - .../System/StorageSystemPartsColumns.cpp | 39 +++++++- .../01780_column_sparse_pk.reference | 9 ++ .../0_stateless/01780_column_sparse_pk.sql | 38 ++++++++ .../01780_column_sparse_tuple.reference | 19 ++++ .../0_stateless/01780_column_sparse_tuple.sql | 22 +++++ 13 files changed, 266 insertions(+), 62 deletions(-) create mode 100644 tests/queries/0_stateless/01780_column_sparse_pk.reference create mode 100644 tests/queries/0_stateless/01780_column_sparse_pk.sql diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 29abd09ac6d..109cbaea5de 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -66,7 +67,7 @@ Field ColumnSparse::operator[](size_t n) const void ColumnSparse::get(size_t n, Field & res) const { - values->get(n, res); + values->get(getValueIndex(n), res); } bool ColumnSparse::getBool(size_t n) const @@ -130,7 +131,7 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len const auto & src_values = src_sparse->getValuesColumn(); size_t offset_start = std::lower_bound(src_offsets.begin(), src_offsets.end(), start) - src_offsets.begin(); - size_t offset_end = std::upper_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); + size_t offset_end = std::lower_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); if (offset_start != offset_end) { @@ -235,8 +236,6 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const return res; } - const auto & offsets_data = getOffsetsData(); - auto res_offsets = offsets->cloneEmpty(); auto & res_offsets_data = assert_cast(*res_offsets).getData(); @@ -245,12 +244,11 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const values_filter.push_back(1); size_t values_result_size_hint = 1; - size_t offset_pos = 0; size_t res_offset = 0; - - for (size_t i = 0; i < _size; ++i) + auto offset_it = begin(); + for (size_t i = 0; i < _size; ++i, ++offset_it) { - if (offset_pos < offsets_data.size() && i == offsets_data[offset_pos]) + if (!offset_it.isDefault()) { if (filt[i]) { @@ -263,8 +261,6 @@ ColumnPtr ColumnSparse::filter(const Filter & filt, ssize_t) const { values_filter.push_back(0); } - - ++offset_pos; } else { @@ -400,9 +396,9 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc size_t num_of_defaults = getNumberOfDefaults(); size_t row = 0; - size_t current_offset = 0; - size_t current_default_row = 0; + const auto & offsets_data = getOffsetsData(); + auto offset_it = begin(); for (size_t i = 0; i < perm.size() && row < limit; ++i) { @@ -413,22 +409,19 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc while (row < limit) { - while (current_offset < offsets_data.size() && current_default_row == offsets_data[current_offset]) - { - ++current_offset; - ++current_default_row; - } + while (offset_it != end() && !offset_it.isDefault()) + ++offset_it; - if (current_default_row == _size) + if (offset_it.getCurrentRow() == _size) break; - res[row++] = current_default_row++; + res[row++] = offset_it.getCurrentRow(); + ++offset_it; } } else { - res[row] = offsets_data[perm[i] - 1]; - ++row; + res[row++] = offsets_data[perm[i] - 1]; } } @@ -497,26 +490,23 @@ ColumnPtr ColumnSparse::replicate(const Offsets & replicate_offsets) const if (_size == 0) return ColumnSparse::create(values->cloneEmpty()); - const auto & offsets_data = getOffsetsData(); auto res_offsets = offsets->cloneEmpty(); auto & res_offsets_data = assert_cast(*res_offsets).getData(); auto res_values = values->cloneEmpty(); res_values->insertDefault(); - size_t current_offset = 0; - for (size_t i = 0; i < _size; ++i) + auto offset_it = begin(); + for (size_t i = 0; i < _size; ++i, ++offset_it) { - if (current_offset < offsets_data.size() && i == offsets_data[current_offset]) + if (!offset_it.isDefault()) { size_t replicate_size = replicate_offsets[i] - replicate_offsets[i - 1]; res_offsets_data.reserve(res_offsets_data.size() + replicate_size); for (size_t row = replicate_offsets[i - 1]; row < replicate_offsets[i]; ++row) { res_offsets_data.push_back(row); - res_values->insertFrom(*values, current_offset + 1); + res_values->insertFrom(*values, offset_it.getValueIndex()); } - - ++current_offset; } } @@ -530,8 +520,21 @@ void ColumnSparse::updateHashWithValue(size_t n, SipHash & hash) const void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const { - UNUSED(hash); - throwMustBeDense(); + if (hash.getData().size() != _size) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of WeakHash32 does not match size of column: " + "column size is {}, hash size is {}", _size, hash.getData().size()); + + auto offset_it = begin(); + auto & hash_data = hash.getData(); + for (size_t i = 0; i < _size; ++i, ++offset_it) + { + size_t value_index = 0; + if (!offset_it.isDefault()) + value_index = offset_it.getValueIndex(); + + auto data_ref = values->getDataAt(value_index); + hash_data[i] = ::updateWeakHash32(reinterpret_cast(data_ref.data), data_ref.size, hash_data[i]); + } } void ColumnSparse::updateHashFast(SipHash & hash) const @@ -543,6 +546,31 @@ void ColumnSparse::updateHashFast(SipHash & hash) const void ColumnSparse::getExtremes(Field & min, Field & max) const { + if (_size == 0) + { + values->get(0, min); + values->get(0, max); + return; + } + + if (getNumberOfDefaults() == 0) + { + size_t min_idx = 1; + size_t max_idx = 1; + + for (size_t i = 2; i < values->size(); ++i) + { + if (values->compareAt(i, min_idx, *values, 1) < 0) + min_idx = i; + else if (values->compareAt(i, max_idx, *values, 1) > 0) + max_idx = i; + } + + values->get(min_idx, min); + values->get(max_idx, max); + return; + } + values->getExtremes(min, max); } @@ -562,9 +590,7 @@ size_t ColumnSparse::getNumberOfDefaultRows(size_t step) const MutableColumns ColumnSparse::scatter(ColumnIndex num_columns, const Selector & selector) const { - UNUSED(num_columns); - UNUSED(selector); - throwMustBeDense(); + return scatterImpl(num_columns, selector); } void ColumnSparse::gather(ColumnGathererStream & gatherer_stream) diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 87123ff0057..3bd0741df4c 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -162,6 +162,45 @@ private: throw Exception("Not implemented for ColumnSparse", ErrorCodes::LOGICAL_ERROR); } + class Iterator + { + public: + Iterator(const PaddedPODArray & offsets_, size_t size_, size_t current_offset_, size_t current_row_) + : offsets(offsets_), size(size_), current_offset(current_offset_), current_row(current_row_) + { + } + + bool isDefault() const { return current_offset == offsets.size() || current_row != offsets[current_offset]; } + size_t getValueIndex() const { return isDefault() ? 0 : current_offset + 1; } + size_t getCurrentRow() const { return current_row; } + + bool operator==(const Iterator & other) const + { + return size == other.size + && current_offset == other.current_offset + && current_row == other.current_row; + } + + bool operator!=(const Iterator & other) const { return !(*this == other); } + + Iterator operator++() + { + if (!isDefault()) + ++current_offset; + ++current_row; + return *this; + } + + private: + const PaddedPODArray & offsets; + const size_t size; + size_t current_offset; + size_t current_row; + }; + + Iterator begin() const { return Iterator(getOffsetsData(), _size, 0, 0); } + Iterator end() const { return Iterator(getOffsetsData(), _size, getOffsetsData().size(), _size); } + WrappedPtr values; WrappedPtr offsets; size_t _size; diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index b0b461894d0..e6078417a94 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -303,17 +303,6 @@ TEST(ColumnSparse, GetPermutation) DUMP_COLUMN(sparse_sorted); DUMP_COLUMN(full_sorted); DUMP_NON_DEFAULTS(full_sorted); - - std::cerr << "sparse: "; - for (size_t i = 0; i < sparse_sorted->size(); ++i) - std::cerr << applyVisitor(FieldVisitorToString(), (*sparse_sorted)[i]) << " "; - std::cerr << "\n"; - - std::cerr << "full: "; - for (size_t i = 0; i < full_sorted->size(); ++i) - std::cerr << applyVisitor(FieldVisitorToString(), (*full_sorted)[i]) << " "; - std::cerr << "\n"; - throw Exception(ErrorCodes::LOGICAL_ERROR, "Sorted columns are unequal"); } }; diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 0fbb98ff933..c8006a65a79 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -172,7 +172,7 @@ MutableColumnPtr DataTypeTuple::createColumn(const ISerialization & serializatio assert(element_serializations.size() == size); MutableColumns tuple_columns(size); for (size_t i = 0; i < size; ++i) - tuple_columns[i] = elems[i]->createColumn(*element_serializations[i]); + tuple_columns[i] = elems[i]->createColumn(*element_serializations[i]->getNested()); return ColumnTuple::create(std::move(tuple_columns)); } @@ -350,16 +350,24 @@ SerializationPtr DataTypeTuple::getSerialization(const String & column_name, con for (size_t i = 0; i < elems.size(); ++i) { auto subcolumn_name = Nested::concatenateName(column_name, names[i]); - - ISerialization::Settings settings = + SerializationPtr serialization; + if (const auto * type_tuple = typeid_cast(elems[i].get())) { - .num_rows = info.getNumberOfRows(), - .num_default_rows = info.getNumberOfDefaultRows(subcolumn_name), - .ratio_for_sparse_serialization = info.getRatioForSparseSerialization() - }; + serialization = type_tuple->getSerialization(subcolumn_name, info); + } + else + { + ISerialization::Settings settings = + { + .num_rows = info.getNumberOfRows(), + .num_default_rows = info.getNumberOfDefaultRows(subcolumn_name), + .ratio_for_sparse_serialization = info.getRatioForSparseSerialization() + }; - auto serializaion = elems[i]->getSerialization(settings); - serializations[i] = std::make_shared(serializaion, names[i]); + serialization = elems[i]->getSerialization(settings); + } + + serializations[i] = std::make_shared(serialization, names[i]); } return std::make_shared(std::move(serializations), have_explicit_names); diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index eb4bcb0f1c2..a160d23c4a1 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -7,8 +7,6 @@ #include #include -#include - namespace DB { @@ -22,6 +20,12 @@ struct DeserializeStateSparse : public ISerialization::DeserializeBinaryBulkStat size_t num_trailing_defaults = 0; bool has_value_after_defaults = false; ISerialization::DeserializeBinaryBulkStatePtr nested; + + void reset() + { + num_trailing_defaults = 0; + has_value_after_defaults = false; + } }; void serializeOffsets(const IColumn::Offsets & offsets, WriteBuffer & ostr, size_t start, size_t end) @@ -54,8 +58,7 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, size_t total_rows = state.num_trailing_defaults; if (state.has_value_after_defaults) { - size_t start_of_group = offsets.empty() ? start : offsets.back() + 1; - offsets.push_back(start_of_group + state.num_trailing_defaults); + offsets.push_back(start + state.num_trailing_defaults); state.has_value_after_defaults = false; state.num_trailing_defaults = 0; @@ -63,6 +66,7 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } size_t group_size; + bool first = true; while (!istr.eof()) { readIntBinary(group_size, istr); @@ -87,7 +91,12 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } else { - size_t start_of_group = offsets.empty() ? start : offsets.back() + 1; + size_t start_of_group = start; + if (!first && !offsets.empty()) + start_of_group = offsets.back() + 1; + if (first) + first = false; + offsets.push_back(start_of_group + group_size); state.num_trailing_defaults = 0; @@ -197,6 +206,9 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( { auto * state_sparse = checkAndGetDeserializeState(state, *this); + if (!settings.continuous_reading) + state_sparse->reset(); + auto mutable_column = column->assumeMutable(); auto & column_sparse = assert_cast(*mutable_column); auto & offsets_data = column_sparse.getOffsetsData(); diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 6a887884293..12363ac72af 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -14,6 +14,8 @@ protected: public: SerializationWrapper(const SerializationPtr & nested_serialization_) : nested_serialization(nested_serialization_) {} + const SerializationPtr & getNested() const { return nested_serialization; } + Kind getKind() const override { return nested_serialization->getKind(); } void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 8b1a7ebd641..07df083f290 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -21,6 +21,8 @@ #include #include +#include + #if !defined(ARCADIA_BUILD) # include #endif @@ -469,7 +471,10 @@ ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & argu for (size_t i = 0; i < arguments.size(); ++i) { - if (typeid_cast(arguments[i].column.get())) + const auto * column_sparse = checkAndGetColumn(arguments[i].column.get()); + /// In rare case, when sparse column doesn't have default values, + /// it's more convinient to convert it to full before execution of function. + if (column_sparse && column_sparse->getNumberOfDefaults()) { sparse_column_position = i; ++num_sparse_columns; diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index d962fa74ce5..3e853de7c5e 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -291,8 +291,6 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) writeCString(grid_symbols.bar, out); - std::cerr << "current row: " << toString((*columns[0])[i]) << "\n"; - for (size_t j = 0; j < num_columns; ++j) { if (j != 0) diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 05999e3dda4..e1cb77327f3 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -57,7 +59,10 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_ {"column_data_compressed_bytes", std::make_shared()}, {"column_data_uncompressed_bytes", std::make_shared()}, {"column_marks_bytes", std::make_shared()}, - {"serialization_kind", std::make_shared()} + {"serialization_kind", std::make_shared()}, + {"subcolumns.names", std::make_shared(std::make_shared())}, + {"subcolumns.types", std::make_shared(std::make_shared())}, + {"subcolumns.serializations", std::make_shared(std::make_shared())} } ) { @@ -219,6 +224,38 @@ void StorageSystemPartsColumns::processNextStorage( columns[res_index++]->insert(ISerialization::kindToString(kind)); } + auto subcolumns = column.type->getSubcolumnNames(); + if (columns_mask[src_index++]) + { + Array array(subcolumns.begin(), subcolumns.end()); + columns[res_index++]->insert(array); + } + + if (columns_mask[src_index++]) + { + Array array; + array.reserve(subcolumns.size()); + for (const auto & name : subcolumns) + array.push_back(column.type->getSubcolumnType(name)->getName()); + + columns[res_index++]->insert(array); + } + + if (columns_mask[src_index++]) + { + Array array; + array.reserve(subcolumns.size()); + for (const auto & name : subcolumns) + { + auto subtype = column.type->getSubcolumnType(name); + auto subcolumn = NameAndTypePair(column.name, name, column.type, subtype); + auto kind = part->getSerializationForColumn(subcolumn)->getKind(); + array.push_back(ISerialization::kindToString(kind)); + } + + columns[res_index++]->insert(array); + } + if (has_state_column) columns[res_index++]->insert(part->stateString()); } diff --git a/tests/queries/0_stateless/01780_column_sparse_pk.reference b/tests/queries/0_stateless/01780_column_sparse_pk.reference new file mode 100644 index 00000000000..56131dcfaed --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_pk.reference @@ -0,0 +1,9 @@ +2 +2 e +0 a +0 b +3 f +200 84 +200 84 +800 167 +800 167 diff --git a/tests/queries/0_stateless/01780_column_sparse_pk.sql b/tests/queries/0_stateless/01780_column_sparse_pk.sql new file mode 100644 index 00000000000..adb132232b8 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_pk.sql @@ -0,0 +1,38 @@ +DROP TABLE IF EXISTS t_sparse_pk; +DROP TABLE IF EXISTS t_full_pk; + +CREATE TABLE t_sparse_pk (k UInt64, s String) +ENGINE = MergeTree ORDER BY k +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.0, index_granularity = 1; + +INSERT INTO t_sparse_pk VALUES (0, 'a'), (0, 'b'), (1, ''), (2, ''), (2, 'e'), (3, 'f'), (4, 'g'); + +SET force_primary_key = 1; + +SELECT k, s FROM t_sparse_pk WHERE k = 2 ORDER BY k, s; +SELECT k, s FROM t_sparse_pk WHERE k = 0 OR k = 3 ORDER BY k, s; + +DROP TABLE IF EXISTS t_sparse_pk; + +CREATE TABLE t_sparse_pk (k UInt64, v UInt64 CODEC(NONE)) +ENGINE = MergeTree ORDER BY k +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.0, index_granularity = 30; + +CREATE TABLE t_full_pk (k UInt64, v UInt64) +ENGINE = MergeTree ORDER BY k +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 1.1, index_granularity = 30; + +INSERT INTO t_sparse_pk SELECT number % 10, number % 4 = 0 FROM numbers(1000); +INSERT INTO t_full_pk SELECT number % 10, number % 4 = 0 FROM numbers(1000); + +INSERT INTO t_sparse_pk SELECT number % 10, number % 6 = 0 FROM numbers(1000); +INSERT INTO t_full_pk SELECT number % 10, number % 6 = 0 FROM numbers(1000); + + +set max_threads = 1; + +SELECT count(v), sum(v) FROM t_sparse_pk WHERE k = 0; +SELECT count(v), sum(v) FROM t_full_pk WHERE k = 0; + +SELECT count(v), sum(v) FROM t_sparse_pk WHERE k = 0 OR k = 3 OR k = 7 OR k = 8; +SELECT count(v), sum(v) FROM t_full_pk WHERE k = 0 OR k = 3 OR k = 7 OR k = 8; \ No newline at end of file diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.reference b/tests/queries/0_stateless/01780_column_sparse_tuple.reference index 7291d39ab2d..c19d48be991 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.reference +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.reference @@ -1,3 +1,5 @@ +id [] [] [] +t ['s','a'] ['String','UInt64'] ['Default','Sparse'] (0,'a') (0,'aa') (0,'aaa') @@ -38,3 +40,20 @@ a a a a +id [] [] [] +t ['b.u','b.s','b','a'] ['UInt32','String','Tuple(u UInt32, s String)','UInt64'] ['Sparse','Default','Default','Sparse'] +0 +0 +0 +60 +0 +a +aa +aaa +aaaa +aaaaa +aaaaaa +a +aaaaaa +a +aaaaaa diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.sql b/tests/queries/0_stateless/01780_column_sparse_tuple.sql index 1ecea77cc0e..ffb9430a0a4 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.sql +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.sql @@ -6,6 +6,11 @@ SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.5; INSERT INTO sparse_tuple SELECT number, (if (number % 20 = 0, number, 0), repeat('a', number % 10 + 1)) FROM numbers(1000); +SELECT column, subcolumns.names, subcolumns.types, subcolumns.serializations +FROM system.parts_columns +WHERE table = 'sparse_tuple' AND database = currentDatabase() +ORDER BY column; + SELECT t FROM sparse_tuple ORDER BY id LIMIT 5; SELECT t FROM sparse_tuple WHERE t.a != 0 ORDER BY id LIMIT 5; SELECT t FROM sparse_tuple WHERE t.a != 0 ORDER BY t.a LIMIT 5; @@ -17,5 +22,22 @@ SELECT t.a FROM sparse_tuple WHERE t.a != 0 ORDER BY t.a LIMIT 5; SELECT t.s FROM sparse_tuple ORDER BY id LIMIT 5; SELECT t.s FROM sparse_tuple WHERE t.a != 0 ORDER BY id LIMIT 5; +DROP TABLE IF EXISTS sparse_tuple; + +CREATE TABLE sparse_tuple (id UInt64, t Tuple(a UInt64, b Tuple(u UInt32, s String))) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.5; + +INSERT INTO sparse_tuple SELECT number, (if (number % 20 = 0, number, 0), (if (number % 15 = 0, number, 0), repeat('a', number % 10 + 1))) FROM numbers(1000); + +SELECT column, subcolumns.names, subcolumns.types, subcolumns.serializations +FROM system.parts_columns +WHERE table = 'sparse_tuple' AND database = currentDatabase() +ORDER BY column; + +SELECT t.a FROM sparse_tuple WHERE t.b.u != 0 ORDER BY id LIMIT 5; + +SELECT t.b.s FROM sparse_tuple ORDER BY id LIMIT 5; +SELECT t.b.s FROM sparse_tuple WHERE t.b.u != 0 ORDER BY id LIMIT 5; DROP TABLE IF EXISTS sparse_tuple; From 860ed203688d964bf81ecf9add60eb9926bdc510 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 15 Apr 2021 05:16:09 +0300 Subject: [PATCH 013/358] ColumnSparse: more fixes --- src/Columns/ColumnNullable.cpp | 23 +++++++++++++++++++++-- src/Columns/ColumnNullable.h | 2 ++ src/Columns/ColumnString.h | 6 ++++++ src/Columns/IColumn.h | 5 +++++ src/DataStreams/materializeBlock.cpp | 6 +++--- src/DataTypes/DataTypeLowCardinality.h | 1 + src/Functions/IFunction.cpp | 7 ++++--- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/MergeJoin.cpp | 8 ++++---- src/Interpreters/Set.cpp | 4 +++- src/Interpreters/join_common.cpp | 20 +++++++++++++------- src/Interpreters/join_common.h | 4 ++-- 12 files changed, 65 insertions(+), 23 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index d7cc7f8a36a..1f4f6fcaa78 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -688,12 +688,31 @@ void ColumnNullable::checkConsistency() const size_t ColumnNullable::getNumberOfDefaultRows(size_t step) const { - return null_map->getNumberOfDefaultRows(step); + size_t res = 0; + const auto & null_map_data = getNullMapData(); + for (size_t i = 0; i < null_map_data.size(); i += step) + res += (null_map_data != 0); + + return res; } void ColumnNullable::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const { - return null_map->getIndicesOfNonDefaultValues(indices, from, limit); + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + + const auto & null_map_data = getNullMapData(); + for (size_t i = from; i < to; ++i) + if (null_map_data[i] == 0) + indices.push_back(i); +} + +ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const +{ + auto new_values = nested_column->createWithOffsets(offsets, total_rows); + auto new_null_map = null_map->createWithOffsets(offsets, total_rows); + + return ColumnNullable::create(new_values, new_null_map); } ColumnPtr makeNullable(const ColumnPtr & column) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 6f6f678b76b..0a8ed4c0458 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -54,6 +54,7 @@ public: void get(size_t n, Field & res) const override; bool getBool(size_t n) const override { return isNullAt(n) ? false : nested_column->getBool(n); } UInt64 get64(size_t n) const override { return nested_column->get64(n); } + bool isDefaultAt(size_t n) const override { return isNullAt(n); } /** * If isNullAt(n) returns false, returns the nested column's getDataAt(n), otherwise returns a special value @@ -173,6 +174,7 @@ public: size_t getNumberOfDefaultRows(size_t step) const override; void getIndicesOfNonDefaultValues(Offsets & offsets, size_t from, size_t limit) const override; + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const override; private: WrappedPtr nested_column; diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 14edde5d63b..0890a5bde05 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -110,6 +110,12 @@ public: return StringRef(&chars[offsetAt(n)], sizeAt(n)); } + bool isDefaultAt(size_t n) const override + { + assert(n < size()); + return offsetAt(n) == 0; + } + /// Suppress gcc 7.3.1 warning: '*((void*)& +8)' may be used uninitialized in this function #if !__clang__ #pragma GCC diagnostic push diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index d3162794b1f..9c7d11cf270 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -69,6 +69,11 @@ public: virtual Ptr convertToFullColumnIfSparse() const { return getPtr(); } + Ptr convertToFullIfNeeded() const + { + return convertToFullColumnIfSparse()->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality(); + } + /// Creates empty column with the same type. virtual MutablePtr cloneEmpty() const { return cloneResized(0); } diff --git a/src/DataStreams/materializeBlock.cpp b/src/DataStreams/materializeBlock.cpp index 6b47cb87baa..7c8756be1a1 100644 --- a/src/DataStreams/materializeBlock.cpp +++ b/src/DataStreams/materializeBlock.cpp @@ -1,5 +1,5 @@ #include - +#include namespace DB { @@ -14,7 +14,7 @@ Block materializeBlock(const Block & block) for (size_t i = 0; i < columns; ++i) { auto & element = res.getByPosition(i); - element.column = element.column->convertToFullColumnIfConst(); + element.column = recursiveRemoveSparse(element.column->convertToFullColumnIfConst()); } return res; @@ -23,7 +23,7 @@ Block materializeBlock(const Block & block) void materializeBlockInplace(Block & block) { for (size_t i = 0; i < block.columns(); ++i) - block.getByPosition(i).column = block.getByPosition(i).column->convertToFullColumnIfConst(); + block.getByPosition(i).column = recursiveRemoveSparse(block.getByPosition(i).column->convertToFullColumnIfConst()); } } diff --git a/src/DataTypes/DataTypeLowCardinality.h b/src/DataTypes/DataTypeLowCardinality.h index 1266174c6d6..cfc14cadfb0 100644 --- a/src/DataTypes/DataTypeLowCardinality.h +++ b/src/DataTypes/DataTypeLowCardinality.h @@ -51,6 +51,7 @@ public: bool isNullable() const override { return false; } bool onlyNull() const override { return false; } bool lowCardinality() const override { return true; } + bool supportsSparseSerialization() const override { return false; } static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type); static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys); diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 07df083f290..306f9e1de94 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -21,8 +21,6 @@ #include #include -#include - #if !defined(ARCADIA_BUILD) # include #endif @@ -508,8 +506,11 @@ ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & argu auto res = executeWithoutSparseColumns(columns_without_sparse, result_type, values_size, dry_run); + if (isColumnConst(*res)) + return res->cloneResized(input_rows_count); + /// If default of sparse column was changed after execution of function, convert to full column. - if (res->compareAt(0, 0, *arg_with_sparse.column, 0) != 0) + if (!res->isDefaultAt(0)) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); return res->createWithOffsets(offsets_data, input_rows_count); diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index fcd89aed84d..98d3dab8dc7 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -193,7 +193,7 @@ HashJoin::HashJoin(std::shared_ptr table_join_, const Block & right_s table_join->splitAdditionalColumns(right_sample_block, right_table_keys, sample_block_with_columns_to_add); required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources); - JoinCommon::removeLowCardinalityInplace(right_table_keys); + JoinCommon::convertToFullColumnsInplace(right_table_keys); initRightBlockStructure(data->sample_block); ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right); diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index a9f50cdda0e..8065c86a2d1 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -465,8 +465,8 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right } table_join->splitAdditionalColumns(right_sample_block, right_table_keys, right_columns_to_add); - JoinCommon::removeLowCardinalityInplace(right_table_keys); - JoinCommon::removeLowCardinalityInplace(right_sample_block, table_join->keyNamesRight()); + JoinCommon::convertToFullColumnsInplace(right_table_keys); + JoinCommon::convertToFullColumnsInplace(right_sample_block, table_join->keyNamesRight()); const NameSet required_right_keys = table_join->requiredRightKeys(); for (const auto & column : right_table_keys) @@ -593,7 +593,7 @@ bool MergeJoin::saveRightBlock(Block && block) Block MergeJoin::modifyRightBlock(const Block & src_block) const { Block block = materializeBlock(src_block); - JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesRight()); + JoinCommon::convertToFullColumnsInplace(block, table_join->keyNamesRight()); return block; } @@ -611,7 +611,7 @@ void MergeJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { JoinCommon::checkTypesOfKeys(block, table_join->keyNamesLeft(), right_table_keys, table_join->keyNamesRight()); materializeBlockInplace(block); - JoinCommon::removeLowCardinalityInplace(block, table_join->keyNamesLeft(), false); + JoinCommon::convertToFullColumnsInplace(block, table_join->keyNamesLeft(), false); sortBlock(block, left_sort_description); diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index b150eabda65..dd474ef0aa2 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -181,7 +181,9 @@ bool Set::insertFromBlock(const Block & block) /// Remember the columns we will work with for (size_t i = 0; i < keys_size; ++i) { - materialized_columns.emplace_back(block.safeGetByPosition(i).column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality()); + materialized_columns.emplace_back( + block.safeGetByPosition(i).column->convertToFullIfNeeded()); + key_columns.emplace_back(materialized_columns.back().get()); } diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index 9a9253cee75..e10b9491919 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -53,7 +54,7 @@ void convertColumnToNullable(ColumnWithTypeAndName & column, bool low_card_nulla { if (low_card_nullability && column.type->lowCardinality()) { - column.column = recursiveRemoveLowCardinality(column.column); + column.column = recursiveRemoveLowCardinality(recursiveRemoveSparse(column.column)); column.type = recursiveRemoveLowCardinality(column.type); } @@ -145,7 +146,7 @@ ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names) for (const auto & column_name : names) { auto & column = block.getByName(column_name).column; - column = recursiveRemoveLowCardinality(column->convertToFullColumnIfConst()); + column = recursiveRemoveLowCardinality(recursiveRemoveSparse(column->convertToFullColumnIfConst())); ptrs.push_back(column.get()); } @@ -160,7 +161,9 @@ Columns materializeColumns(const Block & block, const Names & names) for (const auto & column_name : names) { const auto & src_column = block.getByName(column_name).column; - materialized.emplace_back(recursiveRemoveLowCardinality(src_column->convertToFullColumnIfConst())); + materialized.emplace_back( + recursiveRemoveLowCardinality( + recursiveRemoveSparse(src_column->convertToFullColumnIfConst()))); } return materialized; @@ -177,22 +180,22 @@ ColumnRawPtrs getRawPointers(const Columns & columns) return ptrs; } -void removeLowCardinalityInplace(Block & block) +void convertToFullColumnsInplace(Block & block) { for (size_t i = 0; i < block.columns(); ++i) { auto & col = block.getByPosition(i); - col.column = recursiveRemoveLowCardinality(col.column); + col.column = recursiveRemoveLowCardinality(recursiveRemoveSparse(col.column)); col.type = recursiveRemoveLowCardinality(col.type); } } -void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type) +void convertToFullColumnsInplace(Block & block, const Names & names, bool change_type) { for (const String & column_name : names) { auto & col = block.getByName(column_name); - col.column = recursiveRemoveLowCardinality(col.column); + col.column = recursiveRemoveLowCardinality(recursiveRemoveSparse(col.column)); if (change_type) col.type = recursiveRemoveLowCardinality(col.type); } @@ -221,6 +224,9 @@ ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_nam /// We will join only keys, where all components are not NULL. if (const auto * nullable = checkAndGetColumn(*key_columns[i])) key_columns[i] = &nullable->getNestedColumn(); + + if (const auto * sparse = checkAndGetColumn(*key_columns[i])) + key_columns[i] = &sparse->getValuesColumn(); } return key_columns; diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index cec41438448..e930ae40e93 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -24,8 +24,8 @@ ColumnPtr emptyNotNullableClone(const ColumnPtr & column); Columns materializeColumns(const Block & block, const Names & names); ColumnRawPtrs materializeColumnsInplace(Block & block, const Names & names); ColumnRawPtrs getRawPointers(const Columns & columns); -void removeLowCardinalityInplace(Block & block); -void removeLowCardinalityInplace(Block & block, const Names & names, bool change_type = true); +void convertToFullColumnsInplace(Block & block); +void convertToFullColumnsInplace(Block & block, const Names & names, bool change_type = true); void restoreLowCardinalityInplace(Block & block); ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_names_right); From be6672a25e3205860d981031dad35266516eb3ff Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 15 Apr 2021 17:34:55 +0300 Subject: [PATCH 014/358] ColumnSparse: fixes --- src/Interpreters/InterpreterInsertQuery.cpp | 6 ++++++ src/Processors/Transforms/MaterializingTransform.cpp | 3 ++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 933a5bc6c77..aa7d459e3f1 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -306,6 +307,11 @@ BlockIO InterpreterInsertQuery::execute() return std::make_shared(in_header, actions); }); + res.pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr + { + return std::make_shared(in_header); + }); + res.pipeline.setSinks([&](const Block &, QueryPipeline::StreamType type) -> ProcessorPtr { if (type != QueryPipeline::StreamType::Main) diff --git a/src/Processors/Transforms/MaterializingTransform.cpp b/src/Processors/Transforms/MaterializingTransform.cpp index f13d5376ebe..aa6252e1eb9 100644 --- a/src/Processors/Transforms/MaterializingTransform.cpp +++ b/src/Processors/Transforms/MaterializingTransform.cpp @@ -1,5 +1,6 @@ #include #include +#include namespace DB { @@ -13,7 +14,7 @@ void MaterializingTransform::transform(Chunk & chunk) auto columns = chunk.detachColumns(); for (auto & col : columns) - col = col->convertToFullColumnIfConst(); + col = recursiveRemoveSparse(col->convertToFullColumnIfConst()); chunk.setColumns(std::move(columns), num_rows); } From aa617c6b3cba35a4c4c93ff835ec05fcec56fc00 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 16 Apr 2021 00:47:11 +0300 Subject: [PATCH 015/358] ColumnSparse: fix vertical merge --- src/Columns/ColumnSparse.cpp | 6 +++--- src/DataStreams/ColumnGathererStream.cpp | 10 ++++++---- src/DataStreams/ColumnGathererStream.h | 4 +++- .../Serializations/SerializationInfo.cpp | 12 ++++++++++++ src/DataTypes/Serializations/SerializationInfo.h | 1 + src/Storages/MergeTree/DataPartsExchange.cpp | 4 +++- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + .../MergeTree/IMergedBlockOutputStream.cpp | 1 + .../MergeTree/IMergedBlockOutputStream.h | 1 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 16 ++++++++++++---- .../MergeTree/MergeTreeDataPartCompact.cpp | 3 ++- .../MergeTree/MergeTreeDataPartCompact.h | 1 + .../MergeTree/MergeTreeDataPartInMemory.cpp | 3 ++- .../MergeTree/MergeTreeDataPartInMemory.h | 1 + src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataPartWide.h | 1 + .../MergeTree/MergeTreeDataPartWriterCompact.cpp | 3 ++- .../MergeTree/MergeTreeDataPartWriterCompact.h | 1 + .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 2 ++ .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 2 ++ .../MergeTree/MergeTreeDataPartWriterWide.cpp | 5 +++-- .../MergeTree/MergeTreeDataPartWriterWide.h | 1 + src/Storages/MergeTree/MergeTreeDataWriter.cpp | 13 ++++++++++--- .../MergeTree/MergeTreeWriteAheadLog.cpp | 3 ++- .../MergeTree/MergedBlockOutputStream.cpp | 13 +++++++------ src/Storages/MergeTree/MergedBlockOutputStream.h | 2 +- .../MergeTree/MergedColumnOnlyOutputStream.cpp | 4 ++++ .../MergeTree/MergedColumnOnlyOutputStream.h | 1 + 28 files changed, 88 insertions(+), 30 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 109cbaea5de..f952cd0565b 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -595,8 +596,7 @@ MutableColumns ColumnSparse::scatter(ColumnIndex num_columns, const Selector & s void ColumnSparse::gather(ColumnGathererStream & gatherer_stream) { - UNUSED(gatherer_stream); - throwMustBeDense(); + gatherer_stream.gather(*this); } ColumnPtr ColumnSparse::compress() const @@ -615,7 +615,7 @@ ColumnPtr ColumnSparse::compress() const bool ColumnSparse::structureEquals(const IColumn & rhs) const { - if (auto rhs_sparse = typeid_cast(&rhs)) + if (const auto * rhs_sparse = typeid_cast(&rhs)) return values->structureEquals(*rhs_sparse->values); return false; } diff --git a/src/DataStreams/ColumnGathererStream.cpp b/src/DataStreams/ColumnGathererStream.cpp index 2468ae872a2..3ce2d93e5fd 100644 --- a/src/DataStreams/ColumnGathererStream.cpp +++ b/src/DataStreams/ColumnGathererStream.cpp @@ -18,9 +18,11 @@ namespace ErrorCodes } ColumnGathererStream::ColumnGathererStream( - const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_, + const String & column_name_, const SerializationPtr & serialization_, + const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_, size_t block_preferred_size_) - : column_name(column_name_), sources(source_streams.size()), row_sources_buf(row_sources_buf_) + : column_name(column_name_), serialization(serialization_) + , sources(source_streams.size()), row_sources_buf(row_sources_buf_) , block_preferred_size(block_preferred_size_), log(&Poco::Logger::get("ColumnGathererStream")) { if (source_streams.empty()) @@ -42,9 +44,9 @@ ColumnGathererStream::ColumnGathererStream( { column.name = column_name; column.type = header.getByName(column_name).type; - column.column = column.type->createColumn(); + column.column = column.type->createColumn(*serialization); } - else if (header.getByName(column_name).column->getName() != column.column->getName()) + else if (!header.getByName(column_name).type->equals(*column.type)) throw Exception("Column types don't match", ErrorCodes::INCOMPATIBLE_COLUMNS); } } diff --git a/src/DataStreams/ColumnGathererStream.h b/src/DataStreams/ColumnGathererStream.h index 05665ab3f42..4e1a80c5154 100644 --- a/src/DataStreams/ColumnGathererStream.h +++ b/src/DataStreams/ColumnGathererStream.h @@ -57,7 +57,8 @@ class ColumnGathererStream : public IBlockInputStream { public: ColumnGathererStream( - const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_, + const String & column_name_, const SerializationPtr & serialization_, + const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_, size_t block_preferred_size_ = DEFAULT_BLOCK_SIZE); String getName() const override { return "ColumnGatherer"; } @@ -93,6 +94,7 @@ private: String column_name; ColumnWithTypeAndName column; + SerializationPtr serialization; std::vector sources; ReadBuffer & row_sources_buf; diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 124ee30729e..c6d5bfb12db 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -47,6 +47,18 @@ void SerializationInfo::add(const SerializationInfo & other) default_rows[name] += num; } +void SerializationInfo::update(const SerializationInfo & other) +{ + if (number_of_rows && number_of_rows != other.number_of_rows) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot update SerializationInfo with {} rows by SerializationInfo with {} rows", + number_of_rows, other.number_of_rows); + + number_of_rows = other.number_of_rows; + for (const auto & [name, num] : other.default_rows) + default_rows[name] = num; +} + size_t SerializationInfo::getNumberOfDefaultRows(const String & column_name) const { auto it = default_rows.find(column_name); diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 6205c3c7e0c..0af31f08063 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -17,6 +17,7 @@ public: size_t default_rows_search_step_ = IColumn::DEFAULT_ROWS_SEARCH_STEP); void add(const Block & block); + void update(const SerializationInfo & other); void add(const SerializationInfo & other); size_t getNumberOfDefaultRows(const String & column_name) const; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 862a3088f89..7ed036756b8 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -517,7 +517,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( new_data_part->minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); new_data_part->partition.create(metadata_snapshot, block, 0); - MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {})); + MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, + block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}), new_data_part->serialization_info); + part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_data_part); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 79079455a17..289e89fbc51 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -99,6 +99,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity = {}) const = 0; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index e334cd486ef..3419adeb16e 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -11,6 +11,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , metadata_snapshot(metadata_snapshot_) , volume(data_part->volume) , part_path(data_part->isStoredOnDisk() ? data_part->getFullRelativePath() : "") + , new_serialization_info(data_part->storage.getSettings()->ratio_for_sparse_serialization) { } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index b2ad5309017..0146a74a5ea 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -43,6 +43,7 @@ protected: String part_path; IMergeTreeDataPart::MergeTreeWriterPtr writer; + SerializationInfo new_serialization_info; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 9d2556d34ee..6674fa8d737 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -716,6 +716,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor bool need_remove_expired_values = false; bool force_ttl = false; + SerializationInfo new_serialization_info(data_settings->ratio_for_sparse_serialization); + for (const auto & part : parts) { new_data_part->ttl_infos.update(part->ttl_infos); @@ -726,7 +728,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor force_ttl = true; } - new_data_part->serialization_info.add(part->serialization_info); + new_serialization_info.add(part->serialization_info); } const auto & part_min_ttl = new_data_part->ttl_infos.part_min_ttl; @@ -923,6 +925,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merging_columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, + new_serialization_info, blocks_are_granules_size}; merged_stream->readPrefix(); @@ -1001,7 +1004,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor column_num < gathering_column_names_size; ++column_num, ++it_name_and_type) { - const String & column_name = it_name_and_type->name; + const auto & [column_name, column_type] = *it_name_and_type; Names column_names{column_name}; Float64 progress_before = merge_entry->progress.load(std::memory_order_relaxed); @@ -1023,13 +1026,16 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor } rows_sources_read_buf.seek(0, 0); - ColumnGathererStream column_gathered_stream(column_name, column_part_streams, rows_sources_read_buf); + ColumnGathererStream column_gathered_stream( + column_name, column_type->getSerialization(column_name, new_serialization_info), + column_part_streams, rows_sources_read_buf); MergedColumnOnlyOutputStream column_to( new_data_part, metadata_snapshot, column_gathered_stream.getHeader(), compression_codec, + new_serialization_info, /// we don't need to recalc indices here /// because all of them were already recalculated and written /// as key part of vertical merge @@ -1780,7 +1786,8 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( metadata_snapshot, new_data_part->getColumns(), skip_indices, - compression_codec}; + compression_codec, + new_data_part->serialization_info}; mutating_stream->readPrefix(); out.writePrefix(); @@ -1825,6 +1832,7 @@ void MergeTreeDataMergerMutator::mutateSomePartColumns( metadata_snapshot, mutation_header, compression_codec, + source_part->serialization_info, std::vector(indices_to_recalc.begin(), indices_to_recalc.end()), nullptr, source_part->index_granularity, diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 32f54e3b782..76f9e1fa9b8 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -58,6 +58,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const { @@ -72,7 +73,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( return std::make_unique( shared_from_this(), ordered_columns_list, metadata_snapshot, indices_to_recalc, index_granularity_info.marks_file_extension, - default_codec_, writer_settings, computed_index_granularity); + default_codec_, serialization_info_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 564d59c9198..cea8fdaeb43 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -49,6 +49,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 96fa411339c..b1ea903976f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -59,6 +59,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & /* indices_to_recalc */, const CompressionCodecPtr & /* default_codec */, + const SerializationInfo & /* serialization_info */, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & /* computed_index_granularity */) const { @@ -90,7 +91,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri auto compression_codec = storage.global_context.chooseCompressionCodec(0, 0); auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); - MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec); + MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec, new_data_part->serialization_info); out.writePrefix(); out.write(block); out.writeSuffixAndFinalizePart(new_data_part); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 397d3d2036c..cd11db6aa1b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -38,6 +38,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index b029c3c6912..450c6f4b02d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -63,13 +63,14 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const { return std::make_unique( shared_from_this(), columns_list, metadata_snapshot, indices_to_recalc, index_granularity_info.marks_file_extension, - default_codec_, writer_settings, computed_index_granularity); + default_codec_, serialization_info_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 30d3021d003..fe34a581aa6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -43,6 +43,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 2aa517a9833..6da769ccc0d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -16,11 +16,12 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_) + default_codec_, serialization_info_, settings_, index_granularity_) , plain_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, settings.max_compress_block_size, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 8b86a9701c9..e5500c11912 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -15,6 +15,7 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 2494195223a..c9403c5db92 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -73,6 +73,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const MergeTreeIndices & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter(data_part_, @@ -81,6 +82,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( , part_path(data_part_->getFullRelativePath()) , marks_file_extension(marks_file_extension_) , default_codec(default_codec_) + , serialization_info(serialization_info_) , compute_granularity(index_granularity.empty()) { if (settings.blocks_are_granules_size && !index_granularity.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index d952950e461..8281b6d8982 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -88,6 +88,7 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); @@ -125,6 +126,7 @@ protected: const String part_path; const String marks_file_extension; const CompressionCodecPtr default_codec; + const SerializationInfo serialization_info; const bool compute_granularity; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 7e7714d0bdd..1d904cdad85 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -74,16 +74,17 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, - default_codec_, settings_, index_granularity_) + default_codec_, serialization_info_, settings_, index_granularity_) { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) { - serializations.emplace(it.name, it.type->getSerialization(it.name, data_part->serialization_info)); + serializations.emplace(it.name, it.type->getSerialization(it.name, serialization_info)); addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec)); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 5eaaa0c1bbe..6cda78f14cf 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -24,6 +24,7 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, + const SerializationInfo & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index e2f075070a8..f84f1404f24 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -393,15 +393,22 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); new_data_part->ttl_infos.update(move_ttl_infos); - new_data_part->serialization_info.add(block); /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.global_context.chooseCompressionCodec(0, 0); + const auto & data_settings = data.getSettings(); + + SerializationInfo serialization_info(data_settings->ratio_for_sparse_serialization); + serialization_info.add(block); + const auto & index_factory = MergeTreeIndexFactory::instance(); - MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); - bool sync_on_insert = data.getSettings()->fsync_after_insert; + MergedBlockOutputStream out(new_data_part, metadata_snapshot,columns, + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), + compression_codec, serialization_info); + + bool sync_on_insert = data_settings->fsync_after_insert; out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 4ca20572e90..b78a398b95f 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -189,7 +189,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor if (action_type == ActionType::ADD_PART) { - MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {})); + MergedBlockOutputStream part_out(part, metadata_snapshot, + block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {}), part->serialization_info); part->minmax_idx.update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 18813c8892c..62430956512 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -20,11 +20,11 @@ MergedBlockOutputStream::MergedBlockOutputStream( const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, + const SerializationInfo & serialization_info, bool blocks_are_granules_size) : IMergedBlockOutputStream(data_part, metadata_snapshot_) , columns_list(columns_list_) , default_codec(default_codec_) - , serialization_info(storage.getSettings()->ratio_for_sparse_serialization) { MergeTreeWriterSettings writer_settings( storage.global_context.getSettings(), @@ -36,7 +36,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( if (!part_path.empty()) volume->getDisk()->createDirectories(part_path); - writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, writer_settings); + writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, serialization_info, writer_settings); } /// If data is pre-sorted. @@ -79,12 +79,13 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( else part_columns = *total_columns_list; + new_part->serialization_info.update(new_serialization_info); + if (new_part->isStoredOnDisk()) finalizePartOnDisk(new_part, part_columns, checksums, sync); new_part->setColumns(part_columns); new_part->rows_count = rows_count; - new_part->serialization_info = serialization_info; new_part->modification_time = time(nullptr); new_part->index = writer->releaseIndexColumns(); new_part->checksums = checksums; @@ -149,11 +150,11 @@ void MergedBlockOutputStream::finalizePartOnDisk( removeEmptyColumnsFromPart(new_part, part_columns, checksums); - if (serialization_info.getNumberOfRows() > 0) + if (new_part->serialization_info.getNumberOfRows() > 0) { auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); HashingWriteBuffer out_hashing(*out); - serialization_info.write(out_hashing); + new_part->serialization_info.write(out_hashing); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash(); out->finalize(); @@ -200,7 +201,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm return; writer->write(block, permutation); - serialization_info.add(block); + new_serialization_info.add(block); rows_count += rows; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 288d38baf4f..31e64193730 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -19,6 +19,7 @@ public: const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, + const SerializationInfo & serialization_info, bool blocks_are_granules_size = false); Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } @@ -57,7 +58,6 @@ private: IMergeTreeDataPart::MinMaxIndex minmax_idx; size_t rows_count = 0; CompressionCodecPtr default_codec; - SerializationInfo serialization_info; }; } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 41479f104f3..5f0c4143362 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -14,6 +14,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec, + const SerializationInfo & serialization_info, const MergeTreeIndices & indices_to_recalc, WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, @@ -35,6 +36,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( metadata_snapshot_, indices_to_recalc, default_codec, + serialization_info, std::move(writer_settings), index_granularity); @@ -51,6 +53,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) return; writer->write(block, nullptr); + new_serialization_info.add(block); } void MergedColumnOnlyOutputStream::writeSuffix() @@ -76,6 +79,7 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( all_checksums.files.erase(removed_file); new_part->setColumns(columns); + new_part->serialization_info.update(new_serialization_info); return checksums; } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 393058dc82e..cc911db20f9 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -18,6 +18,7 @@ public: const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec_, + const SerializationInfo & serialization_info, const MergeTreeIndices & indices_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, const MergeTreeIndexGranularity & index_granularity = {}, From 2afa1590e09ce7cc0c9f94843bd71c82f04f804a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 17 Apr 2021 04:06:59 +0300 Subject: [PATCH 016/358] ColumnSparse: fix MergeTree in old syntax --- src/Columns/ColumnSparse.cpp | 9 +++++++++ src/Columns/ColumnSparse.h | 5 ----- .../Serializations/SerializationSparse.cpp | 3 ++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 18 +++++++++++++++--- .../MergeTree/MergedBlockOutputStream.cpp | 4 +++- 5 files changed, 29 insertions(+), 10 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index f952cd0565b..890bae741f7 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -7,6 +7,8 @@ #include #include +#include + namespace DB { @@ -41,6 +43,13 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs if (_size < offsets->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of sparse column ({}) cannot be lower than number of non-default values ({})", _size, offsets->size()); + +#ifndef NDEBUG + const auto & offsets_data = getOffsetsData(); + auto it = std::adjacent_find(offsets_data.begin(), offsets_data.end(), std::greater_equal()); + if (it != offsets_data.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Offsets of ColumnSparse must be strictly sorted"); +#endif } MutableColumnPtr ColumnSparse::cloneResized(size_t new_size) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 3bd0741df4c..591011665bb 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -157,11 +157,6 @@ public: IColumn & getOffsetsColumn() { return *offsets; } private: - [[noreturn]] void throwMustBeDense() const - { - throw Exception("Not implemented for ColumnSparse", ErrorCodes::LOGICAL_ERROR); - } - class Iterator { public: diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index a160d23c4a1..dd39d51b409 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -55,10 +55,12 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, /// TODO: offsets.reserve(limit / 10); + bool first = true; size_t total_rows = state.num_trailing_defaults; if (state.has_value_after_defaults) { offsets.push_back(start + state.num_trailing_defaults); + first = false; state.has_value_after_defaults = false; state.num_trailing_defaults = 0; @@ -66,7 +68,6 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } size_t group_size; - bool first = true; while (!istr.eof()) { readIntBinary(group_size, istr); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 13f53616090..8b5a49c6c3b 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -811,6 +811,14 @@ void IMergeTreeDataPart::loadChecksums(bool require) void IMergeTreeDataPart::loadRowsCount() { String path = getFullRelativePath() + "count.txt"; + + auto read_rows_count = [&]() + { + auto buf = openForReading(volume->getDisk(), path); + readIntText(rows_count, *buf); + assertEOF(*buf); + }; + if (index_granularity.empty()) { rows_count = 0; @@ -820,9 +828,7 @@ void IMergeTreeDataPart::loadRowsCount() if (!volume->getDisk()->exists(path)) throw Exception("No count.txt in part " + name, ErrorCodes::NO_FILE_IN_DATA_PART); - auto buf = openForReading(volume->getDisk(), path); - readIntText(rows_count, *buf); - assertEOF(*buf); + read_rows_count(); #ifndef NDEBUG /// columns have to be loaded @@ -875,6 +881,12 @@ void IMergeTreeDataPart::loadRowsCount() } else { + if (volume->getDisk()->exists(path)) + { + read_rows_count(); + return; + } + for (const NameAndTypePair & column : columns) { ColumnPtr column_col = column.type->createColumn(*getSerializationForColumn(column)); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 4cca8a3cd6c..111b0c948c3 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -115,7 +115,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( out->sync(); } - if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING || isCompactPart(new_part)) + if (storage.format_version >= MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { new_part->partition.store(storage, volume->getDisk(), part_path, checksums); if (new_part->minmax_idx.initialized) @@ -123,7 +123,9 @@ void MergedBlockOutputStream::finalizePartOnDisk( else if (rows_count) throw Exception("MinMax index was not initialized for new non-empty part " + new_part->name + ". It is a bug.", ErrorCodes::LOGICAL_ERROR); + } + { auto count_out = volume->getDisk()->writeFile(part_path + "count.txt", 4096); HashingWriteBuffer count_out_hashing(*count_out); writeIntText(rows_count, count_out_hashing); From b5b624f3d7e9bf13be177fca350e8d9ba8a69811 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 17 Apr 2021 05:25:22 +0300 Subject: [PATCH 017/358] ColumnSparse: fixes for dictionaries --- src/Columns/ColumnSparse.cpp | 20 ++++++++++++++++++++ src/Columns/ColumnSparse.h | 4 ++++ src/Dictionaries/DictionaryHelpers.h | 5 +++-- 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 890bae741f7..bb9409e6860 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -85,6 +85,26 @@ bool ColumnSparse::getBool(size_t n) const return values->getBool(getValueIndex(n)); } +Float64 ColumnSparse::getFloat64(size_t n) const +{ + return values->getFloat64(getValueIndex(n)); +} + +Float32 ColumnSparse::getFloat32(size_t n) const +{ + return values->getFloat32(getValueIndex(n)); +} + +UInt64 ColumnSparse::getUInt(size_t n) const +{ + return values->getUInt(getValueIndex(n)); +} + +Int64 ColumnSparse::getInt(size_t n) const +{ + return values->getInt(getValueIndex(n)); +} + UInt64 ColumnSparse::get64(size_t n) const { return values->get64(getValueIndex(n)); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 591011665bb..0f1cd433884 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -62,6 +62,10 @@ public: Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; bool getBool(size_t n) const override; + Float64 getFloat64(size_t n) const override; + Float32 getFloat32(size_t n) const override; + UInt64 getUInt(size_t n) const override; + Int64 getInt(size_t n) const override; UInt64 get64(size_t n) const override; StringRef getDataAt(size_t n) const override; diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index e4406c9a22b..285c8708b1e 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -347,7 +348,7 @@ public: if constexpr (key_type == DictionaryKeyType::simple) { - key_columns[0] = key_columns[0]->convertToFullColumnIfConst(); + key_columns[0] = recursiveRemoveSparse(key_columns[0]->convertToFullColumnIfConst()); const auto * vector_col = checkAndGetColumn>(key_columns[0].get()); if (!vector_col) @@ -543,7 +544,7 @@ static const PaddedPODArray & getColumnVectorData( PaddedPODArray & backup_storage) { bool is_const_column = isColumnConst(*column); - auto full_column = column->convertToFullColumnIfConst(); + auto full_column = recursiveRemoveSparse(column->convertToFullColumnIfConst()); auto vector_col = checkAndGetColumn>(full_column.get()); if (!vector_col) From 65aceaa668f1bafeedcff072d0eb2f8ab71e1a9a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 19 Apr 2021 15:00:31 +0300 Subject: [PATCH 018/358] ColumnSparse: fixes --- src/DataTypes/IDataType.cpp | 13 +++++++++++-- src/Interpreters/join_common.cpp | 6 +++++- .../Transforms/AggregatingInOrderTransform.cpp | 1 + 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index f2aa20514c4..ee995f43bb2 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -217,8 +217,17 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, con return subcolumn_type.getSerialization(column.name, callback); }; - auto type_in_storage = column.getTypeInStorage(); - return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), base_serialization_getter); + const auto & type_in_storage = column.getTypeInStorage(); + auto subcolumn_serialization = type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), base_serialization_getter); + + if (type_in_storage->supportsSparseSerialization()) + { + auto sparse_idx_name = escapeForFileName(column.getNameInStorage()) + ".sparse.idx"; + if (callback(sparse_idx_name)) + subcolumn_serialization = std::make_shared(subcolumn_serialization); + + return subcolumn_serialization; + } } return column.type->getSerialization(column.name, callback); diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index e10b9491919..bdb59fa0f15 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -52,9 +52,13 @@ namespace JoinCommon void convertColumnToNullable(ColumnWithTypeAndName & column, bool low_card_nullability) { + + if (column.column->isSparse()) + column.column = recursiveRemoveSparse(column.column); + if (low_card_nullability && column.type->lowCardinality()) { - column.column = recursiveRemoveLowCardinality(recursiveRemoveSparse(column.column)); + column.column = recursiveRemoveLowCardinality(column.column); column.type = recursiveRemoveLowCardinality(column.type); } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 392e27166ef..24c1ce18924 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -230,6 +230,7 @@ IProcessor::Status AggregatingInOrderTransform::prepare() return Status::NeedData; } current_chunk = input.pull(!is_consume_finished); + convertToFullIfSparse(current_chunk); return Status::Ready; } From f2c4e0ff01f61ca20821936c22f748493a9e809e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 19 Apr 2021 16:01:26 +0300 Subject: [PATCH 019/358] fix subcolumns --- src/DataTypes/IDataType.cpp | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index ee995f43bb2..f2aa20514c4 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -217,17 +217,8 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, con return subcolumn_type.getSerialization(column.name, callback); }; - const auto & type_in_storage = column.getTypeInStorage(); - auto subcolumn_serialization = type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), base_serialization_getter); - - if (type_in_storage->supportsSparseSerialization()) - { - auto sparse_idx_name = escapeForFileName(column.getNameInStorage()) + ".sparse.idx"; - if (callback(sparse_idx_name)) - subcolumn_serialization = std::make_shared(subcolumn_serialization); - - return subcolumn_serialization; - } + auto type_in_storage = column.getTypeInStorage(); + return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), base_serialization_getter); } return column.type->getSerialization(column.name, callback); From d571d099bd2c1f2e26a32faa0e59eccc3d59ed3e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 20 Apr 2021 15:13:46 +0300 Subject: [PATCH 020/358] ColumnSparse: fix TTL --- src/DataStreams/TTLBlockInputStream.cpp | 3 +++ src/Storages/MergeTree/IMergedBlockOutputStream.cpp | 13 ++++++++++--- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 8b31da6d2f1..6da60f708c8 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -104,6 +104,9 @@ Block TTLBlockInputStream::readImpl() return {}; auto block = children.at(0)->read(); + for (auto & column : block) + column.column = recursiveRemoveSparse(column.column); + for (const auto & algorithm : algorithms) algorithm->execute(block); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 3419adeb16e..1c1745b9711 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -29,15 +29,23 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. std::map stream_counts; + std::unordered_map serialziations; for (const NameAndTypePair & column : columns) { - auto serialization = data_part->getSerializationForColumn(column); + auto serialization = IDataType::getSerialization(column, [&](const String & stream_name) + { + /// Checksums of data_part are not initialized here. + return checksums.files.count(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION) != 0; + }); + serialization->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; }, {}); + + serialziations[column.name] = std::move(serialization); } NameSet remove_files; @@ -59,8 +67,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( } }; - auto serialization = data_part->getSerializationForColumn(*column_with_type); - serialization->enumerateStreams(callback); + serialziations[column_name]->enumerateStreams(callback); } /// Remove files on disk and checksums From 0b65552e63ff0e21fa329eb9082addd1f24b430a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 20 Apr 2021 17:20:39 +0300 Subject: [PATCH 021/358] ColumnSparse: fixes --- src/AggregateFunctions/IAggregateFunction.h | 25 +++++---------------- src/Columns/ColumnSparse.cpp | 5 +---- src/Columns/ColumnSparse.h | 2 +- src/Dictionaries/CacheDictionary.cpp | 3 +++ 4 files changed, 10 insertions(+), 25 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index e37f982fd3b..9986111fa4c 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -306,22 +306,9 @@ public: { const auto & column_sparse = assert_cast(*columns[0]); const auto * values = &column_sparse.getValuesColumn(); - const auto & offsets_data = column_sparse.getOffsetsData(); - size_t offset_pos = 0; - size_t offsets_size = offsets_data.size(); - for (size_t i = 0; i < column_sparse.size(); ++i) - { - if (offset_pos < offsets_size && i == offsets_data[offset_pos]) - { - static_cast(this)->add(places[i] + place_offset, &values, offset_pos + 1, arena); - ++offset_pos; - } - else - { - static_cast(this)->add(places[i] + place_offset, &values, 0, arena); - } - } + for (auto it = column_sparse.begin(); it != column_sparse.end(); ++it) + static_cast(this)->add(places[it.getCurrentRow()] + place_offset, &values, it.getValueIndex(), arena); } void addBatchSinglePlace( @@ -346,14 +333,12 @@ public: void addBatchSparseSinglePlace( AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override { + /// TODO: add values and defaults separately if order of adding isn't important. const auto & column_sparse = assert_cast(*columns[0]); const auto * values = &column_sparse.getValuesColumn(); - for (size_t i = 1; i < values->size(); ++i) - static_cast(this)->add(place, &values, i, arena); - - for (size_t i = 0; i < column_sparse.getNumberOfDefaults(); ++i) - static_cast(this)->add(place, &values, 0, arena); + for (auto it = column_sparse.begin(); it != column_sparse.end(); ++it) + static_cast(this)->add(place, &values, it.getValueIndex(), arena); } void addBatchSinglePlaceNotNull( diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index bb9409e6860..2cccbe545af 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -558,10 +558,7 @@ void ColumnSparse::updateWeakHash32(WeakHash32 & hash) const auto & hash_data = hash.getData(); for (size_t i = 0; i < _size; ++i, ++offset_it) { - size_t value_index = 0; - if (!offset_it.isDefault()) - value_index = offset_it.getValueIndex(); - + size_t value_index = offset_it.getValueIndex(); auto data_ref = values->getDataAt(value_index); hash_data[i] = ::updateWeakHash32(reinterpret_cast(data_ref.data), data_ref.size, hash_data[i]); } diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 0f1cd433884..9b09e8a9d85 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -160,7 +160,6 @@ public: const IColumn & getOffsetsColumn() const { return *offsets; } IColumn & getOffsetsColumn() { return *offsets; } -private: class Iterator { public: @@ -200,6 +199,7 @@ private: Iterator begin() const { return Iterator(getOffsetsData(), _size, 0, 0); } Iterator end() const { return Iterator(getOffsetsData(), _size, getOffsetsData().size(), _size); } +private: WrappedPtr values; WrappedPtr offsets; size_t _size; diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index b1b8ebed5bd..43931da4b92 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -586,6 +586,9 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr Date: Wed, 21 Apr 2021 03:23:02 +0300 Subject: [PATCH 022/358] ColumnSparse: fixes and better performance --- src/AggregateFunctions/IAggregateFunction.h | 13 ++++-- src/Columns/ColumnSparse.cpp | 46 ++++++++++++++++++--- src/Columns/ColumnSparse.h | 6 +-- src/Columns/ya.make | 1 + src/DataTypes/ya.make | 3 +- src/Dictionaries/DictionaryHelpers.h | 3 ++ src/Dictionaries/DirectDictionary.cpp | 5 ++- src/Dictionaries/FlatDictionary.cpp | 5 ++- src/Dictionaries/HashedDictionary.cpp | 5 ++- src/Functions/ya.make | 1 + 10 files changed, 71 insertions(+), 17 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 9986111fa4c..af2a3656463 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -306,9 +306,12 @@ public: { const auto & column_sparse = assert_cast(*columns[0]); const auto * values = &column_sparse.getValuesColumn(); + size_t batch_size = column_sparse.size(); + auto offset_it = column_sparse.begin(); - for (auto it = column_sparse.begin(); it != column_sparse.end(); ++it) - static_cast(this)->add(places[it.getCurrentRow()] + place_offset, &values, it.getValueIndex(), arena); + for (size_t i = 0; i < batch_size; ++i, ++offset_it) + static_cast(this)->add(places[offset_it.getCurrentRow()] + place_offset, + &values, offset_it.getValueIndex(), arena); } void addBatchSinglePlace( @@ -336,9 +339,11 @@ public: /// TODO: add values and defaults separately if order of adding isn't important. const auto & column_sparse = assert_cast(*columns[0]); const auto * values = &column_sparse.getValuesColumn(); + size_t batch_size = column_sparse.size(); + auto offset_it = column_sparse.begin(); - for (auto it = column_sparse.begin(); it != column_sparse.end(); ++it) - static_cast(this)->add(place, &values, it.getValueIndex(), arena); + for (size_t i = 0; i < batch_size; ++i, ++offset_it) + static_cast(this)->add(place, &values, offset_it.getValueIndex(), arena); } void addBatchSinglePlaceNotNull( diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 2cccbe545af..5ca818fa148 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -242,7 +242,7 @@ void ColumnSparse::popBack(size_t n) size_t new_size = _size - n; size_t removed_values = 0; - while(!offsets_data.empty() && offsets_data.back() >= new_size) + while (!offsets_data.empty() && offsets_data.back() >= new_size) { offsets_data.pop_back(); ++removed_values; @@ -309,18 +309,52 @@ ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const if (perm.size() < limit) throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + if (offsets->empty()) + { + auto res = cloneEmpty(); + res->insertManyDefaults(limit); + return res; + } + auto res_offsets = offsets->cloneEmpty(); auto & res_offsets_data = assert_cast(*res_offsets).getData(); auto res_values = values->cloneEmpty(); res_values->insertDefault(); - for (size_t i = 0; i < limit; ++i) + /// If we need to permute full column, or if limit is large enough, + /// it's better to save indexes of values in O(size) + /// and avoid binary search for obtaining every index. + /// 3 is just a guess for overhead on copying indexes. + bool execute_linear = + limit == _size || limit * log2(offsets->size() + 1) > _size * 3; + + if (execute_linear) { - size_t index = getValueIndex(perm[i]); - if (index != 0) + PaddedPODArray indexes(_size); + auto offset_it = begin(); + for (size_t i = 0; i < _size; ++i, ++offset_it) + indexes[i] = offset_it.getValueIndex(); + + for (size_t i = 0; i < limit; ++i) { - res_values->insertFrom(*values, index); - res_offsets_data.push_back(i); + size_t index = indexes[perm[i]]; + if (index != 0) + { + res_values->insertFrom(*values, index); + res_offsets_data.push_back(i); + } + } + } + else + { + for (size_t i = 0; i < limit; ++i) + { + size_t index = getValueIndex(perm[i]); + if (index != 0) + { + res_values->insertFrom(*values, index); + res_offsets_data.push_back(i); + } } } diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 9b09e8a9d85..cb7daa5482a 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -168,9 +168,9 @@ public: { } - bool isDefault() const { return current_offset == offsets.size() || current_row != offsets[current_offset]; } - size_t getValueIndex() const { return isDefault() ? 0 : current_offset + 1; } - size_t getCurrentRow() const { return current_row; } + bool ALWAYS_INLINE isDefault() const { return current_offset == offsets.size() || current_row != offsets[current_offset]; } + size_t ALWAYS_INLINE getValueIndex() const { return isDefault() ? 0 : current_offset + 1; } + size_t ALWAYS_INLINE getCurrentRow() const { return current_row; } bool operator==(const Iterator & other) const { diff --git a/src/Columns/ya.make b/src/Columns/ya.make index 54dd02609ff..964f9aa00b6 100644 --- a/src/Columns/ya.make +++ b/src/Columns/ya.make @@ -29,6 +29,7 @@ SRCS( ColumnLowCardinality.cpp ColumnMap.cpp ColumnNullable.cpp + ColumnSparse.cpp ColumnString.cpp ColumnTuple.cpp ColumnVector.cpp diff --git a/src/DataTypes/ya.make b/src/DataTypes/ya.make index bd57e1a5a47..d2800891512 100644 --- a/src/DataTypes/ya.make +++ b/src/DataTypes/ya.make @@ -15,7 +15,6 @@ SRCS( DataTypeCustomGeo.cpp DataTypeCustomIPv4AndIPv6.cpp DataTypeCustomSimpleAggregateFunction.cpp - DataTypeCustomSimpleTextSerialization.cpp DataTypeDate.cpp DataTypeDateTime.cpp DataTypeDateTime64.cpp @@ -44,6 +43,7 @@ SRCS( Serializations/ISerialization.cpp Serializations/SerializationAggregateFunction.cpp Serializations/SerializationArray.cpp + Serializations/SerializationCustomSimpleText.cpp Serializations/SerializationDate.cpp Serializations/SerializationDateTime.cpp Serializations/SerializationDateTime64.cpp @@ -51,6 +51,7 @@ SRCS( Serializations/SerializationDecimalBase.cpp Serializations/SerializationEnum.cpp Serializations/SerializationFixedString.cpp + Serializations/SerializationIP.cpp Serializations/SerializationInfo.cpp Serializations/SerializationLowCardinality.cpp Serializations/SerializationMap.cpp diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 285c8708b1e..63f7f76e18b 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -483,6 +483,9 @@ void mergeBlockWithStream( while (Block block = stream->read()) { + for (auto & column : block) + column.column = recursiveRemoveSparse(column.column); + Columns block_key_columns; block_key_columns.reserve(key_column_size); diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index 2038704414e..352315fc302 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -70,8 +70,11 @@ Columns DirectDictionary::getColumns( stream->readPrefix(); - while (const auto block = stream->read()) + while (auto block = stream->read()) { + for (auto & column : block) + column.column = recursiveRemoveSparse(column.column); + /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) block_key_columns.emplace_back(block.safeGetByPosition(i).column); diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 1540a3a876b..a71f4601e9c 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -278,8 +278,11 @@ void FlatDictionary::updateData() auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); - while (const auto block = stream->read()) + while (auto block = stream->read()) { + for (auto & column : block) + column.column = recursiveRemoveSparse(column.column); + /// We are using this to keep saved data if input stream consists of multiple blocks if (!previously_loaded_block) previously_loaded_block = std::make_shared(block.cloneEmpty()); diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 2a403554a80..d4dd0ef7e45 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -348,8 +348,11 @@ void HashedDictionary::updateData() auto stream = source_ptr->loadUpdatedAll(); stream->readPrefix(); - while (const auto block = stream->read()) + while (auto block = stream->read()) { + for (auto & column : block) + column.column = recursiveRemoveSparse(column.column); + /// We are using this to keep saved data if input stream consists of multiple blocks if (!previously_loaded_block) previously_loaded_block = std::make_shared(block.cloneEmpty()); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 660f7b115bf..0f4f7a321b2 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -532,6 +532,7 @@ SRCS( upper.cpp upperUTF8.cpp uptime.cpp + validateNestedArraySizes.cpp version.cpp visibleWidth.cpp visitParamExtractBool.cpp From e1e1435b007bf6fcf09c06131db3853d9b43e244 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 12 May 2021 04:36:47 +0300 Subject: [PATCH 023/358] ColumnSparse: use varint for offsets --- src/DataTypes/Serializations/SerializationSparse.cpp | 9 +++++---- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index dd39d51b409..80e55ec19e1 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB { @@ -13,7 +14,7 @@ namespace DB namespace { -static constexpr auto END_OF_GRANULE_FLAG = 1ULL << 63; +static constexpr auto END_OF_GRANULE_FLAG = 1ULL << 62; struct DeserializeStateSparse : public ISerialization::DeserializeBinaryBulkState { @@ -34,13 +35,13 @@ void serializeOffsets(const IColumn::Offsets & offsets, WriteBuffer & ostr, size for (size_t i = 0; i < size; ++i) { size_t group_size = offsets[i] - start; - writeIntBinary(group_size, ostr); + writeVarUInt(group_size, ostr); start += group_size + 1; } size_t group_size = start < end ? end - start : 0; group_size |= END_OF_GRANULE_FLAG; - writeIntBinary(group_size, ostr); + writeVarUInt(group_size, ostr); } size_t deserializeOffsets(IColumn::Offsets & offsets, @@ -70,7 +71,7 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, size_t group_size; while (!istr.eof()) { - readIntBinary(group_size, istr); + readVarUInt(group_size, istr); bool end_of_granule = group_size & END_OF_GRANULE_FLAG; group_size &= ~END_OF_GRANULE_FLAG; diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b9d9f1473ef..4e877980f1d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,7 +33,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_for_sparse_serialization, 0, "", 0) \ + M(Float, ratio_for_sparse_serialization, 1.1, "", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ From 8ae1533f8f0a3d352cfbce814997e200d7476882 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 14 May 2021 23:29:48 +0300 Subject: [PATCH 024/358] better serialization in native format --- src/AggregateFunctions/AggregateFunctionSum.h | 2 - src/AggregateFunctions/IAggregateFunction.h | 5 - src/Columns/IColumn.h | 1 + src/Common/SparseArray.h | 0 src/Core/Defines.h | 2 + src/DataStreams/NativeBlockInputStream.cpp | 25 ++-- src/DataStreams/NativeBlockInputStream.h | 2 +- src/DataStreams/NativeBlockOutputStream.cpp | 16 ++- src/DataTypes/DataTypeCustom_fwd.h | 18 --- src/DataTypes/DataTypeTuple.cpp | 35 ++++-- src/DataTypes/DataTypeTuple.h | 2 + src/DataTypes/IDataType.cpp | 16 ++- src/DataTypes/IDataType.h | 21 +++- .../Serializations/ISerialization.cpp | 44 +++++++ src/DataTypes/Serializations/ISerialization.h | 15 +++ .../SerializationCustomSimpleText.h | 2 +- .../Serializations/SerializationInfo.cpp | 2 - .../Serializations/SerializationSparse.cpp | 110 ++++++++++++++++-- .../Serializations/SerializationSparse.h | 29 ++++- .../Serializations/SerializationTuple.cpp | 10 ++ .../Serializations/SerializationTuple.h | 6 +- .../Serializations/SerializationUUID.h | 2 + .../Serializations/SerializationWrapper.h | 2 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- .../MergeTree/IMergedBlockOutputStream.cpp | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/MergeTree/MergeTreeSettings.h | 4 +- ...system_columns_and_system_tables.reference | 2 +- ...804_test_delta_codec_compression.reference | 2 +- ...ms_in_system_parts_columns_table.reference | 2 +- .../0_stateless/01780_column_sparse.reference | 8 +- .../0_stateless/01780_column_sparse.sql | 4 +- .../0_stateless/01780_column_sparse_pk.sql | 6 +- .../0_stateless/01780_column_sparse_tuple.sql | 4 +- 35 files changed, 307 insertions(+), 100 deletions(-) delete mode 100644 src/Common/SparseArray.h delete mode 100644 src/DataTypes/DataTypeCustom_fwd.h diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index debc9e6dafb..1748458f6d8 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -363,8 +363,6 @@ public: } } - void addManyDefaults(size_t /* length */) const override {} - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).merge(this->data(rhs)); diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index af2a3656463..6bf4173fa1a 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -227,11 +227,6 @@ public: virtual bool supportsSparseArguments() const { return false; } - virtual void addManyDefaults(size_t /* length */) const - { - throw Exception("Method addManyDefaults is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); - } - const DataTypes & getArgumentTypes() const { return argument_types; } const Array & getParameters() const { return parameters; } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 9c7d11cf270..8585aca4043 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -376,6 +376,7 @@ public: static constexpr auto DEFAULT_ROWS_SEARCH_STEP = 8; static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; + static constexpr auto DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION = 0.95; virtual size_t getNumberOfDefaultRows(size_t /* step */) const { return 0; } diff --git a/src/Common/SparseArray.h b/src/Common/SparseArray.h deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 668a60f9be8..c9d3c62b4bc 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -83,6 +83,8 @@ #define DBMS_MIN_REVISION_WITH_X_FORWARDED_FOR_IN_CLIENT_INFO 54443 #define DBMS_MIN_REVISION_WITH_REFERER_IN_CLIENT_INFO 54447 +#define DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION 54452 + /// Version of ClickHouse TCP protocol. Increment it manually when you change the protocol. #define DBMS_TCP_PROTOCOL_VERSION 54448 diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index 5780d160dc3..7d3c288f6a5 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -73,7 +73,7 @@ void NativeBlockInputStream::resetParser() is_killed.store(false); } -void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) +void NativeBlockInputStream::readData(const ISerialization & serialization, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) { ISerialization::DeserializeBinaryBulkSettings settings; settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; @@ -81,10 +81,9 @@ void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column settings.position_independent_encoding = false; ISerialization::DeserializeBinaryBulkStatePtr state; - auto serialization = type.getSerialization(*column); - serialization->deserializeBinaryBulkStatePrefix(settings, state); - serialization->deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state, nullptr); + serialization.deserializeBinaryBulkStatePrefix(settings, state); + serialization.deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state, nullptr); if (column->size() != rows) throw Exception("Cannot read all data in NativeBlockInputStream. Rows read: " + toString(column->size()) + ". Rows expected: " + toString(rows) + ".", @@ -152,10 +151,6 @@ Block NativeBlockInputStream::readImpl() readBinary(type_name, istr); column.type = data_type_factory.get(type_name); - /// TODO: check revision. - ISerialization::Kind serialization_kind; - readIntBinary(serialization_kind, istr); - if (use_index) { /// Index allows to do more checks. @@ -165,21 +160,23 @@ Block NativeBlockInputStream::readImpl() throw Exception("Index points to column with wrong type: corrupted index or data", ErrorCodes::INCORRECT_INDEX); } + /// Serialization + ISerialization::Kinds serialization_kinds; + if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + serialization_kinds.readBinary(istr); + /// Data - ColumnPtr read_column = column.type->createColumn(); - if (serialization_kind == ISerialization::Kind::SPARSE) - read_column = ColumnSparse::create(read_column); + auto serialization = column.type->getSerialization(serialization_kinds); + ColumnPtr read_column = column.type->createColumn(*serialization); 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(*serialization, read_column, istr, rows, avg_value_size_hint); /// TODO: maybe remove. read_column = recursiveRemoveSparse(read_column); column.column = std::move(read_column); - // std::cerr << "column.column: " << column.column->dumpStructure() << "\n"; - if (header) { /// Support insert from old clients without low cardinality type. diff --git a/src/DataStreams/NativeBlockInputStream.h b/src/DataStreams/NativeBlockInputStream.h index 8f3d2843e0f..43c5cc5ec57 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, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); + static void readData(const ISerialization & serialization, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); Block getHeader() const override; diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index b05e80354b4..849a6ce8dfa 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -120,11 +120,17 @@ void NativeBlockOutputStream::write(const Block & block) writeStringBinary(type_name, ostr); - column.column = recursiveRemoveSparse(column.column); - - /// TODO: add revision - auto serialization = column.type->getSerialization(*column.column); - writeIntBinary(serialization->getKind(), ostr); + SerializationPtr serialization; + if (client_revision < DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + { + serialization = column.type->getDefaultSerialization(); + column.column = recursiveRemoveSparse(column.column); + } + else + { + serialization = column.type->getSerialization(*column.column); + serialization->getKinds().writeBinary(ostr); + } /// Data if (rows) /// Zero items of data is always represented as zero number of bytes. diff --git a/src/DataTypes/DataTypeCustom_fwd.h b/src/DataTypes/DataTypeCustom_fwd.h deleted file mode 100644 index 39c82bc4366..00000000000 --- a/src/DataTypes/DataTypeCustom_fwd.h +++ /dev/null @@ -1,18 +0,0 @@ -// #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/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index c8006a65a79..e5dcc327d5c 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -318,7 +318,6 @@ SerializationPtr DataTypeTuple::getSubcolumnSerialization( throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } - SerializationPtr DataTypeTuple::doGetDefaultSerialization() const { SerializationTuple::ElementSerializations serializations(elems.size()); @@ -350,21 +349,35 @@ SerializationPtr DataTypeTuple::getSerialization(const String & column_name, con for (size_t i = 0; i < elems.size(); ++i) { auto subcolumn_name = Nested::concatenateName(column_name, names[i]); + auto serialization = elems[i]->getSerialization(subcolumn_name, info); + serializations[i] = std::make_shared(serialization, names[i]); + } + + return std::make_shared(std::move(serializations), have_explicit_names); +} + +SerializationPtr DataTypeTuple::getSerialization(const ISerialization::Kinds & kinds) const +{ + if (kinds.subcolumns.empty()) + return doGetDefaultSerialization(); + + if (kinds.subcolumns.size() != elems.size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong nmber of kinds of serializations for subcolumns. " + "Expected: {}, got {}", elems.size(), kinds.subcolumns.size()); + + SerializationTuple::ElementSerializations serializations(elems.size()); + for (size_t i = 0; i < elems.size(); ++i) + { SerializationPtr serialization; - if (const auto * type_tuple = typeid_cast(elems[i].get())) + if (const auto * elem_kinds = std::get_if(&kinds.subcolumns[i])) { - serialization = type_tuple->getSerialization(subcolumn_name, info); + serialization = elems[i]->getSerialization(*elem_kinds); } else { - ISerialization::Settings settings = - { - .num_rows = info.getNumberOfRows(), - .num_default_rows = info.getNumberOfDefaultRows(subcolumn_name), - .ratio_for_sparse_serialization = info.getRatioForSparseSerialization() - }; - - serialization = elems[i]->getSerialization(settings); + auto elem_kind = std::get(kinds.subcolumns[i]); + serialization = elem_kind == ISerialization::Kind::SPARSE + ? elems[i]->getSparseSerialization() : getDefaultSerialization(); } serializations[i] = std::make_shared(serialization, names[i]); diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 5e19bb3d77d..2abaeb28753 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -62,6 +62,8 @@ public: SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; + SerializationPtr getSerialization(const ISerialization::Kinds & kinds) const override; + SerializationPtr doGetDefaultSerialization() const override; const DataTypes & getElements() const { return elems; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index f2aa20514c4..be54d4d0b1a 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -189,18 +189,30 @@ SerializationPtr IDataType::getSerialization(const IColumn & column) const { .num_rows = column.size(), .num_default_rows = column.getNumberOfDefaultRows(IColumn::DEFAULT_ROWS_SEARCH_STEP), - .ratio_for_sparse_serialization = 10 + .ratio_for_sparse_serialization = IColumn::DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION }; return getSerialization(settings); } +SerializationPtr IDataType::getSerialization(const ISerialization::Kinds & kinds) const +{ + if (!kinds.subcolumns.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR,"Data type {} doesn't support " + "custom kinds of serialization for subcolumns ot doesn't have subcolumns at all.", getName()); + + if (kinds.main == ISerialization::Kind::SPARSE) + return getSparseSerialization(); + + return getDefaultSerialization(); +} + SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const { if (supportsSparseSerialization()) { double ratio = settings.num_rows ? std::min(static_cast(settings.num_default_rows) / settings.num_rows, 1.0) : 0.0; - if (ratio >= settings.ratio_for_sparse_serialization) + if (ratio > settings.ratio_for_sparse_serialization) return getSparseSerialization(); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index bb59bc8b3c6..9fb8ad01d1f 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -31,7 +31,12 @@ class SerializationInfo; /** Properties of data type. - * Contains methods for serialization/deserialization. + * + * Contains methods for getting serialization instances. + * One data type may have different serializations, which can be chosen + * dynamically before reading or writing, according to information about + * column content (see `getSerialization` methods). + * * Implementations of this interface represent a data type (example: UInt8) * or parametric family of data types (example: Array(...)). * @@ -69,15 +74,18 @@ public: SerializationPtr getDefaultSerialization() const; SerializationPtr getSparseSerialization() const; - /// Asks wether the stream with given name exists in table. + /// Asks whether the stream with given name exists in table. /// If callback returned true for all streams, which are required for /// one of serialization types, that serialization will be chosen for reading. /// If callback always returned false, the default serialization will be chosen. using StreamExistenceCallback = std::function; using BaseSerializationGetter = std::function; - virtual SerializationPtr getSerialization(const IColumn & column) const; + /// Chooses serialization for reading of one column or subcolumns by + /// checking existence of substreams using callback. virtual SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const; + + /// Returns serialization wrapper for reading one particular subcolumn of data type. virtual SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; @@ -85,10 +93,15 @@ public: const NameAndTypePair & column, const StreamExistenceCallback & callback = [](const String &) { return false; }); + /// Chooses serialziation accordind to column content. + virtual SerializationPtr getSerialization(const IColumn & column) const; + + /// Chooses serialization accosrding to collected information about content of columns. virtual SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; + virtual SerializationPtr getSerialization(const ISerialization::Kinds & kinds) const; + SerializationPtr getSerialization(const ISerialization::Settings & settings) const; - // SerializationPtr getSerialization(const IColumn & column) const; using StreamCallbackWithType = std::function; diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 68316ef7650..978ee05f3c2 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -12,6 +12,7 @@ namespace DB namespace ErrorCodes { extern const int MULTIPLE_STREAMS_REQUIRED; + extern const int LOGICAL_ERROR; } String ISerialization::kindToString(Kind kind) @@ -25,6 +26,49 @@ String ISerialization::kindToString(Kind kind) } } +void ISerialization::Kinds::writeBinary(WriteBuffer & ostr) const +{ + writeIntBinary(static_cast(main), ostr); + writeIntBinary(subcolumns.size(), ostr); + for (size_t i = 0; i < subcolumns.size(); ++i) + { + writeIntBinary(subcolumns[i].index(), ostr); + if (const auto * elem_kinds = std::get_if(&subcolumns[i])) + elem_kinds->writeBinary(ostr); + else + writeIntBinary(static_cast(main), ostr); + } +} + +void ISerialization::Kinds::readBinary(ReadBuffer & istr) +{ + readIntBinary(main, istr); + size_t num_subcolumns; + readIntBinary(num_subcolumns, istr); + subcolumns.reserve(num_subcolumns); + for (size_t i = 0; i < num_subcolumns; ++i) + { + size_t index; + readIntBinary(index, istr); + if (index == 0) + { + Kinds elem_kinds; + elem_kinds.readBinary(istr); + subcolumns.emplace_back(elem_kinds); + } + else if (index == 1) + { + Kind elem_kind; + readIntBinary(elem_kind, istr); + subcolumns.emplace_back(elem_kind); + } + else + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index of variant: {}, must be 0 or 1", index); + } + } +} + String ISerialization::Substream::toString() const { switch (type) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index a6b780f3780..729ada2a150 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -7,6 +7,7 @@ #include #include +#include namespace DB { @@ -47,6 +48,20 @@ public: virtual Kind getKind() const { return Kind::DEFAULT; } static String kindToString(Kind kind); + struct Kinds + { + Kinds() = default; + Kinds(Kind main_) : main(main_) {} + + Kind main = Kind::DEFAULT; + std::vector> subcolumns; + + void writeBinary(WriteBuffer & ostr) const; + void readBinary(ReadBuffer & istr); + }; + + virtual Kinds getKinds() const { return Kinds(getKind()); } + /** Binary serialization for range of values in column - for writing to disk/network, etc. * * Some data types are represented in multiple streams while being serialized. diff --git a/src/DataTypes/Serializations/SerializationCustomSimpleText.h b/src/DataTypes/Serializations/SerializationCustomSimpleText.h index ae938b1104b..637dc92f208 100644 --- a/src/DataTypes/Serializations/SerializationCustomSimpleText.h +++ b/src/DataTypes/Serializations/SerializationCustomSimpleText.h @@ -10,7 +10,7 @@ class WriteBuffer; struct FormatSettings; class IColumn; -/** Simple IDataTypeCustomTextSerialization that uses serializeText/deserializeText +/** Simple ISerialization that uses serializeText/deserializeText * for all serialization and deserialization. */ class SerializationCustomSimpleText : public SerializationWrapper { diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index c6d5bfb12db..1586dc712e9 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -78,8 +78,6 @@ constexpr auto KEY_VERSION = "version"; } -/// TODO: add all fields. - void SerializationInfo::fromJSON(const String & json_str) { Poco::JSON::Parser parser; diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 80e55ec19e1..139ecc7d850 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -11,6 +11,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; +} + namespace { @@ -53,8 +59,8 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, return limit; } - /// TODO: - offsets.reserve(limit / 10); + /// Just try to guess number of offsets. + offsets.reserve(static_cast(limit * IColumn::DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION)); bool first = true; size_t total_rows = state.num_trailing_defaults; @@ -114,8 +120,8 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } -SerializationSparse::SerializationSparse(const SerializationPtr & nested_serialization_) - : SerializationWrapper(nested_serialization_) +SerializationSparse::SerializationSparse(const SerializationPtr & nested_) + : nested(nested_) { } @@ -124,7 +130,7 @@ void SerializationSparse::enumerateStreams(const StreamCallback & callback, Subs path.push_back(Substream::SparseOffsets); callback(path); path.back() = Substream::SparseElements; - nested_serialization->enumerateStreams(callback, path); + nested->enumerateStreams(callback, path); path.pop_back(); } @@ -133,7 +139,7 @@ void SerializationSparse::serializeBinaryBulkStatePrefix( SerializeBinaryBulkStatePtr & state) const { settings.path.push_back(Substream::SparseElements); - nested_serialization->serializeBinaryBulkStatePrefix(settings, state); + nested->serializeBinaryBulkStatePrefix(settings, state); settings.path.pop_back(); } @@ -165,12 +171,12 @@ void SerializationSparse::serializeBinaryBulkWithMultipleStreams( const auto & values = column_sparse->getValuesColumn(); size_t begin = column_sparse->getValueIndex(offsets_data[0]); size_t end = column_sparse->getValueIndex(offsets_data.back()); - nested_serialization->serializeBinaryBulkWithMultipleStreams(values, begin, end - begin + 1, settings, state); + nested->serializeBinaryBulkWithMultipleStreams(values, begin, end - begin + 1, settings, state); } else { auto values = column.index(*offsets_column, 0); - nested_serialization->serializeBinaryBulkWithMultipleStreams(*values, 0, values->size(), settings, state); + nested->serializeBinaryBulkWithMultipleStreams(*values, 0, values->size(), settings, state); } } @@ -182,7 +188,7 @@ void SerializationSparse::serializeBinaryBulkStateSuffix( SerializeBinaryBulkStatePtr & state) const { settings.path.push_back(Substream::SparseElements); - nested_serialization->serializeBinaryBulkStateSuffix(settings, state); + nested->serializeBinaryBulkStateSuffix(settings, state); settings.path.pop_back(); } @@ -193,7 +199,7 @@ void SerializationSparse::deserializeBinaryBulkStatePrefix( auto state_sparse = std::make_shared(); settings.path.push_back(Substream::SparseElements); - nested_serialization->deserializeBinaryBulkStatePrefix(settings, state_sparse->nested); + nested->deserializeBinaryBulkStatePrefix(settings, state_sparse->nested); settings.path.pop_back(); state = std::move(state_sparse); @@ -226,7 +232,7 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( size_t values_limit = offsets_data.size() - old_size; settings.path.back() = Substream::SparseElements; - nested_serialization->deserializeBinaryBulkWithMultipleStreams(values_column, values_limit, settings, state_sparse->nested, cache); + nested->deserializeBinaryBulkWithMultipleStreams(values_column, values_limit, settings, state_sparse->nested, cache); settings.path.pop_back(); if (offsets_data.size() + 1 != values_column->size()) @@ -237,4 +243,86 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( column = std::move(mutable_column); } +void SerializationSparse::serializeBinary(const Field & field, WriteBuffer & ostr) const +{ + nested->serializeBinary(field, ostr); +} + +void SerializationSparse::deserializeBinary(Field & field, ReadBuffer & istr) const +{ + nested->deserializeBinary(field, istr); +} + +void SerializationSparse::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +{ + const auto & column_sparse = assert_cast(column); + nested->serializeBinary(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr); +} + +void SerializationSparse::deserializeBinary(IColumn &, ReadBuffer &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeBinary' is not implemented for SerializationSparse"); +} + +void SerializationSparse::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_sparse = assert_cast(column); + nested->serializeTextEscaped(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings); +} + +void SerializationSparse::deserializeTextEscaped(IColumn &, ReadBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeTextEscaped' is not implemented for SerializationSparse"); +} + +void SerializationSparse::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_sparse = assert_cast(column); + nested->serializeTextQuoted(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings); +} + +void SerializationSparse::deserializeTextQuoted(IColumn &, ReadBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeTextQuoted' is not implemented for SerializationSparse"); +} + +void SerializationSparse::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_sparse = assert_cast(column); + nested->serializeTextCSV(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings); +} + +void SerializationSparse::deserializeTextCSV(IColumn &, ReadBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeTextCSV' is not implemented for SerializationSparse"); +} + +void SerializationSparse::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_sparse = assert_cast(column); + nested->serializeText(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings); +} + +void SerializationSparse::deserializeWholeText(IColumn &, ReadBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeWholeText' is not implemented for SerializationSparse"); +} + +void SerializationSparse::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_sparse = assert_cast(column); + nested->serializeTextJSON(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings); +} + +void SerializationSparse::deserializeTextJSON(IColumn &, ReadBuffer &, const FormatSettings &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'deserializeTextJSON' is not implemented for SerializationSparse"); +} + +void SerializationSparse::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto & column_sparse = assert_cast(column); + nested->serializeTextXML(column_sparse.getValuesColumn(), column_sparse.getValueIndex(row_num), ostr, settings); +} + } diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index 24a306de8ee..8d19c94c58b 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -1,11 +1,11 @@ #pragma once -#include +#include namespace DB { -class SerializationSparse final : public SerializationWrapper +class SerializationSparse final : public ISerialization { public: SerializationSparse(const SerializationPtr & nested_); @@ -40,8 +40,31 @@ public: DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const override; - // void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + 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; + void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + + void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + +private: + SerializationPtr nested; }; } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 0a0d1a8ec32..0657b1bd43d 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -33,6 +33,16 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_ return assert_cast(column).getColumn(idx); } +ISerialization::Kinds SerializationTuple::getKinds() const +{ + Kinds kinds(Kind::DEFAULT); + kinds.subcolumns.reserve(elems.size()); + for (const auto & elem : elems) + kinds.subcolumns.push_back(elem->getKind()); + + return kinds; +} + void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const { const auto & tuple = get(field); diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index afe3ab3a38f..f3aaa474f7a 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -13,7 +13,11 @@ public: using ElementSerializations = std::vector; SerializationTuple(const ElementSerializations & elems_, bool have_explicit_names_) - : elems(elems_), have_explicit_names(have_explicit_names_) {} + : elems(elems_), have_explicit_names(have_explicit_names_) + { + } + + Kinds getKinds() const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; diff --git a/src/DataTypes/Serializations/SerializationUUID.h b/src/DataTypes/Serializations/SerializationUUID.h index fee6cd31853..93bf166bbd9 100644 --- a/src/DataTypes/Serializations/SerializationUUID.h +++ b/src/DataTypes/Serializations/SerializationUUID.h @@ -1,3 +1,5 @@ +#pragma once + #include namespace DB diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 12363ac72af..4f72877ea36 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -6,6 +6,8 @@ namespace DB { +/// Wrapper for serialization, which calls methods, which are not overridden, from nested serialization. +/// You can inherit this class, when you need to override bunch of methods, to avoid boilerplate code. class SerializationWrapper : public ISerialization { protected: diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 81053a5929a..4f65df7b7a1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -280,7 +280,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , volume(volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) - , serialization_info(storage_.getSettings()->ratio_for_sparse_serialization) + , serialization_info(storage_.getSettings()->ratio_of_defaults_for_sparse_serialization) , part_type(part_type_) { incrementStateMetric(state); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 1c1745b9711..0059c27fbbd 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -11,7 +11,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , metadata_snapshot(metadata_snapshot_) , volume(data_part->volume) , part_path(data_part->isStoredOnDisk() ? data_part->getFullRelativePath() : "") - , new_serialization_info(data_part->storage.getSettings()->ratio_for_sparse_serialization) + , new_serialization_info(data_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization) { } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c97594b0d62..18d2b9cefb6 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -716,7 +716,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor bool need_remove_expired_values = false; bool force_ttl = false; - SerializationInfo new_serialization_info(data_settings->ratio_for_sparse_serialization); + SerializationInfo new_serialization_info(data_settings->ratio_of_defaults_for_sparse_serialization); for (const auto & part : parts) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 55d42fed20c..3cf0964efca 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -400,7 +400,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa const auto & data_settings = data.getSettings(); - SerializationInfo serialization_info(data_settings->ratio_for_sparse_serialization); + SerializationInfo serialization_info(data_settings->ratio_of_defaults_for_sparse_serialization); serialization_info.add(block); const auto & index_factory = MergeTreeIndexFactory::instance(); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 4e877980f1d..dcebaa2480d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -27,13 +27,13 @@ struct Settings; M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Data storing format settings. */ \ - M(UInt64, min_bytes_for_wide_part, 0, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ M(UInt64, min_bytes_for_compact_part, 0, "Experimental. Minimal uncompressed size in bytes to create part in compact format instead of saving it in RAM", 0) \ M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_for_sparse_serialization, 1.1, "", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index 12af231d18c..afee7a3a793 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -231 1 +374 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┠│ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference index 949d37ed27a..79302586e92 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference @@ -1,4 +1,4 @@ -84 +83 1 46 1 diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index 186f2feab79..cbdcf08ef7f 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_1_1_0 test_00961 b5fce9c4ef1ca42ce4ed027389c208d2 fc3b062b646cd23d4c23d7f5920f89ae da96ff1e527a8a1f908ddf2b1d0af239 +20000101_1_1_0 test_00961 3c1f8e1105208b1abfaf4cd66cad0fe7 3c19e0fca8269335c6134d5ac06f4b5a da96ff1e527a8a1f908ddf2b1d0af239 diff --git a/tests/queries/0_stateless/01780_column_sparse.reference b/tests/queries/0_stateless/01780_column_sparse.reference index 6823c57d410..cc3e2317635 100644 --- a/tests/queries/0_stateless/01780_column_sparse.reference +++ b/tests/queries/0_stateless/01780_column_sparse.reference @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS t_sparse; DROP TABLE IF EXISTS t_sparse_1; CREATE TABLE t_sparse (id UInt64, u UInt64, s String, arr1 Array(String), arr2 Array(UInt64)) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.1; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.1; INSERT INTO t_sparse SELECT number, if (number % 10 = 0, number, 0), @@ -15,8 +15,8 @@ FROM numbers (200); SELECT column, serialization_kind FROM system.parts_columns WHERE table = 't_sparse' AND database = currentDatabase() ORDER BY column; -arr1 Sparse -arr2 Sparse +arr1 Default +arr2 Default id Default s Sparse u Sparse @@ -140,7 +140,7 @@ SELECT arrayFilter(x -> x % 2 = 1, arr2) FROM t_sparse WHERE arr2 != [] LIMIT 5; [1] CREATE TABLE t_sparse_1 (id UInt64, v Int64) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0; INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0); SELECT * FROM t_sparse_1 ORDER BY v; 7 -2 diff --git a/tests/queries/0_stateless/01780_column_sparse.sql b/tests/queries/0_stateless/01780_column_sparse.sql index 3efa263d2a8..9ad2d213d25 100644 --- a/tests/queries/0_stateless/01780_column_sparse.sql +++ b/tests/queries/0_stateless/01780_column_sparse.sql @@ -5,7 +5,7 @@ DROP TABLE IF EXISTS t_sparse_1; CREATE TABLE t_sparse (id UInt64, u UInt64, s String, arr1 Array(String), arr2 Array(UInt64)) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.1; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.1; INSERT INTO t_sparse SELECT number, @@ -31,7 +31,7 @@ SELECT arrayFilter(x -> x % 2 = 1, arr2) FROM t_sparse WHERE arr2 != [] LIMIT 5; CREATE TABLE t_sparse_1 (id UInt64, v Int64) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0; INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0); diff --git a/tests/queries/0_stateless/01780_column_sparse_pk.sql b/tests/queries/0_stateless/01780_column_sparse_pk.sql index adb132232b8..c57988fa8d7 100644 --- a/tests/queries/0_stateless/01780_column_sparse_pk.sql +++ b/tests/queries/0_stateless/01780_column_sparse_pk.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS t_full_pk; CREATE TABLE t_sparse_pk (k UInt64, s String) ENGINE = MergeTree ORDER BY k -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.0, index_granularity = 1; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.0, index_granularity = 1; INSERT INTO t_sparse_pk VALUES (0, 'a'), (0, 'b'), (1, ''), (2, ''), (2, 'e'), (3, 'f'), (4, 'g'); @@ -16,11 +16,11 @@ DROP TABLE IF EXISTS t_sparse_pk; CREATE TABLE t_sparse_pk (k UInt64, v UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY k -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.0, index_granularity = 30; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.0, index_granularity = 30; CREATE TABLE t_full_pk (k UInt64, v UInt64) ENGINE = MergeTree ORDER BY k -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 1.1, index_granularity = 30; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1.1, index_granularity = 30; INSERT INTO t_sparse_pk SELECT number % 10, number % 4 = 0 FROM numbers(1000); INSERT INTO t_full_pk SELECT number % 10, number % 4 = 0 FROM numbers(1000); diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.sql b/tests/queries/0_stateless/01780_column_sparse_tuple.sql index ffb9430a0a4..df4fe080329 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.sql +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS sparse_tuple; CREATE TABLE sparse_tuple (id UInt64, t Tuple(a UInt64, s String)) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.5; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.5; INSERT INTO sparse_tuple SELECT number, (if (number % 20 = 0, number, 0), repeat('a', number % 10 + 1)) FROM numbers(1000); @@ -26,7 +26,7 @@ DROP TABLE IF EXISTS sparse_tuple; CREATE TABLE sparse_tuple (id UInt64, t Tuple(a UInt64, b Tuple(u UInt32, s String))) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_for_sparse_serialization = 0.5; +SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.5; INSERT INTO sparse_tuple SELECT number, (if (number % 20 = 0, number, 0), (if (number % 15 = 0, number, 0), repeat('a', number % 10 + 1))) FROM numbers(1000); From 78dc7bf8fe28d0ec6ca2e16614d7eee3a60b2280 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 15 May 2021 00:45:13 +0300 Subject: [PATCH 025/358] fix build --- src/Storages/MergeTree/DataPartsExchange.cpp | 4 +++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 ++- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 9 ++++++++- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index fb6f797aa2f..a383ec2a6ce 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -600,7 +600,9 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( metadata_snapshot->projections.get(projection_name).metadata, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {})); + CompressionCodecFactory::instance().get("NONE", {}), + new_data_part->serialization_info); + part_out.writePrefix(); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_projection_part); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index f5ecf60367a..4c638c25720 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -255,7 +255,7 @@ static void decrementTypeMetric(MergeTreeDataPartType type) IMergeTreeDataPart::IMergeTreeDataPart( - MergeTreeData & storage_, + const MergeTreeData & storage_, const String & name_, const VolumePtr & volume_, const std::optional & relative_path_, @@ -267,6 +267,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , volume(parent_part_ ? parent_part_->volume : volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) + , serialization_info(storage_.getSettings()->ratio_of_defaults_for_sparse_serialization) , part_type(part_type_) , parent_part(parent_part_) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index b74c4ba64f1..0cd5a6a2550 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -516,12 +516,16 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); + SerializationInfo serialization_info(data.getSettings()->ratio_of_defaults_for_sparse_serialization); + serialization_info.add(block); + MergedBlockOutputStream out( new_data_part, metadata_snapshot, columns, {}, - compression_codec); + compression_codec, + serialization_info); out.writePrefix(); out.writeWithPermutation(block, perm_ptr); @@ -560,6 +564,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempProjectionPart( const IMergeTreeDataPart * parent_part, size_t block_num) { + + + /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); From 5c7371333e2e880c491f333ea5f5ad18e00412bb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 15 May 2021 02:20:00 +0300 Subject: [PATCH 026/358] fix bugs in serializations after merge --- src/Core/Block.cpp | 2 +- src/DataTypes/IDataType.cpp | 9 +-------- .../Transforms/AggregatingInOrderTransform.cpp | 3 +-- 3 files changed, 3 insertions(+), 11 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 4905c8f80b7..79e32c26728 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -545,7 +545,7 @@ static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, cons actual_column = &column_const->getDataColumn(); } - if (getNameOfBaseColumn(*actual.column) != getNameOfBaseColumn(*expected.column)) + if (getNameOfBaseColumn(*actual_column) != getNameOfBaseColumn(*expected.column)) return on_error("Block structure mismatch in " + context_description + " stream: different columns:\n" + lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR); diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index be54d4d0b1a..2b844c05109 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -185,14 +185,7 @@ SerializationPtr IDataType::getSerialization(const IColumn & column) const if (column.isSparse()) return getSparseSerialization(); - ISerialization::Settings settings = - { - .num_rows = column.size(), - .num_default_rows = column.getNumberOfDefaultRows(IColumn::DEFAULT_ROWS_SEARCH_STEP), - .ratio_for_sparse_serialization = IColumn::DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION - }; - - return getSerialization(settings); + return getDefaultSerialization(); } SerializationPtr IDataType::getSerialization(const ISerialization::Kinds & kinds) const diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 531d5819acb..87d5bd07eb0 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -229,11 +229,10 @@ IProcessor::Status AggregatingInOrderTransform::prepare() input.setNeeded(); return Status::NeedData; } - current_chunk = input.pull(!is_consume_finished); - convertToFullIfSparse(current_chunk); assert(!is_consume_finished); current_chunk = input.pull(true /* set_not_needed */); + convertToFullIfSparse(current_chunk); return Status::Ready; } From 86d6673d615daacb0cc2071e4ac6f2d5ba6d6a3c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 16 May 2021 21:59:43 +0300 Subject: [PATCH 027/358] minor fixes --- src/Columns/ColumnSparse.h | 5 --- src/Columns/IColumn.cpp | 2 +- src/Columns/tests/gtest_column_sparse.cpp | 33 ++++++++++--------- src/Common/ya.make | 1 - src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp | 6 ---- src/DataTypes/DataTypeFixedString.cpp | 1 - src/DataTypes/DataTypeTuple.cpp | 1 + src/DataTypes/IDataType.cpp | 2 +- .../Serializations/ISerialization.cpp | 2 ++ .../Serializations/SerializationIP.cpp | 1 + .../Serializations/SerializationNothing.h | 2 ++ .../Serializations/SerializationTuple.cpp | 1 - src/Functions/IFunction.cpp | 2 +- src/Functions/IFunctionAdaptors.h | 2 +- src/Interpreters/Aggregator.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 3 -- tests/integration/test_merge_tree_s3/test.py | 4 +-- .../test_replicated_merge_tree_s3/test.py | 4 +-- .../test.py | 4 +-- 20 files changed, 36 insertions(+), 44 deletions(-) diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index cb7daa5482a..4167e6a7cc0 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -11,11 +11,6 @@ class Collator; namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - class ColumnSparse final : public COWHelper { private: diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index e3626fd2c82..76334e0da6e 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -56,7 +56,7 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows) } ssize_t offsets_diff = static_cast(total_rows) - current_offset; - if(offsets_diff > 1) + if (offsets_diff > 1) res->insertManyFrom(*this, 0, offsets_diff - 1); return res; diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index e6078417a94..ac6351a8b21 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -54,6 +54,9 @@ bool checkEquals(const IColumn & lhs, const IColumn & rhs) return true; } +// Can't use ErrorCodes, because of 'using namespace DB'. +constexpr int error_code = 12345; + constexpr size_t T = 5000; constexpr size_t MAX_ROWS = 10000; constexpr size_t sparse_ratios[] = {1, 2, 5, 10, 32, 50, 64, 100, 256, 500, 1000, 5000, 10000}; @@ -79,7 +82,7 @@ TEST(ColumnSparse, InsertRangeFrom) DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); DUMP_NON_DEFAULTS(full_dst); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + throw Exception(error_code, "Columns are unequal"); } }; @@ -102,7 +105,7 @@ TEST(ColumnSparse, InsertRangeFrom) test_case(n1, k1, n2, k2, from, to - from); } } - catch(const Exception & e) + catch (const Exception & e) { FAIL() << e.displayText(); } @@ -122,7 +125,7 @@ TEST(ColumnSparse, PopBack) DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); DUMP_NON_DEFAULTS(full_dst); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + throw Exception(error_code, "Columns are unequal"); } }; @@ -137,7 +140,7 @@ TEST(ColumnSparse, PopBack) test_case(n, k, m); } } - catch(const Exception & e) + catch (const Exception & e) { FAIL() << e.displayText(); } @@ -163,7 +166,7 @@ TEST(ColumnSparse, Filter) DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); DUMP_NON_DEFAULTS(full_dst); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + throw Exception(error_code, "Columns are unequal"); } }; @@ -178,7 +181,7 @@ TEST(ColumnSparse, Filter) test_case(n, k, m); } } - catch(const Exception & e) + catch (const Exception & e) { FAIL() << e.displayText(); } @@ -190,7 +193,7 @@ TEST(ColumnSparse, Permute) { auto [sparse_src, full_src] = createColumns(n, k); - PaddedPODArray perm(n); + IColumn::Permutation perm; std::iota(perm.begin(), perm.end(), 0); std::shuffle(perm.begin(), perm.end(), rng); @@ -210,7 +213,7 @@ TEST(ColumnSparse, Permute) DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); DUMP_NON_DEFAULTS(full_dst); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Columns are unequal"); + throw Exception(error_code, "Columns are unequal"); } }; @@ -225,7 +228,7 @@ TEST(ColumnSparse, Permute) test_case(n, k, limit); } } - catch(const Exception & e) + catch (const Exception & e) { FAIL() << e.displayText(); } @@ -250,7 +253,7 @@ TEST(ColumnSparse, CompareColumn) DUMP_COLUMN(full_src1); DUMP_COLUMN(sparse_src2); DUMP_COLUMN(full_src2); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Compare results are unequal"); + throw Exception(error_code, "Compare results are unequal"); } }; @@ -269,7 +272,7 @@ TEST(ColumnSparse, CompareColumn) test_case(n1, k1, n2, k2, row_num); } } - catch(const Exception & e) + catch (const Exception & e) { FAIL() << e.displayText(); } @@ -281,8 +284,8 @@ TEST(ColumnSparse, GetPermutation) { auto [sparse_src, full_src] = createColumns(n, k); - PaddedPODArray perm_sparse; - PaddedPODArray perm_full; + IColumn::Permutation perm_sparse; + IColumn::Permutation perm_full; sparse_src->getPermutation(reverse, limit, 1, perm_sparse); full_src->getPermutation(reverse, limit, 1, perm_full); @@ -303,7 +306,7 @@ TEST(ColumnSparse, GetPermutation) DUMP_COLUMN(sparse_sorted); DUMP_COLUMN(full_sorted); DUMP_NON_DEFAULTS(full_sorted); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Sorted columns are unequal"); + throw Exception(error_code, "Sorted columns are unequal"); } }; @@ -320,7 +323,7 @@ TEST(ColumnSparse, GetPermutation) test_case(n, k, limit, reverse); } } - catch(const Exception & e) + catch (const Exception & e) { FAIL() << e.displayText(); } diff --git a/src/Common/ya.make b/src/Common/ya.make index dde1e6ae013..f12b17827f7 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -18,7 +18,6 @@ PEERDIR( contrib/libs/openssl contrib/libs/poco/NetSSL_OpenSSL contrib/libs/re2 - contrib/libs/cxxsupp/libcxxabi-parts contrib/restricted/dragonbox ) diff --git a/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp b/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp index f0eeaa21be8..808aa43528e 100644 --- a/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp +++ b/src/DataTypes/DataTypeCustomIPv4AndIPv6.cpp @@ -5,12 +5,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; -} - void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory) { factory.registerSimpleDataTypeCustom("IPv4", [] diff --git a/src/DataTypes/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp index 4b6042febab..a40592ba023 100644 --- a/src/DataTypes/DataTypeFixedString.cpp +++ b/src/DataTypes/DataTypeFixedString.cpp @@ -25,7 +25,6 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int UNEXPECTED_AST_STRUCTURE; - extern const int TOO_LARGE_STRING_SIZE; } diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index e5dcc327d5c..33a763e8fcf 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 2b844c05109..3c916cb80cb 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -192,7 +192,7 @@ SerializationPtr IDataType::getSerialization(const ISerialization::Kinds & kinds { if (!kinds.subcolumns.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR,"Data type {} doesn't support " - "custom kinds of serialization for subcolumns ot doesn't have subcolumns at all.", getName()); + "custom kinds of serialization for subcolumns or doesn't have subcolumns at all.", getName()); if (kinds.main == ISerialization::Kind::SPARSE) return getSparseSerialization(); diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 978ee05f3c2..a0bc0947dfa 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -24,6 +24,8 @@ String ISerialization::kindToString(Kind kind) case Kind::SPARSE: return "Sparse"; } + + __builtin_unreachable(); } void ISerialization::Kinds::writeBinary(WriteBuffer & ostr) const diff --git a/src/DataTypes/Serializations/SerializationIP.cpp b/src/DataTypes/Serializations/SerializationIP.cpp index d14122671d9..ec49f960c77 100644 --- a/src/DataTypes/Serializations/SerializationIP.cpp +++ b/src/DataTypes/Serializations/SerializationIP.cpp @@ -10,6 +10,7 @@ namespace DB namespace ErrorCodes { extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; + extern const int ILLEGAL_COLUMN; } SerializationIPv4::SerializationIPv4(const SerializationPtr & nested_) diff --git a/src/DataTypes/Serializations/SerializationNothing.h b/src/DataTypes/Serializations/SerializationNothing.h index e9d25c62ee0..a7b26c117bc 100644 --- a/src/DataTypes/Serializations/SerializationNothing.h +++ b/src/DataTypes/Serializations/SerializationNothing.h @@ -1,3 +1,5 @@ +#pragma once + #include #include diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 0657b1bd43d..7893b5e3808 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -19,7 +19,6 @@ namespace ErrorCodes { extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH; extern const int NOT_FOUND_COLUMN_IN_BLOCK; - extern const int LOGICAL_ERROR; } diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 5fb4428227e..f1f25579f8b 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -467,7 +467,7 @@ ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & argu { const auto * column_sparse = checkAndGetColumn(arguments[i].column.get()); /// In rare case, when sparse column doesn't have default values, - /// it's more convinient to convert it to full before execution of function. + /// it's more convenient to convert it to full before execution of function. if (column_sparse && column_sparse->getNumberOfDefaults()) { sparse_column_position = i; diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 13886fb8660..9824d2a1659 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -35,7 +35,7 @@ private: const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const; ColumnPtr executeWithoutSparseColumns( - const ColumnsWithTypeAndName & args, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const; + const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, bool dry_run) const; }; class FunctionBaseAdaptor final : public IFunctionBase diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index f9e62bb5c50..54a47205d94 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -613,7 +613,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( if (inst->offsets) inst->batch_that->addBatchSinglePlace( inst->offsets[static_cast(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena); - else if(inst->has_sparse_arguments) + else if (inst->has_sparse_arguments) inst->batch_that->addBatchSparseSinglePlace(res + inst->state_offset, inst->batch_arguments, arena); else inst->batch_that->addBatchSinglePlace(rows, res + inst->state_offset, inst->batch_arguments, arena); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index f48feb48355..4c772d93152 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -403,7 +403,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, { const auto & serialization = serializations[name]; - if (!type.isValueRepresentedByNumber() || type.haveSubtypes() || serialization->getKind() != ISerialization::Kind::SPARSE) + if (!type.isValueRepresentedByNumber() || type.haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName()); auto disk = data_part->volume->getDisk(); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 0cd5a6a2550..6cbe772bddf 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -564,9 +564,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempProjectionPart( const IMergeTreeDataPart * parent_part, size_t block_num) { - - - /// Size of part would not be greater than block.bytes() + epsilon size_t expected_size = block.bytes(); diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index c0c05355def..32c48df3db0 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -64,8 +64,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 def random_string(length): diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 3b3540ef1b8..ab5fb41eec5 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -32,8 +32,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 def random_string(length): diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index 793abc53566..6d00a94733b 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -32,8 +32,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 def random_string(length): From 46226a559117a315dd0d7f2f764a92ec990572e4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 17 May 2021 02:17:01 +0300 Subject: [PATCH 028/358] fix unit tests --- src/Columns/tests/gtest_column_sparse.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index ac6351a8b21..8d720cbe0d4 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -193,7 +193,7 @@ TEST(ColumnSparse, Permute) { auto [sparse_src, full_src] = createColumns(n, k); - IColumn::Permutation perm; + IColumn::Permutation perm(n); std::iota(perm.begin(), perm.end(), 0); std::shuffle(perm.begin(), perm.end(), rng); From 76613a5dd10dac4913d04e899d689db863c7f8d3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 19 May 2021 04:48:46 +0300 Subject: [PATCH 029/358] serialization: better interfaces --- src/DataStreams/NativeBlockInputStream.cpp | 22 ++- src/DataStreams/NativeBlockOutputStream.cpp | 10 +- src/DataTypes/DataTypeTuple.cpp | 43 ----- src/DataTypes/DataTypeTuple.h | 4 +- src/DataTypes/IDataType.cpp | 41 +---- src/DataTypes/IDataType.h | 17 +- .../Serializations/ISerialization.cpp | 55 ++---- src/DataTypes/Serializations/ISerialization.h | 18 +- .../Serializations/SerializationInfo.cpp | 165 ++++++++++++++---- .../Serializations/SerializationInfo.h | 69 ++++++-- .../Serializations/SerializationTuple.cpp | 10 -- .../Serializations/SerializationTuple.h | 2 - src/Storages/MergeTree/IMergeTreeDataPart.cpp | 19 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 7 +- src/Storages/MergeTree/IMergeTreeReader.h | 3 + .../MergeTree/IMergedBlockOutputStream.cpp | 14 +- .../MergeTree/IMergedBlockOutputStream.h | 6 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 12 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartCompact.h | 2 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 2 +- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/MergeTreeDataPartWide.cpp | 11 +- .../MergeTree/MergeTreeDataPartWide.h | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 4 +- .../MergeTreeDataPartWriterCompact.h | 2 +- .../MergeTreeDataPartWriterOnDisk.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 12 +- .../MergeTree/MergeTreeReaderCompact.cpp | 20 ++- src/Storages/MergeTree/MergeTreeReaderWide.h | 3 - .../MergeTree/MergedBlockOutputStream.cpp | 14 +- .../MergeTree/MergedBlockOutputStream.h | 2 +- .../MergedColumnOnlyOutputStream.cpp | 10 +- .../MergeTree/MergedColumnOnlyOutputStream.h | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 20 +-- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageTinyLog.cpp | 2 +- .../0_stateless/01780_column_sparse.reference | 4 +- .../0_stateless/01780_column_sparse.sql | 4 +- .../01780_column_sparse_full.reference | 105 +++++++++++ .../0_stateless/01780_column_sparse_full.sql | 97 ++++++++++ .../0_stateless/01780_column_sparse_pk.sql | 9 +- .../0_stateless/01780_column_sparse_tuple.sql | 4 +- 46 files changed, 536 insertions(+), 330 deletions(-) create mode 100644 tests/queries/0_stateless/01780_column_sparse_full.reference create mode 100644 tests/queries/0_stateless/01780_column_sparse_full.sql diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index 7d3c288f6a5..1fde12fdc67 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -10,6 +10,7 @@ #include #include +#include #include @@ -133,6 +134,18 @@ Block NativeBlockInputStream::readImpl() rows = index_block_it->num_rows; } + /// Serialization + SerializationInfoPtr serialization_info; + if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + { + auto serialization_kinds = SerializationInfo::readKindsBinary(istr); + serialization_info = std::make_shared(rows, serialization_kinds); + } + else + { + serialization_info = std::make_shared(); + } + for (size_t i = 0; i < columns; ++i) { if (use_index) @@ -160,21 +173,14 @@ Block NativeBlockInputStream::readImpl() throw Exception("Index points to column with wrong type: corrupted index or data", ErrorCodes::INCORRECT_INDEX); } - /// Serialization - ISerialization::Kinds serialization_kinds; - if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) - serialization_kinds.readBinary(istr); - /// Data - auto serialization = column.type->getSerialization(serialization_kinds); + auto serialization = column.type->getSerialization(column.name, *serialization_info); ColumnPtr read_column = column.type->createColumn(*serialization); 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(*serialization, read_column, istr, rows, avg_value_size_hint); - /// TODO: maybe remove. - read_column = recursiveRemoveSparse(read_column); column.column = std::move(read_column); if (header) diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index a65a050c22a..f7143e3e19f 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -10,6 +10,7 @@ #include #include +#include #include namespace DB @@ -85,6 +86,14 @@ void NativeBlockOutputStream::write(const Block & block) writeVarUInt(rows, *index_ostr); } + + /// Serialization + if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + { + auto serialization_kinds = SerializationInfo::getKinds(block); + SerializationInfo::writeKindsBinary(serialization_kinds, ostr); + } + for (size_t i = 0; i < columns; ++i) { /// For the index. @@ -129,7 +138,6 @@ void NativeBlockOutputStream::write(const Block & block) else { serialization = column.type->getSerialization(*column.column); - serialization->getKinds().writeBinary(ostr); } /// Data diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 33a763e8fcf..333050e080f 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -331,19 +331,6 @@ SerializationPtr DataTypeTuple::doGetDefaultSerialization() const return std::make_shared(std::move(serializations), have_explicit_names); } -SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const StreamExistenceCallback & callback) const -{ - SerializationTuple::ElementSerializations serializations(elems.size()); - for (size_t i = 0; i < elems.size(); ++i) - { - auto subcolumn_name = Nested::concatenateName(column_name, names[i]); - auto serializaion = elems[i]->getSerialization(subcolumn_name, callback); - serializations[i] = std::make_shared(serializaion, names[i]); - } - - return std::make_shared(std::move(serializations), have_explicit_names); -} - SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const SerializationInfo & info) const { SerializationTuple::ElementSerializations serializations(elems.size()); @@ -357,36 +344,6 @@ SerializationPtr DataTypeTuple::getSerialization(const String & column_name, con return std::make_shared(std::move(serializations), have_explicit_names); } -SerializationPtr DataTypeTuple::getSerialization(const ISerialization::Kinds & kinds) const -{ - if (kinds.subcolumns.empty()) - return doGetDefaultSerialization(); - - if (kinds.subcolumns.size() != elems.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong nmber of kinds of serializations for subcolumns. " - "Expected: {}, got {}", elems.size(), kinds.subcolumns.size()); - - SerializationTuple::ElementSerializations serializations(elems.size()); - for (size_t i = 0; i < elems.size(); ++i) - { - SerializationPtr serialization; - if (const auto * elem_kinds = std::get_if(&kinds.subcolumns[i])) - { - serialization = elems[i]->getSerialization(*elem_kinds); - } - else - { - auto elem_kind = std::get(kinds.subcolumns[i]); - serialization = elem_kind == ISerialization::Kind::SPARSE - ? elems[i]->getSparseSerialization() : getDefaultSerialization(); - } - - serializations[i] = std::make_shared(serialization, names[i]); - } - - return std::make_shared(std::move(serializations), have_explicit_names); -} - static DataTypePtr create(const ASTPtr & arguments) { diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 2abaeb28753..7cf6c00f8be 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -56,14 +56,12 @@ public: DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; - SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const override; + // SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const override; SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const override; SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; - SerializationPtr getSerialization(const ISerialization::Kinds & kinds) const override; - SerializationPtr doGetDefaultSerialization() const override; const DataTypes & getElements() const { return elems; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 3c916cb80cb..fa6309493bc 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -170,31 +170,16 @@ SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_n SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const { - ISerialization::Settings settings = - { - .num_rows = info.getNumberOfRows(), - .num_default_rows = info.getNumberOfDefaultRows(column_name), - .ratio_for_sparse_serialization = info.getRatioForSparseSerialization() - }; - - return getSerialization(settings); -} - -SerializationPtr IDataType::getSerialization(const IColumn & column) const -{ - if (column.isSparse()) + auto kind = info.getKind(column_name); + if (supportsSparseSerialization() && kind == ISerialization::Kind::SPARSE) return getSparseSerialization(); return getDefaultSerialization(); } -SerializationPtr IDataType::getSerialization(const ISerialization::Kinds & kinds) const +SerializationPtr IDataType::getSerialization(const IColumn & column) const { - if (!kinds.subcolumns.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR,"Data type {} doesn't support " - "custom kinds of serialization for subcolumns or doesn't have subcolumns at all.", getName()); - - if (kinds.main == ISerialization::Kind::SPARSE) + if (column.isSparse()) return getSparseSerialization(); return getDefaultSerialization(); @@ -213,32 +198,20 @@ SerializationPtr IDataType::getSerialization(const ISerialization::Settings & se } // static -SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, const IDataType::StreamExistenceCallback & callback) +SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, const SerializationInfo & info) { if (column.isSubcolumn()) { auto base_serialization_getter = [&](const IDataType & subcolumn_type) { - return subcolumn_type.getSerialization(column.name, callback); + return subcolumn_type.getSerialization(column.name, info); }; auto type_in_storage = column.getTypeInStorage(); return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), base_serialization_getter); } - return column.type->getSerialization(column.name, callback); -} - -SerializationPtr IDataType::getSerialization(const String & column_name, const StreamExistenceCallback & callback) const -{ - if (supportsSparseSerialization()) - { - auto sparse_idx_name = escapeForFileName(column_name) + ".sparse.idx"; - if (callback(sparse_idx_name)) - return getSparseSerialization(); - } - - return getDefaultSerialization(); + return column.type->getSerialization(column.name, info); } DataTypePtr IDataType::getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 04281f0f5b8..466fbcfcbb9 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -74,35 +74,22 @@ public: SerializationPtr getDefaultSerialization() const; SerializationPtr getSparseSerialization() const; - /// Asks whether the stream with given name exists in table. - /// If callback returned true for all streams, which are required for - /// one of serialization types, that serialization will be chosen for reading. - /// If callback always returned false, the default serialization will be chosen. - using StreamExistenceCallback = std::function; using BaseSerializationGetter = std::function; - /// Chooses serialization for reading of one column or subcolumns by - /// checking existence of substreams using callback. - virtual SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const; - /// Returns serialization wrapper for reading one particular subcolumn of data type. virtual SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; - static SerializationPtr getSerialization( - const NameAndTypePair & column, - const StreamExistenceCallback & callback = [](const String &) { return false; }); - /// Chooses serialziation accordind to column content. virtual SerializationPtr getSerialization(const IColumn & column) const; /// Chooses serialization accosrding to collected information about content of columns. virtual SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; - virtual SerializationPtr getSerialization(const ISerialization::Kinds & kinds) const; - SerializationPtr getSerialization(const ISerialization::Settings & settings) const; + static SerializationPtr getSerialization(const NameAndTypePair & column, const SerializationInfo & info); + using StreamCallbackWithType = std::function; void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback, ISerialization::SubstreamPath & path) const; diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index a0bc0947dfa..4f1ae511bcc 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -15,6 +15,14 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +ISerialization::Kind ISerialization::getKind(const IColumn & column) +{ + if (column.isSparse()) + return Kind::SPARSE; + + return Kind::DEFAULT; +} + String ISerialization::kindToString(Kind kind) { switch (kind) @@ -28,47 +36,14 @@ String ISerialization::kindToString(Kind kind) __builtin_unreachable(); } -void ISerialization::Kinds::writeBinary(WriteBuffer & ostr) const +ISerialization::Kind ISerialization::stringToKind(const String & str) { - writeIntBinary(static_cast(main), ostr); - writeIntBinary(subcolumns.size(), ostr); - for (size_t i = 0; i < subcolumns.size(); ++i) - { - writeIntBinary(subcolumns[i].index(), ostr); - if (const auto * elem_kinds = std::get_if(&subcolumns[i])) - elem_kinds->writeBinary(ostr); - else - writeIntBinary(static_cast(main), ostr); - } -} - -void ISerialization::Kinds::readBinary(ReadBuffer & istr) -{ - readIntBinary(main, istr); - size_t num_subcolumns; - readIntBinary(num_subcolumns, istr); - subcolumns.reserve(num_subcolumns); - for (size_t i = 0; i < num_subcolumns; ++i) - { - size_t index; - readIntBinary(index, istr); - if (index == 0) - { - Kinds elem_kinds; - elem_kinds.readBinary(istr); - subcolumns.emplace_back(elem_kinds); - } - else if (index == 1) - { - Kind elem_kind; - readIntBinary(elem_kind, istr); - subcolumns.emplace_back(elem_kind); - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown index of variant: {}, must be 0 or 1", index); - } - } + if (str == "Default") + return Kind::DEFAULT; + else if (str == "Sparse") + return Kind::SPARSE; + else + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown serialization kind '{}'", str); } String ISerialization::Substream::toString() const diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 729ada2a150..d3b6d953408 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -42,25 +42,13 @@ public: enum class Kind : UInt8 { DEFAULT = 0, - SPARSE = 1 + SPARSE = 1, }; virtual Kind getKind() const { return Kind::DEFAULT; } + static Kind getKind(const IColumn & column); static String kindToString(Kind kind); - - struct Kinds - { - Kinds() = default; - Kinds(Kind main_) : main(main_) {} - - Kind main = Kind::DEFAULT; - std::vector> subcolumns; - - void writeBinary(WriteBuffer & ostr) const; - void readBinary(ReadBuffer & istr); - }; - - virtual Kinds getKinds() const { return Kinds(getKind()); } + static Kind stringToKind(const String & str); /** Binary serialization for range of values in column - for writing to disk/network, etc. * diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 1586dc712e9..01f5909631b 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include @@ -15,66 +16,107 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int NO_SUCH_COLUMN_IN_TABLE; + extern const int UNKNOWN_FORMAT_VERSION; } -SerializationInfo::SerializationInfo( +SerializationInfoBuilder::SerializationInfoBuilder( double ratio_for_sparse_serialization_, size_t default_rows_search_step_) : ratio_for_sparse_serialization(ratio_for_sparse_serialization_) , default_rows_search_step(default_rows_search_step_) + , info(std::make_shared()) { } -void SerializationInfo::add(const Block & block) +void SerializationInfoBuilder::add(const Block & block) { - number_of_rows += block.rows(); + info->number_of_rows += block.rows(); for (const auto & elem : block) { - default_rows[elem.name] += elem.column->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; + info->columns[elem.name].num_defaults += elem.column->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; for (const auto & subname : elem.type->getSubcolumnNames()) { auto subcolumn = elem.type->getSubcolumn(subname, *elem.column); auto full_name = Nested::concatenateName(elem.name, subname); - default_rows[full_name] += subcolumn->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; + info->columns[full_name].num_defaults += subcolumn->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; } } } -void SerializationInfo::add(const SerializationInfo & other) +void SerializationInfoBuilder::add(const SerializationInfo & other) { - number_of_rows += other.number_of_rows; - for (const auto & [name, num] : other.default_rows) - default_rows[name] += num; + info->number_of_rows += other.number_of_rows; + for (const auto & [name, column_info] : other.columns) + info->columns[name].num_defaults += column_info.num_defaults; } -void SerializationInfo::update(const SerializationInfo & other) +SerializationInfoPtr SerializationInfoBuilder::build() { - if (number_of_rows && number_of_rows != other.number_of_rows) + size_t total_rows = info->number_of_rows; + for (auto & [_, column_info] : info->columns) + { + double ratio = total_rows ? std::min(static_cast(column_info.num_defaults) / total_rows, 1.0) : 0.0; + if (ratio > ratio_for_sparse_serialization) + column_info.kind = ISerialization::Kind::SPARSE; + } + + return info; +} + +SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo & other) +{ + if (info->number_of_rows && info->number_of_rows != other.number_of_rows) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot update SerializationInfo with {} rows by SerializationInfo with {} rows", - number_of_rows, other.number_of_rows); + info->number_of_rows, other.number_of_rows); - number_of_rows = other.number_of_rows; - for (const auto & [name, num] : other.default_rows) - default_rows[name] = num; + for (auto & [name, column_info] : other.columns) + { + auto it = info->columns.find(name); + if (it == info->columns.end()) + info->columns[name] = column_info; + else + it->second.kind = column_info.kind; + } + + return info; +} + +SerializationInfo::SerializationInfo(size_t number_of_rows_, const NameToKind & kinds) + : number_of_rows(number_of_rows_) +{ + for (const auto & [name, kind] : kinds) + columns[name].kind = kind; +} + +ISerialization::Kind SerializationInfo::getKind(const String & column_name) const +{ + auto it = columns.find(column_name); + if (it == columns.end()) + return ISerialization::Kind::DEFAULT; + + return it->second.kind; } size_t SerializationInfo::getNumberOfDefaultRows(const String & column_name) const { - auto it = default_rows.find(column_name); - if (it == default_rows.end()) + auto it = columns.find(column_name); + if (it == columns.end()) return 0; - return it->second; + + return it->second.num_defaults; } namespace { -constexpr auto KEY_NUMBER_OF_ROWS = "number_of_rows"; -constexpr auto KEY_NUMBER_OF_default_rows = "number_of_default_rows"; -constexpr auto KEY_NUMBER = "number"; -constexpr auto KEY_NAME = "name"; constexpr auto KEY_VERSION = "version"; +constexpr auto KEY_NUMBER_OF_ROWS = "number_of_rows"; +constexpr auto KEY_COLUMNS = "columns"; +constexpr auto KEY_NUM_DEFAULTS = "num_defaults"; +constexpr auto KEY_KIND = "kind"; +constexpr auto KEY_NAME = "name"; } @@ -86,19 +128,21 @@ void SerializationInfo::fromJSON(const String & json_str) if (object->has(KEY_NUMBER_OF_ROWS)) number_of_rows = object->getValue(KEY_NUMBER_OF_ROWS); - if (object->has(KEY_NUMBER_OF_default_rows)) + if (object->has(KEY_COLUMNS)) { - auto array = object->getArray(KEY_NUMBER_OF_default_rows); + auto array = object->getArray(KEY_COLUMNS); for (const auto & elem : *array) { auto elem_object = elem.extract(); - if (!elem_object->has(KEY_NUMBER) || !elem_object->has(KEY_NAME)) + if (!elem_object->has(KEY_NAME) || !elem_object->has(KEY_NUM_DEFAULTS) || !elem_object->has(KEY_KIND)) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Missed field 'name' or 'number' in SerializationInfo of columns"); + "Missed field '{}' or '{}' or '{}' in SerializationInfo of columns", + KEY_NAME, KEY_NUM_DEFAULTS, KEY_KIND); auto name = elem_object->getValue(KEY_NAME); - auto number = elem_object->getValue(KEY_NUMBER); - default_rows[name] = number; + auto kind = elem_object->getValue(KEY_KIND); + auto num_defaults = elem_object->getValue(KEY_NUM_DEFAULTS); + columns[name] = {ISerialization::stringToKind(kind), num_defaults}; } } } @@ -110,15 +154,16 @@ String SerializationInfo::toJSON() const info.set(KEY_NUMBER_OF_ROWS, number_of_rows); Poco::JSON::Array column_infos; - for (const auto & [name, num] : default_rows) + for (const auto & [name, column_info] : columns) { - Poco::JSON::Object column_info; - column_info.set(KEY_NAME, name); - column_info.set(KEY_NUMBER, num); - column_infos.add(std::move(column_info)); + Poco::JSON::Object column_info_json; + column_info_json.set(KEY_NAME, name); + column_info_json.set(KEY_KIND, ISerialization::kindToString(column_info.kind)); + column_info_json.set(KEY_NUM_DEFAULTS, column_info.num_defaults); + column_infos.add(std::move(column_info_json)); } - info.set(KEY_NUMBER_OF_default_rows, std::move(column_infos)); + info.set(KEY_COLUMNS, std::move(column_infos)); std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM oss.exceptions(std::ios::failbit); @@ -127,16 +172,64 @@ String SerializationInfo::toJSON() const return oss.str(); } -void SerializationInfo::read(ReadBuffer & in) +void SerializationInfo::readText(ReadBuffer & in) { String json_str; readString(json_str, in); fromJSON(json_str); } -void SerializationInfo::write(WriteBuffer & out) const +void SerializationInfo::writeText(WriteBuffer & out) const { writeString(toJSON(), out); } +SerializationInfo::NameToKind SerializationInfo::getKinds(const Block & block) +{ + NameToKind kinds; + for (const auto & elem : block) + { + kinds[elem.name] = ISerialization::getKind(*elem.column); + for (const auto & subname : elem.type->getSubcolumnNames()) + { + auto full_name = Nested::concatenateName(elem.name, subname); + auto subcolumn = elem.type->getSubcolumn(subname, *elem.column); + kinds[full_name] = ISerialization::getKind(*subcolumn); + } + } + + return kinds; +} + +SerializationInfo::NameToKind SerializationInfo::readKindsBinary(ReadBuffer & in) +{ + size_t size = 0; + readVarUInt(size, in); + + NameToKind kinds; + kinds.reserve(size); + for (size_t i = 0; i < size; ++i) + { + String name; + UInt8 kind; + + readBinary(name, in); + readBinary(kind, in); + if (!kinds.emplace(name, static_cast(kind)).second) + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicated name '{}' found in serialization kinds", name); + } + + return kinds; +} + +void SerializationInfo::writeKindsBinary(const NameToKind & kinds, WriteBuffer & out) +{ + writeVarUInt(kinds.size(), out); + for (const auto & [name, kind] : kinds) + { + writeBinary(name, out); + writeBinary(static_cast(kind), out); + } +} + } diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 0af31f08063..ab87d18a2ed 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { @@ -8,34 +9,68 @@ namespace DB class SerializationInfo { public: + using NameToKind = std::unordered_map; + + SerializationInfo() = default; + SerializationInfo(size_t number_of_rows_, const NameToKind & kinds); + static constexpr auto version = 1; - - using NameToNumber = std::unordered_map; - - SerializationInfo( - double ratio_for_sparse_serialization_, - size_t default_rows_search_step_ = IColumn::DEFAULT_ROWS_SEARCH_STEP); - - void add(const Block & block); - void update(const SerializationInfo & other); - void add(const SerializationInfo & other); - size_t getNumberOfDefaultRows(const String & column_name) const; - size_t getNumberOfRows() const { return number_of_rows; } - double getRatioForSparseSerialization() const { return ratio_for_sparse_serialization; } + ISerialization::Kind getKind(const String & column_name) const; - void read(ReadBuffer & in); - void write(WriteBuffer & out) const; + size_t getNumberOfRows() const { return number_of_rows; } + + void readText(ReadBuffer & in); + void writeText(WriteBuffer & out) const; + + static NameToKind getKinds(const Block & block); + static NameToKind readKindsBinary(ReadBuffer & in); + static void writeKindsBinary(const NameToKind & kinds, WriteBuffer & out); private: void fromJSON(const String & json_str); String toJSON() const; + struct Column + { + ISerialization::Kind kind = ISerialization::Kind::DEFAULT; + size_t num_defaults = 0; + }; + + using NameToColumn = std::unordered_map; + + size_t number_of_rows = 0; + NameToColumn columns; + + friend class SerializationInfoBuilder; +}; + +using SerializationInfoPtr = std::shared_ptr; + +class SerializationInfoBuilder +{ +public: + SerializationInfoBuilder(); + SerializationInfoBuilder( + double ratio_for_sparse_serialization_, + size_t default_rows_search_step_ = IColumn::DEFAULT_ROWS_SEARCH_STEP); + + void add(const Block & block); + void add(const SerializationInfo & other); + + SerializationInfoPtr build(); + SerializationInfoPtr buildFrom(const SerializationInfo & other); + static SerializationInfoPtr buildFromBlock(const Block & block); + + double getRatioForSparseSerialization() const { return ratio_for_sparse_serialization; } + +private: double ratio_for_sparse_serialization; size_t default_rows_search_step; - size_t number_of_rows = 0; - NameToNumber default_rows; + SerializationInfoPtr info; }; +using SerializationInfoBuilderPtr = std::shared_ptr; + } diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 7893b5e3808..17971dcfcf6 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -32,16 +32,6 @@ static inline const IColumn & extractElementColumn(const IColumn & column, size_ return assert_cast(column).getColumn(idx); } -ISerialization::Kinds SerializationTuple::getKinds() const -{ - Kinds kinds(Kind::DEFAULT); - kinds.subcolumns.reserve(elems.size()); - for (const auto & elem : elems) - kinds.subcolumns.push_back(elem->getKind()); - - return kinds; -} - void SerializationTuple::serializeBinary(const Field & field, WriteBuffer & ostr) const { const auto & tuple = get(field); diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index f3aaa474f7a..455e6e290aa 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -17,8 +17,6 @@ public: { } - Kinds getKinds() const override; - 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/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 4c638c25720..6a12d35aa10 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -267,7 +267,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , volume(parent_part_ ? parent_part_->volume : volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) - , serialization_info(storage_.getSettings()->ratio_of_defaults_for_sparse_serialization) + , serialization_info(std::make_shared()) , part_type(part_type_) , parent_part(parent_part_) { @@ -291,7 +291,7 @@ IMergeTreeDataPart::IMergeTreeDataPart( , volume(parent_part_ ? parent_part_->volume : volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) - , serialization_info(storage_.getSettings()->ratio_of_defaults_for_sparse_serialization) + , serialization_info(std::make_shared()) , part_type(part_type_) , parent_part(parent_part_) { @@ -747,12 +747,7 @@ 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)) { - auto serialization = IDataType::getSerialization(part_column, - [&](const String & stream_name) - { - return volume->getDisk()->exists(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION); - }); - + auto serialization = IDataType::getSerialization(part_column, *serialization_info); String path_to_data_file; serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { @@ -1020,7 +1015,7 @@ void IMergeTreeDataPart::loadSerializationInfo() if (volume->getDisk()->exists(path)) { auto in = openForReading(volume->getDisk(), path); - serialization_info.read(*in); + serialization_info->readText(*in); } } @@ -1489,11 +1484,7 @@ bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metada SerializationPtr IMergeTreeDataPart::getSerializationForColumn(const NameAndTypePair & column) const { - return IDataType::getSerialization(column, - [&](const String & stream_name) - { - return checksums.files.count(stream_name + DATA_FILE_EXTENSION) != 0; - }); + return IDataType::getSerialization(column, *serialization_info); } String IMergeTreeDataPart::getUniqueId() const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 743a5e9f108..4228b62d3c4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -102,7 +102,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity = {}) const = 0; @@ -192,11 +192,10 @@ public: MergeTreeIndexGranularityInfo index_granularity_info; /// TODO: add comment - SerializationInfo serialization_info; + SerializationInfoPtr serialization_info; size_t rows_count = 0; - time_t modification_time = 0; /// When the part is removed from the working set. Changes once. mutable std::atomic remove_time { std::numeric_limits::max() }; @@ -398,7 +397,7 @@ public: /// part creation (using alter query with materialize_ttl setting). bool checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const; - /// Returns serialization for column according to files in which column is written in part. + /// Returns serialization for column according to serialization_info. SerializationPtr getSerializationForColumn(const NameAndTypePair & column) const; /// Return some uniq string for file diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 0771bc3d5cb..d1cdadde9af 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -85,6 +85,9 @@ protected: using ColumnPosition = std::optional; ColumnPosition findColumnForOffsets(const String & column_name) const; + using Serializations = std::map; + Serializations serializations; + friend class MergeTreeRangeReader::DelayedStream; private: diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 0059c27fbbd..cd9c85aabfd 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -4,14 +4,18 @@ namespace DB { + IMergedBlockOutputStream::IMergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, - const StorageMetadataPtr & metadata_snapshot_) + const StorageMetadataPtr & metadata_snapshot_, + const SerializationInfoPtr & input_serialization_info_) : storage(data_part->storage) , metadata_snapshot(metadata_snapshot_) , volume(data_part->volume) , part_path(data_part->isStoredOnDisk() ? data_part->getFullRelativePath() : "") - , new_serialization_info(data_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization) + , input_serialization_info(input_serialization_info_) + , new_serialization_info(std::make_shared( + data_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization)) { } @@ -32,11 +36,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( std::unordered_map serialziations; for (const NameAndTypePair & column : columns) { - auto serialization = IDataType::getSerialization(column, [&](const String & stream_name) - { - /// Checksums of data_part are not initialized here. - return checksums.files.count(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION) != 0; - }); + auto serialization = IDataType::getSerialization(column, *data_part->serialization_info); serialization->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 0146a74a5ea..2b943f74d65 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -14,7 +14,8 @@ class IMergedBlockOutputStream : public IBlockOutputStream public: IMergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, - const StorageMetadataPtr & metadata_snapshot_); + const StorageMetadataPtr & metadata_snapshot_, + const SerializationInfoPtr & input_serialization_info_); using WrittenOffsetColumns = std::set; @@ -43,7 +44,8 @@ protected: String part_path; IMergeTreeDataPart::MergeTreeWriterPtr writer; - SerializationInfo new_serialization_info; + SerializationInfoPtr input_serialization_info; + SerializationInfoBuilderPtr new_serialization_info; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 42960ec9025..23edb3916ea 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -732,7 +732,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor bool need_remove_expired_values = false; bool force_ttl = false; - SerializationInfo new_serialization_info(data_settings->ratio_of_defaults_for_sparse_serialization); + auto serialization_info_builder = std::make_shared(data_settings->ratio_of_defaults_for_sparse_serialization); for (const auto & part : parts) { @@ -744,9 +744,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor force_ttl = true; } - new_serialization_info.add(part->serialization_info); + serialization_info_builder->add(*part->serialization_info); } + auto input_serialization_info = serialization_info_builder->build(); + const auto & part_min_ttl = new_data_part->ttl_infos.part_min_ttl; if (part_min_ttl && part_min_ttl <= time_of_merge) need_remove_expired_values = true; @@ -942,7 +944,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor merging_columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec, - new_serialization_info, + input_serialization_info, blocks_are_granules_size}; merged_stream->readPrefix(); @@ -1044,7 +1046,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor rows_sources_read_buf.seek(0, 0); ColumnGathererStream column_gathered_stream( - column_name, column_type->getSerialization(column_name, new_serialization_info), + column_name, column_type->getSerialization(column_name, *input_serialization_info), column_part_streams, rows_sources_read_buf); MergedColumnOnlyOutputStream column_to( @@ -1052,7 +1054,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor metadata_snapshot, column_gathered_stream.getHeader(), compression_codec, - new_serialization_info, + input_serialization_info, /// we don't need to recalc indices here /// because all of them were already recalculated and written /// as key part of vertical merge diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 0962d4ff092..31de8eadf1a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -60,7 +60,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const { diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 3f7b9a7bbc1..8f2f27e0a91 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -51,7 +51,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 3754870db21..4af3b13aed3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -61,7 +61,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & /* indices_to_recalc */, const CompressionCodecPtr & /* default_codec */, - const SerializationInfo & /* serialization_info */, + const SerializationInfoPtr & /* serialization_info */, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & /* computed_index_granularity */) const { diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 9713d186ae6..3a767a5f4dc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -40,7 +40,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info, + const SerializationInfoPtr & serialization_info, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 2eb3e9f1b2c..5185e6ff7c7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -65,7 +65,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const { @@ -186,12 +186,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const std::optional marks_size; for (const NameAndTypePair & name_type : columns) { - auto serialization = IDataType::getSerialization(name_type, - [&](const String & stream_name) - { - return volume->getDisk()->exists(stream_name + DATA_FILE_EXTENSION); - }); - + auto serialization = IDataType::getSerialization(name_type, *serialization_info); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { auto file_path = path + ISerialization::getFileNameForStream(name_type, substream_path) + index_granularity_info.marks_file_extension; @@ -227,7 +222,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const }; bool res = true; - auto serialization = IDataType::getSerialization(column, check_stream_exists); + auto serialization = IDataType::getSerialization(column, *serialization_info); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index cfe94259f4f..a3a170ddd80 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -45,7 +45,7 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 55b2177aa5e..15a1f259629 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -16,7 +16,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, @@ -37,7 +37,7 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( serializations.reserve(columns_list.size()); for (const auto & column : columns_list) { - serializations.emplace(column.name, column.type->getDefaultSerialization()); + serializations.emplace(column.name, column.type->getSerialization(column.name, *serialization_info)); addStreams(column, storage_columns.getCodecDescOrDefault(column.name, default_codec)); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index e5500c11912..0acf5d243ad 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -15,7 +15,7 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index c9403c5db92..5ac5f6791e9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -73,7 +73,7 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const MergeTreeIndices & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter(data_part_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 8281b6d8982..89acbfc9b6f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -88,7 +88,7 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); @@ -126,7 +126,7 @@ protected: const String part_path; const String marks_file_extension; const CompressionCodecPtr default_codec; - const SerializationInfo serialization_info; + const SerializationInfoPtr serialization_info; const bool compute_granularity; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 4c772d93152..2f06662a13e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -75,7 +75,7 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, @@ -85,7 +85,7 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) { - serializations.emplace(it.name, it.type->getSerialization(it.name, serialization_info)); + serializations.emplace(it.name, it.type->getSerialization(it.name, *serialization_info)); addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec)); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 6cda78f14cf..7190c4c2655 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -24,7 +24,7 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, - const SerializationInfo & serialization_info_, + const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 6cbe772bddf..b090eeb7627 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -435,13 +435,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart( const auto & data_settings = data.getSettings(); - SerializationInfo serialization_info(data_settings->ratio_of_defaults_for_sparse_serialization); - serialization_info.add(block); + auto serialization_info = std::make_shared(data_settings->ratio_of_defaults_for_sparse_serialization); + serialization_info->add(block); const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot,columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), - compression_codec, serialization_info); + compression_codec, serialization_info->build()); bool sync_on_insert = data_settings->fsync_after_insert; @@ -516,8 +516,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); - SerializationInfo serialization_info(data.getSettings()->ratio_of_defaults_for_sparse_serialization); - serialization_info.add(block); + auto serialization_info = std::make_shared(data.getSettings()->ratio_of_defaults_for_sparse_serialization); + serialization_info->add(block); MergedBlockOutputStream out( new_data_part, @@ -525,7 +525,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl( columns, {}, compression_codec, - serialization_info); + serialization_info->build()); out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index da28f75b57f..8c244323fcd 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -65,6 +65,16 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( } column_positions[i] = std::move(position); + + if (column_from_part.isSubcolumn()) + { + auto name_in_storage = column_from_part.getNameInStorage(); + /// We have to read whole column and extract subcolumn. + serializations.emplace(name_in_storage, data_part->getSerializationForColumn( + {name_in_storage, column_from_part.getTypeInStorage()})); + } + + serializations.emplace(column_from_part.name, data_part->getSerializationForColumn(column_from_part)); } /// Do not use max_read_buffer_size, but try to lower buffer size with maximal size of granule to avoid reading much data. @@ -146,7 +156,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, continue; if (res_columns[i] == nullptr) - res_columns[i] = getColumnFromPart(*column_it).type->createColumn(); + res_columns[i] = getColumnFromPart(*column_it).type->createColumn(*serializations[column_it->name]); } while (read_rows < max_rows_to_read) @@ -221,17 +231,19 @@ void MergeTreeReaderCompact::readData( if (name_and_type.isSubcolumn()) { + auto name_in_storage = name_and_type.getNameInStorage(); auto type_in_storage = name_and_type.getTypeInStorage(); - ColumnPtr temp_column = type_in_storage->createColumn(); - auto serialization = type_in_storage->getDefaultSerialization(); + const auto & serialization = serializations[name_in_storage]; + ColumnPtr temp_column = type_in_storage->createColumn(*serialization); + serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr); column = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), *temp_column); } else { - auto serialization = type->getDefaultSerialization(); + const auto & serialization = serializations[name]; serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr); } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 1afbca4bf41..f4a021ce27f 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -34,10 +34,7 @@ public: private: using FileStreams = std::map>; - using Serializations = std::map; - FileStreams streams; - Serializations serializations; void addStreams(const NameAndTypePair & name_and_type, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 4bb0d949744..6c7dfc69cf1 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -20,9 +20,9 @@ MergedBlockOutputStream::MergedBlockOutputStream( const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, - const SerializationInfo & serialization_info, + const SerializationInfoPtr & input_serialization_info_, bool blocks_are_granules_size) - : IMergedBlockOutputStream(data_part, metadata_snapshot_) + : IMergedBlockOutputStream(data_part, metadata_snapshot_, input_serialization_info_) , columns_list(columns_list_) , default_codec(default_codec_) { @@ -36,7 +36,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( if (!part_path.empty()) volume->getDisk()->createDirectories(part_path); - writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, serialization_info, writer_settings); + writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, input_serialization_info, writer_settings); } /// If data is pre-sorted. @@ -85,7 +85,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( else part_columns = *total_columns_list; - new_part->serialization_info.update(new_serialization_info); + new_part->serialization_info = new_serialization_info->buildFrom(*input_serialization_info); if (new_part->isStoredOnDisk()) finalizePartOnDisk(new_part, part_columns, checksums, sync); @@ -174,11 +174,11 @@ void MergedBlockOutputStream::finalizePartOnDisk( removeEmptyColumnsFromPart(new_part, part_columns, checksums); - if (new_part->serialization_info.getNumberOfRows() > 0) + if (new_part->serialization_info->getNumberOfRows() > 0) { auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); HashingWriteBuffer out_hashing(*out); - new_part->serialization_info.write(out_hashing); + new_part->serialization_info->writeText(out_hashing); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash(); out->finalize(); @@ -225,7 +225,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm return; writer->write(block, permutation); - new_serialization_info.add(block); + new_serialization_info->add(block); rows_count += rows; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 31e64193730..c7789600baf 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -19,7 +19,7 @@ public: const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, - const SerializationInfo & serialization_info, + const SerializationInfoPtr & input_serialization_info_, bool blocks_are_granules_size = false); Block getHeader() const override { return metadata_snapshot->getSampleBlock(); } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 4324f358127..13d4abbc62c 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -14,12 +14,12 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec, - const SerializationInfo & serialization_info, + const SerializationInfoPtr & input_serialization_info_, const MergeTreeIndices & indices_to_recalc, WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) - : IMergedBlockOutputStream(data_part, metadata_snapshot_) + : IMergedBlockOutputStream(data_part, metadata_snapshot_, input_serialization_info_) , header(header_) { const auto & global_settings = data_part->storage.getContext()->getSettings(); @@ -36,7 +36,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( metadata_snapshot_, indices_to_recalc, default_codec, - serialization_info, + input_serialization_info, std::move(writer_settings), index_granularity); @@ -53,7 +53,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) return; writer->write(block, nullptr); - new_serialization_info.add(block); + new_serialization_info->add(block); } void MergedColumnOnlyOutputStream::writeSuffix() @@ -85,7 +85,7 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( all_checksums.files.erase(removed_file); new_part->setColumns(columns); - new_part->serialization_info.update(new_serialization_info); + new_part->serialization_info = new_serialization_info->buildFrom(*input_serialization_info); return checksums; } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index cc911db20f9..fd8eea92fc9 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -18,7 +18,7 @@ public: const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec_, - const SerializationInfo & serialization_info, + const SerializationInfoPtr & input_serialization_info_, const MergeTreeIndices & indices_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, const MergeTreeIndexGranularity & index_granularity = {}, diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 8f0847fc106..158bc9600b9 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -99,6 +99,13 @@ IMergeTreeDataPart::Checksums checkDataPart( }; }; + auto serialization_info = std::make_shared(); + + { + auto serialization_file = disk->readFile(path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME); + serialization_info->readText(*serialization_file); + } + /// This function calculates only checksum of file content (compressed or uncompressed). /// It also calculates checksum of projections. auto checksum_file = [&](const String & file_path, const String & file_name) @@ -132,11 +139,7 @@ IMergeTreeDataPart::Checksums checkDataPart( const NamesAndTypesList & projection_columns_list = projection->getColumns(); for (const auto & projection_column : projection_columns_list) { - auto serialization = IDataType::getSerialization(projection_column, [&](const String & stream_name) - { - return disk->exists(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION); - }); - + auto serialization = IDataType::getSerialization(projection_column, *serialization_info); serialization->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { @@ -210,12 +213,7 @@ IMergeTreeDataPart::Checksums checkDataPart( { for (const auto & column : columns_list) { - auto serialization = IDataType::getSerialization(column, - [&](const String & stream_name) - { - return disk->exists(stream_name + IMergeTreeDataPart::DATA_FILE_EXTENSION); - }); - + auto serialization = IDataType::getSerialization(column, *serialization_info); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path) + ".bin"; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 8ed68e0b44d..ebb2ffc38a6 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -168,7 +168,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = IDataType::getSerialization(name_and_type); + auto serialization = name_and_type.type->getDefaultSerialization(); auto create_stream_getter = [&](bool stream_for_prefix) { diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 41c2961e929..31d5aa3198c 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -166,7 +166,7 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = IDataType::getSerialization(name_and_type); + auto serialization = name_and_type.type->getDefaultSerialization(); settings.getter = [&] (const ISerialization::SubstreamPath & path) -> ReadBuffer * { diff --git a/tests/queries/0_stateless/01780_column_sparse.reference b/tests/queries/0_stateless/01780_column_sparse.reference index cc3e2317635..850ebcee349 100644 --- a/tests/queries/0_stateless/01780_column_sparse.reference +++ b/tests/queries/0_stateless/01780_column_sparse.reference @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS t_sparse; DROP TABLE IF EXISTS t_sparse_1; CREATE TABLE t_sparse (id UInt64, u UInt64, s String, arr1 Array(String), arr2 Array(UInt64)) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.1; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.1; INSERT INTO t_sparse SELECT number, if (number % 10 = 0, number, 0), @@ -140,7 +140,7 @@ SELECT arrayFilter(x -> x % 2 = 1, arr2) FROM t_sparse WHERE arr2 != [] LIMIT 5; [1] CREATE TABLE t_sparse_1 (id UInt64, v Int64) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0; INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0); SELECT * FROM t_sparse_1 ORDER BY v; 7 -2 diff --git a/tests/queries/0_stateless/01780_column_sparse.sql b/tests/queries/0_stateless/01780_column_sparse.sql index 9ad2d213d25..6dc53cf07c6 100644 --- a/tests/queries/0_stateless/01780_column_sparse.sql +++ b/tests/queries/0_stateless/01780_column_sparse.sql @@ -5,7 +5,7 @@ DROP TABLE IF EXISTS t_sparse_1; CREATE TABLE t_sparse (id UInt64, u UInt64, s String, arr1 Array(String), arr2 Array(UInt64)) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.1; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.1; INSERT INTO t_sparse SELECT number, @@ -31,7 +31,7 @@ SELECT arrayFilter(x -> x % 2 = 1, arr2) FROM t_sparse WHERE arr2 != [] LIMIT 5; CREATE TABLE t_sparse_1 (id UInt64, v Int64) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0; INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), (7, -2), (8, 0), (9, 0), (10, 4), (11, 0); diff --git a/tests/queries/0_stateless/01780_column_sparse_full.reference b/tests/queries/0_stateless/01780_column_sparse_full.reference new file mode 100644 index 00000000000..eedc070dd37 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_full.reference @@ -0,0 +1,105 @@ +all_1_1_0 id Default +all_1_1_0 s Sparse +all_1_1_0 u Sparse +all_2_2_0 id Default +all_2_2_0 s Default +all_2_2_0 u Default +0 0 +0 0 +1 0 +1 1 +====== +0 0 +0 0 +1 0 +1 1 +====== +990 990 +980 980 980 +970 970 +====== +990 990 +980 980 980 +970 970 +====== +0 58413 +1 57920 +2 57917 +====== +507 +====== +0 [0,2,1,3] +1 [0,2,1,3] +2 [0,2,1,3] +3 [0,2,1,3] +4 [0,2,1,3] +8 +====== +0 0 0 +0 0 0 +0 0 0 +0 0 0 +1 1 1 +====== +58413 +57920 +57917 + +174250 +====== +174250 +58413 +57920 +57917 +====== +174250 +58413 +57920 +57917 +====== +508413 +57920 +57917 +====== +1075 +====== +1077 +====== +0 +1 +2 +3 +4 +====== +0 0 0 +0 0 0 +1 0 +2 0 +3 0 +====== +0 0 0 +0 0 0 +1 0 +1 1 1 +2 0 +====== +0 0 0 +0 0 0 +1 0 +2 0 +2 2 2 +====== +0 0 0 +0 0 0 +1 1 1 +1 0 +2 2 2 +====== +0 0 0 +0 0 0 +0 0 0 +0 0 0 +0 0 0 +id Default +s Sparse +u Sparse diff --git a/tests/queries/0_stateless/01780_column_sparse_full.sql b/tests/queries/0_stateless/01780_column_sparse_full.sql new file mode 100644 index 00000000000..b51f8f7ce4e --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_full.sql @@ -0,0 +1,97 @@ +-- This test checks, that common SQL operations work +-- with mixed columns (sparse and full) in table. + +DROP TABLE IF EXISTS t_sparse_full; + +CREATE TABLE t_sparse_full (id UInt64, u UInt64, s String) +ENGINE = MergeTree ORDER BY id +SETTINGS index_granularity = 32, +ratio_of_defaults_for_sparse_serialization = 0.1; + +SYSTEM STOP MERGES t_sparse_full; + +INSERT INTO t_sparse_full +SELECT + number, + if (number % 10 = 0, number, 0), + if (number % 7 = 0, toString(number), '') +FROM numbers(1000); + +INSERT INTO t_sparse_full +SELECT + number, + number, + toString(number) +FROM numbers(500); + +SELECT name, column, serialization_kind +FROM system.parts_columns WHERE table = 't_sparse_full' AND database = currentDatabase() AND active +ORDER BY name, column; + +SELECT id, u FROM t_sparse_full ORDER BY id, u LIMIT 4; +SELECT '======'; +SELECT id, u FROM t_sparse_full ORDER BY id, u LIMIT 4 SETTINGS optimize_read_in_order = 0; +SELECT '======'; +SELECT id, u, s FROM t_sparse_full ORDER BY u DESC LIMIT 3; +SELECT '======'; +SELECT id, u, s FROM t_sparse_full WHERE u != 0 ORDER BY u DESC LIMIT 3; +SELECT '======'; +SELECT id % 3 AS k, sum(u) FROM t_sparse_full WHERE u != 0 GROUP BY k ORDER BY k; +SELECT '======'; +SELECT uniqExact(u) FROM t_sparse_full WHERE s != ''; +SELECT '======'; +SELECT toUInt32(s) % 5 AS k, groupUniqArray(u % 4) FROM t_sparse_full WHERE s != '' GROUP BY k ORDER BY k; +SELECT max(range(id % 10)[u]) FROM t_sparse_full; +SELECT '======'; +SELECT id, u, s FROM remote('127.0.0.{1,2}', currentDatabase(), t_sparse_full) ORDER BY id LIMIT 5; +SELECT '======'; +SELECT sum(u) FROM t_sparse_full GROUP BY id % 3 AS k WITH TOTALS ORDER BY k; +SELECT '======'; +SELECT sum(u) FROM t_sparse_full GROUP BY id % 3 AS k WITH ROLLUP ORDER BY k; +SELECT '======'; +SELECT sum(u) FROM t_sparse_full GROUP BY id % 3 AS k WITH CUBE ORDER BY k; +SELECT '======'; +SELECT sum(id) FROM t_sparse_full GROUP BY u % 3 AS k ORDER BY k; +SELECT '======'; +SELECT count() FROM t_sparse_full WHERE u % 4 = 0; +SELECT '======'; +SELECT count() FROM t_sparse_full WHERE u IN (SELECT u FROM t_sparse_full WHERE id % 4 = 2); +SELECT '======'; +SELECT DISTINCT u FROM t_sparse_full ORDER BY id LIMIT 5; + +SELECT '======'; + +SELECT id, u, s FROM t_sparse_full INNER JOIN +( + SELECT number * 3 AS u FROM numbers(10) +) AS t1 USING(u) ORDER BY id, u, s LIMIT 5; + +SELECT '======'; + +SELECT id, u, s FROM t_sparse_full FULL JOIN +( + SELECT number * 3 AS u FROM numbers(10) +) AS t1 USING(u) ORDER BY id, u, s LIMIT 5; + +SELECT '======'; + +SELECT id, u, s FROM (SELECT number * 2 AS u FROM numbers(10)) AS t1 +INNER JOIN t_sparse_full USING(u) ORDER BY id, u, s LIMIT 5; + +SELECT '======'; + +SELECT id, u, s FROM (SELECT number * 2 AS u FROM numbers(10)) AS t1 +FULL JOIN t_sparse_full USING(u) ORDER BY id LIMIT 5; + +SELECT '======'; + +SELECT id, u, s FROM (SELECT u FROM t_sparse_full) AS t1 +FULL JOIN t_sparse_full USING(u) ORDER BY id, u, s LIMIT 5; + +SYSTEM START MERGES t_sparse_full; + +OPTIMIZE TABLE t_sparse_full FINAL; + +SELECT column, serialization_kind +FROM system.parts_columns WHERE table = 't_sparse_full' AND database = currentDatabase() AND active +ORDER BY name, column; diff --git a/tests/queries/0_stateless/01780_column_sparse_pk.sql b/tests/queries/0_stateless/01780_column_sparse_pk.sql index c57988fa8d7..a162f7bdfce 100644 --- a/tests/queries/0_stateless/01780_column_sparse_pk.sql +++ b/tests/queries/0_stateless/01780_column_sparse_pk.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS t_full_pk; CREATE TABLE t_sparse_pk (k UInt64, s String) ENGINE = MergeTree ORDER BY k -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.0, index_granularity = 1; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.0, index_granularity = 1; INSERT INTO t_sparse_pk VALUES (0, 'a'), (0, 'b'), (1, ''), (2, ''), (2, 'e'), (3, 'f'), (4, 'g'); @@ -16,11 +16,11 @@ DROP TABLE IF EXISTS t_sparse_pk; CREATE TABLE t_sparse_pk (k UInt64, v UInt64 CODEC(NONE)) ENGINE = MergeTree ORDER BY k -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.0, index_granularity = 30; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.0, index_granularity = 30; CREATE TABLE t_full_pk (k UInt64, v UInt64) ENGINE = MergeTree ORDER BY k -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 1.1, index_granularity = 30; +SETTINGS ratio_of_defaults_for_sparse_serialization = 1.1, index_granularity = 30; INSERT INTO t_sparse_pk SELECT number % 10, number % 4 = 0 FROM numbers(1000); INSERT INTO t_full_pk SELECT number % 10, number % 4 = 0 FROM numbers(1000); @@ -28,9 +28,6 @@ INSERT INTO t_full_pk SELECT number % 10, number % 4 = 0 FROM numbers(1000); INSERT INTO t_sparse_pk SELECT number % 10, number % 6 = 0 FROM numbers(1000); INSERT INTO t_full_pk SELECT number % 10, number % 6 = 0 FROM numbers(1000); - -set max_threads = 1; - SELECT count(v), sum(v) FROM t_sparse_pk WHERE k = 0; SELECT count(v), sum(v) FROM t_full_pk WHERE k = 0; diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.sql b/tests/queries/0_stateless/01780_column_sparse_tuple.sql index df4fe080329..847e95d1fd8 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.sql +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS sparse_tuple; CREATE TABLE sparse_tuple (id UInt64, t Tuple(a UInt64, s String)) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.5; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; INSERT INTO sparse_tuple SELECT number, (if (number % 20 = 0, number, 0), repeat('a', number % 10 + 1)) FROM numbers(1000); @@ -26,7 +26,7 @@ DROP TABLE IF EXISTS sparse_tuple; CREATE TABLE sparse_tuple (id UInt64, t Tuple(a UInt64, b Tuple(u UInt32, s String))) ENGINE = MergeTree ORDER BY tuple() -SETTINGS min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0.5; +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; INSERT INTO sparse_tuple SELECT number, (if (number % 20 = 0, number, 0), (if (number % 15 = 0, number, 0), repeat('a', number % 10 + 1))) FROM numbers(1000); From a06f2fed9a589c43247f6bd77428c2adf8b546a8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 19 May 2021 19:08:17 +0300 Subject: [PATCH 030/358] serializations: fix mutations --- src/DataTypes/Serializations/SerializationInfo.cpp | 5 ----- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 1 + src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 7 ++++--- src/Storages/MergeTree/checkDataPart.cpp | 4 +++- 6 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 01f5909631b..86e8fb93fed 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -66,11 +66,6 @@ SerializationInfoPtr SerializationInfoBuilder::build() SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo & other) { - if (info->number_of_rows && info->number_of_rows != other.number_of_rows) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot update SerializationInfo with {} rows by SerializationInfo with {} rows", - info->number_of_rows, other.number_of_rows); - for (auto & [name, column_info] : other.columns) { auto it = info->columns.find(name); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 6a12d35aa10..e0877ff9574 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1009,7 +1009,7 @@ void IMergeTreeDataPart::loadUUID() } } -void IMergeTreeDataPart::loadSerializationInfo() +void IMergeTreeDataPart::loadSerializationInfo() const { String path = getFullRelativePath() + SERIALIZATION_FILE_NAME; if (volume->getDisk()->exists(path)) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 4228b62d3c4..567e931a3da 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -462,7 +462,7 @@ private: /// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros void loadTTLInfos(); - void loadSerializationInfo(); + void loadSerializationInfo() const; void loadPartitionAndMinMaxIndex(); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 23edb3916ea..01ab592ada2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1253,6 +1253,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor new_data_part->uuid = future_part.uuid; new_data_part->is_temp = true; new_data_part->ttl_infos = source_part->ttl_infos; + new_data_part->serialization_info = source_part->serialization_info; /// It shouldn't be changed by mutation. new_data_part->index_granularity_info = source_part->index_granularity_info; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 8c244323fcd..f5ebee6b2e4 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -155,8 +155,9 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, if (!column_positions[i]) continue; + auto column_from_part = getColumnFromPart(*column_it); if (res_columns[i] == nullptr) - res_columns[i] = getColumnFromPart(*column_it).type->createColumn(*serializations[column_it->name]); + res_columns[i] = column_from_part.type->createColumn(*serializations.at(column_from_part.name)); } while (read_rows < max_rows_to_read) @@ -234,7 +235,7 @@ void MergeTreeReaderCompact::readData( auto name_in_storage = name_and_type.getNameInStorage(); auto type_in_storage = name_and_type.getTypeInStorage(); - const auto & serialization = serializations[name_in_storage]; + const auto & serialization = serializations.at(name_in_storage); ColumnPtr temp_column = type_in_storage->createColumn(*serialization); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); @@ -243,7 +244,7 @@ void MergeTreeReaderCompact::readData( } else { - const auto & serialization = serializations[name]; + const auto & serialization = serializations.at(name); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr); } diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 158bc9600b9..ba59e4e419a 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -100,9 +100,11 @@ IMergeTreeDataPart::Checksums checkDataPart( }; auto serialization_info = std::make_shared(); + auto serialization_path = path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME; + if (disk->exists(serialization_path)) { - auto serialization_file = disk->readFile(path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME); + auto serialization_file = disk->readFile(serialization_path); serialization_info->readText(*serialization_file); } From cd37cd32ef284d5eef57b0d0570e20714b7a5154 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 19 May 2021 23:07:46 +0300 Subject: [PATCH 031/358] fix StorageLog with subcolumns --- src/Storages/StorageLog.cpp | 3 ++- src/Storages/StorageTinyLog.cpp | 3 ++- .../00753_system_columns_and_system_tables.reference | 2 +- .../00961_checksums_in_system_parts_columns_table.reference | 2 +- 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index ebb2ffc38a6..5d7928acd1c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -15,6 +15,7 @@ #include #include +#include #include #include @@ -168,7 +169,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = name_and_type.type->getDefaultSerialization(); + auto serialization = IDataType::getSerialization(name_and_type, {}); auto create_stream_getter = [&](bool stream_for_prefix) { diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 31d5aa3198c..56befec47ea 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -21,6 +21,7 @@ #include #include +#include #include #include @@ -166,7 +167,7 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = name_and_type.type->getDefaultSerialization(); + auto serialization = IDataType::getSerialization(name_and_type, {}); settings.getter = [&] (const ISerialization::SubstreamPath & path) -> ReadBuffer * { diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference index afee7a3a793..7da6b35beaf 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -374 1 +428 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┠│ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index cbdcf08ef7f..d4f3098ed1e 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_1_1_0 test_00961 3c1f8e1105208b1abfaf4cd66cad0fe7 3c19e0fca8269335c6134d5ac06f4b5a da96ff1e527a8a1f908ddf2b1d0af239 +20000101_1_1_0 test_00961 8f7bf8151cbcee40008cb31dddf60e5f e4118be4d9f9fe6d6b68c2cab74b4a11 da96ff1e527a8a1f908ddf2b1d0af239 From 8718044dbd81a9603464cadca5c9debc3133aef8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 19 May 2021 23:25:55 +0300 Subject: [PATCH 032/358] fix style check --- src/Columns/ColumnNullable.h | 2 +- src/Columns/ColumnSparse.cpp | 8 ++++---- src/DataTypes/DataTypeTuple.cpp | 1 - src/DataTypes/Serializations/SerializationInfo.cpp | 3 +-- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 0a8ed4c0458..86cbc7b9d17 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -173,7 +173,7 @@ public: void checkConsistency() const; size_t getNumberOfDefaultRows(size_t step) const override; - void getIndicesOfNonDefaultValues(Offsets & offsets, size_t from, size_t limit) const override; + void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const override; private: diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 5ca818fa148..22e378b7f7c 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -432,7 +432,7 @@ int ColumnSparse::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs bool ColumnSparse::hasEqualValues() const { - return offsets->size() == 0; + return offsets->empty(); } void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const @@ -638,8 +638,8 @@ void ColumnSparse::getExtremes(Field & min, Field & max) const void ColumnSparse::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const { const auto & offsets_data = getOffsetsData(); - auto start = from ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from) : offsets_data.begin(); - auto end = limit ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from + limit) : offsets_data.end(); + const auto * start = from ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from) : offsets_data.begin(); + const auto * end = limit ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from + limit) : offsets_data.end(); indices.assign(start, end); } @@ -701,7 +701,7 @@ size_t ColumnSparse::getValueIndex(size_t n) const assert(n < _size); const auto & offsets_data = getOffsetsData(); - auto it = std::lower_bound(offsets_data.begin(), offsets_data.end(), n); + const auto * it = std::lower_bound(offsets_data.begin(), offsets_data.end(), n); if (it == offsets_data.end() || *it != n) return 0; diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 333050e080f..44e1d5d874d 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -36,7 +36,6 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH; - extern const int LOGICAL_ERROR; } diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 86e8fb93fed..f3d401afad3 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -16,8 +16,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int NO_SUCH_COLUMN_IN_TABLE; - extern const int UNKNOWN_FORMAT_VERSION; + extern const int INCORRECT_DATA; } SerializationInfoBuilder::SerializationInfoBuilder( From 9412a3a894ea57a41a70b338d9b4c22115697f3c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 21 May 2021 01:06:41 +0300 Subject: [PATCH 033/358] add perf test for sparse column --- tests/performance/sparse_column.xml | 58 +++++++++++++++++++++++++++++ 1 file changed, 58 insertions(+) create mode 100644 tests/performance/sparse_column.xml diff --git a/tests/performance/sparse_column.xml b/tests/performance/sparse_column.xml new file mode 100644 index 00000000000..6523d37df44 --- /dev/null +++ b/tests/performance/sparse_column.xml @@ -0,0 +1,58 @@ + + + + serialization + + sparse + full + + + + ratio + + 10 + 100 + 1000 + + + + + + CREATE TABLE test_full_{ratio} (id UInt64, u8 UInt8, u64 UInt64, str String) + ENGINE = MergeTree ORDER BY id + + + + CREATE TABLE test_sparse_{ratio} (id UInt64, u8 UInt8, u64 UInt64, str String) + ENGINE = MergeTree ORDER BY id + SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9 + + + SYSTEM STOP MERGES test_{serialization}_{ratio} + + + INSERT INTO test_{serialization}_{ratio} SELECT + number, + number % {ratio} = 0 ? rand(1) : 0, + number % {ratio} = 0 ? rand(2) : 0, + number % {ratio} = 0 ? randomPrintableASCII(64, 3) : '' + FROM numbers(100000000) + + + SELECT u8 FROM test_{serialization}_{ratio} FORMAT Null + SELECT u64 FROM test_{serialization}_{ratio} FORMAT Null + SELECT str FROM test_{serialization}_{ratio} FORMAT Null + + SELECT erf(u64) FROM test_{serialization}_{ratio} FORMAT Null + SELECT lower(str) FROM test_{serialization}_{ratio} FORMAT Null + + SELECT id FROM test_{serialization}_{ratio} ORDER BY u64 DESC LIMIT 100 FORMAT Null + SELECT id FROM test_{serialization}_{ratio} ORDER BY str DESC LIMIT 100 FORMAT Null + SELECT id FROM test_{serialization}_{ratio} ORDER BY u8, u64 DESC LIMIT 100 FORMAT Null + SELECT * FROM test_{serialization}_{ratio} ORDER BY u8, u64 DESC LIMIT 100 FORMAT Null + + SELECT sum(u64) FROM test_{serialization}_{ratio} GROUP BY id % 11 FORMAT Null + SELECT uniq(str) FROM test_{serialization}_{ratio} GROUP BY id % 11 FORMAT Null + + + From de4afb0cb9485d58f155ceceb88042a429a659bb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 21 May 2021 03:57:11 +0300 Subject: [PATCH 034/358] ColumnSparse: add comments --- src/AggregateFunctions/IAggregateFunction.h | 4 +-- src/Columns/ColumnArray.cpp | 32 +++++++++---------- src/Columns/ColumnArray.h | 4 +-- src/Columns/ColumnNullable.cpp | 6 ++-- src/Columns/ColumnNullable.h | 2 +- src/Columns/ColumnSparse.cpp | 22 +++++++------ src/Columns/ColumnSparse.h | 19 ++++++++--- src/Columns/ColumnVector.cpp | 8 +++-- src/Columns/ColumnVector.h | 2 +- src/Columns/IColumn.cpp | 13 ++++++-- src/Columns/IColumn.h | 15 +++++++-- src/Core/Block.cpp | 6 ++++ src/Core/Block.h | 2 ++ src/DataStreams/TTLBlockInputStream.cpp | 3 +- src/DataTypes/DataTypeTuple.h | 1 - src/DataTypes/IDataType.h | 11 +++++-- src/DataTypes/Serializations/ISerialization.h | 7 ++++ .../Serializations/SerializationInfo.cpp | 1 + .../Serializations/SerializationInfo.h | 22 ++++++++++++- .../Serializations/SerializationSparse.cpp | 14 ++++++++ .../Serializations/SerializationSparse.h | 17 ++++++++++ src/Dictionaries/CacheDictionary.cpp | 4 +-- src/Dictionaries/DictionaryHelpers.h | 3 +- src/Dictionaries/DirectDictionary.cpp | 3 +- src/Dictionaries/FlatDictionary.cpp | 3 +- src/Dictionaries/HashedDictionary.cpp | 3 +- src/Functions/IFunction.cpp | 2 +- src/Functions/IFunctionAdaptors.h | 2 ++ src/Functions/IFunctionImpl.h | 4 +++ src/Functions/ignore.cpp | 2 ++ src/Interpreters/join_common.cpp | 1 + src/Processors/Transforms/FilterTransform.cpp | 2 ++ .../Transforms/SortingTransform.cpp | 3 ++ src/Storages/MergeTree/MergeTreeSettings.h | 1 + 34 files changed, 180 insertions(+), 64 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 80cb7137a75..98daab717f8 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -154,6 +154,7 @@ public: Arena * arena, ssize_t if_argument_pos = -1) const = 0; + /// The version of "addBatch", that handle sparse columns as arguments. virtual void addBatchSparse( AggregateDataPtr * places, size_t place_offset, @@ -172,6 +173,7 @@ public: virtual void addBatchSinglePlace( size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos = -1) const = 0; + /// The version of "addBatchSinglePlace", that handle sparse columns as arguments. virtual void addBatchSparseSinglePlace( AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0; @@ -232,8 +234,6 @@ public: */ virtual AggregateFunctionPtr getNestedFunction() const { return {}; } - virtual bool supportsSparseArguments() const { return false; } - const DataTypes & getArgumentTypes() const { return argument_types; } const Array & getParameters() const { return parameters; } diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 0b22a7fa7a0..6e10a15d920 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1216,23 +1216,23 @@ void ColumnArray::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } -// size_t ColumnArray::getNumberOfDefaultRows(size_t step) const -// { -// const auto & offsets_data = getOffsets(); -// size_t res = 0; -// for (size_t i = 0; i < offsets_data.size(); i += step) -// res += (offsets_data[i] != offsets_data[i - 1]); +size_t ColumnArray::getNumberOfDefaultRows(size_t step) const +{ + const auto & offsets_data = getOffsets(); + size_t res = 0; + for (size_t i = 0; i < offsets_data.size(); i += step) + res += (offsets_data[i] != offsets_data[i - 1]); -// return res; -// } + return res; +} -// void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const -// { -// const auto & offsets_data = getOffsets(); -// size_t to = limit && from + limit < size() ? from + limit : size(); -// for (size_t i = from; i < to; ++i) -// if (offsets_data[i] != offsets_data[i - 1]) -// indices.push_back(i); -// } +void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const +{ + const auto & offsets_data = getOffsets(); + size_t to = limit && from + limit < size() ? from + limit : size(); + for (size_t i = from; i < to; ++i) + if (offsets_data[i] != offsets_data[i - 1]) + indices.push_back(i); +} } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index cc686855e37..fab6573f159 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -140,8 +140,8 @@ public: return false; } - // size_t getNumberOfDefaultRows(size_t step) const override; - // void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + size_t getNumberOfDefaultRows(size_t step) const override; + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 1f4f6fcaa78..2b6dd1106b3 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -707,10 +707,10 @@ void ColumnNullable::getIndicesOfNonDefaultValues(Offsets & indices, size_t from indices.push_back(i); } -ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const +ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const { - auto new_values = nested_column->createWithOffsets(offsets, total_rows); - auto new_null_map = null_map->createWithOffsets(offsets, total_rows); + auto new_values = nested_column->createWithOffsets(offsets, total_rows, shift); + auto new_null_map = null_map->createWithOffsets(offsets, total_rows, shift); return ColumnNullable::create(new_values, new_null_map); } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 86cbc7b9d17..55282f4569b 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -174,7 +174,7 @@ public: size_t getNumberOfDefaultRows(size_t step) const override; void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; - ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const override; + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; private: WrappedPtr nested_column; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 22e378b7f7c..5aeddd6e2dd 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -36,6 +36,7 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs if (!offsets_concrete) throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::LOGICAL_ERROR); + /// 'values' should contain one extra element: default value at 0 position. if (offsets->size() + 1 != values->size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Values size ({}) is inconsistent with offsets size ({})", values->size(), offsets->size()); @@ -46,7 +47,7 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs #ifndef NDEBUG const auto & offsets_data = getOffsetsData(); - auto it = std::adjacent_find(offsets_data.begin(), offsets_data.end(), std::greater_equal()); + const auto * it = std::adjacent_find(offsets_data.begin(), offsets_data.end(), std::greater_equal()); if (it != offsets_data.end()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Offsets of ColumnSparse must be strictly sorted"); #endif @@ -117,7 +118,7 @@ StringRef ColumnSparse::getDataAt(size_t n) const ColumnPtr ColumnSparse::convertToFullColumnIfSparse() const { - return values->createWithOffsets(getOffsetsData(), _size); + return values->createWithOffsets(getOffsetsData(), _size, 1); } void ColumnSparse::insertData(const char * pos, size_t length) @@ -177,6 +178,8 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len ++_size; } + /// 'end' <= 'src_offsets[offsets_end]', but end is excluded, so index is 'offsets_end' - 1. + /// Since 'end' is excluded need, to substract one more row from result. insertManyDefaults(end - src_offsets[offset_end - 1] - 1); values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start); } @@ -326,7 +329,7 @@ ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const /// and avoid binary search for obtaining every index. /// 3 is just a guess for overhead on copying indexes. bool execute_linear = - limit == _size || limit * log2(offsets->size() + 1) > _size * 3; + limit == _size || limit * std::bit_width(offsets->size()) > _size * 3; if (execute_linear) { @@ -452,7 +455,8 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc limit = _size; Permutation perm; - /// limit + 1 for case when there is 0 default values + /// Firstly we sort all values. + /// limit + 1 for case when there are 0 default values. if (collator) values->getPermutationWithCollation(*collator, reverse, limit + 1, null_direction_hint, perm); else @@ -462,8 +466,8 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc size_t row = 0; const auto & offsets_data = getOffsetsData(); - auto offset_it = begin(); + /// Fill the permutation. for (size_t i = 0; i < perm.size() && row < limit; ++i) { if (perm[i] == 0) @@ -471,9 +475,11 @@ void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direc if (!num_of_defaults) continue; + /// Fill the positions of default values in the required quantity. + auto offset_it = begin(); while (row < limit) { - while (offset_it != end() && !offset_it.isDefault()) + while (offset_it.getCurrentRow() < _size && !offset_it.isDefault()) ++offset_it; if (offset_it.getCurrentRow() == _size) @@ -515,10 +521,6 @@ void ColumnSparse::updatePermutationWithCollation( this_full->updatePermutationWithCollation(collator, reverse, limit, null_direction_hint, res, equal_range); } -void ColumnSparse::reserve(size_t) -{ -} - size_t ColumnSparse::byteSize() const { return values->byteSize() + offsets->byteSize(); diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 4167e6a7cc0..eaef23648c4 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -11,6 +11,13 @@ class Collator; namespace DB { + +/** Column for spare representation. + * It stores column with non-default values and column + * with their sorted positions in original column. Column with + * values contains also one default value at 0 position to make + * implementation of execution of functions and sorting more convinient. + */ class ColumnSparse final : public COWHelper { private: @@ -21,9 +28,6 @@ private: ColumnSparse(const ColumnSparse &) = default; public: - /** Create immutable column using immutable arguments. This arguments may be shared with other columns. - * Use IColumn::mutate in order to make mutable column and mutate shared nested columns. - */ using Base = COWHelper; static Ptr create(const ColumnPtr & values_, const ColumnPtr & offsets_, size_t size_) { @@ -102,7 +106,6 @@ public: void updatePermutationWithCollation( const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const override; - void reserve(size_t n) override; size_t byteSize() const override; size_t byteSizeAt(size_t n) const override; size_t allocatedBytes() const override; @@ -138,6 +141,9 @@ public: return offsets->empty() ? _size : _size - getOffsetsData().back() - 1; } + /// Return position of element in 'values' columns, + /// that corresponds to n-th element of full column. + /// O(log(size)) complexity, size_t getValueIndex(size_t n) const; const IColumn & getValuesColumn() const { return *values; } @@ -155,6 +161,7 @@ public: const IColumn & getOffsetsColumn() const { return *offsets; } IColumn & getOffsetsColumn() { return *offsets; } + /// This class helps to iterate over all values in ColumnSparse. class Iterator { public: @@ -195,7 +202,11 @@ public: Iterator end() const { return Iterator(getOffsetsData(), _size, getOffsetsData().size(), _size); } private: + /// Contains default value at 0 position. WrappedPtr values; + + /// Sorted offsets of non-default values in the full column. + /// 'offsets[i]' corresponds to 'values[i + 1]'. WrappedPtr offsets; size_t _size; }; diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 22b7187efe2..44c6132f23c 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -583,14 +583,18 @@ void ColumnVector::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, s } template -ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const +ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const { + if (offsets.size() + shift != size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Incompatible sizes of offsets ({}), shift ({}) and size of column {}", offsets.size(), shift, size()); + auto res = this->create(); auto & res_data = res->getData(); res_data.resize_fill(total_rows, data[0]); for (size_t i = 0; i < offsets.size(); ++i) - res_data[offsets[i]] = data[i + 1]; + res_data[offsets[i]] = data[i + shift]; return res; } diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index b61bdc4e1ef..3001c88a516 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -323,7 +323,7 @@ public: size_t getNumberOfDefaultRows(size_t step) const override; void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; - ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows) const override; + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; ColumnPtr compress() const override; diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 76334e0da6e..ed81bdf7f1a 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + String IColumn::dumpStructure() const { WriteBufferFromOwnString res; @@ -38,8 +43,12 @@ void IColumn::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_ indices.push_back(i); } -ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows) const +ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows, size_t shift) const { + if (offsets.size() + shift != size()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Incompatible sizes of offsets ({}), shift ({}) and size of column {}", offsets.size(), shift, size()); + auto res = cloneEmpty(); res->reserve(total_rows); @@ -52,7 +61,7 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows) if (offsets_diff > 1) res->insertManyFrom(*this, 0, offsets_diff - 1); - res->insertFrom(*this, i + 1); + res->insertFrom(*this, i + shift); } ssize_t offsets_diff = static_cast(total_rows) - current_offset; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 8585aca4043..7c0b6647a23 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -64,9 +64,11 @@ public: virtual Ptr convertToFullColumnIfConst() const { return getPtr(); } /// If column isn't ColumnLowCardinality, return itself. - /// If column is ColumnLowCardinality, transforms is to full column. + /// If column is ColumnLowCardinality, transforms it to full column. virtual Ptr convertToFullColumnIfLowCardinality() const { return getPtr(); } + /// If column isn't ColumnSparse, return itself. + /// If column is ColumnSparse, transforms it to full column. virtual Ptr convertToFullColumnIfSparse() const { return getPtr(); } Ptr convertToFullIfNeeded() const @@ -378,11 +380,20 @@ public: static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; static constexpr auto DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION = 0.95; + /// Returns number of values in column, that equal to default value of column. + /// Checks every @step-th value. So, if step is not 1, returns number, + /// that lower than actual. 0 means, that such statistic is unknown for column. virtual size_t getNumberOfDefaultRows(size_t /* step */) const { return 0; } + /// Returns indices of values in column, that not equal to default value of column. virtual void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const; - virtual Ptr createWithOffsets(const Offsets & offsets, size_t total_rows) const; + /// Returns column with @total_size elements. + /// In result column values from current column are at positions from @offsets. + /// Other values are filled by defaults. + /// @shift means how much rows to skip from the beginning of current column. + /// Used to create full column from sparse. + virtual Ptr createWithOffsets(const Offsets & offsets, size_t total_rows, size_t shift) const; /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 79e32c26728..29e90af3545 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -677,4 +677,10 @@ void Block::updateHash(SipHash & hash) const col.column->updateHashWithValue(row_no, hash); } +void convertToFullIfSparse(Block & block) +{ + for (auto & column : block) + column.column = recursiveRemoveSparse(column.column); +} + } diff --git a/src/Core/Block.h b/src/Core/Block.h index a21bd290571..e4b1ad0a73a 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -194,4 +194,6 @@ void assertCompatibleHeader(const Block & actual, const Block & desired, const s /// Calculate difference in structure of blocks and write description into output strings. NOTE It doesn't compare values of constant columns. void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out_lhs_diff, std::string & out_rhs_diff); +void convertToFullIfSparse(Block & chunk); + } diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 6da60f708c8..a668389098e 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -104,8 +104,7 @@ Block TTLBlockInputStream::readImpl() return {}; auto block = children.at(0)->read(); - for (auto & column : block) - column.column = recursiveRemoveSparse(column.column); + convertToFullIfSparse(block); for (const auto & algorithm : algorithms) algorithm->execute(block); diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 7cf6c00f8be..3355f4c2aab 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -56,7 +56,6 @@ public: DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; - // SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const override; SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const override; SerializationPtr getSubcolumnSerialization( diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 466fbcfcbb9..9104c6bd4f3 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -80,14 +80,17 @@ public: virtual SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; - /// Chooses serialziation accordind to column content. + /// Chooses serialziation according to column content. virtual SerializationPtr getSerialization(const IColumn & column) const; - /// Chooses serialization accosrding to collected information about content of columns. + /// Chooses serialization according to collected information about content of columns. virtual SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; + /// Chooses serialization according to settings. SerializationPtr getSerialization(const ISerialization::Settings & settings) const; + /// Chooses beetween subcolumn serialization and regular serialization according to @column. + /// This method typically should be used to get serialization for reading column or subcolumn. static SerializationPtr getSerialization(const NameAndTypePair & column, const SerializationInfo & info); using StreamCallbackWithType = std::function; @@ -103,10 +106,12 @@ protected: DataTypePtr getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const; public: - /** Create empty column for corresponding type. + /** Create empty column for corresponding type and default serialization. */ virtual MutableColumnPtr createColumn() const = 0; + /** Create empty column for corresponding type and serialization. + */ virtual MutableColumnPtr createColumn(const ISerialization & serialization) const; /** Create ColumnConst for corresponding type, with specified size and value. diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index d3b6d953408..3138e53f15b 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -33,6 +33,13 @@ class Field; struct FormatSettings; struct NameAndTypePair; +/** Represents serialization of data type. + * Has methods to serialize/deserialize column in binary and several text formats. + * Every data type has default serialization, but can be serialized in different representations. + * Default serialization can be wrapped to one of the special kind of serializations. + * Currently there is only one special serialization: Sparse. + * Each serialization has its own implemetation of IColumn as its in-memory representation. + */ class ISerialization { public: diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index f3d401afad3..b258f889e5e 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -33,6 +33,7 @@ void SerializationInfoBuilder::add(const Block & block) info->number_of_rows += block.rows(); for (const auto & elem : block) { + /// Multiply by step to restore approximate number of default values. info->columns[elem.name].num_defaults += elem.column->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; for (const auto & subname : elem.type->getSubcolumnNames()) { diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index ab87d18a2ed..0685d61e080 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -6,6 +6,15 @@ namespace DB { +/** Contains information about kinds of serialization of columns. + * Also contains information about content of columns, + * that helps to choose kind of serialization of column. + * + * Currently has only information about number of default rows, + * that helps to choose sparse serialization. + * + * Should be extended, when new kinds of serialization will be implemented. + */ class SerializationInfo { public: @@ -31,6 +40,8 @@ private: void fromJSON(const String & json_str); String toJSON() const; + /// Information about one column. + /// Can be extended, when new kinds of serialization will be implemented. struct Column { ISerialization::Kind kind = ISerialization::Kind::DEFAULT; @@ -47,6 +58,7 @@ private: using SerializationInfoPtr = std::shared_ptr; +/// Builder, that helps to create SerializationInfo. class SerializationInfoBuilder { public: @@ -55,12 +67,20 @@ public: double ratio_for_sparse_serialization_, size_t default_rows_search_step_ = IColumn::DEFAULT_ROWS_SEARCH_STEP); + /// Add information about column from block. void add(const Block & block); + + /// Add information about column from other SerializationInfo. void add(const SerializationInfo & other); + /// Choose kind of serialization for every column + /// according its content and return finalized SerializationInfo. SerializationInfoPtr build(); + + /// Create SerializationInfo from other. + /// Respects kinds of serialization for columns, that exist in other SerializationInfo, + /// but keeps information about content of column from current SerializationInfo. SerializationInfoPtr buildFrom(const SerializationInfo & other); - static SerializationInfoPtr buildFromBlock(const Block & block); double getRatioForSparseSerialization() const { return ratio_for_sparse_serialization; } diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 139ecc7d850..080a190e963 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -20,11 +20,14 @@ namespace ErrorCodes namespace { +/// 2^62, because VarInt supports only values < 2^63. static constexpr auto END_OF_GRANULE_FLAG = 1ULL << 62; struct DeserializeStateSparse : public ISerialization::DeserializeBinaryBulkState { + /// Number of default values, that remain from previous read. size_t num_trailing_defaults = 0; + /// Do we have non-default value after @num_trailing_defaults? bool has_value_after_defaults = false; ISerialization::DeserializeBinaryBulkStatePtr nested; @@ -50,6 +53,9 @@ void serializeOffsets(const IColumn::Offsets & offsets, WriteBuffer & ostr, size writeVarUInt(group_size, ostr); } + +/// Returns number of read rows. +/// @start is the size of column before reading offsets. size_t deserializeOffsets(IColumn::Offsets & offsets, ReadBuffer & istr, size_t start, size_t limit, DeserializeStateSparse & state) { @@ -87,6 +93,8 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, if (limit && next_total_rows >= limit) { + /// If it was not last group in granule, + /// we have to add current non-default value at further reads. state.num_trailing_defaults = next_total_rows - limit; state.has_value_after_defaults = !end_of_granule; return limit; @@ -99,6 +107,9 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } else { + /// If we add value to column for first time in current read, + /// start from column's current size, because it can have some defaults after last offset, + /// otherwise just start from previous offset. size_t start_of_group = start; if (!first && !offsets.empty()) start_of_group = offsets.back() + 1; @@ -239,10 +250,13 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( throw Exception(ErrorCodes::LOGICAL_ERROR, "Inconsistent sizes of values and offsets in SerializationSparse." " Offsets size: {}, values size: {}", offsets_data.size(), values_column->size()); + /// 'insertManyDefaults' just increases size of column. column_sparse.insertManyDefaults(read_rows); column = std::move(mutable_column); } +/// All methods below just wrap nested serialization. + void SerializationSparse::serializeBinary(const Field & field, WriteBuffer & ostr) const { nested->serializeBinary(field, ostr); diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index 8d19c94c58b..aa9ee7c2771 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -5,6 +5,21 @@ namespace DB { + +/** Serialization for sparse representation. + * Only '{serialize,deserialize}BinaryBulk' makes sense. + * Format: + * Values and offsets are written to separate substreams. + * There are written only non-default values. + * + * Offsets have position independent format: as i-th offset there + * is written number of default values, that precedes the i-th non-default value. + * Offsets are written in VarInt encoding. + * Additionaly at the end of every call of 'serializeBinaryBulkWithMultipleStreams' + * there is written number of default values in the suffix of part of column, + * that we currently writing. This value also marked with a flag, that means the end of portion of data. + * This value is used, e.g. to allow independent reading of granules in MergeTree. + */ class SerializationSparse final : public ISerialization { public: @@ -26,6 +41,7 @@ public: DeserializeBinaryBulkSettings & settings, DeserializeBinaryBulkStatePtr & state) const override; + /// Allows to write ColumnSparse and other columns in sparse serialization. void serializeBinaryBulkWithMultipleStreams( const IColumn & column, size_t offset, @@ -33,6 +49,7 @@ public: SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; + /// Allows to read only ColumnSparse. void deserializeBinaryBulkWithMultipleStreams( ColumnPtr & column, size_t limit, diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 47eafe8ea99..1ac079edef4 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -590,9 +590,7 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrread()) { - for (auto & column : block) - column.column = recursiveRemoveSparse(column.column); + convertToFullIfSparse(block); Columns block_key_columns; block_key_columns.reserve(key_columns_size); diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index b1a85cd4feb..160a91fde6e 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -72,8 +72,7 @@ Columns DirectDictionary::getColumns( while (auto block = stream->read()) { - for (auto & column : block) - column.column = recursiveRemoveSparse(column.column); + convertToFullIfSparse(block); /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index b901eb945c5..5b750772583 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -294,8 +294,7 @@ void FlatDictionary::updateData() while (auto block = stream->read()) { - for (auto & column : block) - column.column = recursiveRemoveSparse(column.column); + convertToFullIfSparse(block); /// We are using this to keep saved data if input stream consists of multiple blocks if (!update_field_loaded_block) diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index e001229cf69..b948c0284ba 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -366,8 +366,7 @@ void HashedDictionary::updateData() while (auto block = stream->read()) { - for (auto & column : block) - column.column = recursiveRemoveSparse(column.column); + convertToFullIfSparse(block); /// We are using this to keep saved data if input stream consists of multiple blocks if (!update_field_loaded_block) diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index f1f25579f8b..5179568e05b 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -509,7 +509,7 @@ ColumnPtr ExecutableFunctionAdaptor::execute(const ColumnsWithTypeAndName & argu if (!res->isDefaultAt(0)) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); - return res->createWithOffsets(offsets_data, input_rows_count); + return res->createWithOffsets(offsets_data, input_rows_count, 1); } return ColumnSparse::create(res, sparse_offsets, input_rows_count); diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 9824d2a1659..e05af9f348b 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -164,6 +164,7 @@ protected: bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); } bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); } bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); } + bool useDefaultImplementationForSparseColumns() const final { return function->useDefaultImplementationForSparseColumns(); } ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); } bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); } @@ -250,6 +251,7 @@ public: bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); } bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); } + bool useDefaultImplementationForSparseColumns() const override { return function->useDefaultImplementationForSparseColumns(); } bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); } FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const override diff --git a/src/Functions/IFunctionImpl.h b/src/Functions/IFunctionImpl.h index 22846367d72..1bddf83227e 100644 --- a/src/Functions/IFunctionImpl.h +++ b/src/Functions/IFunctionImpl.h @@ -184,6 +184,8 @@ public: */ virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; } + virtual bool useDefaultImplementationForSparseColumns() const { return true; } + /// If it isn't, will convert all ColumnLowCardinality arguments to full columns. virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; } }; @@ -230,6 +232,8 @@ public: /// If it isn't, will convert all ColumnLowCardinality arguments to full columns. virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; } + virtual bool useDefaultImplementationForSparseColumns() const { return true; } + /** Some arguments could remain constant during this implementation. */ virtual ColumnNumbers getArgumentsThatAreAlwaysConstant() const { return {}; } diff --git a/src/Functions/ignore.cpp b/src/Functions/ignore.cpp index 1a8a838036c..9592b5c40e8 100644 --- a/src/Functions/ignore.cpp +++ b/src/Functions/ignore.cpp @@ -35,6 +35,8 @@ public: /// (in getResultIfAlwaysReturnsConstantAndHasArguments) bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + bool useDefaultImplementationForSparseColumns() const override { return false; } + String getName() const override { return name; diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index d4108188fc5..2e3e4f9ca54 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -77,6 +77,7 @@ void convertColumnToNullable(ColumnWithTypeAndName & column, bool remove_low_car if (column.column->isSparse()) column.column = recursiveRemoveSparse(column.column); + if (remove_low_card && column.type->lowCardinality()) { column.column = recursiveRemoveLowCardinality(column.column); diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 364fb8e1958..8c0648fe40a 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -4,6 +4,8 @@ #include #include +#include + namespace DB { diff --git a/src/Processors/Transforms/SortingTransform.cpp b/src/Processors/Transforms/SortingTransform.cpp index 59e59c524af..3a1d171f3d5 100644 --- a/src/Processors/Transforms/SortingTransform.cpp +++ b/src/Processors/Transforms/SortingTransform.cpp @@ -37,6 +37,9 @@ MergeSorter::MergeSorter(Chunks chunks_, SortDescription & description_, size_t if (chunk.getNumRows() == 0) continue; + /// Convert to full column, because sparse column has + /// access to element in O(log(K)), where K is number of non-default rows, + /// which can be inefficient. convertToFullIfSparse(chunk); cursors.emplace_back(chunk.getColumns(), description); diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dcebaa2480d..b4989382503 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,6 +33,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ + M(Bool, allow_dynamic_serializations, false, "", 0) \ M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ From c5d290cddb15bcb8691b040f4d15bcaa2288b2a4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 24 May 2021 17:49:03 +0300 Subject: [PATCH 035/358] remove unused setting --- src/Storages/MergeTree/MergeTreeSettings.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index b4989382503..dcebaa2480d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,7 +33,6 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Bool, allow_dynamic_serializations, false, "", 0) \ M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ From df5c4079502570972812a0a9fa2eae7a6361700d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Jun 2021 19:03:18 +0300 Subject: [PATCH 036/358] fix tests --- src/Columns/ColumnSparse.cpp | 2 +- src/Columns/ColumnSparse.h | 2 +- src/DataTypes/IDataType.h | 2 +- src/DataTypes/Serializations/ISerialization.h | 2 +- src/DataTypes/Serializations/SerializationSparse.h | 2 +- tests/integration/test_merge_tree_hdfs/test.py | 4 ++-- tests/integration/test_merge_tree_s3_failover/test.py | 6 +++--- tests/integration/test_partition/test.py | 6 ++++-- tests/integration/test_s3_zero_copy_replication/test.py | 4 ++-- 9 files changed, 16 insertions(+), 14 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 5aeddd6e2dd..56ce9096a75 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -179,7 +179,7 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len } /// 'end' <= 'src_offsets[offsets_end]', but end is excluded, so index is 'offsets_end' - 1. - /// Since 'end' is excluded need, to substract one more row from result. + /// Since 'end' is excluded need, to subtract one more row from result. insertManyDefaults(end - src_offsets[offset_end - 1] - 1); values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start); } diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index eaef23648c4..69ee66a37e8 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -16,7 +16,7 @@ namespace DB * It stores column with non-default values and column * with their sorted positions in original column. Column with * values contains also one default value at 0 position to make - * implementation of execution of functions and sorting more convinient. + * implementation of execution of functions and sorting more convenient. */ class ColumnSparse final : public COWHelper { diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 121d3a0193d..ce4390dd91c 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -89,7 +89,7 @@ public: /// Chooses serialization according to settings. SerializationPtr getSerialization(const ISerialization::Settings & settings) const; - /// Chooses beetween subcolumn serialization and regular serialization according to @column. + /// Chooses between subcolumn serialization and regular serialization according to @column. /// This method typically should be used to get serialization for reading column or subcolumn. static SerializationPtr getSerialization(const NameAndTypePair & column, const SerializationInfo & info); diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 3138e53f15b..3a8ae37eaae 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -38,7 +38,7 @@ struct NameAndTypePair; * Every data type has default serialization, but can be serialized in different representations. * Default serialization can be wrapped to one of the special kind of serializations. * Currently there is only one special serialization: Sparse. - * Each serialization has its own implemetation of IColumn as its in-memory representation. + * Each serialization has its own implementation of IColumn as its in-memory representation. */ class ISerialization { diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index aa9ee7c2771..eeca6dc7017 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -15,7 +15,7 @@ namespace DB * Offsets have position independent format: as i-th offset there * is written number of default values, that precedes the i-th non-default value. * Offsets are written in VarInt encoding. - * Additionaly at the end of every call of 'serializeBinaryBulkWithMultipleStreams' + * Additionally at the end of every call of 'serializeBinaryBulkWithMultipleStreams' * there is written number of default values in the suffix of part of column, * that we currently writing. This value also marked with a flag, that means the end of portion of data. * This value is used, e.g. to allow independent reading of granules in MergeTree. diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index 2d0d9d9fb1e..36ffda6c36e 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -39,8 +39,8 @@ def create_table(cluster, table_name, additional_settings=None): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 def random_string(length): diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 11a1e464cb6..2ba4e7253e2 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -69,7 +69,7 @@ def drop_table(cluster): # S3 request will be failed for an appropriate part file write. -FILES_PER_PART_BASE = 5 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt +FILES_PER_PART_BASE = 6 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt, serialization.txt FILES_PER_PART_WIDE = FILES_PER_PART_BASE + 1 + 1 + 3 * 2 # Primary index, MinMax, Mark and data file for column(s) FILES_PER_PART_COMPACT = FILES_PER_PART_BASE + 1 + 1 + 2 @@ -159,13 +159,13 @@ def test_move_failover(cluster): # There should be 2 attempts to move part. assert node.query(""" - SELECT count(*) FROM system.part_log + SELECT count(*) FROM system.part_log WHERE event_type='MovePart' AND table='s3_failover_test' """) == '2\n' # First attempt should be failed with expected error. exception = node.query(""" - SELECT exception FROM system.part_log + SELECT exception FROM system.part_log WHERE event_type='MovePart' AND table='s3_failover_test' AND notEmpty(exception) ORDER BY event_time LIMIT 1 diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index b5facb5f4b2..8d275455bf3 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -69,7 +69,6 @@ def partition_complex_assert_checksums(): "13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition/19700102_2_2_0/minmax_p.idx\n" \ "25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition/19700102_2_2_0/partition.dat\n" \ "3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition/19700201_1_1_0/partition.dat\n" \ - "37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition/19700102_2_2_0/checksums.txt\n" \ "38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition/19700102_2_2_0/v1.bin\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/k.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700102_2_2_0/p.mrk\n" \ @@ -78,15 +77,18 @@ def partition_complex_assert_checksums(): "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/p.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition/19700201_1_1_0/v1.mrk\n" \ "55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition/19700201_1_1_0/primary.idx\n" \ - "5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition/19700201_1_1_0/checksums.txt\n" \ + "636c2df435a924297354025e3e203006\tshadow/1/data/test/partition/19700102_2_2_0/checksums.txt\n" \ "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700102_2_2_0/columns.txt\n" \ "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition/19700201_1_1_0/columns.txt\n" \ "88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition/19700201_1_1_0/p.bin\n" \ "9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition/19700102_2_2_0/primary.idx\n" \ + "ac5882732ed524266cfcb45c2cb53781\tshadow/1/data/test/partition/19700201_1_1_0/checksums.txt\n" \ "c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition/19700102_2_2_0/default_compression_codec.txt\n" \ "c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition/19700201_1_1_0/default_compression_codec.txt\n" \ "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700102_2_2_0/count.txt\n" \ "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition/19700201_1_1_0/count.txt\n" \ + "c92f1855b6cc741573952c527dedff3a\tshadow/1/data/test/partition/19700102_2_2_0/serialization.txt\n" \ + "c92f1855b6cc741573952c527dedff3a\tshadow/1/data/test/partition/19700201_1_1_0/serialization.txt\n" \ "cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition/19700102_2_2_0/p.bin\n" \ "e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition/19700201_1_1_0/k.bin\n" \ "f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition/19700201_1_1_0/minmax_p.idx\n" diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index d9f7cca4a3a..ecea3d3592f 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -68,8 +68,8 @@ def test_s3_zero_copy_replication(cluster, policy): assert node1.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data')" assert node2.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data')" - # Based on version 20.x - should be only one file with size 100+ (checksums.txt), used by both nodes - assert get_large_objects_count(cluster) == 1 + # Based on version 21.x - should be only 2 files with size 100+ (checksums.txt, serialization.txt), used by both nodes + assert get_large_objects_count(cluster) == 2 node2.query("INSERT INTO s3_test VALUES (2,'data'),(3,'data')") node1.query("SYSTEM SYNC REPLICA s3_test") From e48922d320d93269640e545054fa81ba570aee84 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 2 Jun 2021 20:45:18 +0300 Subject: [PATCH 037/358] temorarly make all columns sparse --- src/Core/Block.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 29e90af3545..f4922006a24 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -497,8 +497,8 @@ DataTypes Block::getDataTypes() const static String getNameOfBaseColumn(const IColumn & column) { - if (const auto * column_sparse = checkAndGetColumn(&column)) - return column_sparse->getValuesColumn().getName(); + // if (const auto * column_sparse = checkAndGetColumn(&column)) + // return column_sparse->getValuesColumn().getName(); return column.getName(); } diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index af6f0fada94..1681493018c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,7 +33,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, -1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ From 0e26483b218550df413617ecb2b0adc0b52db94f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 3 Jun 2021 16:11:36 +0300 Subject: [PATCH 038/358] fix subcolumns with sparse serialization --- src/DataTypes/DataTypeArray.h | 2 - src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/IDataType.cpp | 3 +- src/DataTypes/IDataType.h | 3 +- src/DataTypes/NestedUtils.cpp | 60 ++++++++++++------- src/DataTypes/NestedUtils.h | 3 +- .../Serializations/SerializationInfo.cpp | 28 +++++++-- src/DataTypes/tests/gtest_split_name.cpp | 32 ++++++++++ 8 files changed, 97 insertions(+), 35 deletions(-) create mode 100644 src/DataTypes/tests/gtest_split_name.cpp diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 8e29bef6830..c720a15d798 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -54,8 +54,6 @@ public: return nested->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion(); } - /// TODO: support arrays with sparse serialization. - bool supportsSparseSerialization() const override { return false; } DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; SerializationPtr getSubcolumnSerialization( diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 3355f4c2aab..2aafc365373 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -36,6 +36,7 @@ public: const char * getFamilyName() const override { return "Tuple"; } bool canBeInsideNullable() const override { return false; } + bool supportsSparseSerialization() const override { return true; } MutableColumnPtr createColumn() const override; MutableColumnPtr createColumn(const ISerialization & serialization) const override; diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index fa6309493bc..c7706e51eab 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -170,8 +170,7 @@ SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_n SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const { - auto kind = info.getKind(column_name); - if (supportsSparseSerialization() && kind == ISerialization::Kind::SPARSE) + if (supportsSparseSerialization() && info.getKind(column_name) == ISerialization::Kind::SPARSE) return getSparseSerialization(); return getDefaultSerialization(); diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 4241d1fb74d..56ee92f63fe 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -69,7 +69,8 @@ public: virtual ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const; Names getSubcolumnNames() const; - virtual bool supportsSparseSerialization() const { return true; } + /// TODO: support more types. + virtual bool supportsSparseSerialization() const { return !haveSubtypes(); } SerializationPtr getDefaultSerialization() const; SerializationPtr getSparseSerialization() const; diff --git a/src/DataTypes/NestedUtils.cpp b/src/DataTypes/NestedUtils.cpp index 6c13eea0a1b..68f17298020 100644 --- a/src/DataTypes/NestedUtils.cpp +++ b/src/DataTypes/NestedUtils.cpp @@ -33,42 +33,56 @@ std::string concatenateName(const std::string & nested_table_name, const std::st return nested_table_name + "." + nested_field_name; } +static const char * extractSimpleIdentifier(const char * begin, const char * end) +{ + const char * pos = begin; + if (pos >= end || !isValidIdentifierBegin(*pos)) + return nullptr; -/** Name can be treated as compound if and only if both parts are simple identifiers. + while (pos < end && isWordCharASCII(*pos)) + ++pos; + + if (pos < end && *pos != '.') + return nullptr; + + return pos; +} + + +/** Name can be treated as compound if and only if one part is simple identifier + * and other part is simple or compound identifier. */ -std::pair splitName(const std::string & name) +std::pair splitName(const std::string & name, bool reverse) { const char * begin = name.data(); const char * pos = begin; const char * end = begin + name.size(); - if (pos >= end || !isValidIdentifierBegin(*pos)) - return {name, {}}; + const char * first_end = nullptr; + const char * second_begin = nullptr; - ++pos; + while (true) + { + pos = extractSimpleIdentifier(pos, end); + if (!pos) + return {name, {}}; + + if (pos >= end) + break; + + if (reverse || !first_end) + first_end = pos; - while (pos < end && isWordCharASCII(*pos)) ++pos; - if (pos >= end || *pos != '.') - return {name, {}}; + if (reverse || !second_begin) + second_begin = pos; + } - const char * first_end = pos; - ++pos; - const char * second_begin = pos; + if (first_end) + return {{ begin, first_end }, { second_begin, end }}; - if (pos >= end || !isValidIdentifierBegin(*pos)) - return {name, {}}; - - ++pos; - - while (pos < end && isWordCharASCII(*pos)) - ++pos; - - if (pos != end) - return {name, {}}; - - return {{ begin, first_end }, { second_begin, end }}; + return {name, {}}; } diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index b8428b96d3e..c3bc9d17589 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -11,7 +11,8 @@ namespace Nested { std::string concatenateName(const std::string & nested_table_name, const std::string & nested_field_name); - std::pair splitName(const std::string & name); + /// Splits name of compound indentifier by firts/last dot (depending on 'reverse' parameter). + std::pair splitName(const std::string & name, bool reverse = false); /// Returns the prefix of the name to the first '.'. Or the name is unchanged if there is no dot. std::string extractTableName(const std::string & nested_name); diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index b258f889e5e..384cba1f735 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -33,12 +33,28 @@ void SerializationInfoBuilder::add(const Block & block) info->number_of_rows += block.rows(); for (const auto & elem : block) { + /// Just skip column and always return default serialization. + if (!elem.type->supportsSparseSerialization()) + continue; + /// Multiply by step to restore approximate number of default values. info->columns[elem.name].num_defaults += elem.column->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; - for (const auto & subname : elem.type->getSubcolumnNames()) + for (const auto & subcolumn_name : elem.type->getSubcolumnNames()) { - auto subcolumn = elem.type->getSubcolumn(subname, *elem.column); - auto full_name = Nested::concatenateName(elem.name, subname); + auto subcolumn_type = elem.type->getSubcolumnType(subcolumn_name); + if (!subcolumn_type->supportsSparseSerialization()) + continue; + + auto parent_subcolumn_name = Nested::splitName(subcolumn_name, /*reverse=*/ true).first; + if (!parent_subcolumn_name.empty()) + { + auto parent_subcolumn_type = elem.type->tryGetSubcolumnType(parent_subcolumn_name); + if (parent_subcolumn_type && !parent_subcolumn_type->supportsSparseSerialization()) + continue; + } + + auto subcolumn = elem.type->getSubcolumn(subcolumn_name, *elem.column); + auto full_name = Nested::concatenateName(elem.name, subcolumn_name); info->columns[full_name].num_defaults += subcolumn->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; } } @@ -185,10 +201,10 @@ SerializationInfo::NameToKind SerializationInfo::getKinds(const Block & block) for (const auto & elem : block) { kinds[elem.name] = ISerialization::getKind(*elem.column); - for (const auto & subname : elem.type->getSubcolumnNames()) + for (const auto & subcolumn_name : elem.type->getSubcolumnNames()) { - auto full_name = Nested::concatenateName(elem.name, subname); - auto subcolumn = elem.type->getSubcolumn(subname, *elem.column); + auto full_name = Nested::concatenateName(elem.name, subcolumn_name); + auto subcolumn = elem.type->getSubcolumn(subcolumn_name, *elem.column); kinds[full_name] = ISerialization::getKind(*subcolumn); } } diff --git a/src/DataTypes/tests/gtest_split_name.cpp b/src/DataTypes/tests/gtest_split_name.cpp new file mode 100644 index 00000000000..d985c8303d8 --- /dev/null +++ b/src/DataTypes/tests/gtest_split_name.cpp @@ -0,0 +1,32 @@ +#include + +#include + +using namespace DB; + +TEST(SplitName, forward) +{ + ASSERT_EQ(Nested::splitName("abc"), (std::pair{"abc", ""})); + ASSERT_EQ(Nested::splitName("a.b"), (std::pair{"a", "b"})); + ASSERT_EQ(Nested::splitName("a.b.c"), (std::pair{"a", "b.c"})); + ASSERT_EQ(Nested::splitName("a.1"), (std::pair{"a.1", ""})); + ASSERT_EQ(Nested::splitName("a.1.b"), (std::pair{"a.1.b", ""})); + ASSERT_EQ(Nested::splitName("1.a"), (std::pair{"1.a", ""})); + ASSERT_EQ(Nested::splitName("a.b1.b2"), (std::pair{"a", "b1.b2"})); + ASSERT_EQ(Nested::splitName("a.b1.2a.3a"), (std::pair{"a.b1.2a.3a", ""})); + ASSERT_EQ(Nested::splitName(".."), (std::pair{"..", ""})); +} + +TEST(SplitName, reverse) +{ + ASSERT_EQ(Nested::splitName("abc", true), (std::pair{"abc", ""})); + ASSERT_EQ(Nested::splitName("a.b", true), (std::pair{"a", "b"})); + ASSERT_EQ(Nested::splitName("a.b.c", true), (std::pair{"a.b", "c"})); + ASSERT_EQ(Nested::splitName("a.1", true), (std::pair{"a.1", ""})); + ASSERT_EQ(Nested::splitName("a.1a.b", true), (std::pair{"a.1a.b", ""})); + ASSERT_EQ(Nested::splitName("1a.b", true), (std::pair{"1a.b", ""})); + ASSERT_EQ(Nested::splitName("a.b1.b2", true), (std::pair{"a.b1", "b2"})); + ASSERT_EQ(Nested::splitName("a.b1.2a.3a", true), (std::pair{"a.b1.2a.3a", ""})); + ASSERT_EQ(Nested::splitName("a.b1.b2.b3", true), (std::pair{"a.b1.b2", "b3"})); + ASSERT_EQ(Nested::splitName("..", true), (std::pair{"..", ""})); +} From 2a2074fae72e29e8228899913dfbef523c867974 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 4 Jun 2021 16:06:57 +0300 Subject: [PATCH 039/358] fix debug asserts --- src/Core/Block.cpp | 25 ++++++++----------- src/DataStreams/ColumnGathererStream.h | 2 ++ src/DataStreams/ConvertingBlockInputStream.h | 1 + src/DataStreams/IBlockInputStream.cpp | 8 +++++- src/DataStreams/IBlockInputStream.h | 4 +++ .../PipelineExecutingBlockInputStream.h | 1 + .../Sources/SourceFromInputStream.cpp | 5 +++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 3 ++- 8 files changed, 32 insertions(+), 17 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index f4922006a24..17ca3a5a2b0 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -495,16 +495,8 @@ DataTypes Block::getDataTypes() const return res; } -static String getNameOfBaseColumn(const IColumn & column) -{ - // if (const auto * column_sparse = checkAndGetColumn(&column)) - // return column_sparse->getValuesColumn().getName(); - - return column.getName(); -} - template -static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description, bool allow_remove_constants) +static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description, bool allow_materialize) { auto on_error = [](const std::string & message [[maybe_unused]], int code [[maybe_unused]]) { @@ -538,14 +530,19 @@ static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, cons const IColumn * actual_column = actual.column.get(); - /// If we allow to remove constants, and expected column is not const, then unwrap actual constant column. - if (allow_remove_constants && !isColumnConst(*expected.column)) + /// If we allow to materialize, and expected column is not const or sparse, then unwrap actual column. + if (allow_materialize) { - if (const auto * column_const = typeid_cast(actual_column)) - actual_column = &column_const->getDataColumn(); + if (!isColumnConst(*expected.column)) + if (const auto * column_const = typeid_cast(actual_column)) + actual_column = &column_const->getDataColumn(); + + if (!expected.column->isSparse()) + if (const auto * column_sparse = typeid_cast(actual_column)) + actual_column = &column_sparse->getValuesColumn(); } - if (getNameOfBaseColumn(*actual_column) != getNameOfBaseColumn(*expected.column)) + if (actual_column->getName() != expected.column->getName()) return on_error("Block structure mismatch in " + context_description + " stream: different columns:\n" + lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::LOGICAL_ERROR); diff --git a/src/DataStreams/ColumnGathererStream.h b/src/DataStreams/ColumnGathererStream.h index 4e1a80c5154..27a64eefd37 100644 --- a/src/DataStreams/ColumnGathererStream.h +++ b/src/DataStreams/ColumnGathererStream.h @@ -67,6 +67,8 @@ public: void readSuffixImpl() override; + bool columnsCanDifferInRepresentationAmongBlocks() const override { return true; } + Block getHeader() const override { return children.at(0)->getHeader(); } /// for use in implementations of IColumn::gather() diff --git a/src/DataStreams/ConvertingBlockInputStream.h b/src/DataStreams/ConvertingBlockInputStream.h index b0324618408..0e5d183ee32 100644 --- a/src/DataStreams/ConvertingBlockInputStream.h +++ b/src/DataStreams/ConvertingBlockInputStream.h @@ -38,6 +38,7 @@ public: String getName() const override { return "Converting"; } Block getHeader() const override { return header; } + bool columnsCanDifferInRepresentationAmongBlocks() const override { return true; } private: Block readImpl() override; diff --git a/src/DataStreams/IBlockInputStream.cpp b/src/DataStreams/IBlockInputStream.cpp index a6484c41b4f..f58f95f4034 100644 --- a/src/DataStreams/IBlockInputStream.cpp +++ b/src/DataStreams/IBlockInputStream.cpp @@ -88,7 +88,13 @@ Block IBlockInputStream::read() { Block header = getHeader(); if (header) - assertBlocksHaveEqualStructure(res, header, getName()); + { + if (columnsCanDifferInRepresentationAmongBlocks()) + assertCompatibleHeader(res, header, getName()); + else + assertBlocksHaveEqualStructure(res, header, getName()); + } + } #endif diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h index 090ea394fd6..1b5e98294ad 100644 --- a/src/DataStreams/IBlockInputStream.h +++ b/src/DataStreams/IBlockInputStream.h @@ -192,6 +192,10 @@ public: quota = new_quota; } + /// If true, columns with same name and type may have different representation (normal, const, sparse) + /// in different blocks, that was read during stream execution. + virtual bool columnsCanDifferInRepresentationAmongBlocks() const { return false; } + /// Enable calculation of minimums and maximums by the result columns. void enableExtremes() { enabled_extremes = true; } diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.h b/src/Processors/Executors/PipelineExecutingBlockInputStream.h index 68497938ad4..adfabc7ad4e 100644 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.h +++ b/src/Processors/Executors/PipelineExecutingBlockInputStream.h @@ -18,6 +18,7 @@ public: String getName() const override { return "PipelineExecuting"; } Block getHeader() const override; + bool columnsCanDifferInRepresentationAmongBlocks() const override { return true; } void cancel(bool kill) override; diff --git a/src/Processors/Sources/SourceFromInputStream.cpp b/src/Processors/Sources/SourceFromInputStream.cpp index 7c88c6dfbeb..9ec3e6e8f50 100644 --- a/src/Processors/Sources/SourceFromInputStream.cpp +++ b/src/Processors/Sources/SourceFromInputStream.cpp @@ -180,7 +180,10 @@ Chunk SourceFromInputStream::generate() return {}; #ifndef NDEBUG - assertBlocksHaveEqualStructure(getPort().getHeader(), block, "SourceFromInputStream"); + if (stream->columnsCanDifferInRepresentationAmongBlocks()) + assertCompatibleHeader(getPort().getHeader(), block, "SourceFromInputStream"); + else + assertBlocksHaveEqualStructure(getPort().getHeader(), block, "SourceFromInputStream"); #endif UInt64 num_rows = block.rows(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 92e11738ce6..eba8c3f6f17 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -585,6 +585,7 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadUUID(); loadColumns(require_columns_checksums); loadChecksums(require_columns_checksums); + loadSerializationInfo(); loadIndexGranularity(); calculateColumnsSizesOnDisk(); loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity` @@ -598,8 +599,8 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks if (check_consistency) checkConsistency(require_columns_checksums); + loadDefaultCompressionCodec(); - loadSerializationInfo(); } void IMergeTreeDataPart::loadProjections(bool require_columns_checksums, bool check_consistency) From e41749997d2d2ea59261e9fe873ef76982b8fa2f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 7 Jun 2021 16:50:27 +0300 Subject: [PATCH 040/358] better interface for getting ratio of default values --- src/Columns/ColumnAggregateFunction.h | 10 +++++++ src/Columns/ColumnArray.cpp | 17 +++++------ src/Columns/ColumnArray.h | 7 ++++- src/Columns/ColumnCompressed.h | 2 ++ src/Columns/ColumnConst.h | 10 +++++++ src/Columns/ColumnDecimal.cpp | 10 ------- src/Columns/ColumnDecimal.h | 7 +++-- src/Columns/ColumnFixedString.cpp | 30 +++++++++++++++++++ src/Columns/ColumnFixedString.h | 7 +++++ src/Columns/ColumnFunction.h | 10 +++++++ src/Columns/ColumnLowCardinality.h | 6 ++++ src/Columns/ColumnMap.cpp | 10 +++++++ src/Columns/ColumnMap.h | 2 ++ src/Columns/ColumnNullable.cpp | 10 ------- src/Columns/ColumnNullable.h | 12 +++++--- src/Columns/ColumnSparse.cpp | 9 ++++-- src/Columns/ColumnSparse.h | 7 ++++- src/Columns/ColumnString.cpp | 9 ------ src/Columns/ColumnString.h | 8 +++-- src/Columns/ColumnTuple.cpp | 14 +++++++++ src/Columns/ColumnTuple.h | 2 ++ src/Columns/ColumnUnique.h | 6 ++++ src/Columns/ColumnVector.cpp | 10 ------- src/Columns/ColumnVector.h | 8 ++++- src/Columns/IColumn.h | 12 ++++---- src/Columns/IColumnDummy.h | 6 ++++ src/Columns/IColumnImpl.h | 24 +++++++++++++++ src/Columns/tests/gtest_column_sparse.cpp | 6 ---- .../Serializations/SerializationInfo.cpp | 15 ++++++---- .../Serializations/SerializationInfo.h | 5 ++-- .../Serializations/SerializationSparse.cpp | 2 +- 31 files changed, 210 insertions(+), 83 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index e75af280ed5..c2ecd27cc7e 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -132,6 +132,11 @@ public: void get(size_t n, Field & res) const override; + bool isDefaultAt(size_t) const override + { + throw Exception("Method isDefaultAt is not supported for ColumnAggregateFunction", ErrorCodes::NOT_IMPLEMENTED); + } + StringRef getDataAt(size_t n) const override; void insertData(const char * pos, size_t length) override; @@ -205,6 +210,11 @@ public: throw Exception("Method hasEqualValues is not supported for ColumnAggregateFunction", ErrorCodes::NOT_IMPLEMENTED); } + double getRatioOfDefaultRows(double) const override + { + throw Exception("Method getRatioOfDefaultRows is not supported for ColumnAggregateFunction", ErrorCodes::NOT_IMPLEMENTED); + } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 6e10a15d920..6d3e86388ad 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -181,6 +181,13 @@ StringRef ColumnArray::getDataAt(size_t n) const } +bool ColumnArray::isDefaultAt(size_t n) const +{ + const auto & offsets_data = getOffsets(); + return offsets_data[n] == offsets_data[n - 1]; +} + + void ColumnArray::insertData(const char * pos, size_t length) { /** Similarly - only for arrays of fixed length values. @@ -1216,16 +1223,6 @@ void ColumnArray::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } -size_t ColumnArray::getNumberOfDefaultRows(size_t step) const -{ - const auto & offsets_data = getOffsets(); - size_t res = 0; - for (size_t i = 0; i < offsets_data.size(); i += step) - res += (offsets_data[i] != offsets_data[i - 1]); - - return res; -} - void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const { const auto & offsets_data = getOffsets(); diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index fab6573f159..e3bde2ed570 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -58,6 +58,7 @@ public: Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; StringRef getDataAt(size_t n) const override; + bool isDefaultAt(size_t n) const override; void insertData(const char * pos, size_t length) override; StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override; const char * deserializeAndInsertFromArena(const char * pos) override; @@ -140,7 +141,11 @@ public: return false; } - size_t getNumberOfDefaultRows(size_t step) const override; + double getRatioOfDefaultRows(double sample_ratio) const override + { + return getRatioOfDefaultRowsImpl(sample_ratio); + } + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index 3cc2c014732..916473c373b 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -78,6 +78,7 @@ public: Field operator[](size_t) const override { throwMustBeDecompressed(); } void get(size_t, Field &) const override { throwMustBeDecompressed(); } StringRef getDataAt(size_t) const override { throwMustBeDecompressed(); } + bool isDefaultAt(size_t) const override { throwMustBeDecompressed(); } void insert(const Field &) override { throwMustBeDecompressed(); } void insertRangeFrom(const IColumn &, size_t, size_t) override { throwMustBeDecompressed(); } void insertData(const char *, size_t) override { throwMustBeDecompressed(); } @@ -108,6 +109,7 @@ public: void gather(ColumnGathererStream &) override { throwMustBeDecompressed(); } void getExtremes(Field &, Field &) const override { throwMustBeDecompressed(); } size_t byteSizeAt(size_t) const override { throwMustBeDecompressed(); } + double getRatioOfDefaultRows(double) const override { throwMustBeDecompressed(); } protected: size_t rows; diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 01d5b235a2b..c5223be1757 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -115,6 +115,11 @@ public: return data->getFloat32(0); } + bool isDefaultAt(size_t) const override + { + return data->isDefaultAt(0); + } + bool isNullAt(size_t) const override { return data->isNullAt(0); @@ -237,6 +242,11 @@ public: return false; } + double getRatioOfDefaultRows(double) const override + { + return data->isDefaultAt(0) ? 1.0 : 0.0; + } + bool isNullable() const override { return isColumnNullable(*data); } bool onlyNull() const override { return data->isNullAt(0); } bool isNumeric() const override { return data->isNumeric(); } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 3f3dffeb5b2..023a7577f84 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -383,16 +383,6 @@ ColumnPtr ColumnDecimal::compress() const }); } -template -size_t ColumnDecimal::getNumberOfDefaultRows(size_t step) const -{ - size_t res = 0; - for (size_t i = 0; i < data.size(); i += step) - res += (data[i] == T{}); - - return res; -} - template void ColumnDecimal::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 7886af58e84..9e274610f78 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -174,11 +174,14 @@ public: return false; } - ColumnPtr compress() const override; + double getRatioOfDefaultRows(double sample_ratio) const override + { + return this->template getRatioOfDefaultRowsImpl(sample_ratio); + } - size_t getNumberOfDefaultRows(size_t step) const override; void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + ColumnPtr compress() const override; void insertValue(const T value) { data.push_back(value); } Container & getData() { return data; } diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 4d54a46c924..4165776a6b9 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -51,6 +51,36 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const return new_col_holder; } +bool ColumnFixedString::isDefaultAt(size_t index) const +{ + static constexpr size_t SIMD_BYTES = 16; + + const UInt8 * pos = chars.data() + index * n; + const UInt8 * end = pos + n; + const UInt8 * end_sse = pos + n / SIMD_BYTES * SIMD_BYTES; + + const __m128i zero16 = _mm_setzero_si128(); + + while (pos < end_sse) + { + if (0xFFFF != _mm_movemask_epi8(_mm_cmpeq_epi8( + _mm_loadu_si128(reinterpret_cast(pos)), zero16))) + return false; + + pos += SIMD_BYTES; + } + + while (pos < end) + { + if (*pos != 0) + return false; + + ++pos; + } + + return true; +} + void ColumnFixedString::insert(const Field & x) { const String & s = DB::get(x); diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 5fd482aef6e..d037776ea9b 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -87,6 +87,8 @@ public: return StringRef(&chars[n * index], n); } + bool isDefaultAt(size_t index) const override; + void insert(const Field & x) override; void insertFrom(const IColumn & src_, size_t index) override; @@ -179,6 +181,11 @@ public: return false; } + double getRatioOfDefaultRows(double sample_ratio) const override + { + return getRatioOfDefaultRowsImpl(sample_ratio); + } + bool canBeInsideNullable() const override { return true; } bool isFixedAndContiguous() const override { return true; } diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index fa605e741aa..73fa3b5822b 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -68,6 +68,11 @@ public: throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + bool isDefaultAt(size_t) const override + { + throw Exception("isDefaultAt is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + void insert(const Field &) override { throw Exception("Cannot insert into " + getName(), ErrorCodes::NOT_IMPLEMENTED); @@ -153,6 +158,11 @@ public: throw Exception("Method gather is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + double getRatioOfDefaultRows(double) const override + { + throw Exception("Method getRatioOfDefaultRows is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + private: size_t size_; FunctionBasePtr function; diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index fc607021ccf..6736614d60b 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -64,6 +64,7 @@ public: return getDictionary().getDataAtWithTerminatingZero(getIndexes().getUInt(n)); } + bool isDefaultAt(size_t n) const override { return getDictionary().isDefaultAt(getIndexes().getUInt(n)); } UInt64 get64(size_t n) const override { return getDictionary().get64(getIndexes().getUInt(n)); } UInt64 getUInt(size_t n) const override { return getDictionary().getUInt(getIndexes().getUInt(n)); } Int64 getInt(size_t n) const override { return getDictionary().getInt(getIndexes().getUInt(n)); } @@ -175,6 +176,11 @@ public: return false; } + double getRatioOfDefaultRows(double sample_ratio) const override + { + return getRatioOfDefaultRowsImpl(sample_ratio); + } + bool valuesHaveFixedSize() const override { return getDictionary().valuesHaveFixedSize(); } bool isFixedAndContiguous() const override { return false; } size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 05c0e0458d8..2acdcf33080 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -81,6 +81,11 @@ void ColumnMap::get(size_t n, Field & res) const getNestedData().get(offset + i, map[i]); } +bool ColumnMap::isDefaultAt(size_t n) const +{ + return nested->isDefaultAt(n); +} + StringRef ColumnMap::getDataAt(size_t) const { throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); @@ -268,6 +273,11 @@ bool ColumnMap::structureEquals(const IColumn & rhs) const return false; } +double ColumnMap::getRatioOfDefaultRows(double sample_ratio) const +{ + return getRatioOfDefaultRowsImpl(sample_ratio); +} + ColumnPtr ColumnMap::compress() const { auto compressed = nested->compress(); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 17f0ccc422c..54c18eb98a1 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -51,6 +51,7 @@ public: Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; + bool isDefaultAt(size_t n) const override; StringRef getDataAt(size_t n) const override; void insertData(const char * pos, size_t length) override; void insert(const Field & x) override; @@ -84,6 +85,7 @@ public: void protect() override; void forEachSubcolumn(ColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; + double getRatioOfDefaultRows(double sample_ratio) const override; const ColumnArray & getNestedColumn() const { return assert_cast(*nested); } ColumnArray & getNestedColumn() { return assert_cast(*nested); } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 2b6dd1106b3..eeead6fe67c 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -686,16 +686,6 @@ void ColumnNullable::checkConsistency() const ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT); } -size_t ColumnNullable::getNumberOfDefaultRows(size_t step) const -{ - size_t res = 0; - const auto & null_map_data = getNullMapData(); - for (size_t i = 0; i < null_map_data.size(); i += step) - res += (null_map_data != 0); - - return res; -} - void ColumnNullable::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const { size_t to = limit && from + limit < size() ? from + limit : size(); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 55282f4569b..7d0918a6497 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -135,6 +135,14 @@ public: return false; } + double getRatioOfDefaultRows(double sample_ratio) const override + { + return null_map->getRatioOfDefaultRows(sample_ratio); + } + + void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; + bool isNullable() const override { return true; } bool isFixedAndContiguous() const override { return false; } bool valuesHaveFixedSize() const override { return nested_column->valuesHaveFixedSize(); } @@ -172,10 +180,6 @@ public: /// Check that size of null map equals to size of nested column. void checkConsistency() const; - size_t getNumberOfDefaultRows(size_t step) const override; - void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; - ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; - private: WrappedPtr nested_column; WrappedPtr null_map; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 56ce9096a75..9f182a5e992 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -66,6 +66,11 @@ MutableColumnPtr ColumnSparse::cloneResized(size_t new_size) const return res; } +bool ColumnSparse::isDefaultAt(size_t n) const +{ + return getValueIndex(n) == 0; +} + bool ColumnSparse::isNullAt(size_t n) const { return values->isNullAt(getValueIndex(n)); @@ -646,9 +651,9 @@ void ColumnSparse::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size indices.assign(start, end); } -size_t ColumnSparse::getNumberOfDefaultRows(size_t step) const +double ColumnSparse::getRatioOfDefaultRows(double) const { - return getNumberOfDefaults() / step; + return static_cast(getNumberOfDefaults()) / _size; } MutableColumns ColumnSparse::scatter(ColumnIndex num_columns, const Selector & selector) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 69ee66a37e8..0d2b17b790d 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -28,6 +28,10 @@ private: ColumnSparse(const ColumnSparse &) = default; public: + static constexpr auto DEFAULT_ROWS_SEARCH_SAMPLE_RATIO = 0.1; + static constexpr auto DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION = 0.95; + // static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; + using Base = COWHelper; static Ptr create(const ColumnPtr & values_, const ColumnPtr & offsets_, size_t size_) { @@ -57,6 +61,7 @@ public: TypeIndex getDataType() const override { return values->getDataType(); } MutableColumnPtr cloneResized(size_t new_size) const override; size_t size() const override { return _size; } + bool isDefaultAt(size_t n) const override; bool isNullAt(size_t n) const override; Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; @@ -117,7 +122,7 @@ public: void getExtremes(Field & min, Field & max) const override; void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; - size_t getNumberOfDefaultRows(size_t step) const override; + double getRatioOfDefaultRows(double sample_ratio) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 437e7e4bd7a..4370e7a9491 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -545,15 +545,6 @@ void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, indices.push_back(i); } -size_t ColumnString::getNumberOfDefaultRows(size_t step) const -{ - size_t res = 0; - for (size_t i = 0; i < offsets.size(); i += step) - res += (offsets[i] == offsets[i - 1] + 1); - - return res; -} - ColumnPtr ColumnString::compress() const { size_t source_chars_size = chars.size(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 6dc7c9ecc57..503c84a2d6d 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -113,7 +113,7 @@ public: bool isDefaultAt(size_t n) const override { assert(n < size()); - return offsetAt(n) == 0; + return sizeAt(n) == 1; } /// Suppress gcc 7.3.1 warning: '*((void*)& +8)' may be used uninitialized in this function @@ -285,8 +285,12 @@ public: return typeid(rhs) == typeid(ColumnString); } + double getRatioOfDefaultRows(double sample_ratio) const override + { + return getRatioOfDefaultRowsImpl(sample_ratio); + } + void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; - size_t getNumberOfDefaultRows(size_t step) const override; Chars & getChars() { return chars; } const Chars & getChars() const { return chars; } diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index bb59d58b75d..33423419a17 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -113,6 +113,15 @@ void ColumnTuple::get(size_t n, Field & res) const res = tuple; } +bool ColumnTuple::isDefaultAt(size_t n) const +{ + const size_t tuple_size = columns.size(); + for (size_t i = 0; i < tuple_size; ++i) + if (!columns[i]->isDefaultAt(n)) + return false; + return true; +} + StringRef ColumnTuple::getDataAt(size_t) const { throw Exception("Method getDataAt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); @@ -530,4 +539,9 @@ ColumnPtr ColumnTuple::compress() const }); } +double ColumnTuple::getRatioOfDefaultRows(double sample_ratio) const +{ + return getRatioOfDefaultRowsImpl(sample_ratio); +} + } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 3f5422c7719..14db57bf5a0 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -53,6 +53,7 @@ public: Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; + bool isDefaultAt(size_t n) const override; StringRef getDataAt(size_t n) const override; void insertData(const char * pos, size_t length) override; void insert(const Field & x) override; @@ -92,6 +93,7 @@ public: bool structureEquals(const IColumn & rhs) const override; bool isCollationSupported() const override; ColumnPtr compress() const override; + double getRatioOfDefaultRows(double sample_ratio) const override; size_t tupleSize() const { return columns.size(); } diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 51b45be53fd..6d6aa26797d 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -67,6 +67,7 @@ public: Field operator[](size_t n) const override { return (*getNestedColumn())[n]; } void get(size_t n, Field & res) const override { getNestedColumn()->get(n, res); } + bool isDefaultAt(size_t n) const override { return getNestedColumn()->isDefaultAt(n); } StringRef getDataAt(size_t n) const override { return getNestedColumn()->getDataAt(n); } StringRef getDataAtWithTerminatingZero(size_t n) const override { @@ -121,6 +122,11 @@ public: return false; } + double getRatioOfDefaultRows(double sample_ratio) const override + { + return getNestedColumn()->getRatioOfDefaultRows(sample_ratio); + } + const UInt64 * tryGetSavedHash() const override { return reverse_index.tryGetSavedHash(); } UInt128 getHash() const override { return hash.getHash(*getRawColumnPtr()); } diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 44c6132f23c..e4581709f8e 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -562,16 +562,6 @@ ColumnPtr ColumnVector::compress() const }); } -template -size_t ColumnVector::getNumberOfDefaultRows(size_t step) const -{ - size_t res = 0; - for (size_t i = 0; i < data.size(); i += step) - res += (data[i] == T{}); - - return res; -} - template void ColumnVector::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const { diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 3001c88a516..9aadc937402 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -316,12 +316,18 @@ public: return StringRef(reinterpret_cast(&data[n]), sizeof(data[n])); } + bool isDefaultAt(size_t n) const override { return data[n] == T{}; } + bool structureEquals(const IColumn & rhs) const override { return typeid(rhs) == typeid(ColumnVector); } - size_t getNumberOfDefaultRows(size_t step) const override; + double getRatioOfDefaultRows(double sample_ratio) const override + { + return this->template getRatioOfDefaultRowsImpl(sample_ratio); + } + void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 7c0b6647a23..617a4351a51 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -142,7 +142,7 @@ public: throw Exception("Method getInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - virtual bool isDefaultAt(size_t n) const { return get64(n) == 0; } + virtual bool isDefaultAt(size_t n) const = 0; virtual bool isNullAt(size_t /*n*/) const { return false; } /** If column is numeric, return value of n-th element, casted to bool. @@ -376,14 +376,10 @@ public: throw Exception("Method structureEquals is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - static constexpr auto DEFAULT_ROWS_SEARCH_STEP = 8; - static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; - static constexpr auto DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION = 0.95; - /// Returns number of values in column, that equal to default value of column. /// Checks every @step-th value. So, if step is not 1, returns number, /// that lower than actual. 0 means, that such statistic is unknown for column. - virtual size_t getNumberOfDefaultRows(size_t /* step */) const { return 0; } + virtual double getRatioOfDefaultRows(double sample_ratio = 1.0) const = 0; /// Returns indices of values in column, that not equal to default value of column. virtual void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const; @@ -490,7 +486,6 @@ public: String dumpStructure() const; protected: - /// Template is to devirtualize calls to insertFrom method. /// In derived classes (that use final keyword), implement scatter method as call to scatterImpl. template @@ -510,6 +505,9 @@ protected: template bool hasEqualValuesImpl() const; + + template + double getRatioOfDefaultRowsImpl(double sample_ratio) const; }; using ColumnPtr = IColumn::Ptr; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 7e1958f077e..016866a5156 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -46,6 +46,7 @@ public: Field operator[](size_t) const override { throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); } void get(size_t, Field &) const override { throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); } void insert(const Field &) override { throw Exception("Cannot insert element into " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + bool isDefaultAt(size_t) const override { throw Exception("isDefaultAt is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } StringRef getDataAt(size_t) const override { @@ -152,6 +153,11 @@ public: return res; } + double getRatioOfDefaultRows(double) const override + { + throw Exception("Method getRatioOfDefaultRows is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + void gather(ColumnGathererStream &) override { throw Exception("Method gather is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index a1ee6a17982..a47c7a1485c 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -139,4 +139,28 @@ bool IColumn::hasEqualValuesImpl() const return true; } + +template +double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const +{ + assert(sample_ratio > 0 && sample_ratio <= 1.0); + + size_t num_rows = size(); + size_t num_sampled_rows = static_cast(num_rows * sample_ratio); + if (num_sampled_rows == 0) + return 0.0; + + size_t step = num_rows / num_sampled_rows; + std::uniform_int_distribution dist(1, step); + + size_t res = 0; + for (size_t i = 0; i < num_rows; i += step) + { + size_t idx = std::min(i + dist(thread_local_rng), num_rows - 1); + res += static_cast(*this).isDefaultAt(idx); + } + + return static_cast(res) / num_sampled_rows; +} + } diff --git a/src/Columns/tests/gtest_column_sparse.cpp b/src/Columns/tests/gtest_column_sparse.cpp index 8d720cbe0d4..56284b75204 100644 --- a/src/Columns/tests/gtest_column_sparse.cpp +++ b/src/Columns/tests/gtest_column_sparse.cpp @@ -63,7 +63,6 @@ constexpr size_t sparse_ratios[] = {1, 2, 5, 10, 32, 50, 64, 100, 256, 500, 1000 constexpr size_t K = sizeof(sparse_ratios) / sizeof(sparse_ratios[0]); #define DUMP_COLUMN(column) std::cerr << #column << ": " << (column)->dumpStructure() << "\n" -#define DUMP_NON_DEFAULTS(column) std::cerr << "non-default values in " #column ": " << (column)->size() - (column)->getNumberOfDefaultRows(1) << "\n" TEST(ColumnSparse, InsertRangeFrom) { @@ -81,7 +80,6 @@ TEST(ColumnSparse, InsertRangeFrom) DUMP_COLUMN(full_src); DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); - DUMP_NON_DEFAULTS(full_dst); throw Exception(error_code, "Columns are unequal"); } }; @@ -124,7 +122,6 @@ TEST(ColumnSparse, PopBack) { DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); - DUMP_NON_DEFAULTS(full_dst); throw Exception(error_code, "Columns are unequal"); } }; @@ -165,7 +162,6 @@ TEST(ColumnSparse, Filter) DUMP_COLUMN(full_src); DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); - DUMP_NON_DEFAULTS(full_dst); throw Exception(error_code, "Columns are unequal"); } }; @@ -212,7 +208,6 @@ TEST(ColumnSparse, Permute) DUMP_COLUMN(full_src); DUMP_COLUMN(sparse_dst); DUMP_COLUMN(full_dst); - DUMP_NON_DEFAULTS(full_dst); throw Exception(error_code, "Columns are unequal"); } }; @@ -305,7 +300,6 @@ TEST(ColumnSparse, GetPermutation) DUMP_COLUMN(full_src); DUMP_COLUMN(sparse_sorted); DUMP_COLUMN(full_sorted); - DUMP_NON_DEFAULTS(full_sorted); throw Exception(error_code, "Sorted columns are unequal"); } }; diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 384cba1f735..fe8e9f52ad6 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -21,16 +21,17 @@ namespace ErrorCodes SerializationInfoBuilder::SerializationInfoBuilder( double ratio_for_sparse_serialization_, - size_t default_rows_search_step_) + double default_rows_search_sample_ratio_) : ratio_for_sparse_serialization(ratio_for_sparse_serialization_) - , default_rows_search_step(default_rows_search_step_) + , default_rows_search_sample_ratio(default_rows_search_sample_ratio_) , info(std::make_shared()) { } void SerializationInfoBuilder::add(const Block & block) { - info->number_of_rows += block.rows(); + size_t num_rows = block.rows(); + info->number_of_rows += num_rows; for (const auto & elem : block) { /// Just skip column and always return default serialization. @@ -38,7 +39,9 @@ void SerializationInfoBuilder::add(const Block & block) continue; /// Multiply by step to restore approximate number of default values. - info->columns[elem.name].num_defaults += elem.column->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; + info->columns[elem.name].num_defaults += static_cast( + num_rows * elem.column->getRatioOfDefaultRows(default_rows_search_sample_ratio)); + for (const auto & subcolumn_name : elem.type->getSubcolumnNames()) { auto subcolumn_type = elem.type->getSubcolumnType(subcolumn_name); @@ -55,7 +58,9 @@ void SerializationInfoBuilder::add(const Block & block) auto subcolumn = elem.type->getSubcolumn(subcolumn_name, *elem.column); auto full_name = Nested::concatenateName(elem.name, subcolumn_name); - info->columns[full_name].num_defaults += subcolumn->getNumberOfDefaultRows(default_rows_search_step) * default_rows_search_step; + + info->columns[full_name].num_defaults += static_cast( + num_rows * subcolumn->getRatioOfDefaultRows(default_rows_search_sample_ratio)); } } } diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 0685d61e080..5d22e39de84 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -65,7 +66,7 @@ public: SerializationInfoBuilder(); SerializationInfoBuilder( double ratio_for_sparse_serialization_, - size_t default_rows_search_step_ = IColumn::DEFAULT_ROWS_SEARCH_STEP); + double default_rows_search_sample_ratio_ = ColumnSparse::DEFAULT_ROWS_SEARCH_SAMPLE_RATIO); /// Add information about column from block. void add(const Block & block); @@ -86,7 +87,7 @@ public: private: double ratio_for_sparse_serialization; - size_t default_rows_search_step; + double default_rows_search_sample_ratio; SerializationInfoPtr info; }; diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 080a190e963..480618a3532 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -66,7 +66,7 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } /// Just try to guess number of offsets. - offsets.reserve(static_cast(limit * IColumn::DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION)); + offsets.reserve(static_cast(limit * ColumnSparse::DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION)); bool first = true; size_t total_rows = state.num_trailing_defaults; From eaf806a731e3a988fed7f0bce79cb05842a95065 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 7 Jun 2021 18:34:22 +0300 Subject: [PATCH 041/358] better interface for getIndicesOfNonDefaultRows --- src/Columns/ColumnAggregateFunction.h | 5 +++++ src/Columns/ColumnArray.cpp | 9 --------- src/Columns/ColumnArray.h | 5 ++++- src/Columns/ColumnCompressed.h | 1 + src/Columns/ColumnConst.h | 11 +++++++++++ src/Columns/ColumnDecimal.cpp | 10 ---------- src/Columns/ColumnDecimal.h | 5 ++++- src/Columns/ColumnFixedString.h | 5 +++++ src/Columns/ColumnFunction.h | 5 +++++ src/Columns/ColumnLowCardinality.h | 5 +++++ src/Columns/ColumnMap.cpp | 5 +++++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 11 ----------- src/Columns/ColumnNullable.h | 6 +++++- src/Columns/ColumnSparse.cpp | 4 ++-- src/Columns/ColumnSparse.h | 2 +- src/Columns/ColumnString.cpp | 9 --------- src/Columns/ColumnString.h | 5 ++++- src/Columns/ColumnTuple.cpp | 5 +++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnUnique.h | 5 +++++ src/Columns/ColumnVector.cpp | 10 ---------- src/Columns/ColumnVector.h | 6 +++++- src/Columns/IColumn.cpp | 8 -------- src/Columns/IColumn.h | 11 +++++++---- src/Columns/IColumnDummy.h | 5 +++++ src/Columns/IColumnImpl.h | 13 +++++++++++++ src/DataTypes/NestedUtils.h | 2 +- .../Serializations/SerializationSparse.cpp | 2 +- .../0_stateless/01780_column_sparse.reference | 4 ++-- tests/queries/0_stateless/01780_column_sparse.sql | 4 ++-- 31 files changed, 105 insertions(+), 75 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index c2ecd27cc7e..ce53e39ff33 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -215,6 +215,11 @@ public: throw Exception("Method getRatioOfDefaultRows is not supported for ColumnAggregateFunction", ErrorCodes::NOT_IMPLEMENTED); } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override + { + throw Exception("Method getIndicesOfNonDefaultRows is not supported for ColumnAggregateFunction", ErrorCodes::NOT_IMPLEMENTED); + } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index 6d3e86388ad..9d5570cb5da 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -1223,13 +1223,4 @@ void ColumnArray::gather(ColumnGathererStream & gatherer) gatherer.gather(*this); } -void ColumnArray::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const -{ - const auto & offsets_data = getOffsets(); - size_t to = limit && from + limit < size() ? from + limit : size(); - for (size_t i = from; i < to; ++i) - if (offsets_data[i] != offsets_data[i - 1]) - indices.push_back(i); -} - } diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index e3bde2ed570..f01ff88daef 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -146,7 +146,10 @@ public: return getRatioOfDefaultRowsImpl(sample_ratio); } - void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override + { + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); + } bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index 916473c373b..8c24898747f 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -110,6 +110,7 @@ public: void getExtremes(Field &, Field &) const override { throwMustBeDecompressed(); } size_t byteSizeAt(size_t) const override { throwMustBeDecompressed(); } double getRatioOfDefaultRows(double) const override { throwMustBeDecompressed(); } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override { throwMustBeDecompressed(); } protected: size_t rows; diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index c5223be1757..59e1e79696e 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -247,6 +248,16 @@ public: return data->isDefaultAt(0) ? 1.0 : 0.0; } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override + { + if (!data->isDefaultAt(0)) + { + indices.reserve(indices.size() + limit); + for (size_t i = from; i < from + limit; ++i) + indices.push_back(i); + } + } + bool isNullable() const override { return isColumnNullable(*data); } bool onlyNull() const override { return data->isNullAt(0); } bool isNumeric() const override { return data->isNumeric(); } diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 023a7577f84..ec08db274b3 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -383,16 +383,6 @@ ColumnPtr ColumnDecimal::compress() const }); } -template -void ColumnDecimal::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const -{ - size_t to = limit && from + limit < size() ? from + limit : size(); - indices.reserve(indices.size() + to - from); - for (size_t i = from; i < to; ++i) - if (data[i] != T{}) - indices.push_back(i); -} - template void ColumnDecimal::getExtremes(Field & min, Field & max) const { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 9e274610f78..fb91e946cb2 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -179,7 +179,10 @@ public: return this->template getRatioOfDefaultRowsImpl(sample_ratio); } - void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override + { + return this->template getIndicesOfNonDefaultRowsImpl(indices, from, limit); + } ColumnPtr compress() const override; diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index d037776ea9b..cc8e2c06fe8 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -186,6 +186,11 @@ public: return getRatioOfDefaultRowsImpl(sample_ratio); } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override + { + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); + } + bool canBeInsideNullable() const override { return true; } bool isFixedAndContiguous() const override { return true; } diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index 73fa3b5822b..dda828ab78a 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -163,6 +163,11 @@ public: throw Exception("Method getRatioOfDefaultRows is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override + { + throw Exception("Method getIndicesOfNonDefaultRows is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + private: size_t size_; FunctionBasePtr function; diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 6736614d60b..f968b5d1a7c 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -181,6 +181,11 @@ public: return getRatioOfDefaultRowsImpl(sample_ratio); } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override + { + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); + } + bool valuesHaveFixedSize() const override { return getDictionary().valuesHaveFixedSize(); } bool isFixedAndContiguous() const override { return false; } size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); } diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 2acdcf33080..3946f771963 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -278,6 +278,11 @@ double ColumnMap::getRatioOfDefaultRows(double sample_ratio) const return getRatioOfDefaultRowsImpl(sample_ratio); } +void ColumnMap::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const +{ + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); +} + ColumnPtr ColumnMap::compress() const { auto compressed = nested->compress(); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 54c18eb98a1..cf29aa3930c 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -86,6 +86,7 @@ public: void forEachSubcolumn(ColumnCallback callback) override; bool structureEquals(const IColumn & rhs) const override; double getRatioOfDefaultRows(double sample_ratio) const override; + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; const ColumnArray & getNestedColumn() const { return assert_cast(*nested); } ColumnArray & getNestedColumn() { return assert_cast(*nested); } diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index eeead6fe67c..206f913439e 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -686,17 +686,6 @@ void ColumnNullable::checkConsistency() const ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT); } -void ColumnNullable::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const -{ - size_t to = limit && from + limit < size() ? from + limit : size(); - indices.reserve(indices.size() + to - from); - - const auto & null_map_data = getNullMapData(); - for (size_t i = from; i < to; ++i) - if (null_map_data[i] == 0) - indices.push_back(i); -} - ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const { auto new_values = nested_column->createWithOffsets(offsets, total_rows, shift); diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 7d0918a6497..5609b43a16a 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -140,7 +140,11 @@ public: return null_map->getRatioOfDefaultRows(sample_ratio); } - void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override + { + null_map->getIndicesOfNonDefaultRows(indices, from, limit); + } + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; bool isNullable() const override { return true; } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 9f182a5e992..3802f16b76d 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -642,13 +642,13 @@ void ColumnSparse::getExtremes(Field & min, Field & max) const values->getExtremes(min, max); } -void ColumnSparse::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const +void ColumnSparse::getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const { const auto & offsets_data = getOffsetsData(); const auto * start = from ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from) : offsets_data.begin(); const auto * end = limit ? std::lower_bound(offsets_data.begin(), offsets_data.end(), from + limit) : offsets_data.end(); - indices.assign(start, end); + indices.insert(start, end); } double ColumnSparse::getRatioOfDefaultRows(double) const diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 0d2b17b790d..e3088032361 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -121,7 +121,7 @@ public: void updateHashFast(SipHash & hash) const override; void getExtremes(Field & min, Field & max) const override; - void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override; double getRatioOfDefaultRows(double sample_ratio) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 4370e7a9491..fcd0516d465 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -536,15 +536,6 @@ void ColumnString::getExtremes(Field & min, Field & max) const get(max_idx, max); } -void ColumnString::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const -{ - size_t to = limit && from + limit < size() ? from + limit : size(); - indices.reserve(indices.size() + to - from); - for (size_t i = from; i < to; ++i) - if (offsets[i] - offsets[i - 1] > 1) - indices.push_back(i); -} - ColumnPtr ColumnString::compress() const { size_t source_chars_size = chars.size(); diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 503c84a2d6d..dfd2b5debcc 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -290,7 +290,10 @@ public: return getRatioOfDefaultRowsImpl(sample_ratio); } - void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const override; + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override + { + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); + } Chars & getChars() { return chars; } const Chars & getChars() const { return chars; } diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 33423419a17..d13a09a42db 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -544,4 +544,9 @@ double ColumnTuple::getRatioOfDefaultRows(double sample_ratio) const return getRatioOfDefaultRowsImpl(sample_ratio); } +void ColumnTuple::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const +{ + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); +} + } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 14db57bf5a0..416e02998b0 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -94,6 +94,7 @@ public: bool isCollationSupported() const override; ColumnPtr compress() const override; double getRatioOfDefaultRows(double sample_ratio) const override; + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; size_t tupleSize() const { return columns.size(); } diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 6d6aa26797d..ebf8b747d55 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -127,6 +127,11 @@ public: return getNestedColumn()->getRatioOfDefaultRows(sample_ratio); } + void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override + { + return getNestedColumn()->getIndicesOfNonDefaultRows(indices, from, limit); + } + const UInt64 * tryGetSavedHash() const override { return reverse_index.tryGetSavedHash(); } UInt128 getHash() const override { return hash.getHash(*getRawColumnPtr()); } diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index e4581709f8e..3a8c24b8353 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -562,16 +562,6 @@ ColumnPtr ColumnVector::compress() const }); } -template -void ColumnVector::getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const -{ - size_t to = limit && from + limit < size() ? from + limit : size(); - indices.reserve(indices.size() + to - from); - for (size_t i = from; i < to; ++i) - if (data[i] != T{}) - indices.push_back(i); -} - template ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const { diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 9aadc937402..74537da676e 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -328,7 +328,11 @@ public: return this->template getRatioOfDefaultRowsImpl(sample_ratio); } - void getIndicesOfNonDefaultValues(IColumn::Offsets & indices, size_t from, size_t limit) const override; + void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override + { + return this->template getIndicesOfNonDefaultRowsImpl(indices, from, limit); + } + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; ColumnPtr compress() const override; diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 45c7967072f..94fbe04b51a 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -36,14 +36,6 @@ void IColumn::insertFrom(const IColumn & src, size_t n) insert(src[n]); } -void IColumn::getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const -{ - size_t to = limit && from + limit < size() ? from + limit : size(); - indices.reserve(indices.size() + to - from); - for (size_t i = from; i < to; ++i) - indices.push_back(i); -} - ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows, size_t shift) const { if (offsets.size() + shift != size()) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 617a4351a51..e7c79d6fbbc 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -376,13 +376,12 @@ public: throw Exception("Method structureEquals is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /// Returns number of values in column, that equal to default value of column. - /// Checks every @step-th value. So, if step is not 1, returns number, - /// that lower than actual. 0 means, that such statistic is unknown for column. + /// Returns ration of values in column, that equal to default value of column. + /// Checks only @sample_ratio ratio of rows. virtual double getRatioOfDefaultRows(double sample_ratio = 1.0) const = 0; /// Returns indices of values in column, that not equal to default value of column. - virtual void getIndicesOfNonDefaultValues(Offsets & indices, size_t from, size_t limit) const; + virtual void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const = 0; /// Returns column with @total_size elements. /// In result column values from current column are at positions from @offsets. @@ -506,8 +505,12 @@ protected: template bool hasEqualValuesImpl() const; + /// Template is to devirtualize calls to 'isDefaultAt' method. template double getRatioOfDefaultRowsImpl(double sample_ratio) const; + + template + void getIndicesOfNonDefaultRowsImpl(Offsets & indices, size_t from, size_t limit) const; }; using ColumnPtr = IColumn::Ptr; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 016866a5156..81e066a65c4 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -158,6 +158,11 @@ public: throw Exception("Method getRatioOfDefaultRows is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + void getIndicesOfNonDefaultRows(Offsets &, size_t, size_t) const override + { + throw Exception("Method getIndicesOfNonDefaultRows is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + void gather(ColumnGathererStream &) override { throw Exception("Method gather is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index a47c7a1485c..fe9ad251111 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -163,4 +163,17 @@ double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const return static_cast(res) / num_sampled_rows; } +template +void IColumn::getIndicesOfNonDefaultRowsImpl(Offsets & indices, size_t from, size_t limit) const +{ + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + + for (size_t i = from; i < to; ++i) + { + if (!static_cast(*this).isDefaultAt(i)) + indices.push_back(i); + } +} + } diff --git a/src/DataTypes/NestedUtils.h b/src/DataTypes/NestedUtils.h index c3bc9d17589..3c203fcd911 100644 --- a/src/DataTypes/NestedUtils.h +++ b/src/DataTypes/NestedUtils.h @@ -11,7 +11,7 @@ namespace Nested { std::string concatenateName(const std::string & nested_table_name, const std::string & nested_field_name); - /// Splits name of compound indentifier by firts/last dot (depending on 'reverse' parameter). + /// Splits name of compound identifier by first/last dot (depending on 'reverse' parameter). std::pair splitName(const std::string & name, bool reverse = false); /// Returns the prefix of the name to the first '.'. Or the name is unchanged if there is no dot. diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 480618a3532..abdd869b6d0 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -165,7 +165,7 @@ void SerializationSparse::serializeBinaryBulkWithMultipleStreams( auto offsets_column = DataTypeNumber().createColumn(); auto & offsets_data = assert_cast &>(*offsets_column).getData(); - column.getIndicesOfNonDefaultValues(offsets_data, offset, limit); + column.getIndicesOfNonDefaultRows(offsets_data, offset, limit); settings.path.push_back(Substream::SparseOffsets); if (auto * stream = settings.getter(settings.path)) diff --git a/tests/queries/0_stateless/01780_column_sparse.reference b/tests/queries/0_stateless/01780_column_sparse.reference index 850ebcee349..08aef433172 100644 --- a/tests/queries/0_stateless/01780_column_sparse.reference +++ b/tests/queries/0_stateless/01780_column_sparse.reference @@ -166,13 +166,13 @@ SELECT * FROM t_sparse_1 ORDER BY v DESC; 11 0 4 -1 7 -2 -SELECT * FROM t_sparse_1 ORDER BY v LIMIT 5; +SELECT * FROM t_sparse_1 ORDER BY v, id LIMIT 5; 7 -2 4 -1 3 0 5 0 6 0 -SELECT * FROM t_sparse_1 ORDER BY v DESC LIMIT 5; +SELECT * FROM t_sparse_1 ORDER BY v DESC, id LIMIT 5; 1 6 10 4 2 1 diff --git a/tests/queries/0_stateless/01780_column_sparse.sql b/tests/queries/0_stateless/01780_column_sparse.sql index 6dc53cf07c6..480321c6d14 100644 --- a/tests/queries/0_stateless/01780_column_sparse.sql +++ b/tests/queries/0_stateless/01780_column_sparse.sql @@ -37,8 +37,8 @@ INSERT INTO t_sparse_1 VALUES (1, 6), (2, 1), (3, 0), (4, -1), (5, 0), (6, 0), ( SELECT * FROM t_sparse_1 ORDER BY v; SELECT * FROM t_sparse_1 ORDER BY v DESC; -SELECT * FROM t_sparse_1 ORDER BY v LIMIT 5; -SELECT * FROM t_sparse_1 ORDER BY v DESC LIMIT 5; +SELECT * FROM t_sparse_1 ORDER BY v, id LIMIT 5; +SELECT * FROM t_sparse_1 ORDER BY v DESC, id LIMIT 5; DROP TABLE t_sparse; DROP TABLE t_sparse_1; From 384ea35ed9ebbe94ccd74b73c0ede9ba063b4298 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 7 Jun 2021 18:44:47 +0300 Subject: [PATCH 042/358] fix getIndicesOfNonDefaultRows with ColumnConst --- src/Columns/ColumnConst.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 59e1e79696e..81d5808c73a 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -252,8 +252,9 @@ public: { if (!data->isDefaultAt(0)) { - indices.reserve(indices.size() + limit); - for (size_t i = from; i < from + limit; ++i) + size_t to = limit && from + limit < size() ? from + limit : size(); + indices.reserve(indices.size() + to - from); + for (size_t i = from; i < to; ++i) indices.push_back(i); } } From 5d64df1e39709da44ebf90ca50a8ddb54fc55a2d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 8 Jun 2021 12:35:58 +0300 Subject: [PATCH 043/358] disable sparse columns by default --- src/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 6e48209c490..d71268a7b2c 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -33,7 +33,7 @@ struct Settings; M(UInt64, min_rows_for_compact_part, 0, "Experimental. Minimal number of rows to create part in compact format instead of saving it in RAM", 0) \ M(Bool, in_memory_parts_enable_wal, true, "Whether to write blocks in Native format to write-ahead-log before creation in-memory part", 0) \ M(UInt64, write_ahead_log_max_bytes, 1024 * 1024 * 1024, "Rotate WAL, if it exceeds that amount of bytes", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, -1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + M(Float, ratio_of_defaults_for_sparse_serialization, 1.0, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ \ /** Merge settings. */ \ M(UInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ From 2b0ec88f5a56fd200ad843a209b809ee5bbe77a2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 20 Jul 2021 20:02:41 +0300 Subject: [PATCH 044/358] disable jit aggregation for sparse columns --- src/Interpreters/Aggregator.cpp | 22 ++++++++++------------ src/Interpreters/Aggregator.h | 2 ++ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index a51ab7f236b..170a08519af 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -579,6 +579,14 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const } } +bool Aggregator::hasSparseArguments(AggregateFunctionInstruction * aggregate_instructions) +{ + for (auto * inst = aggregate_instructions; inst->that; ++inst) + if (inst->has_sparse_arguments) + return true; + return false; +} + /** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%). * (Probably because after the inline of this function, more internal functions no longer be inlined.) * Inline does not make sense, since the inner loop is entirely inside this function. @@ -598,7 +606,7 @@ void NO_INLINE Aggregator::executeImpl( if (!no_more_keys) { #if USE_EMBEDDED_COMPILER - if (compiled_aggregate_functions_holder) + if (compiled_aggregate_functions_holder && !hasSparseArguments(aggregate_instructions)) { executeImplBatch(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row); } @@ -650,17 +658,7 @@ void NO_INLINE Aggregator::executeImplBatch( } } - bool has_sparse = false; - for (auto * inst = aggregate_instructions; inst->that; ++inst) - { - if (inst->has_sparse_arguments) - { - has_sparse = true; - break; - } - } - - if (!has_arrays && !has_sparse) + if (!has_arrays && !hasSparseArguments(aggregate_instructions)) { for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) { diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 780b8e0f372..4d2e439bc57 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1318,6 +1318,8 @@ private: AggregatedDataVariants & data_variants, Columns & key_columns, size_t key_row, MutableColumns & final_key_columns) const; + + static bool hasSparseArguments(AggregateFunctionInstruction * aggregate_instructions); }; From 272c066cf9f6a81031f60bd2867788fa554bbcef Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 20 Jul 2021 20:10:19 +0300 Subject: [PATCH 045/358] fix special builds --- src/Columns/ColumnFixedString.cpp | 7 ++++--- src/DataTypes/Serializations/SerializationInfo.cpp | 2 +- src/DataTypes/Serializations/SerializationSparse.cpp | 2 +- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index a0cf8782a9a..457a7d28df6 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -53,12 +53,12 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const bool ColumnFixedString::isDefaultAt(size_t index) const { - static constexpr size_t SIMD_BYTES = 16; - const UInt8 * pos = chars.data() + index * n; const UInt8 * end = pos + n; - const UInt8 * end_sse = pos + n / SIMD_BYTES * SIMD_BYTES; +#ifdef __SSE2__ + static constexpr size_t SIMD_BYTES = 16; + const UInt8 * end_sse = pos + n / SIMD_BYTES * SIMD_BYTES; const __m128i zero16 = _mm_setzero_si128(); while (pos < end_sse) @@ -69,6 +69,7 @@ bool ColumnFixedString::isDefaultAt(size_t index) const pos += SIMD_BYTES; } +#endif while (pos < end) { diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index fe8e9f52ad6..366a657eb71 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -87,7 +87,7 @@ SerializationInfoPtr SerializationInfoBuilder::build() SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo & other) { - for (auto & [name, column_info] : other.columns) + for (const auto & [name, column_info] : other.columns) { auto it = info->columns.find(name); if (it == info->columns.end()) diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index abdd869b6d0..bc638a54fdc 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -21,7 +21,7 @@ namespace { /// 2^62, because VarInt supports only values < 2^63. -static constexpr auto END_OF_GRANULE_FLAG = 1ULL << 62; +constexpr auto END_OF_GRANULE_FLAG = 1ULL << 62; struct DeserializeStateSparse : public ISerialization::DeserializeBinaryBulkState { From 2e593658cb9a1215509d593bc4efd21952fd4db3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 2 Aug 2021 18:42:16 +0300 Subject: [PATCH 046/358] fix tests --- .../test.py | 2 +- .../test_s3_zero_copy_replication/test.py | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py index f426c3619a4..552d6b16e83 100644 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py @@ -9,7 +9,7 @@ from pyhdfs import HdfsClient SHARDS = 2 FILES_OVERHEAD_PER_TABLE = 1 # format_version.txt -FILES_OVERHEAD_PER_PART_COMPACT = 7 +FILES_OVERHEAD_PER_PART_COMPACT = 8 def wait_for_hdfs_objects(cluster, fp, expected, num_tries=30): diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index ecea3d3592f..994e8082502 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -77,16 +77,16 @@ def test_s3_zero_copy_replication(cluster, policy): assert node2.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" assert node1.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" - # Based on version 20.x - two parts - wait_for_large_objects_count(cluster, 2) + # Based on version 21.x - two parts + wait_for_large_objects_count(cluster, 4) node1.query("OPTIMIZE TABLE s3_test FINAL") - # Based on version 20.x - after merge, two old parts and one merged - wait_for_large_objects_count(cluster, 3) + # Based on version 21.x - after merge, two old parts and one merged + wait_for_large_objects_count(cluster, 6) - # Based on version 20.x - after cleanup - only one merged part - wait_for_large_objects_count(cluster, 1, timeout=60) + # Based on version 21.x - after cleanup - only one merged part + wait_for_large_objects_count(cluster, 2, timeout=60) node1.query("DROP TABLE IF EXISTS s3_test NO DELAY") node2.query("DROP TABLE IF EXISTS s3_test NO DELAY") From f3a77bb20d3a84b432c7e73505601722afde2351 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 20 Aug 2021 02:22:50 +0300 Subject: [PATCH 047/358] slightly better --- src/DataTypes/Serializations/SerializationInfo.cpp | 4 ++-- src/DataTypes/Serializations/SerializationInfo.h | 7 +++---- src/Storages/MergeTree/IMergedBlockOutputStream.cpp | 3 +-- src/Storages/MergeTree/IMergedBlockOutputStream.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 12 ++++++------ src/Storages/MergeTree/MergedBlockOutputStream.cpp | 4 ++-- .../MergeTree/MergedColumnOnlyOutputStream.cpp | 4 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 9 files changed, 22 insertions(+), 24 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 366a657eb71..47624a9d121 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -72,7 +72,7 @@ void SerializationInfoBuilder::add(const SerializationInfo & other) info->columns[name].num_defaults += column_info.num_defaults; } -SerializationInfoPtr SerializationInfoBuilder::build() +SerializationInfoPtr SerializationInfoBuilder::build() && { size_t total_rows = info->number_of_rows; for (auto & [_, column_info] : info->columns) @@ -85,7 +85,7 @@ SerializationInfoPtr SerializationInfoBuilder::build() return info; } -SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo & other) +SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo & other) && { for (const auto & [name, column_info] : other.columns) { diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 5d22e39de84..b0bc866670c 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -28,6 +28,7 @@ public: size_t getNumberOfDefaultRows(const String & column_name) const; ISerialization::Kind getKind(const String & column_name) const; + bool empty() const { return !number_of_rows && columns.empty(); } size_t getNumberOfRows() const { return number_of_rows; } void readText(ReadBuffer & in); @@ -76,12 +77,12 @@ public: /// Choose kind of serialization for every column /// according its content and return finalized SerializationInfo. - SerializationInfoPtr build(); + SerializationInfoPtr build() &&; /// Create SerializationInfo from other. /// Respects kinds of serialization for columns, that exist in other SerializationInfo, /// but keeps information about content of column from current SerializationInfo. - SerializationInfoPtr buildFrom(const SerializationInfo & other); + SerializationInfoPtr buildFrom(const SerializationInfo & other) &&; double getRatioForSparseSerialization() const { return ratio_for_sparse_serialization; } @@ -92,6 +93,4 @@ private: SerializationInfoPtr info; }; -using SerializationInfoBuilderPtr = std::shared_ptr; - } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index cd9c85aabfd..43d7a6cb3be 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -14,8 +14,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , volume(data_part->volume) , part_path(data_part->isStoredOnDisk() ? data_part->getFullRelativePath() : "") , input_serialization_info(input_serialization_info_) - , new_serialization_info(std::make_shared( - data_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization)) + , new_serialization_info(data_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization) { } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 2b943f74d65..3a9cadf53b5 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -45,7 +45,7 @@ protected: IMergeTreeDataPart::MergeTreeWriterPtr writer; SerializationInfoPtr input_serialization_info; - SerializationInfoBuilderPtr new_serialization_info; + SerializationInfoBuilder new_serialization_info; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 3bd509f8f72..d379cf58c80 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -750,7 +750,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor bool need_remove_expired_values = false; bool force_ttl = false; - auto serialization_info_builder = std::make_shared(data_settings->ratio_of_defaults_for_sparse_serialization); + SerializationInfoBuilder serialization_info_builder(data_settings->ratio_of_defaults_for_sparse_serialization); for (const auto & part : parts) { @@ -765,10 +765,10 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTempor new_data_part->ttl_infos.update(part->ttl_infos); } - serialization_info_builder->add(*part->serialization_info); + serialization_info_builder.add(*part->serialization_info); } - auto input_serialization_info = serialization_info_builder->build(); + auto input_serialization_info = std::move(serialization_info_builder).build(); const auto & part_min_ttl = new_data_part->ttl_infos.part_min_ttl; if (part_min_ttl && part_min_ttl <= time_of_merge) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index ead3a8f90bd..6b0e972dcf2 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -435,13 +435,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart( const auto & data_settings = data.getSettings(); - auto serialization_info = std::make_shared(data_settings->ratio_of_defaults_for_sparse_serialization); - serialization_info->add(block); + SerializationInfoBuilder serialization_info(data_settings->ratio_of_defaults_for_sparse_serialization); + serialization_info.add(block); const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot,columns, index_factory.getMany(metadata_snapshot->getSecondaryIndices()), - compression_codec, serialization_info->build()); + compression_codec, std::move(serialization_info).build()); bool sync_on_insert = data_settings->fsync_after_insert; @@ -516,8 +516,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); - auto serialization_info = std::make_shared(data.getSettings()->ratio_of_defaults_for_sparse_serialization); - serialization_info->add(block); + SerializationInfoBuilder serialization_info(data.getSettings()->ratio_of_defaults_for_sparse_serialization); + serialization_info.add(block); MergedBlockOutputStream out( new_data_part, @@ -525,7 +525,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl( columns, {}, compression_codec, - serialization_info->build()); + std::move(serialization_info).build()); out.writePrefix(); out.writeWithPermutation(block, perm_ptr); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 3c2d1caf2b4..34217763362 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -84,7 +84,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( else part_columns = *total_columns_list; - new_part->serialization_info = new_serialization_info->buildFrom(*input_serialization_info); + new_part->serialization_info = std::move(new_serialization_info).buildFrom(*input_serialization_info); if (new_part->isStoredOnDisk()) finalizePartOnDisk(new_part, part_columns, checksums, sync); @@ -224,7 +224,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm return; writer->write(block, permutation); - new_serialization_info->add(block); + new_serialization_info.add(block); rows_count += rows; } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 13d4abbc62c..7cb17cfcbae 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -53,7 +53,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) return; writer->write(block, nullptr); - new_serialization_info->add(block); + new_serialization_info.add(block); } void MergedColumnOnlyOutputStream::writeSuffix() @@ -85,7 +85,7 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( all_checksums.files.erase(removed_file); new_part->setColumns(columns); - new_part->serialization_info = new_serialization_info->buildFrom(*input_serialization_info); + new_part->serialization_info = std::move(new_serialization_info).buildFrom(*input_serialization_info); return checksums; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7cc01f181e8..35cbb47d702 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7489,8 +7489,8 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP new_data_part->is_temp = true; /// Create empty serialization_info. - auto serialization_info_builder = std::make_shared(getSettings()->ratio_of_defaults_for_sparse_serialization); - new_data_part->serialization_info = serialization_info_builder->build(); + auto ratio = getSettings()->ratio_of_defaults_for_sparse_serialization; + new_data_part->serialization_info = SerializationInfoBuilder(ratio).build(); SyncGuardPtr sync_guard; if (new_data_part->isStoredOnDisk()) From d80a52a59d1c21b58bc11d2f3e54a76e83261e72 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 14 Sep 2021 21:01:05 +0300 Subject: [PATCH 048/358] fix performance of inserts --- src/DataTypes/Serializations/SerializationInfo.cpp | 4 ++++ src/DataTypes/Serializations/SerializationInfo.h | 1 + 2 files changed, 5 insertions(+) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 47624a9d121..60b2e2ccfe8 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -32,6 +32,10 @@ void SerializationInfoBuilder::add(const Block & block) { size_t num_rows = block.rows(); info->number_of_rows += num_rows; + + if (!canHaveSparseSerialization()) + return; + for (const auto & elem : block) { /// Just skip column and always return default serialization. diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index b0bc866670c..ae5962f7689 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -85,6 +85,7 @@ public: SerializationInfoPtr buildFrom(const SerializationInfo & other) &&; double getRatioForSparseSerialization() const { return ratio_for_sparse_serialization; } + bool canHaveSparseSerialization() const { return ratio_for_sparse_serialization < 1.0; } private: double ratio_for_sparse_serialization; From 9e8c2197995d3a34a38c943be083e6411fe862c5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Sep 2021 14:54:10 +0300 Subject: [PATCH 049/358] fix tests --- .../00753_system_columns_and_system_tables_long.reference | 2 +- .../00961_checksums_in_system_parts_columns_table.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference index 7da6b35beaf..2b05c417fae 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -428 1 +276 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┠│ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index d4f3098ed1e..30ab6c68aaa 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_1_1_0 test_00961 8f7bf8151cbcee40008cb31dddf60e5f e4118be4d9f9fe6d6b68c2cab74b4a11 da96ff1e527a8a1f908ddf2b1d0af239 +20000101_1_1_0 test_00961 21d49af89b4905a5d1c76d4f1b02e20e d816caf0abffc3f9aabdb4f92bcf7bfb da96ff1e527a8a1f908ddf2b1d0af239 From 3adc2a9aea957a3892f7dbde5743bddc5f8b976b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Sep 2021 15:52:35 +0300 Subject: [PATCH 050/358] fix distinct with sparse columns --- src/Columns/ColumnSparse.cpp | 3 ++- .../Serializations/SerializationSparse.cpp | 3 ++- src/Interpreters/Aggregator.cpp | 2 +- .../Transforms/DistinctTransform.cpp | 2 ++ .../01780_column_sparse_distinct.reference | 7 +++++++ .../01780_column_sparse_distinct.sql | 20 +++++++++++++++++++ 6 files changed, 34 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01780_column_sparse_distinct.reference create mode 100644 tests/queries/0_stateless/01780_column_sparse_distinct.sql diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 8c229a12018..071a95a1f93 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -171,6 +171,7 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len if (offset_start != offset_end) { + offsets_data.reserve(offsets_data.size() + offset_end - offset_start); insertManyDefaults(src_offsets[offset_start] - start); offsets_data.push_back(_size); ++_size; @@ -184,7 +185,7 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len } /// 'end' <= 'src_offsets[offsets_end]', but end is excluded, so index is 'offsets_end' - 1. - /// Since 'end' is excluded need, to subtract one more row from result. + /// Since 'end' is excluded, need to subtract one more row from result. insertManyDefaults(end - src_offsets[offset_end - 1] - 1); values->insertRangeFrom(src_values, offset_start + 1, offset_end - offset_start); } diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index bc638a54fdc..d9ad66d1f35 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -66,7 +66,8 @@ size_t deserializeOffsets(IColumn::Offsets & offsets, } /// Just try to guess number of offsets. - offsets.reserve(static_cast(limit * ColumnSparse::DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION)); + offsets.reserve(offsets.size() + + static_cast(limit * (1.0 - ColumnSparse::DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION))); bool first = true; size_t total_rows = state.num_trailing_defaults; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 355cb61cbb7..61c65fd2c2d 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -907,7 +907,7 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns aggregate_columns[i][j] = materialized_columns.back().get(); } - if (typeid_cast(aggregate_columns[i][j])) + if (aggregate_columns[i][j]->isSparse()) has_sparse_arguments = true; } diff --git a/src/Processors/Transforms/DistinctTransform.cpp b/src/Processors/Transforms/DistinctTransform.cpp index fddfe663af5..5cff91eabde 100644 --- a/src/Processors/Transforms/DistinctTransform.cpp +++ b/src/Processors/Transforms/DistinctTransform.cpp @@ -54,6 +54,8 @@ void DistinctTransform::buildFilter( void DistinctTransform::transform(Chunk & chunk) { + convertToFullIfSparse(chunk); + auto num_rows = chunk.getNumRows(); auto columns = chunk.detachColumns(); diff --git a/tests/queries/0_stateless/01780_column_sparse_distinct.reference b/tests/queries/0_stateless/01780_column_sparse_distinct.reference new file mode 100644 index 00000000000..bb0cebc6540 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_distinct.reference @@ -0,0 +1,7 @@ +all_1_1_0 v Default +all_2_2_0 v Sparse +0 +1 +2 +3 +4 diff --git a/tests/queries/0_stateless/01780_column_sparse_distinct.sql b/tests/queries/0_stateless/01780_column_sparse_distinct.sql new file mode 100644 index 00000000000..502ca7600d4 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_distinct.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS t_sparse_distinct; + +CREATE TABLE t_sparse_distinct (id UInt32, v UInt64) +ENGINE = MergeTree +ORDER BY id +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.9; + +SYSTEM STOP MERGES t_sparse_distinct; + +INSERT INTO t_sparse_distinct SELECT number, number % 5 FROM numbers(100000); +INSERT INTO t_sparse_distinct SELECT number, number % 100 = 0 FROM numbers(100000); + +SELECT name, column, serialization_kind +FROM system.parts_columns +WHERE table = 't_sparse_distinct' AND database = currentDatabase() AND column = 'v' +ORDER BY name; + +SELECT DISTINCT v FROM t_sparse_distinct ORDER BY v; + +DROP TABLE t_sparse_distinct; From 348537faee71f6f0a80c99cd31f56290689ef3c2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Sep 2021 16:57:45 +0300 Subject: [PATCH 051/358] better IColumn::createWithOffsets --- src/Columns/ColumnNullable.cpp | 21 ++++++++++++++++++--- src/Columns/ColumnNullable.h | 2 +- src/Columns/ColumnSparse.cpp | 2 +- src/Columns/ColumnVector.cpp | 5 +++-- src/Columns/ColumnVector.h | 2 +- src/Columns/IColumn.cpp | 6 +++--- src/Columns/IColumn.h | 11 +++++++++-- src/Functions/IFunction.cpp | 2 +- 8 files changed, 37 insertions(+), 14 deletions(-) diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index e68c9b61608..89e443b72cb 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -648,10 +648,25 @@ void ColumnNullable::checkConsistency() const ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT); } -ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const +ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const { - auto new_values = nested_column->createWithOffsets(offsets, total_rows, shift); - auto new_null_map = null_map->createWithOffsets(offsets, total_rows, shift); + ColumnPtr new_values; + ColumnPtr new_null_map; + + if (default_field.getType() == Field::Types::Null) + { + auto default_column = nested_column->cloneEmpty(); + default_column->insertDefault(); + + /// Value in main column, when null map is 1 is implementation defined. So, take any value. + new_values = nested_column->createWithOffsets(offsets, (*default_column)[0], total_rows, shift); + new_null_map = null_map->createWithOffsets(offsets, Field(1u), total_rows, shift); + } + else + { + new_values = nested_column->createWithOffsets(offsets, default_field, total_rows, shift); + new_null_map = null_map->createWithOffsets(offsets, Field(0u), total_rows, shift); + } return ColumnNullable::create(new_values, new_null_map); } diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index a3005f52ae2..3e99a25a445 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -148,7 +148,7 @@ public: null_map->getIndicesOfNonDefaultRows(indices, from, limit); } - ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const override; bool isNullable() const override { return true; } bool isFixedAndContiguous() const override { return false; } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 071a95a1f93..d39f98671b4 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -123,7 +123,7 @@ StringRef ColumnSparse::getDataAt(size_t n) const ColumnPtr ColumnSparse::convertToFullColumnIfSparse() const { - return values->createWithOffsets(getOffsetsData(), _size, 1); + return values->createWithOffsets(getOffsetsData(), (*values)[0], _size, /*shift=*/ 1); } void ColumnSparse::insertData(const char * pos, size_t length) diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 07c1a05e8df..ace28f06704 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -570,7 +570,7 @@ ColumnPtr ColumnVector::compress() const } template -ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const +ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const { if (offsets.size() + shift != size()) throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -579,7 +579,8 @@ ColumnPtr ColumnVector::createWithOffsets(const IColumn::Offsets & offsets, s auto res = this->create(); auto & res_data = res->getData(); - res_data.resize_fill(total_rows, data[0]); + T default_value = safeGet(default_field); + res_data.resize_fill(total_rows, default_value); for (size_t i = 0; i < offsets.size(); ++i) res_data[offsets[i]] = data[i + shift]; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index 49bfe4e48d7..467b52399e0 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -336,7 +336,7 @@ public: return this->template getIndicesOfNonDefaultRowsImpl(indices, from, limit); } - ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, size_t total_rows, size_t shift) const override; + ColumnPtr createWithOffsets(const IColumn::Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const override; ColumnPtr compress() const override; diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index f20a4885cdb..555f4e112c9 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -35,7 +35,7 @@ void IColumn::insertFrom(const IColumn & src, size_t n) insert(src[n]); } -ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows, size_t shift) const +ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const { if (offsets.size() + shift != size()) throw Exception(ErrorCodes::LOGICAL_ERROR, @@ -51,14 +51,14 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, size_t total_rows, current_offset = offsets[i]; if (offsets_diff > 1) - res->insertManyFrom(*this, 0, offsets_diff - 1); + res->insertMany(default_field, offsets_diff - 1); res->insertFrom(*this, i + shift); } ssize_t offsets_diff = static_cast(total_rows) - current_offset; if (offsets_diff > 1) - res->insertManyFrom(*this, 0, offsets_diff - 1); + res->insertMany(default_field, offsets_diff - 1); return res; } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 088a630eb65..a21bf94370a 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -182,6 +182,13 @@ public: insertFrom(src, position); } + /// Appends one field multiple times. Can be optimized in inherited classes. + virtual void insertMany(const Field & field, size_t length) + { + for (size_t i = 0; i < length; ++i) + insert(field); + } + /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). /// Is used to optimize some computations (in aggregation, for example). /// Parameter length could be ignored if column values have fixed size. @@ -393,10 +400,10 @@ public: /// Returns column with @total_size elements. /// In result column values from current column are at positions from @offsets. - /// Other values are filled by defaults. + /// Other values are filled by @default_value. /// @shift means how much rows to skip from the beginning of current column. /// Used to create full column from sparse. - virtual Ptr createWithOffsets(const Offsets & offsets, size_t total_rows, size_t shift) const; + virtual Ptr createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const; /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 29595e3ec6f..3250101c3cc 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -326,7 +326,7 @@ ColumnPtr IExecutableFunction::execute(const ColumnsWithTypeAndName & arguments, if (!res->isDefaultAt(0)) { const auto & offsets_data = assert_cast &>(*sparse_offsets).getData(); - return res->createWithOffsets(offsets_data, input_rows_count, /*shift=*/ 1); + return res->createWithOffsets(offsets_data, (*res)[0], input_rows_count, /*shift=*/ 1); } return ColumnSparse::create(res, sparse_offsets, input_rows_count); From 219193966707112d510fe99217a4084b7469d34c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Sep 2021 18:31:54 +0300 Subject: [PATCH 052/358] slightly better --- src/DataTypes/Serializations/SerializationInfo.cpp | 13 ++++++------- .../0_stateless/01780_column_sparse_tuple.reference | 4 ++-- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 60b2e2ccfe8..14d59ec5069 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -46,18 +46,17 @@ void SerializationInfoBuilder::add(const Block & block) info->columns[elem.name].num_defaults += static_cast( num_rows * elem.column->getRatioOfDefaultRows(default_rows_search_sample_ratio)); - for (const auto & subcolumn_name : elem.type->getSubcolumnNames()) + elem.type->forEachSubcolumn([&](const auto & subcolumn_name, const auto & subcolumn_type, const auto &) { - auto subcolumn_type = elem.type->getSubcolumnType(subcolumn_name); if (!subcolumn_type->supportsSparseSerialization()) - continue; + return; auto parent_subcolumn_name = Nested::splitName(subcolumn_name, /*reverse=*/ true).first; if (!parent_subcolumn_name.empty()) { auto parent_subcolumn_type = elem.type->tryGetSubcolumnType(parent_subcolumn_name); if (parent_subcolumn_type && !parent_subcolumn_type->supportsSparseSerialization()) - continue; + return; } auto subcolumn = elem.type->getSubcolumn(subcolumn_name, *elem.column); @@ -65,7 +64,7 @@ void SerializationInfoBuilder::add(const Block & block) info->columns[full_name].num_defaults += static_cast( num_rows * subcolumn->getRatioOfDefaultRows(default_rows_search_sample_ratio)); - } + }); } } @@ -86,7 +85,7 @@ SerializationInfoPtr SerializationInfoBuilder::build() && column_info.kind = ISerialization::Kind::SPARSE; } - return info; + return std::move(info); } SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo & other) && @@ -100,7 +99,7 @@ SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo it->second.kind = column_info.kind; } - return info; + return std::move(info); } SerializationInfo::SerializationInfo(size_t number_of_rows_, const NameToKind & kinds) diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.reference b/tests/queries/0_stateless/01780_column_sparse_tuple.reference index c19d48be991..394fc148ef6 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.reference +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.reference @@ -1,5 +1,5 @@ id [] [] [] -t ['s','a'] ['String','UInt64'] ['Default','Sparse'] +t ['a','s'] ['UInt64','String'] ['Sparse','Default'] (0,'a') (0,'aa') (0,'aaa') @@ -41,7 +41,7 @@ a a a id [] [] [] -t ['b.u','b.s','b','a'] ['UInt32','String','Tuple(u UInt32, s String)','UInt64'] ['Sparse','Default','Default','Sparse'] +t ['a','b','b.u','b.s'] ['UInt64','Tuple(u UInt32, s String)','UInt32','String'] ['Sparse','Default','Sparse','Default'] 0 0 0 From dce55260e0b3130a908f86e7c83994c23c16f500 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 20 Sep 2021 16:09:20 +0300 Subject: [PATCH 053/358] fix tests --- src/Functions/IFunction.h | 6 +++--- tests/integration/test_partition/test.py | 8 ++++---- .../integration/test_s3_zero_copy_replication/test.py | 10 +++++----- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index ba3b5dfaf12..9bfdae7df93 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -81,7 +81,7 @@ protected: /** If function arguments has single sparse column and all other arguments are constants, call function on nested column. * Otherwise, convert all sparse columns to ordinary columns. * If default value doesn't change after function execution, returns sparse column as a result. - * Otherwide, result column is converted to full. + * Otherwise, result column is converted to full. */ virtual bool useDefaultImplementationForSparseColumns() const { return true; } @@ -367,7 +367,7 @@ protected: /** If function arguments has single sparse column and all other arguments are constants, call function on nested column. * Otherwise, convert all sparse columns to ordinary columns. * If default value doesn't change after function execution, returns sparse column as a result. - * Otherwide, result column is converted to full. + * Otherwise, result column is converted to full. */ virtual bool useDefaultImplementationForSparseColumns() const { return true; } @@ -425,7 +425,7 @@ public: /** If function arguments has single sparse column and all other arguments are constants, call function on nested column. * Otherwise, convert all sparse columns to ordinary columns. * If default value doesn't change after function execution, returns sparse column as a result. - * Otherwide, result column is converted to full. + * Otherwise, result column is converted to full. */ virtual bool useDefaultImplementationForSparseColumns() const { return true; } diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index d1078ab524c..dc47d47c131 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -62,6 +62,7 @@ def partition_complex_assert_checksums(): "25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/partition.dat\n" \ "3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition_complex/19700201_1_1_0/partition.dat\n" \ "38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.bin\n" \ + "43d32c3316e810e0231ee4f93dbf2875\tshadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.mrk\n" \ @@ -69,20 +70,19 @@ def partition_complex_assert_checksums(): "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.mrk\n" \ "55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition_complex/19700201_1_1_0/primary.idx\n" \ - "636c2df435a924297354025e3e203006\tshadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt\n" \ "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700102_2_2_0/columns.txt\n" \ "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700201_1_1_0/columns.txt\n" \ "88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.bin\n" \ "9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition_complex/19700102_2_2_0/primary.idx\n" \ - "ac5882732ed524266cfcb45c2cb53781\tshadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt\n" \ + "b0f1c38fe8a3e0b38d75f4d9c142bc45\tshadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt\n" \ "c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700102_2_2_0/default_compression_codec.txt\n" \ "c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700201_1_1_0/default_compression_codec.txt\n" \ "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700102_2_2_0/count.txt\n" \ "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700201_1_1_0/count.txt\n" \ - "c92f1855b6cc741573952c527dedff3a\tshadow/1/data/test/partition_complex/19700102_2_2_0/serialization.txt\n" \ - "c92f1855b6cc741573952c527dedff3a\tshadow/1/data/test/partition_complex/19700201_1_1_0/serialization.txt\n" \ "cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.bin\n" \ "e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.bin\n" \ + "e8490b8552f8b9b774db2f9eb1c90349\tshadow/1/data/test/partition_complex/19700102_2_2_0/serialization.txt\n" \ + "e8490b8552f8b9b774db2f9eb1c90349\tshadow/1/data/test/partition_complex/19700201_1_1_0/serialization.txt\n" \ "f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition_complex/19700201_1_1_0/minmax_p.idx\n" assert TSV(instance.exec_in_container(cmd).replace(' ', '\t')) == TSV(checksums) diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 994e8082502..8dd144e4e1a 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -68,8 +68,8 @@ def test_s3_zero_copy_replication(cluster, policy): assert node1.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data')" assert node2.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data')" - # Based on version 21.x - should be only 2 files with size 100+ (checksums.txt, serialization.txt), used by both nodes - assert get_large_objects_count(cluster) == 2 + # Based on version 21.x - should be only 1 file with size 100+ (checksums.txt), used by both nodes + assert get_large_objects_count(cluster) == 1 node2.query("INSERT INTO s3_test VALUES (2,'data'),(3,'data')") node1.query("SYSTEM SYNC REPLICA s3_test") @@ -78,15 +78,15 @@ def test_s3_zero_copy_replication(cluster, policy): assert node1.query("SELECT * FROM s3_test order by id FORMAT Values") == "(0,'data'),(1,'data'),(2,'data'),(3,'data')" # Based on version 21.x - two parts - wait_for_large_objects_count(cluster, 4) + wait_for_large_objects_count(cluster, 2) node1.query("OPTIMIZE TABLE s3_test FINAL") # Based on version 21.x - after merge, two old parts and one merged - wait_for_large_objects_count(cluster, 6) + wait_for_large_objects_count(cluster, 3) # Based on version 21.x - after cleanup - only one merged part - wait_for_large_objects_count(cluster, 2, timeout=60) + wait_for_large_objects_count(cluster, 1, timeout=60) node1.query("DROP TABLE IF EXISTS s3_test NO DELAY") node2.query("DROP TABLE IF EXISTS s3_test NO DELAY") From dc94d2239e33c432e8465d2ff3648470feb589a4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 29 Sep 2021 17:18:41 +0300 Subject: [PATCH 054/358] better ColumnSparse --- src/Columns/ColumnArray.cpp | 10 ++++ src/Columns/ColumnArray.h | 10 +--- src/Columns/ColumnFixedString.cpp | 29 +----------- src/Columns/ColumnLowCardinality.h | 4 +- src/Columns/ColumnSparse.cpp | 73 ++++++++++++++++++++++-------- src/Columns/ColumnSparse.h | 11 ++++- src/Columns/ColumnUnique.h | 10 ++-- src/Columns/IColumnImpl.h | 4 +- 8 files changed, 88 insertions(+), 63 deletions(-) diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index f33354748a1..867728d5c8b 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -985,6 +985,16 @@ ColumnPtr ColumnArray::compress() const }); } +double ColumnArray::getRatioOfDefaultRows(double sample_ratio) const +{ + return getRatioOfDefaultRowsImpl(sample_ratio); +} + +void ColumnArray::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const +{ + return getIndicesOfNonDefaultRowsImpl(indices, from, limit); +} + ColumnPtr ColumnArray::replicate(const Offsets & replicate_offsets) const { diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index d8368f955aa..adfb4788b93 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -142,15 +142,9 @@ public: return false; } - double getRatioOfDefaultRows(double sample_ratio) const override - { - return getRatioOfDefaultRowsImpl(sample_ratio); - } + double getRatioOfDefaultRows(double sample_ratio) const override; - void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override - { - return getIndicesOfNonDefaultRowsImpl(indices, from, limit); - } + void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index ab581e87dab..89f93c83596 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -53,33 +53,8 @@ MutableColumnPtr ColumnFixedString::cloneResized(size_t size) const bool ColumnFixedString::isDefaultAt(size_t index) const { - const UInt8 * pos = chars.data() + index * n; - const UInt8 * end = pos + n; - -#ifdef __SSE2__ - static constexpr size_t SIMD_BYTES = 16; - const UInt8 * end_sse = pos + n / SIMD_BYTES * SIMD_BYTES; - const __m128i zero16 = _mm_setzero_si128(); - - while (pos < end_sse) - { - if (0xFFFF != _mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(pos)), zero16))) - return false; - - pos += SIMD_BYTES; - } -#endif - - while (pos < end) - { - if (*pos != 0) - return false; - - ++pos; - } - - return true; + assert(index < size()); + return memoryIsZero(chars.data() + index * n, n); } void ColumnFixedString::insert(const Field & x) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 3575a7f8877..d3ee54778e0 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -183,12 +183,12 @@ public: double getRatioOfDefaultRows(double sample_ratio) const override { - return getRatioOfDefaultRowsImpl(sample_ratio); + return getIndexes().getRatioOfDefaultRows(sample_ratio); } void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override { - return getIndicesOfNonDefaultRowsImpl(indices, from, limit); + return getIndexes().getIndicesOfNonDefaultRows(indices, from, limit); } bool valuesHaveFixedSize() const override { return getDictionary().valuesHaveFixedSize(); } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index d39f98671b4..6aecd4cecd5 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -34,7 +34,7 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs const ColumnUInt64 * offsets_concrete = typeid_cast(offsets.get()); if (!offsets_concrete) - throw Exception("offsets_column must be a ColumnUInt64", ErrorCodes::LOGICAL_ERROR); + throw Exception( ErrorCodes::LOGICAL_ERROR, "'offsets' column must be a ColumnUInt64, got: {}", offsets->getName()); /// 'values' should contain one extra element: default value at 0 position. if (offsets->size() + 1 != values->size()) @@ -45,6 +45,11 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs throw Exception(ErrorCodes::LOGICAL_ERROR, "Size of sparse column ({}) cannot be lower than number of non-default values ({})", _size, offsets->size()); + if (!offsets_concrete->empty() && _size <= offsets_concrete->getData().back()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Size sparse columns ({}) should be greater than last position of non-default value ({})", + _size, offsets_concrete->getData().back()); + #ifndef NDEBUG const auto & offsets_data = getOffsetsData(); const auto * it = std::adjacent_find(offsets_data.begin(), offsets_data.end(), std::greater_equal()); @@ -126,13 +131,24 @@ ColumnPtr ColumnSparse::convertToFullColumnIfSparse() const return values->createWithOffsets(getOffsetsData(), (*values)[0], _size, /*shift=*/ 1); } -void ColumnSparse::insertData(const char * pos, size_t length) +void ColumnSparse::insertSingleValue(const Inserter & inserter) { - _size += length; - return values->insertData(pos, length); + inserter(*values); + + size_t last_idx = values->size() - 1; + if (values->isDefaultAt(last_idx)) + values->popBack(1); + else + getOffsetsData().push_back(_size); + + ++_size; +} + +void ColumnSparse::insertData(const char * pos, size_t length) +{ + insertSingleValue([&](IColumn & column) { column.insertData(pos, length); }); } -/// TODO: maybe need to reimplement it. StringRef ColumnSparse::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { return values->serializeValueIntoArena(getValueIndex(n), arena, begin); @@ -140,8 +156,9 @@ StringRef ColumnSparse::serializeValueIntoArena(size_t n, Arena & arena, char co const char * ColumnSparse::deserializeAndInsertFromArena(const char * pos) { - ++_size; - return values->deserializeAndInsertFromArena(pos); + const char * res = nullptr; + insertSingleValue([&](IColumn & column) { res = column.deserializeAndInsertFromArena(pos); }); + return res; } const char * ColumnSparse::skipSerializedInArena(const char * pos) const @@ -168,6 +185,7 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len size_t offset_start = std::lower_bound(src_offsets.begin(), src_offsets.end(), start) - src_offsets.begin(); size_t offset_end = std::lower_bound(src_offsets.begin(), src_offsets.end(), end) - src_offsets.begin(); + assert(offset_start <= offset_end); if (offset_start != offset_end) { @@ -198,24 +216,24 @@ void ColumnSparse::insertRangeFrom(const IColumn & src, size_t start, size_t len { for (size_t i = start; i < end; ++i) { - offsets_data.push_back(_size); + if (!src.isDefaultAt(i)) + { + values->insertFrom(src, i); + offsets_data.push_back(_size); + } + ++_size; } - - values->insertRangeFrom(src, start, length); } } void ColumnSparse::insert(const Field & x) { - getOffsetsData().push_back(_size); - values->insert(x); - ++_size; + insertSingleValue([&](IColumn & column) { column.insert(x); }); } void ColumnSparse::insertFrom(const IColumn & src, size_t n) { - if (const auto * src_sparse = typeid_cast(&src)) { if (size_t value_index = src_sparse->getValueIndex(n)) @@ -226,8 +244,11 @@ void ColumnSparse::insertFrom(const IColumn & src, size_t n) } else { - getOffsetsData().push_back(_size); - values->insertFrom(src, n); + if (!src.isDefaultAt(n)) + { + values->insertFrom(src, n); + getOffsetsData().push_back(_size); + } } ++_size; @@ -467,7 +488,21 @@ int ColumnSparse::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs bool ColumnSparse::hasEqualValues() const { - return offsets->empty(); + size_t num_defaults = getNumberOfDefaults(); + if (num_defaults == _size) + return true; + + /// Have at least 1 default and 1 non-default values. + if (num_defaults != 0) + return false; + + /// Check that probably all non-default values are equal. + /// It's suboptiomal, but it's a rare case. + for (size_t i = 2; i < values->size(); ++i) + if (values->compareAt(1, i, *values, 1) != 0) + return false; + + return true; } void ColumnSparse::getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const @@ -555,7 +590,7 @@ void ColumnSparse::updatePermutationWithCollation( size_t ColumnSparse::byteSize() const { - return values->byteSize() + offsets->byteSize(); + return values->byteSize() + offsets->byteSize() + sizeof(_size); } size_t ColumnSparse::byteSizeAt(size_t n) const @@ -570,7 +605,7 @@ size_t ColumnSparse::byteSizeAt(size_t n) const size_t ColumnSparse::allocatedBytes() const { - return values->allocatedBytes() + offsets->allocatedBytes(); + return values->allocatedBytes() + offsets->allocatedBytes() + sizeof(_size); } void ColumnSparse::protect() diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 037b967e8dd..eb24664c7bb 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -149,7 +149,7 @@ public: /// Return position of element in 'values' columns, /// that corresponds to n-th element of full column. - /// O(log(size)) complexity, + /// O(log(offsets.size())) complexity, size_t getValueIndex(size_t n) const; const IColumn & getValuesColumn() const { return *values; } @@ -209,7 +209,16 @@ public: Iterator end() const { return Iterator(getOffsetsData(), _size, getOffsetsData().size(), _size); } private: + using Inserter = std::function; + + /// Inserts value to 'values' column via callback. + /// Properly handles cases, when inserted value is default. + /// Used, when it's unknown in advance if inserted value is default. + void insertSingleValue(const Inserter & inserter); + /// Contains default value at 0 position. + /// It's convenient, because it allows to execute, e.g functions or sorting, + /// for this column without handling different cases. WrappedPtr values; /// Sorted offsets of non-default values in the full column. diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 6f97acd9df1..6eef745a355 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -68,7 +68,7 @@ public: Field operator[](size_t n) const override { return (*getNestedColumn())[n]; } void get(size_t n, Field & res) const override { getNestedColumn()->get(n, res); } - bool isDefaultAt(size_t n) const override { return getNestedColumn()->isDefaultAt(n); } + bool isDefaultAt(size_t n) const override { return n == 0; } StringRef getDataAt(size_t n) const override { return getNestedColumn()->getDataAt(n); } StringRef getDataAtWithTerminatingZero(size_t n) const override { @@ -123,14 +123,14 @@ public: return false; } - double getRatioOfDefaultRows(double sample_ratio) const override + double getRatioOfDefaultRows(double) const override { - return getNestedColumn()->getRatioOfDefaultRows(sample_ratio); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getRatioOfDefaultRows' not implemeted for ColumnUnique"); } - void getIndicesOfNonDefaultRows(IColumn::Offsets & indices, size_t from, size_t limit) const override + void getIndicesOfNonDefaultRows(IColumn::Offsets &, size_t, size_t) const override { - return getNestedColumn()->getIndicesOfNonDefaultRows(indices, from, limit); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getIndicesOfNonDefaultRows' not implemeted for ColumnUnique"); } const UInt64 * tryGetSavedHash() const override { return reverse_index.tryGetSavedHash(); } diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index fe9ad251111..7d9f338f8e2 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -143,7 +143,9 @@ bool IColumn::hasEqualValuesImpl() const template double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const { - assert(sample_ratio > 0 && sample_ratio <= 1.0); + if (sample_ratio <= 0.0 || sample_ratio > 1.0) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Value of 'sample_ratio' must be in interval (0.0; 1.0], but got: {}", sample_ratio); size_t num_rows = size(); size_t num_sampled_rows = static_cast(num_rows * sample_ratio); From 07e1224a56f536836ef50736c89f4e7e0ff5352e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 4 Oct 2021 18:21:38 +0300 Subject: [PATCH 055/358] better serialization of serialization kinds in native protocol --- src/DataStreams/NativeBlockInputStream.cpp | 32 ++++++----- src/DataStreams/NativeBlockOutputStream.cpp | 30 +++++----- src/DataTypes/DataTypeTuple.cpp | 4 +- src/DataTypes/DataTypeTuple.h | 2 +- src/DataTypes/IDataType.cpp | 17 ++++-- src/DataTypes/IDataType.h | 11 +++- .../Serializations/ISerialization.cpp | 28 ++-------- src/DataTypes/Serializations/ISerialization.h | 1 + .../Serializations/SerializationInfo.cpp | 55 ------------------- .../Serializations/SerializationInfo.h | 7 --- 10 files changed, 66 insertions(+), 121 deletions(-) diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index e39674968c7..c1a2abb8679 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -135,18 +135,6 @@ Block NativeBlockInputStream::readImpl() rows = index_block_it->num_rows; } - /// Serialization - SerializationInfoPtr serialization_info; - if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) - { - auto serialization_kinds = SerializationInfo::readKindsBinary(istr); - serialization_info = std::make_shared(rows, serialization_kinds); - } - else - { - serialization_info = std::make_shared(); - } - for (size_t i = 0; i < columns; ++i) { if (use_index) @@ -165,6 +153,25 @@ Block NativeBlockInputStream::readImpl() readBinary(type_name, istr); column.type = data_type_factory.get(type_name); + SerializationPtr serialization; + if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + { + serialization = column.type->getSerialization(column.name, [&](const String & /*name*/) + { + UInt8 kind_num; + readBinary(kind_num, istr); + auto kind = magic_enum::enum_cast(kind_num); + if (!kind) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown serialization kind " + std::to_string(kind_num)); + + return *kind; + }); + } + else + { + serialization = column.type->getDefaultSerialization(); + } + if (use_index) { /// Index allows to do more checks. @@ -175,7 +182,6 @@ Block NativeBlockInputStream::readImpl() } /// Data - auto serialization = column.type->getSerialization(column.name, *serialization_info); ColumnPtr read_column = column.type->createColumn(*serialization); double avg_value_size_hint = avg_value_size_hints.empty() ? 0 : avg_value_size_hints[i]; diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index f7143e3e19f..2809e1ad69e 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -10,7 +10,7 @@ #include #include -#include +#include #include namespace DB @@ -86,14 +86,6 @@ void NativeBlockOutputStream::write(const Block & block) writeVarUInt(rows, *index_ostr); } - - /// Serialization - if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) - { - auto serialization_kinds = SerializationInfo::getKinds(block); - SerializationInfo::writeKindsBinary(serialization_kinds, ostr); - } - for (size_t i = 0; i < columns; ++i) { /// For the index. @@ -129,15 +121,27 @@ void NativeBlockOutputStream::write(const Block & block) writeStringBinary(type_name, ostr); + /// Serialization. Dynamic, if client supports it. SerializationPtr serialization; - if (client_revision < DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) + if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) { - serialization = column.type->getDefaultSerialization(); - column.column = recursiveRemoveSparse(column.column); + serialization = column.type->getSerialization(column.name, [&](const String & name) + { + auto split = Nested::splitName(name); + ISerialization::Kind kind; + if (!split.second.empty() && column.type->tryGetSubcolumnType(split.second)) + kind = ISerialization::getKind(*column.type->getSubcolumn(split.second, *column.column)); + else + kind = ISerialization::getKind(*column.column); + + writeBinary(static_cast(kind), ostr); + return kind; + }); } else { - serialization = column.type->getSerialization(*column.column); + serialization = column.type->getDefaultSerialization(); + column.column = recursiveRemoveSparse(column.column); } /// Data diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 1e2c180522e..bae0e3b7109 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -330,7 +330,7 @@ SerializationPtr DataTypeTuple::doGetDefaultSerialization() const return std::make_shared(std::move(serializations), use_explicit_names); } -SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const SerializationInfo & info) const +SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const SerializationCallback & callback) const { SerializationTuple::ElementSerializations serializations(elems.size()); bool use_explicit_names = have_explicit_names && serialize_names; @@ -338,7 +338,7 @@ SerializationPtr DataTypeTuple::getSerialization(const String & column_name, con { String elem_name = use_explicit_names ? names[i] : toString(i + 1); auto subcolumn_name = Nested::concatenateName(column_name, elem_name); - auto serializaion = elems[i]->getSerialization(subcolumn_name, info); + auto serializaion = elems[i]->getSerialization(subcolumn_name, callback); serializations[i] = std::make_shared(serializaion, elem_name); } diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 2aafc365373..1b3c6289f22 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -57,7 +57,7 @@ public: DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override; ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override; - SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const override; + SerializationPtr getSerialization(const String & column_name, const SerializationCallback & callback) const override; SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override; diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 5c090580ba9..d94faa69c55 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -166,9 +166,9 @@ SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_n throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); } -SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const +SerializationPtr IDataType::getSerialization(ISerialization::Kind kind) const { - if (supportsSparseSerialization() && info.getKind(column_name) == ISerialization::Kind::SPARSE) + if (supportsSparseSerialization() && kind == ISerialization::Kind::SPARSE) return getSparseSerialization(); return getDefaultSerialization(); @@ -176,10 +176,17 @@ SerializationPtr IDataType::getSerialization(const String & column_name, const S SerializationPtr IDataType::getSerialization(const IColumn & column) const { - if (column.isSparse()) - return getSparseSerialization(); + return getSerialization(ISerialization::getKind(column)); +} - return getDefaultSerialization(); +SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const +{ + return getSerialization(column_name, [&info](const auto & name) { return info.getKind(name); }); +} + +SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationCallback & callback) const +{ + return getSerialization(callback(column_name)); } SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index c216912c52c..868a5e08b45 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -97,15 +97,22 @@ public: virtual SerializationPtr getSubcolumnSerialization( const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const; + /// Chooses serialziation according to serialization kind. + SerializationPtr getSerialization(ISerialization::Kind kind) const; + /// Chooses serialziation according to column content. - virtual SerializationPtr getSerialization(const IColumn & column) const; + SerializationPtr getSerialization(const IColumn & column) const; /// Chooses serialization according to collected information about content of columns. - virtual SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; + SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; /// Chooses serialization according to settings. SerializationPtr getSerialization(const ISerialization::Settings & settings) const; + using SerializationCallback = std::function; + + virtual SerializationPtr getSerialization(const String & column_name, const SerializationCallback & callback) const; + /// Chooses between subcolumn serialization and regular serialization according to @column. /// This method typically should be used to get serialization for reading column or subcolumn. static SerializationPtr getSerialization(const NameAndTypePair & column, const SerializationInfo & info); diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 97ddcc4a352..bb3e95857ca 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -49,30 +50,11 @@ ISerialization::Kind ISerialization::stringToKind(const String & str) String ISerialization::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"; - case SparseElements: - return "SparseElements"; - case SparseOffsets: - return "SparseOffsets"; - } + if (type == TupleElement) + return fmt::format("TupleElement({}, escape_tuple_delimiter = {})", + tuple_element_name, escape_tuple_delimiter ? "true" : "false"); - __builtin_unreachable(); + return String(magic_enum::enum_name(type)); } String ISerialization::SubstreamPath::toString() const diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 1e5f34a39c9..8ed4a44b482 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -98,6 +98,7 @@ public: SparseElements, SparseOffsets, }; + Type type; /// Index of tuple element, starting at 1 or name. diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 14d59ec5069..1959647b10f 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -102,13 +102,6 @@ SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo return std::move(info); } -SerializationInfo::SerializationInfo(size_t number_of_rows_, const NameToKind & kinds) - : number_of_rows(number_of_rows_) -{ - for (const auto & [name, kind] : kinds) - columns[name].kind = kind; -} - ISerialization::Kind SerializationInfo::getKind(const String & column_name) const { auto it = columns.find(column_name); @@ -203,52 +196,4 @@ void SerializationInfo::writeText(WriteBuffer & out) const writeString(toJSON(), out); } -SerializationInfo::NameToKind SerializationInfo::getKinds(const Block & block) -{ - NameToKind kinds; - for (const auto & elem : block) - { - kinds[elem.name] = ISerialization::getKind(*elem.column); - for (const auto & subcolumn_name : elem.type->getSubcolumnNames()) - { - auto full_name = Nested::concatenateName(elem.name, subcolumn_name); - auto subcolumn = elem.type->getSubcolumn(subcolumn_name, *elem.column); - kinds[full_name] = ISerialization::getKind(*subcolumn); - } - } - - return kinds; -} - -SerializationInfo::NameToKind SerializationInfo::readKindsBinary(ReadBuffer & in) -{ - size_t size = 0; - readVarUInt(size, in); - - NameToKind kinds; - kinds.reserve(size); - for (size_t i = 0; i < size; ++i) - { - String name; - UInt8 kind; - - readBinary(name, in); - readBinary(kind, in); - if (!kinds.emplace(name, static_cast(kind)).second) - throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicated name '{}' found in serialization kinds", name); - } - - return kinds; -} - -void SerializationInfo::writeKindsBinary(const NameToKind & kinds, WriteBuffer & out) -{ - writeVarUInt(kinds.size(), out); - for (const auto & [name, kind] : kinds) - { - writeBinary(name, out); - writeBinary(static_cast(kind), out); - } -} - } diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index ae5962f7689..e67e81edade 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -19,10 +19,7 @@ namespace DB class SerializationInfo { public: - using NameToKind = std::unordered_map; - SerializationInfo() = default; - SerializationInfo(size_t number_of_rows_, const NameToKind & kinds); static constexpr auto version = 1; size_t getNumberOfDefaultRows(const String & column_name) const; @@ -34,10 +31,6 @@ public: void readText(ReadBuffer & in); void writeText(WriteBuffer & out) const; - static NameToKind getKinds(const Block & block); - static NameToKind readKindsBinary(ReadBuffer & in); - static void writeKindsBinary(const NameToKind & kinds, WriteBuffer & out); - private: void fromJSON(const String & json_str); String toJSON() const; From 2360a5cace6a86c777c0ee824280781296d734c4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Oct 2021 20:03:21 +0300 Subject: [PATCH 056/358] better getter for serialization states --- src/DataTypes/Serializations/ISerialization.h | 28 +++++-------------- .../SerializationLowCardinality.cpp | 6 ++-- .../Serializations/SerializationSparse.cpp | 2 +- .../Serializations/SerializationTuple.cpp | 6 ++-- 4 files changed, 14 insertions(+), 28 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index d18577df80a..8a2815dfe9c 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -289,22 +289,20 @@ public: static bool isSpecialCompressionAllowed(const SubstreamPath & path); - template - static State * checkAndGetSerializeState(SerializeBinaryBulkStatePtr & state, const Serialization &); - - template - static State * checkAndGetDeserializeState(DeserializeBinaryBulkStatePtr & state, const Serialization &); +protected: + template + State * checkAndGetState(const StatePtr & state) const; }; using SerializationPtr = std::shared_ptr; using Serializations = std::vector; -template -static State * checkAndGetState(StatePtr & state) +template +State * ISerialization::checkAndGetState(const StatePtr & state) const { if (!state) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Got empty state for {}", demangle(typeid(Serialization).name())); + "Got empty state for {}", demangle(typeid(*this).name())); auto * state_concrete = typeid_cast(state.get()); if (!state_concrete) @@ -312,7 +310,7 @@ static State * checkAndGetState(StatePtr & state) auto & state_ref = *state; throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid State for {}. Expected: {}, got {}", - demangle(typeid(Serialization).name()), + demangle(typeid(*this).name()), demangle(typeid(State).name()), demangle(typeid(state_ref).name())); } @@ -320,16 +318,4 @@ static State * checkAndGetState(StatePtr & state) return state_concrete; } -template -State * ISerialization::checkAndGetSerializeState(SerializeBinaryBulkStatePtr & state, const Serialization &) -{ - return checkAndGetState(state); -} - -template -State * ISerialization::checkAndGetDeserializeState(DeserializeBinaryBulkStatePtr & state, const Serialization &) -{ - return checkAndGetState(state); -} - } diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 3dff6cb4e4b..5095acca430 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -230,7 +230,7 @@ void SerializationLowCardinality::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * low_cardinality_state = checkAndGetSerializeState(state, *this); + auto * low_cardinality_state = checkAndGetState(state); KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); if (low_cardinality_state->shared_dictionary && settings.low_cardinality_max_dictionary_size) @@ -469,7 +469,7 @@ void SerializationLowCardinality::serializeBinaryBulkWithMultipleStreams( const ColumnLowCardinality & low_cardinality_column = typeid_cast(column); - auto * low_cardinality_state = checkAndGetSerializeState(state, *this); + auto * low_cardinality_state = checkAndGetState(state); auto & global_dictionary = low_cardinality_state->shared_dictionary; KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); @@ -568,7 +568,7 @@ void SerializationLowCardinality::deserializeBinaryBulkWithMultipleStreams( if (!indexes_stream) throw Exception("Got empty stream for SerializationLowCardinality indexes.", ErrorCodes::LOGICAL_ERROR); - auto * low_cardinality_state = checkAndGetDeserializeState(state, *this); + auto * low_cardinality_state = checkAndGetState(state); KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value); auto read_dictionary = [this, low_cardinality_state, keys_stream]() diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index d9ad66d1f35..458921f6e9d 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -224,7 +224,7 @@ void SerializationSparse::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { - auto * state_sparse = checkAndGetDeserializeState(state, *this); + auto * state_sparse = checkAndGetState(state); if (!settings.continuous_reading) state_sparse->reset(); diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 12ef7d9422f..19f1060dd3a 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -314,7 +314,7 @@ void SerializationTuple::serializeBinaryBulkStateSuffix( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * tuple_state = checkAndGetSerializeState(state, *this); + auto * tuple_state = checkAndGetState(state); for (size_t i = 0; i < elems.size(); ++i) elems[i]->serializeBinaryBulkStateSuffix(settings, tuple_state->states[i]); @@ -340,7 +340,7 @@ void SerializationTuple::serializeBinaryBulkWithMultipleStreams( SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - auto * tuple_state = checkAndGetSerializeState(state, *this); + auto * tuple_state = checkAndGetState(state); for (const auto i : collections::range(0, elems.size())) { @@ -356,7 +356,7 @@ void SerializationTuple::deserializeBinaryBulkWithMultipleStreams( DeserializeBinaryBulkStatePtr & state, SubstreamsCache * cache) const { - auto * tuple_state = checkAndGetDeserializeState(state, *this); + auto * tuple_state = checkAndGetState(state); auto mutable_column = column->assumeMutable(); auto & column_tuple = assert_cast(*mutable_column); From e46396331dfed6554241d5aa38c3c7ba001f9529 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 14 Oct 2021 20:28:31 +0300 Subject: [PATCH 057/358] better permute in ColumnSparse --- src/Columns/ColumnSparse.cpp | 52 ++++++++++++------------------------ 1 file changed, 17 insertions(+), 35 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index f01d9e12e1e..5a0087af719 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -360,10 +360,20 @@ void ColumnSparse::expand(const Filter & mask, bool inverted) ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const { - limit = limit ? std::min(limit, _size) : _size; + return permuteImpl(*this, perm, limit); +} - if (perm.size() < limit) - throw Exception("Size of permutation is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); +ColumnPtr ColumnSparse::index(const IColumn & indexes, size_t limit) const +{ + return selectIndexImpl(*this, indexes, limit); +} + +template +ColumnPtr ColumnSparse::indexImpl(const PaddedPODArray & indexes, size_t limit) const +{ + assert(limit <= indexes.size()); + if (limit == 0) + return ColumnSparse::create(values->cloneEmpty()); if (offsets->empty()) { @@ -386,14 +396,14 @@ ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const if (execute_linear) { - PaddedPODArray indexes(_size); + PaddedPODArray values_index(_size); auto offset_it = begin(); for (size_t i = 0; i < _size; ++i, ++offset_it) - indexes[i] = offset_it.getValueIndex(); + values_index[i] = offset_it.getValueIndex(); for (size_t i = 0; i < limit; ++i) { - size_t index = indexes[perm[i]]; + size_t index = values_index[indexes[i]]; if (index != 0) { res_values->insertFrom(*values, index); @@ -405,7 +415,7 @@ ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const { for (size_t i = 0; i < limit; ++i) { - size_t index = getValueIndex(perm[i]); + size_t index = getValueIndex(indexes[i]); if (index != 0) { res_values->insertFrom(*values, index); @@ -417,34 +427,6 @@ ColumnPtr ColumnSparse::permute(const Permutation & perm, size_t limit) const return ColumnSparse::create(std::move(res_values), std::move(res_offsets), limit); } -ColumnPtr ColumnSparse::index(const IColumn & indexes, size_t limit) const -{ - return selectIndexImpl(*this, indexes, limit); -} - -template -ColumnPtr ColumnSparse::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - limit = limit ? std::min(limit, indexes.size()) : indexes.size(); - - auto res_offsets = offsets->cloneEmpty(); - auto & res_offsets_data = assert_cast(*res_offsets).getData(); - auto res_values = values->cloneEmpty(); - res_values->insertDefault(); - - for (size_t i = 0; i < limit; ++i) - { - size_t index = getValueIndex(indexes[i]); - if (index != 0) - { - res_values->insertFrom(*values, index); - res_offsets_data.push_back(i); - } - } - - return ColumnSparse::create(std::move(res_values), std::move(res_offsets), limit); -} - int ColumnSparse::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const { if (const auto * rhs_sparse = typeid_cast(&rhs_)) From f1a5f798499a1954a7bfe6b5c2666c2c1932252f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 19 Oct 2021 19:54:20 +0300 Subject: [PATCH 058/358] fix sparse serialization --- .../Serializations/SerializationInfo.cpp | 3 +- .../Serializations/SerializationSparse.cpp | 6 ++- .../System/StorageSystemPartsColumns.cpp | 46 ++++++------------- 3 files changed, 22 insertions(+), 33 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 3495c1043a4..0b35ad48b3b 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -46,7 +46,7 @@ void SerializationInfoBuilder::add(const Block & block) info->columns[elem.name].num_defaults += static_cast( num_rows * elem.column->getRatioOfDefaultRows(default_rows_search_sample_ratio)); - elem.type->forEachSubcolumn([&](const auto &, const auto & name, const auto & data) + IDataType::forEachSubcolumn([&](const auto &, const auto & name, const auto & data) { if (!data.type->supportsSparseSerialization()) return; @@ -62,6 +62,7 @@ void SerializationInfoBuilder::add(const Block & block) auto full_name = Nested::concatenateName(elem.name, name); info->columns[full_name].num_defaults += static_cast( num_rows * data.column->getRatioOfDefaultRows(default_rows_search_sample_ratio)); + }, elem.type->getDefaultSerialization(), elem.type, elem.column); } } diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index a39200fd0b6..1a703ea2075 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -159,11 +159,15 @@ void SerializationSparse::enumerateStreams( data.type = type ? std::make_shared() : nullptr; data.serialization = std::make_shared>(); data.column = column_sparse ? column_sparse->getOffsetsPtr() : nullptr; + size_t column_size = column_sparse ? column_sparse->size() : 0; + + path.push_back(Substream::SparseOffsets); + path.back().data = data; callback(path); path.back() = Substream::SparseElements; - path.back().data = {type, column, getPtr(), std::make_shared(data.column, column->size())}; + path.back().data = {type, column, getPtr(), std::make_shared(data.column, column_size)}; auto next_column = column_sparse ? column_sparse->getValuesPtr() : nullptr; nested->enumerateStreams(path, callback, type, next_column); diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 715c822aece..8401ab321d1 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -222,43 +222,27 @@ void StorageSystemPartsColumns::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.marks); + auto serialization = part->getSerializationForColumn(column); if (columns_mask[src_index++]) - { - auto kind = part->getSerializationForColumn(column)->getKind(); - columns[res_index++]->insert(ISerialization::kindToString(kind)); - } + columns[res_index++]->insert(ISerialization::kindToString(serialization->getKind())); - auto subcolumns = column.type->getSubcolumnNames(); - if (columns_mask[src_index++]) + Array subcolumn_names; + Array subcolumn_types; + Array subcolumn_sers; + + IDataType::forEachSubcolumn([&](const auto &, const auto & name, const auto & data) { - Array array(subcolumns.begin(), subcolumns.end()); - columns[res_index++]->insert(array); - } + subcolumn_names.push_back(name); + subcolumn_types.push_back(data.type->getName()); + subcolumn_sers.push_back(ISerialization::kindToString(data.serialization->getKind())); + }, serialization, column.type, nullptr); if (columns_mask[src_index++]) - { - Array array; - array.reserve(subcolumns.size()); - for (const auto & name : subcolumns) - array.push_back(column.type->getSubcolumnType(name)->getName()); - - columns[res_index++]->insert(array); - } - + columns[res_index++]->insert(subcolumn_names); if (columns_mask[src_index++]) - { - Array array; - array.reserve(subcolumns.size()); - for (const auto & name : subcolumns) - { - auto subtype = column.type->getSubcolumnType(name); - auto subcolumn = NameAndTypePair(column.name, name, column.type, subtype); - auto kind = part->getSerializationForColumn(subcolumn)->getKind(); - array.push_back(ISerialization::kindToString(kind)); - } - - columns[res_index++]->insert(array); - } + columns[res_index++]->insert(subcolumn_types); + if (columns_mask[src_index++]) + columns[res_index++]->insert(subcolumn_sers); if (has_state_column) columns[res_index++]->insert(part->stateString()); From 0099dfd52381248a0aa212821a694ed646691eaa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 29 Oct 2021 20:21:02 +0300 Subject: [PATCH 059/358] refactoring of SerializationInfo --- src/Columns/ColumnSparse.h | 1 - src/Columns/ColumnTuple.cpp | 12 + src/Columns/ColumnTuple.h | 1 + src/Columns/IColumn.cpp | 6 + src/Columns/IColumn.h | 6 +- .../CompressionFactoryAdditions.cpp | 2 +- src/DataTypes/DataTypeTuple.cpp | 20 +- src/DataTypes/DataTypeTuple.h | 3 +- src/DataTypes/IDataType.cpp | 109 +++--- src/DataTypes/IDataType.h | 38 ++- .../Serializations/ISerialization.cpp | 15 +- src/DataTypes/Serializations/ISerialization.h | 23 +- .../Serializations/SerializationArray.cpp | 27 +- .../Serializations/SerializationArray.h | 3 +- .../Serializations/SerializationInfo.cpp | 314 +++++++++--------- .../Serializations/SerializationInfo.h | 116 ++++--- .../Serializations/SerializationInfoTuple.cpp | 116 +++++++ .../Serializations/SerializationInfoTuple.h | 31 ++ .../SerializationLowCardinality.cpp | 22 +- .../SerializationLowCardinality.h | 3 +- .../Serializations/SerializationMap.cpp | 14 +- .../Serializations/SerializationMap.h | 3 +- .../Serializations/SerializationNamed.cpp | 9 +- .../Serializations/SerializationNamed.h | 3 +- .../Serializations/SerializationNullable.cpp | 25 +- .../Serializations/SerializationNullable.h | 3 +- .../Serializations/SerializationSparse.cpp | 31 +- .../Serializations/SerializationSparse.h | 3 +- .../Serializations/SerializationTuple.cpp | 20 +- .../Serializations/SerializationTuple.h | 3 +- .../Serializations/SerializationWrapper.cpp | 5 +- .../Serializations/SerializationWrapper.h | 3 +- src/Formats/NativeReader.cpp | 16 +- src/Formats/NativeWriter.cpp | 18 +- src/Interpreters/InterpreterDescribeQuery.cpp | 2 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 2 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 80 +++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 20 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 1 + src/Storages/MergeTree/IMergeTreeReader.h | 3 +- .../MergeTree/IMergedBlockOutputStream.cpp | 20 +- .../MergeTree/IMergedBlockOutputStream.h | 8 +- src/Storages/MergeTree/MergeTask.cpp | 17 +- src/Storages/MergeTree/MergeTask.h | 1 - .../MergeTree/MergeTreeDataPartCompact.cpp | 3 +- .../MergeTree/MergeTreeDataPartCompact.h | 1 - .../MergeTree/MergeTreeDataPartInMemory.cpp | 5 +- .../MergeTree/MergeTreeDataPartInMemory.h | 1 - .../MergeTree/MergeTreeDataPartWide.cpp | 20 +- .../MergeTree/MergeTreeDataPartWide.h | 1 - .../MergeTreeDataPartWriterCompact.cpp | 11 +- .../MergeTreeDataPartWriterCompact.h | 1 - .../MergeTreeDataPartWriterOnDisk.cpp | 3 +- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 6 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 27 +- .../MergeTree/MergeTreeDataPartWriterWide.h | 1 - .../MergeTree/MergeTreeDataWriter.cpp | 28 +- .../MergeTree/MergeTreeReaderCompact.cpp | 10 - .../MergeTree/MergeTreeReaderWide.cpp | 10 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 3 +- .../MergeTree/MergedBlockOutputStream.cpp | 18 +- .../MergeTree/MergedBlockOutputStream.h | 2 +- .../MergedColumnOnlyOutputStream.cpp | 9 +- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 - src/Storages/MergeTree/MutateTask.cpp | 26 +- src/Storages/MergeTree/checkDataPart.cpp | 8 +- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 7 +- src/Storages/StorageTinyLog.cpp | 2 +- .../System/StorageSystemPartsColumns.cpp | 4 +- .../01780_column_sparse_tuple.reference | 7 + .../0_stateless/01780_column_sparse_tuple.sql | 10 + 74 files changed, 798 insertions(+), 615 deletions(-) create mode 100644 src/DataTypes/Serializations/SerializationInfoTuple.cpp create mode 100644 src/DataTypes/Serializations/SerializationInfoTuple.h diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index eb24664c7bb..6eb09642510 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -30,7 +30,6 @@ private: public: static constexpr auto DEFAULT_ROWS_SEARCH_SAMPLE_RATIO = 0.1; static constexpr auto DEFAULT_RATIO_FOR_SPARSE_SERIALIZATION = 0.95; - // static constexpr auto MIN_ROWS_TO_SEARCH_DEFAULTS = DEFAULT_ROWS_SEARCH_STEP * 16; using Base = COWHelper; static Ptr create(const ColumnPtr & values_, const ColumnPtr & offsets_, size_t size_) diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 852c069e1da..d667b264d55 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -555,4 +556,15 @@ void ColumnTuple::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, siz return getIndicesOfNonDefaultRowsImpl(indices, from, limit); } +SerializationInfoPtr ColumnTuple::getSerializationInfo() const +{ + MutableSerializationInfos infos; + infos.reserve(columns.size()); + + for (const auto & column : columns) + infos.push_back(const_pointer_cast(column->getSerializationInfo())); + + return std::make_shared(std::move(infos), SerializationInfo::Settings{}); +} + } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index abc713d6ee5..b4c7f6bdf9a 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -96,6 +96,7 @@ public: ColumnPtr compress() const override; double getRatioOfDefaultRows(double sample_ratio) const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; + SerializationInfoPtr getSerializationInfo() const override; size_t tupleSize() const { return columns.size(); } diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 555f4e112c9..e7caee3b23d 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -63,6 +64,11 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const Field & defa return res; } +SerializationInfoPtr IColumn::getSerializationInfo() const +{ + return std::make_shared(ISerialization::getKind(*this), SerializationInfo::Settings{}); +} + bool isColumnNullable(const IColumn & column) { return checkColumn(column); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index abb86075250..b1a6e83ee98 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -26,8 +26,8 @@ class ColumnGathererStream; class Field; class WeakHash32; -class ISerialization; -using SerializationPtr = std::shared_ptr; +class SerializationInfo; +using SerializationInfoPtr = std::shared_ptr; /* * Represents a set of equal ranges in previous column to perform sorting in current column. @@ -404,6 +404,8 @@ public: /// Used to create full column from sparse. virtual Ptr createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const; + virtual SerializationInfoPtr getSerializationInfo() const; + /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. virtual Ptr compress() const diff --git a/src/Compression/CompressionFactoryAdditions.cpp b/src/Compression/CompressionFactoryAdditions.cpp index b5f00c60827..d87d0f8b4ee 100644 --- a/src/Compression/CompressionFactoryAdditions.cpp +++ b/src/Compression/CompressionFactoryAdditions.cpp @@ -117,7 +117,7 @@ ASTPtr CompressionCodecFactory::validateCodecAndGetPreprocessedAST( }; ISerialization::SubstreamPath path; - column_type->getDefaultSerialization()->enumerateStreams(path, callback, column_type, nullptr); + column_type->getDefaultSerialization()->enumerateStreams(path, callback, column_type); if (!result_codec) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find any substream with data type for type {}. It's a bug", column_type->getName()); diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index aef9ef9039f..fbc6ef880a0 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -254,21 +255,32 @@ SerializationPtr DataTypeTuple::doGetDefaultSerialization() const return std::make_shared(std::move(serializations), use_explicit_names); } -SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const SerializationCallback & callback) const +SerializationPtr DataTypeTuple::getSerialization(const SerializationInfo & info) const { SerializationTuple::ElementSerializations serializations(elems.size()); + const auto & info_tuple = assert_cast(info); bool use_explicit_names = have_explicit_names && serialize_names; + for (size_t i = 0; i < elems.size(); ++i) { String elem_name = use_explicit_names ? names[i] : toString(i + 1); - auto subcolumn_name = Nested::concatenateName(column_name, elem_name); - auto serializaion = elems[i]->getSerialization(subcolumn_name, callback); - serializations[i] = std::make_shared(serializaion, elem_name); + auto serialization = elems[i]->getSerialization(*info_tuple.getElementInfo(i)); + serializations[i] = std::make_shared(serialization, elem_name); } return std::make_shared(std::move(serializations), use_explicit_names); } +MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const SerializationInfo::Settings & settings) const +{ + MutableSerializationInfos infos; + infos.reserve(elems.size()); + for (const auto & elem : elems) + infos.push_back(elem->createSerializationInfo(settings)); + + return std::make_shared(std::move(infos), settings); +} + static DataTypePtr create(const ASTPtr & arguments) { diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 9c0419d9dcf..0ac029a63ea 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -54,8 +54,9 @@ public: size_t getMaximumSizeOfValueInMemory() const override; size_t getSizeOfValueInMemory() const override; - SerializationPtr getSerialization(const String & column_name, const SerializationCallback & callback) const override; SerializationPtr doGetDefaultSerialization() const override; + SerializationPtr getSerialization(const SerializationInfo & info) const override; + MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const override; const DataTypePtr & getElement(size_t i) const { return elems[i]; } const DataTypes & getElements() const { return elems; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 5e913f4d211..57199f7188f 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -77,9 +77,7 @@ size_t IDataType::getSizeOfValueInMemory() const void IDataType::forEachSubcolumn( const SubcolumnCallback & callback, - const SerializationPtr & serialization, - const DataTypePtr & type, - const ColumnPtr & column) + const SubstreamData & data) { ISerialization::StreamCallback callback_with_data = [&](const auto & subpath) { @@ -88,66 +86,59 @@ void IDataType::forEachSubcolumn( if (!subpath[i].visited && ISerialization::hasSubcolumnForPath(subpath, i + 1)) { auto name = ISerialization::getSubcolumnNameForStream(subpath, i + 1); - auto data = ISerialization::createFromPath(subpath, i); - callback(subpath, name, data); + auto subdata = ISerialization::createFromPath(subpath, i); + callback(subpath, name, subdata); } subpath[i].visited = true; } }; - ISerialization::SubstreamPath path; - serialization->enumerateStreams(path, callback_with_data, type, column); + SubstreamPath path; + data.serialization->enumerateStreams(path, callback_with_data, data); } -DataTypePtr IDataType::tryGetSubcolumnType(const String & subcolumn_name) const +template +Ptr IDataType::getForSubcolumn( + const String & subcolumn_name, + const SubstreamData & data, + Ptr SubstreamData::*member, + bool throw_if_null) const { - DataTypePtr res; - forEachSubcolumn([&](const auto &, const auto & name, const auto & data) + Ptr res; + forEachSubcolumn([&](const auto &, const auto & name, const auto & subdata) { if (name == subcolumn_name) - res = data.type; - }, getDefaultSerialization(), getPtr(), nullptr); + res = subdata.*member; + }, data); + + if (!res && throw_if_null) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); return res; } +DataTypePtr IDataType::tryGetSubcolumnType(const String & subcolumn_name) const +{ + SubstreamData data = { getDefaultSerialization(), getPtr(), nullptr, nullptr }; + return getForSubcolumn(subcolumn_name, data, &SubstreamData::type, false); +} + 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()); + SubstreamData data = { getDefaultSerialization(), getPtr(), nullptr, nullptr }; + return getForSubcolumn(subcolumn_name, data, &SubstreamData::type); } SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_name, const SerializationPtr & serialization) const { - SerializationPtr res; - forEachSubcolumn([&](const auto &, const auto & name, const auto & data) - { - if (name == subcolumn_name) - res = data.serialization; - }, serialization, nullptr, nullptr); - - if (res) - return res; - - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); + SubstreamData data = { serialization, nullptr, nullptr, nullptr }; + return getForSubcolumn(subcolumn_name, data, &SubstreamData::serialization); } ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const { - ColumnPtr res; - forEachSubcolumn([&](const auto &, const auto & name, const auto & data) - { - if (name == subcolumn_name) - res = data.column; - }, getDefaultSerialization(), nullptr, column); - - if (res) - return res; - - throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName()); + SubstreamData data = { getDefaultSerialization(), nullptr, column, nullptr }; + return getForSubcolumn(subcolumn_name, data, &SubstreamData::column); } Names IDataType::getSubcolumnNames() const @@ -156,7 +147,7 @@ Names IDataType::getSubcolumnNames() const forEachSubcolumn([&](const auto &, const auto & name, const auto &) { res.push_back(name); - }, getDefaultSerialization(), nullptr, nullptr); + }, { getDefaultSerialization(), nullptr, nullptr, nullptr }); return res; } @@ -175,6 +166,12 @@ void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const custom_serialization = std::move(custom_desc_->serialization); } +MutableSerializationInfoPtr IDataType::createSerializationInfo( + const SerializationInfo::Settings & settings) const +{ + return std::make_shared(ISerialization::Kind::DEFAULT, settings); +} + SerializationPtr IDataType::getDefaultSerialization() const { if (custom_serialization) @@ -196,31 +193,9 @@ SerializationPtr IDataType::getSerialization(ISerialization::Kind kind) const return getDefaultSerialization(); } -SerializationPtr IDataType::getSerialization(const IColumn & column) const +SerializationPtr IDataType::getSerialization(const SerializationInfo & info) const { - return getSerialization(ISerialization::getKind(column)); -} - -SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const -{ - return getSerialization(column_name, [&info](const auto & name) { return info.getKind(name); }); -} - -SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationCallback & callback) const -{ - return getSerialization(callback(column_name)); -} - -SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const -{ - if (supportsSparseSerialization()) - { - double ratio = settings.num_rows ? std::min(static_cast(settings.num_default_rows) / settings.num_rows, 1.0) : 0.0; - if (ratio > settings.ratio_for_sparse_serialization) - return getSparseSerialization(); - } - - return getDefaultSerialization(); + return getSerialization(info.getKind()); } // static @@ -229,11 +204,11 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, con if (column.isSubcolumn()) { const auto & type_in_storage = column.getTypeInStorage(); - auto default_serialization = type_in_storage->getDefaultSerialization(); - return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), default_serialization); + auto serialization = type_in_storage->getSerialization(info); + return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), serialization); } - return column.type->getSerialization(column.name, info); + return column.type->getSerialization(info); } } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 1e05dd75067..41b624c5655 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB { @@ -27,7 +28,6 @@ using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; struct NameAndTypePair; -class SerializationInfo; struct DataTypeWithConstInfo { @@ -84,19 +84,23 @@ public: SerializationPtr getSubcolumnSerialization(const String & subcolumn_name, const SerializationPtr & serialization) const; ColumnPtr getSubcolumn(const String & subcolumn_name, const ColumnPtr & column) const; + using SubstreamData = ISerialization::SubstreamData; + using SubstreamPath = ISerialization::SubstreamPath; + using SubcolumnCallback = std::function; + const SubstreamData &)>; static void forEachSubcolumn( const SubcolumnCallback & callback, - const SerializationPtr & serialization, - const DataTypePtr & type, - const ColumnPtr & column); + const SubstreamData & data); Names getSubcolumnNames() const; + virtual MutableSerializationInfoPtr createSerializationInfo( + const SerializationInfo::Settings & settings) const; + /// TODO: support more types. virtual bool supportsSparseSerialization() const { return !haveSubtypes(); } @@ -106,18 +110,8 @@ public: /// Chooses serialziation according to serialization kind. SerializationPtr getSerialization(ISerialization::Kind kind) const; - /// Chooses serialziation according to column content. - SerializationPtr getSerialization(const IColumn & column) const; - - /// Chooses serialization according to collected information about content of columns. - SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const; - - /// Chooses serialization according to settings. - SerializationPtr getSerialization(const ISerialization::Settings & settings) const; - - using SerializationCallback = std::function; - - virtual SerializationPtr getSerialization(const String & column_name, const SerializationCallback & callback) const; + /// Chooses serialization according to collected information about content of column. + virtual SerializationPtr getSerialization(const SerializationInfo & info) const; /// Chooses between subcolumn serialization and regular serialization according to @column. /// This method typically should be used to get serialization for reading column or subcolumn. @@ -302,6 +296,14 @@ protected: public: const IDataTypeCustomName * getCustomName() const { return custom_name.get(); } const ISerialization * getCustomSerialization() const { return custom_serialization.get(); } + +private: + template + Ptr getForSubcolumn( + const String & subcolumn_name, + const SubstreamData & data, + Ptr SubstreamData::*member, + bool throw_if_null = true) const; }; diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 33a9750f63a..b09e606080f 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -72,18 +72,22 @@ String ISerialization::SubstreamPath::toString() const void ISerialization::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { path.push_back(Substream::Regular); - path.back().data = {type, column, getPtr(), nullptr}; + path.back().data = data; callback(path); path.pop_back(); } void ISerialization::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const { - enumerateStreams(path, callback, nullptr, nullptr); + enumerateStreams(path, callback, {getPtr(), nullptr, nullptr, nullptr}); +} + +void ISerialization::enumerateStreams(SubstreamPath & path, const StreamCallback & callback, const DataTypePtr & type) const +{ + enumerateStreams(path, callback, {getPtr(), type, nullptr, nullptr}); } void ISerialization::serializeBinaryBulk(const IColumn & column, WriteBuffer &, size_t, size_t) const @@ -268,10 +272,9 @@ ISerialization::SubstreamData ISerialization::createFromPath(const SubstreamPath assert(prefix_len < path.size()); SubstreamData res = path[prefix_len].data; - res.creator.reset(); for (ssize_t i = static_cast(prefix_len) - 1; i >= 0; --i) { - const auto & creator = path[i].data.creator; + const auto & creator = path[i].creator; if (creator) { res.type = res.type ? creator->create(res.type) : res.type; diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index 58ed331da8b..0bc79a9ae94 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -32,6 +32,9 @@ using DataTypePtr = std::shared_ptr; class ISerialization; using SerializationPtr = std::shared_ptr; +class SerializationInfo; +using SerializationInfoPtr = std::shared_ptr; + class Field; struct FormatSettings; @@ -98,10 +101,10 @@ public: struct SubstreamData { + SerializationPtr serialization; DataTypePtr type; ColumnPtr column; - SerializationPtr serialization; - SubcolumnCreatorPtr creator; + SerializationInfoPtr serialization_info; }; struct Substream @@ -136,6 +139,9 @@ public: /// Data for current substream. SubstreamData data; + /// Creator of subcolumn for current substream. + SubcolumnCreatorPtr creator = nullptr; + /// Flag, that may help to traverse substream paths. mutable bool visited = false; @@ -158,13 +164,14 @@ public: virtual void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const; + const SubstreamData & data) const; 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, {}); } + void enumerateStreams(SubstreamPath & path, const StreamCallback & callback, const DataTypePtr & type) const; + using OutputStreamGetter = std::function; using InputStreamGetter = std::function; @@ -208,13 +215,6 @@ public: double avg_value_size_hint = 0; }; - struct Settings - { - size_t num_rows; - size_t num_default_rows; - double ratio_for_sparse_serialization; - }; - /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. virtual void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & /*settings*/, @@ -339,6 +339,7 @@ protected: using SerializationPtr = std::shared_ptr; using Serializations = std::vector; +using SerializationByName = std::unordered_map; template State * ISerialization::checkAndGetState(const StatePtr & state) const diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 4ccee54c294..15972ba241c 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -198,33 +198,38 @@ ColumnPtr SerializationArray::SubcolumnCreator::create(const ColumnPtr & prev) c void SerializationArray::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { - const auto * type_array = type ? &assert_cast(*type) : nullptr; - const auto * column_array = column ? &assert_cast(*column) : nullptr; + const auto * type_array = data.type ? &assert_cast(*data.type) : nullptr; + const auto * column_array = data.column ? &assert_cast(*data.column) : nullptr; auto offsets_column = column_array ? column_array->getOffsetsPtr() : nullptr; path.push_back(Substream::ArraySizes); path.back().data = { - type ? std::make_shared() : nullptr, - offsets_column ? arrayOffsetsToSizes(*offsets_column) : nullptr, std::make_shared( std::make_shared>(), "size" + std::to_string(getArrayLevel(path)), false), - nullptr, + data.type ? std::make_shared() : nullptr, + offsets_column ? arrayOffsetsToSizes(*offsets_column) : nullptr, + data.serialization_info, }; callback(path); path.back() = Substream::ArrayElements; - path.back().data = {type, column, getPtr(), std::make_shared(offsets_column)}; + path.back().data = data; + path.back().creator = std::make_shared(offsets_column); - auto next_type = type_array ? type_array->getNestedType() : nullptr; - auto next_column = column_array ? column_array->getDataPtr() : nullptr; + SubstreamData next_data = + { + nested, + type_array ? type_array->getNestedType() : nullptr, + column_array ? column_array->getDataPtr() : nullptr, + data.serialization_info, + }; - nested->enumerateStreams(path, callback, next_type, next_column); + nested->enumerateStreams(path, callback, next_data); path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index 83045d4c033..6ff0f426b8a 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -38,8 +38,7 @@ public: void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 0b35ad48b3b..ab94127dc0c 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -1,8 +1,10 @@ #include #include +#include #include #include #include +#include #include #include @@ -17,113 +19,14 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INCORRECT_DATA; -} - -SerializationInfoBuilder::SerializationInfoBuilder( - double ratio_for_sparse_serialization_, - double default_rows_search_sample_ratio_) - : ratio_for_sparse_serialization(ratio_for_sparse_serialization_) - , default_rows_search_sample_ratio(default_rows_search_sample_ratio_) - , info(std::make_shared()) -{ -} - -void SerializationInfoBuilder::add(const Block & block) -{ - size_t num_rows = block.rows(); - info->number_of_rows += num_rows; - - if (!canHaveSparseSerialization()) - return; - - for (const auto & elem : block) - { - /// Just skip column and always return default serialization. - if (!elem.type->supportsSparseSerialization()) - continue; - - /// Multiply by step to restore approximate number of default values. - info->columns[elem.name].num_defaults += static_cast( - num_rows * elem.column->getRatioOfDefaultRows(default_rows_search_sample_ratio)); - - IDataType::forEachSubcolumn([&](const auto &, const auto & name, const auto & data) - { - if (!data.type->supportsSparseSerialization()) - return; - - auto parent_subcolumn_name = Nested::splitName(name, /*reverse=*/ true).first; - if (!parent_subcolumn_name.empty()) - { - auto parent_subcolumn_type = elem.type->tryGetSubcolumnType(parent_subcolumn_name); - if (parent_subcolumn_type && !parent_subcolumn_type->supportsSparseSerialization()) - return; - } - - auto full_name = Nested::concatenateName(elem.name, name); - info->columns[full_name].num_defaults += static_cast( - num_rows * data.column->getRatioOfDefaultRows(default_rows_search_sample_ratio)); - - }, elem.type->getDefaultSerialization(), elem.type, elem.column); - } -} - -void SerializationInfoBuilder::add(const SerializationInfo & other) -{ - info->number_of_rows += other.number_of_rows; - for (const auto & [name, column_info] : other.columns) - info->columns[name].num_defaults += column_info.num_defaults; -} - -SerializationInfoPtr SerializationInfoBuilder::build() && -{ - size_t total_rows = info->number_of_rows; - for (auto & [_, column_info] : info->columns) - { - double ratio = total_rows ? std::min(static_cast(column_info.num_defaults) / total_rows, 1.0) : 0.0; - if (ratio > ratio_for_sparse_serialization) - column_info.kind = ISerialization::Kind::SPARSE; - } - - return std::move(info); -} - -SerializationInfoPtr SerializationInfoBuilder::buildFrom(const SerializationInfo & other) && -{ - for (const auto & [name, column_info] : other.columns) - { - auto it = info->columns.find(name); - if (it == info->columns.end()) - info->columns[name] = column_info; - else - it->second.kind = column_info.kind; - } - - return std::move(info); -} - -ISerialization::Kind SerializationInfo::getKind(const String & column_name) const -{ - auto it = columns.find(column_name); - if (it == columns.end()) - return ISerialization::Kind::DEFAULT; - - return it->second.kind; -} - -size_t SerializationInfo::getNumberOfDefaultRows(const String & column_name) const -{ - auto it = columns.find(column_name); - if (it == columns.end()) - return 0; - - return it->second.num_defaults; + extern const int CORRUPTED_DATA; } namespace { constexpr auto KEY_VERSION = "version"; -constexpr auto KEY_NUMBER_OF_ROWS = "number_of_rows"; +constexpr auto KEY_NUM_ROWS = "num_rows"; constexpr auto KEY_COLUMNS = "columns"; constexpr auto KEY_NUM_DEFAULTS = "num_defaults"; constexpr auto KEY_KIND = "kind"; @@ -131,68 +34,181 @@ constexpr auto KEY_NAME = "name"; } -void SerializationInfo::fromJSON(const String & json_str) +void SerializationInfo::Data::add(const IColumn & column) { + size_t rows = column.size(); + double ratio = column.getRatioOfDefaultRows(ColumnSparse::DEFAULT_ROWS_SEARCH_SAMPLE_RATIO); + + num_rows += rows; + num_defaults += static_cast(ratio * rows); +} + +void SerializationInfo::Data::add(const Data & other) +{ + num_rows += other.num_rows; + num_defaults += other.num_defaults; +} + +SerializationInfo::SerializationInfo(ISerialization::Kind kind_, const Settings & settings_) + : settings(settings_) + , kind(kind_) +{ +} + +void SerializationInfo::add(const IColumn & column) +{ + data.add(column); + if (settings.choose_kind) + kind = chooseKind(data, settings); +} + +void SerializationInfo::add(const SerializationInfo & other) +{ + data.add(other.data); + if (settings.choose_kind) + kind = chooseKind(data, settings); +} + +void SerializationInfo::replaceData(const SerializationInfo & other) +{ + data = other.data; +} + +MutableSerializationInfoPtr SerializationInfo::clone() const +{ + auto res = std::make_shared(kind, settings); + res->data = data; + return res; +} + +void SerializationInfo::serialializeKindBinary(WriteBuffer & out) const +{ + writeBinary(static_cast(kind), out); +} + +void SerializationInfo::deserializeFromKindsBinary(ReadBuffer & in) +{ + UInt8 kind_num; + readBinary(kind_num, in); + auto maybe_kind = magic_enum::enum_cast(kind_num); + if (!maybe_kind) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown serialization kind " + std::to_string(kind_num)); + + kind = *maybe_kind; +} + +Poco::JSON::Object SerializationInfo::toJSON() const +{ + Poco::JSON::Object object; + object.set(KEY_KIND, ISerialization::kindToString(kind)); + object.set(KEY_NUM_DEFAULTS, data.num_defaults); + object.set(KEY_NUM_ROWS, data.num_rows); + return object; +} + +void SerializationInfo::fromJSON(const Poco::JSON::Object & object) +{ + if (!object.has(KEY_KIND) || !object.has(KEY_NUM_DEFAULTS) || !object.has(KEY_NUM_ROWS)) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Missed field '{}' or '{}' or '{}' in SerializationInfo of columns", + KEY_KIND, KEY_NUM_DEFAULTS, KEY_NUM_ROWS); + + data.num_rows = object.getValue(KEY_NUM_ROWS); + data.num_defaults = object.getValue(KEY_NUM_DEFAULTS); + kind = ISerialization::stringToKind(object.getValue(KEY_KIND)); +} + +ISerialization::Kind SerializationInfo::chooseKind(const Data & data, const Settings & settings) +{ + double ratio = data.num_rows ? std::min(static_cast(data.num_defaults) / data.num_rows, 1.0) : 0.0; + return ratio > settings.ratio_for_sparse ? ISerialization::Kind::SPARSE : ISerialization::Kind::DEFAULT; +} + +SerializationInfoByName::SerializationInfoByName( + const NamesAndTypesList & columns, + const SerializationInfo::Settings & settings) +{ + for (const auto & column : columns) + emplace(column.name, column.type->createSerializationInfo(settings)); +} + +void SerializationInfoByName::add(const Block & block) +{ + for (const auto & column : block) + { + auto it = find(column.name); + if (it == end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Not found column {} in serialization infos", column.name); + + it->second->add(*column.column); + } +} + +void SerializationInfoByName::add(const SerializationInfoByName & other) +{ + for (const auto & [name, info] : other) + { + auto it = find(name); + if (it == end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Not found column {} in serialization infos", name); + + it->second->add(*info); + } +} + +void SerializationInfoByName::writeText(WriteBuffer & out) const +{ + Poco::JSON::Object object; + object.set(KEY_VERSION, SERIALIZATION_INFO_VERSION); + + Poco::JSON::Array column_infos; + for (const auto & [name, info] : *this) + { + auto info_json = info->toJSON(); + info_json.set(KEY_NAME, name); + column_infos.add(std::move(info_json)); + } + + object.set(KEY_COLUMNS, std::move(column_infos)); + + std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + oss.exceptions(std::ios::failbit); + Poco::JSON::Stringifier::stringify(object, oss); + + return writeString(oss.str(), out); +} + +void SerializationInfoByName::readText(ReadBuffer & in) +{ + String json_str; + readString(json_str, in); + Poco::JSON::Parser parser; auto object = parser.parse(json_str).extract(); - if (object->has(KEY_NUMBER_OF_ROWS)) - number_of_rows = object->getValue(KEY_NUMBER_OF_ROWS); - if (object->has(KEY_COLUMNS)) { auto array = object->getArray(KEY_COLUMNS); for (const auto & elem : *array) { auto elem_object = elem.extract(); - if (!elem_object->has(KEY_NAME) || !elem_object->has(KEY_NUM_DEFAULTS) || !elem_object->has(KEY_KIND)) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Missed field '{}' or '{}' or '{}' in SerializationInfo of columns", - KEY_NAME, KEY_NUM_DEFAULTS, KEY_KIND); + + if (!elem_object->has(KEY_NAME)) + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Missed field '{}' in SerializationInfo of columns", KEY_NAME); auto name = elem_object->getValue(KEY_NAME); - auto kind = elem_object->getValue(KEY_KIND); - auto num_defaults = elem_object->getValue(KEY_NUM_DEFAULTS); - columns[name] = {ISerialization::stringToKind(kind), num_defaults}; + auto it = find(name); + + if (it == end()) + throw Exception(ErrorCodes::CORRUPTED_DATA, + "There is not column {} in serialization infos", name); + + it->second->fromJSON(*elem_object); } } } -String SerializationInfo::toJSON() const -{ - Poco::JSON::Object info; - info.set(KEY_VERSION, version); - info.set(KEY_NUMBER_OF_ROWS, number_of_rows); - - Poco::JSON::Array column_infos; - for (const auto & [name, column_info] : columns) - { - Poco::JSON::Object column_info_json; - column_info_json.set(KEY_NAME, name); - column_info_json.set(KEY_KIND, ISerialization::kindToString(column_info.kind)); - column_info_json.set(KEY_NUM_DEFAULTS, column_info.num_defaults); - column_infos.add(std::move(column_info_json)); - } - - info.set(KEY_COLUMNS, std::move(column_infos)); - - std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - oss.exceptions(std::ios::failbit); - Poco::JSON::Stringifier::stringify(info, oss); - - return oss.str(); -} - -void SerializationInfo::readText(ReadBuffer & in) -{ - String json_str; - readString(json_str, in); - fromJSON(json_str); -} - -void SerializationInfo::writeText(WriteBuffer & out) const -{ - writeString(toJSON(), out); -} - } diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index e67e81edade..0635bfc9cbf 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -1,13 +1,21 @@ #pragma once -#include +#include #include -#include +#include + namespace DB { -/** Contains information about kinds of serialization of columns. +class ReadBuffer; +class WriteBuffer; +class NamesAndTypesList; +class Block; + +constexpr auto SERIALIZATION_INFO_VERSION = 1; + +/** Contains information about kind of serialization of column and its subcolumns. * Also contains information about content of columns, * that helps to choose kind of serialization of column. * @@ -19,72 +27,70 @@ namespace DB class SerializationInfo { public: - SerializationInfo() = default; - - static constexpr auto version = 1; - size_t getNumberOfDefaultRows(const String & column_name) const; - ISerialization::Kind getKind(const String & column_name) const; - - bool empty() const { return !number_of_rows && columns.empty(); } - size_t getNumberOfRows() const { return number_of_rows; } - - void readText(ReadBuffer & in); - void writeText(WriteBuffer & out) const; - -private: - void fromJSON(const String & json_str); - String toJSON() const; - - /// Information about one column. - /// Can be extended, when new kinds of serialization will be implemented. - struct Column + struct Data { - ISerialization::Kind kind = ISerialization::Kind::DEFAULT; + size_t num_rows = 0; size_t num_defaults = 0; + + void add(const IColumn & column); + void add(const Data & other); }; - using NameToColumn = std::unordered_map; + struct Settings + { + const double ratio_for_sparse = 1.0; + const bool choose_kind = false; + }; - size_t number_of_rows = 0; - NameToColumn columns; + SerializationInfo(ISerialization::Kind kind_, const Settings & settings_); - friend class SerializationInfoBuilder; + virtual ~SerializationInfo() = default; + + virtual bool hasCustomSerialization() const { return kind != ISerialization::Kind::DEFAULT; } + + virtual void add(const IColumn & column); + virtual void add(const SerializationInfo & other); + virtual void replaceData(const SerializationInfo & other); + virtual std::shared_ptr clone() const; + + virtual void serialializeKindBinary(WriteBuffer & out) const; + virtual void deserializeFromKindsBinary(ReadBuffer & in); + + virtual Poco::JSON::Object toJSON() const; + virtual void fromJSON(const Poco::JSON::Object & object); + + const Settings & getSettings() const { return settings; } + const Data & getData() const { return data; } + ISerialization::Kind getKind() const { return kind; } + + static ISerialization::Kind chooseKind(const Data & data, const Settings & settings); + +protected: + const Settings settings; + + ISerialization::Kind kind; + Data data; }; -using SerializationInfoPtr = std::shared_ptr; +using SerializationInfoPtr = std::shared_ptr; +using MutableSerializationInfoPtr = std::shared_ptr; -/// Builder, that helps to create SerializationInfo. -class SerializationInfoBuilder +using SerializationInfos = std::vector; +using MutableSerializationInfos = std::vector; + +class SerializationInfoByName : public std::unordered_map { public: - SerializationInfoBuilder(); - SerializationInfoBuilder( - double ratio_for_sparse_serialization_, - double default_rows_search_sample_ratio_ = ColumnSparse::DEFAULT_ROWS_SEARCH_SAMPLE_RATIO); + SerializationInfoByName() = default; + SerializationInfoByName( + const NamesAndTypesList & columns, + const SerializationInfo::Settings & settings); - /// Add information about column from block. void add(const Block & block); + void add(const SerializationInfoByName & other); - /// Add information about column from other SerializationInfo. - void add(const SerializationInfo & other); - - /// Choose kind of serialization for every column - /// according its content and return finalized SerializationInfo. - SerializationInfoPtr build() &&; - - /// Create SerializationInfo from other. - /// Respects kinds of serialization for columns, that exist in other SerializationInfo, - /// but keeps information about content of column from current SerializationInfo. - SerializationInfoPtr buildFrom(const SerializationInfo & other) &&; - - double getRatioForSparseSerialization() const { return ratio_for_sparse_serialization; } - bool canHaveSparseSerialization() const { return ratio_for_sparse_serialization < 1.0; } - -private: - double ratio_for_sparse_serialization; - double default_rows_search_sample_ratio; - - SerializationInfoPtr info; + void writeText(WriteBuffer & out) const; + void readText(ReadBuffer & in); }; } diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp new file mode 100644 index 00000000000..bb41fb8a30c --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -0,0 +1,116 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int CORRUPTED_DATA; + extern const int THERE_IS_NO_COLUMN; +} + +SerializationInfoTuple::SerializationInfoTuple( + MutableSerializationInfos elems_, const Settings & settings_) + : SerializationInfo(ISerialization::Kind::DEFAULT, settings_) + , elems(std::move(elems_)) +{ +} + +bool SerializationInfoTuple::hasCustomSerialization() const +{ + return std::any_of(elems.begin(), elems.end(), [](const auto & elem) { return elem->hasCustomSerialization(); }); +} + +void SerializationInfoTuple::add(const IColumn & column) +{ + SerializationInfo::add(column); + + const auto & column_tuple = assert_cast(column); + const auto & right_elems = column_tuple.getColumns(); + assert(elems.size() == right_elems.size()); + + for (size_t i = 0; i < elems.size(); ++i) + elems[i]->add(*right_elems[i]); +} + +void SerializationInfoTuple::add(const SerializationInfo & other) +{ + SerializationInfo::add(other); + + const auto & info_tuple = assert_cast(other); + assert(elems.size() == info_tuple.elems.size()); + + for (size_t i = 0; i < elems.size(); ++i) + elems[i]->add(*info_tuple.elems[i]); +} + +void SerializationInfoTuple::replaceData(const SerializationInfo & other) +{ + SerializationInfo::add(other); + + const auto & info_tuple = assert_cast(other); + assert(elems.size() == info_tuple.elems.size()); + + for (size_t i = 0; i < elems.size(); ++i) + elems[i]->replaceData(*info_tuple.elems[i]); +} +MutableSerializationInfoPtr SerializationInfoTuple::clone() const +{ + MutableSerializationInfos elems_cloned; + elems_cloned.reserve(elems.size()); + for (const auto & elem : elems) + elems_cloned.push_back(elem->clone()); + + return std::make_shared(std::move(elems_cloned), settings); +} + +void SerializationInfoTuple::serialializeKindBinary(WriteBuffer & out) const +{ + SerializationInfo::serialializeKindBinary(out); + + for (const auto & elem : elems) + elem->serialializeKindBinary(out); +} + +void SerializationInfoTuple::deserializeFromKindsBinary(ReadBuffer & in) +{ + SerializationInfo::deserializeFromKindsBinary(in); + + for (const auto & elem : elems) + elem->deserializeFromKindsBinary(in); +} + +Poco::JSON::Object SerializationInfoTuple::toJSON() const +{ + auto object = SerializationInfo::toJSON(); + Poco::JSON::Array subcolumns; + for (const auto & elem : elems) + subcolumns.add(elem->toJSON()); + + object.set("subcolumns", std::move(subcolumns)); + return object; +} + +void SerializationInfoTuple::fromJSON(const Poco::JSON::Object & object) +{ + SerializationInfo::fromJSON(object); + + if (!object.has("subcolumns")) + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Missed field '{}' in SerializationInfo of columns SerializationInfoTuple"); + + auto subcolumns = object.getArray("subcolumns"); + if (elems.size() != subcolumns->size()) + throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, + "Mismatched number of subcolumns beetween JSON and SerializationInfoTuple." + "Expected: {}, got: {}", elems.size(), subcolumns->size()); + + for (size_t i = 0; i < elems.size(); ++i) + elems[i]->fromJSON(*subcolumns->getObject(i)); +} + +} diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.h b/src/DataTypes/Serializations/SerializationInfoTuple.h new file mode 100644 index 00000000000..d196f80393e --- /dev/null +++ b/src/DataTypes/Serializations/SerializationInfoTuple.h @@ -0,0 +1,31 @@ +#pragma once +#include + +namespace DB +{ + +class SerializationInfoTuple : public SerializationInfo +{ +public: + SerializationInfoTuple(MutableSerializationInfos elems_, const Settings & settings_); + + bool hasCustomSerialization() const override; + void add(const IColumn & column) override; + void add(const SerializationInfo & other) override; + void replaceData(const SerializationInfo & other) override; + + MutableSerializationInfoPtr clone() const override; + void serialializeKindBinary(WriteBuffer & out) const override; + void deserializeFromKindsBinary(ReadBuffer & in) override; + + Poco::JSON::Object toJSON() const override; + void fromJSON(const Poco::JSON::Object & object) override; + + MutableSerializationInfoPtr getElementInfo(size_t i) const { return elems[i]; } + ISerialization::Kind getElementKind(size_t i) const { return elems[i]->getKind(); } + +private: + MutableSerializationInfos elems; +}; + +} diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index c2b0d239c12..a119ccc03ea 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -43,23 +43,23 @@ SerializationLowCardinality::SerializationLowCardinality(const DataTypePtr & dic void SerializationLowCardinality::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { - const auto * column_lc = column ? &getColumnLowCardinality(*column) : nullptr; - - SubstreamData data; - data.type = type ? dictionary_type : nullptr; - data.column = column_lc ? column_lc->getDictionary().getNestedColumn() : nullptr; - data.serialization = dict_inner_serialization; + const auto * column_lc = data.column ? &getColumnLowCardinality(*data.column) : nullptr; path.push_back(Substream::DictionaryKeys); - path.back().data = data; + path.back().data = + { + dict_inner_serialization, + data.type ? dictionary_type : nullptr, + column_lc ? column_lc->getDictionary().getNestedColumn() : nullptr, + data.serialization_info, + }; - dict_inner_serialization->enumerateStreams(path, callback, data.type, data.column); + dict_inner_serialization->enumerateStreams(path, callback, path.back().data); path.back() = Substream::DictionaryIndexes; - path.back().data = {type, column, getPtr(), nullptr}; + path.back().data = data; callback(path); path.pop_back(); diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index f82b35a52d5..f0f4fa1a4cc 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -20,8 +20,7 @@ public: void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index 2e436070e1c..f767b162604 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -254,13 +254,17 @@ void SerializationMap::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c void SerializationMap::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { - auto next_type = type ? assert_cast(*type).getNestedType() : nullptr; - auto next_column = column ? assert_cast(*column).getNestedColumnPtr() : nullptr; + SubstreamData next_data = + { + nested, + data.type ? assert_cast(*data.type).getNestedType() : nullptr, + data.column ? assert_cast(*data.column).getNestedColumnPtr() : nullptr, + data.serialization_info, + }; - nested->enumerateStreams(path, callback, next_type, next_column); + nested->enumerateStreams(path, callback, next_data); } void SerializationMap::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index b6a003139ec..8df4e4d573b 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -34,8 +34,7 @@ public: void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationNamed.cpp b/src/DataTypes/Serializations/SerializationNamed.cpp index 4ef4d4527f8..097e9cedfbe 100644 --- a/src/DataTypes/Serializations/SerializationNamed.cpp +++ b/src/DataTypes/Serializations/SerializationNamed.cpp @@ -6,12 +6,13 @@ namespace DB void SerializationNamed::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { addToPath(path); - path.back().data = {type, column, getPtr(), std::make_shared(name, escape_delimiter)}; - nested_serialization->enumerateStreams(path, callback, type, column); + path.back().data = data; + path.back().creator = std::make_shared(name, escape_delimiter); + + nested_serialization->enumerateStreams(path, callback, data); path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationNamed.h b/src/DataTypes/Serializations/SerializationNamed.h index 20dd15a20ba..91db0cf67f4 100644 --- a/src/DataTypes/Serializations/SerializationNamed.h +++ b/src/DataTypes/Serializations/SerializationNamed.h @@ -23,8 +23,7 @@ public: void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 560a4812123..de26f573863 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -41,30 +41,35 @@ ColumnPtr SerializationNullable::SubcolumnCreator::create(const ColumnPtr & prev void SerializationNullable::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { - const auto * type_nullable = type ? &assert_cast(*type) : nullptr; - const auto * column_nullable = column ? &assert_cast(*column) : nullptr; + const auto * type_nullable = data.type ? &assert_cast(*data.type) : nullptr; + const auto * column_nullable = data.column ? &assert_cast(*data.column) : nullptr; path.push_back(Substream::NullMap); path.back().data = { + std::make_shared(std::make_shared>(), "null", false), type_nullable ? std::make_shared() : nullptr, column_nullable ? column_nullable->getNullMapColumnPtr() : nullptr, - std::make_shared(std::make_shared>(), "null", false), - nullptr, + data.serialization_info, }; callback(path); path.back() = Substream::NullableElements; - path.back().data = {type, column, getPtr(), std::make_shared(path.back().data.column)}; + path.back().creator = std::make_shared(path.back().data.column); + path.back().data = data; - auto next_type = type_nullable ? type_nullable->getNestedType() : nullptr; - auto next_column = column_nullable ? column_nullable->getNestedColumnPtr() : nullptr; + SubstreamData next_data = + { + nested, + type_nullable ? type_nullable->getNestedType() : nullptr, + column_nullable ? column_nullable->getNestedColumnPtr() : nullptr, + data.serialization_info, + }; - nested->enumerateStreams(path, callback, next_type, next_column); + nested->enumerateStreams(path, callback, next_data); path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index c39c4dd6573..10eb8a56d37 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -16,8 +16,7 @@ public: void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 1a703ea2075..64db248c5fc 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -150,27 +150,36 @@ ColumnPtr SerializationSparse::SubcolumnCreator::create(const ColumnPtr & prev) void SerializationSparse::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { - const auto * column_sparse = column ? &assert_cast(*column) : nullptr; + const auto * column_sparse = data.column ? &assert_cast(*data.column) : nullptr; - SubstreamData data; - data.type = type ? std::make_shared() : nullptr; - data.serialization = std::make_shared>(); - data.column = column_sparse ? column_sparse->getOffsetsPtr() : nullptr; size_t column_size = column_sparse ? column_sparse->size() : 0; path.push_back(Substream::SparseOffsets); - path.back().data = data; + path.back().data = + { + std::make_shared>(), + data.type ? std::make_shared() : nullptr, + column_sparse ? column_sparse->getOffsetsPtr() : nullptr, + data.serialization_info, + }; callback(path); path.back() = Substream::SparseElements; - path.back().data = {type, column, getPtr(), std::make_shared(data.column, column_size)}; + path.back().creator = std::make_shared(path.back().data.column, column_size); + path.back().data = data; - auto next_column = column_sparse ? column_sparse->getValuesPtr() : nullptr; - nested->enumerateStreams(path, callback, type, next_column); + SubstreamData next_data = + { + nested, + data.type, + column_sparse ? column_sparse->getValuesPtr() : nullptr, + data.serialization_info, + }; + + nested->enumerateStreams(path, callback, next_data); path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index 4a5116f7ebd..51d9df2cb5d 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -30,8 +30,7 @@ public: virtual void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 9583bf1fbfa..6b4f32f4212 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -284,18 +285,23 @@ void SerializationTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, void SerializationTuple::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { - const auto * type_tuple = type ? &assert_cast(*type) : nullptr; - const auto * column_tuple = column ? &assert_cast(*column) : nullptr; + const auto * type_tuple = data.type ? &assert_cast(*data.type) : nullptr; + const auto * column_tuple = data.column ? &assert_cast(*data.column) : nullptr; + const auto * info_tuple = data.serialization_info ? &assert_cast(*data.serialization_info) : nullptr; for (size_t i = 0; i < elems.size(); ++i) { - auto next_type = type_tuple ? type_tuple->getElement(i) : nullptr; - auto next_column = column_tuple ? column_tuple->getColumnPtr(i) : nullptr; + SubstreamData next_data = + { + elems[i], + type_tuple ? type_tuple->getElement(i) : nullptr, + column_tuple ? column_tuple->getColumnPtr(i) : nullptr, + info_tuple ? info_tuple->getElementInfo(i) : nullptr, + }; - elems[i]->enumerateStreams(path, callback, next_type, next_column); + elems[i]->enumerateStreams(path, callback, next_data); } } diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index 8261e09b363..10ca8a05501 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -36,8 +36,7 @@ public: void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/DataTypes/Serializations/SerializationWrapper.cpp b/src/DataTypes/Serializations/SerializationWrapper.cpp index c0829ab1b26..271c53dfcf1 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.cpp +++ b/src/DataTypes/Serializations/SerializationWrapper.cpp @@ -7,10 +7,9 @@ namespace DB void SerializationWrapper::enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const + const SubstreamData & data) const { - nested_serialization->enumerateStreams(path, callback, type, column); + nested_serialization->enumerateStreams(path, callback, data); } void SerializationWrapper::serializeBinaryBulkStatePrefix( diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index ba86b24b1cf..4cdcffc21a8 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -23,8 +23,7 @@ public: void enumerateStreams( SubstreamPath & path, const StreamCallback & callback, - DataTypePtr type, - ColumnPtr column) const override; + const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( SerializeBinaryBulkSettings & settings, diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 8903e8730b7..97091b11724 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -149,16 +149,14 @@ Block NativeReader::read() SerializationPtr serialization; if (server_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) { - serialization = column.type->getSerialization(column.name, [&](const String & /*name*/) - { - UInt8 kind_num; - readBinary(kind_num, istr); - auto kind = magic_enum::enum_cast(kind_num); - if (!kind) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown serialization kind " + std::to_string(kind_num)); + auto info = column.type->createSerializationInfo({}); - return *kind; - }); + UInt8 has_custom; + readBinary(has_custom, istr); + if (has_custom) + info->deserializeFromKindsBinary(istr); + + serialization = column.type->getSerialization(*info); } else { diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index 0dc54f295a9..20550a698fe 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -125,18 +126,13 @@ void NativeWriter::write(const Block & block) SerializationPtr serialization; if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) { - serialization = column.type->getSerialization(column.name, [&](const String & name) - { - auto split = Nested::splitName(name); - ISerialization::Kind kind; - if (!split.second.empty() && column.type->tryGetSubcolumnType(split.second)) - kind = ISerialization::getKind(*column.type->getSubcolumn(split.second, column.column)); - else - kind = ISerialization::getKind(*column.column); + auto info = column.column->getSerializationInfo(); + serialization = column.type->getSerialization(*info); - writeBinary(static_cast(kind), ostr); - return kind; - }); + bool has_custom = info->hasCustomSerialization(); + writeBinary(static_cast(has_custom), ostr); + if (has_custom) + info->serialializeKindBinary(ostr); } else { diff --git a/src/Interpreters/InterpreterDescribeQuery.cpp b/src/Interpreters/InterpreterDescribeQuery.cpp index 5370aee1096..72f58b652b4 100644 --- a/src/Interpreters/InterpreterDescribeQuery.cpp +++ b/src/Interpreters/InterpreterDescribeQuery.cpp @@ -150,7 +150,7 @@ BlockIO InterpreterDescribeQuery::execute() res_columns[6]->insertDefault(); res_columns[7]->insert(1u); - }, column.type->getDefaultSerialization(), column.type, nullptr); + }, {column.type->getDefaultSerialization(), column.type, nullptr, nullptr}); } } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 3227a07e5ea..fda25f0992c 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -160,7 +160,7 @@ void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind) Serializations serializations(num_columns); for (size_t i = 0; i < num_columns; ++i) - serializations[i] = header.getByPosition(i).type->getSerialization(*columns[i]); + serializations[i] = header.getByPosition(i).type->getSerialization(*columns[i]->getSerializationInfo()); WidthsPerColumn widths; Widths max_widths; diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index d5700c8e148..2663f4de3c5 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -26,7 +26,7 @@ void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind Serializations serializations(num_columns); for (size_t i = 0; i < num_columns; ++i) - serializations[i] = header.getByPosition(i).type->getSerialization(*columns[i]); + serializations[i] = header.getByPosition(i).type->getSerialization(*columns[i]->getSerializationInfo()); WidthsPerColumn widths; Widths max_widths; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 729c8ca21f7..30278395b25 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -588,8 +588,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( metadata_snapshot->projections.get(projection_name).metadata, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {}), - new_data_part->serialization_info); + CompressionCodecFactory::instance().get("NONE", {})); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_projection_part); @@ -613,8 +612,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( MergedBlockOutputStream part_out( new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {}), - new_data_part->serialization_info); + CompressionCodecFactory::instance().get("NONE", {})); part_out.write(block); part_out.writeSuffixAndFinalizePart(new_data_part); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0e77d8654f3..c1f7adef620 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -281,7 +281,6 @@ IMergeTreeDataPart::IMergeTreeDataPart( , volume(parent_part_ ? parent_part_->volume : volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) - , serialization_info(std::make_shared()) , part_type(part_type_) , parent_part(parent_part_) { @@ -307,7 +306,6 @@ IMergeTreeDataPart::IMergeTreeDataPart( , volume(parent_part_ ? parent_part_->volume : volume_) , relative_path(relative_path_.value_or(name_)) , index_granularity_info(storage_, part_type_) - , serialization_info(std::make_shared()) , part_type(part_type_) , parent_part(parent_part_) { @@ -408,17 +406,47 @@ std::pair IMergeTreeDataPart::getMinMaxTime() const } -void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) +void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos) { columns = new_columns; + column_name_to_position.clear(); column_name_to_position.reserve(new_columns.size()); size_t pos = 0; + for (const auto & column : columns) { + auto & serialization = serializations[column.name]; 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); + + auto it = new_infos.find(column.name); + if (it != new_infos.end()) + { + auto & old_info = serialization_infos[column.name]; + const auto & new_info = it->second; + + if (old_info) + { + old_info->replaceData(*new_info); + } + else + { + old_info = new_info->clone(); + serialization = column.type->getSerialization(*old_info); + } + } + else + { + serialization = column.type->getDefaultSerialization(); + } + + IDataType::forEachSubcolumn([&](const auto &, const auto & subname, const auto & subdata) + { + auto subcolumn_name = Nested::concatenateName(column.name, subname); + column_name_to_position.emplace(subcolumn_name, pos); + serializations.emplace(subcolumn_name, subdata.serialization); + }, {serialization, column.type, nullptr, nullptr}); + ++pos; } } @@ -589,7 +617,6 @@ void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checks loadUUID(); loadColumns(require_columns_checksums); loadChecksums(require_columns_checksums); - loadSerializationInfo(); loadIndexGranularity(); calculateColumnsAndSecondaryIndicesSizesOnDisk(); loadIndex(); /// Must be called after loadIndexGranularity as it uses the value of `index_granularity` @@ -655,13 +682,13 @@ void IMergeTreeDataPart::loadIndex() size_t marks_count = index_granularity.getMarksCount(); - Serializations serializations(key_size); + Serializations key_serializations(key_size); for (size_t j = 0; j < key_size; ++j) - serializations[j] = primary_key.data_types[j]->getDefaultSerialization(); + key_serializations[j] = primary_key.data_types[j]->getDefaultSerialization(); for (size_t i = 0; i < marks_count; ++i) //-V756 for (size_t j = 0; j < key_size; ++j) - serializations[j]->deserializeBinary(*loaded_index[j], *index_file); + key_serializations[j]->deserializeBinary(*loaded_index[j], *index_file); for (size_t i = 0; i < key_size; ++i) { @@ -752,9 +779,8 @@ 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)) { - auto serialization = IDataType::getSerialization(part_column, *serialization_info); String path_to_data_file; - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + serializations.at(part_column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (path_to_data_file.empty()) { @@ -888,7 +914,7 @@ void IMergeTreeDataPart::loadRowsCount() /// Most trivial types if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && getSerializationForColumn(column)->getKind() == ISerialization::Kind::DEFAULT) + && getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) { auto size = getColumnSize(column.name, *column.type); @@ -940,7 +966,7 @@ void IMergeTreeDataPart::loadRowsCount() for (const NameAndTypePair & column : columns) { - ColumnPtr column_col = column.type->createColumn(*getSerializationForColumn(column)); + ColumnPtr column_col = column.type->createColumn(*serializations.at(column.name)); if (!column_col->isFixedAndContiguous() || column_col->lowCardinality()) continue; @@ -1014,16 +1040,6 @@ void IMergeTreeDataPart::loadUUID() } } -void IMergeTreeDataPart::loadSerializationInfo() const -{ - String path = getFullRelativePath() + SERIALIZATION_FILE_NAME; - if (volume->getDisk()->exists(path)) - { - auto in = openForReading(volume->getDisk(), path); - serialization_info->readText(*in); - } -} - void IMergeTreeDataPart::loadColumns(bool require) { String path = fs::path(getFullRelativePath()) / "columns.txt"; @@ -1058,7 +1074,18 @@ void IMergeTreeDataPart::loadColumns(bool require) loaded_columns.readText(*volume->getDisk()->readFile(path)); } - setColumns(loaded_columns); + SerializationInfo::Settings settings = + { + .ratio_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .choose_kind = false, + }; + + SerializationInfoByName infos(loaded_columns, settings); + path = getFullRelativePath() + SERIALIZATION_FILE_NAME; + if (volume->getDisk()->exists(path)) + infos.readText(*volume->getDisk()->readFile(path)); + + setColumns(loaded_columns, infos); } bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const @@ -1563,11 +1590,6 @@ bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metada return true; } -SerializationPtr IMergeTreeDataPart::getSerializationForColumn(const NameAndTypePair & column) const -{ - return IDataType::getSerialization(column, *serialization_info); -} - String IMergeTreeDataPart::getUniqueId() const { auto disk = volume->getDisk(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 4500a964b42..1695988ea32 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -93,7 +93,6 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity = {}) const = 0; @@ -127,9 +126,12 @@ public: String getTypeName() const { return getType().toString(); } - void setColumns(const NamesAndTypesList & new_columns); + void setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos = {}); const NamesAndTypesList & getColumns() const { return columns; } + const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } + const SerializationByName & getSerializations() const { return serializations; } + const SerializationPtr & getSerialization(const String & column_name) const { return serializations.at(column_name); } /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; @@ -190,9 +192,6 @@ public: mutable String relative_path; MergeTreeIndexGranularityInfo index_granularity_info; - /// TODO: add comment - SerializationInfoPtr serialization_info; - size_t rows_count = 0; time_t modification_time = 0; @@ -399,8 +398,8 @@ public: /// part creation (using alter query with materialize_ttl setting). bool checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const; - /// Returns serialization for column according to serialization_info. - SerializationPtr getSerializationForColumn(const NameAndTypePair & column) const; + // /// Returns serialization for column according to serialization_info. + // SerializationPtr getSerializationForColumn(const NameAndTypePair & column) const; /// Return some uniq string for file /// Required for distinguish different copies of the same part on S3 @@ -424,6 +423,11 @@ protected: /// Columns description. Cannot be changed, after part initialization. NamesAndTypesList columns; + + SerializationInfoByName serialization_infos; + + SerializationByName serializations; + const Type part_type; /// Not null when it's a projection part. @@ -470,8 +474,6 @@ private: /// Loads ttl infos in json format from file ttl.txt. If file doesn't exists assigns ttl infos with all zeros void loadTTLInfos(); - void loadSerializationInfo() const; - void loadPartitionAndMinMaxIndex(); void calculateColumnsSizesOnDisk(); diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 75eb01ed73c..c52d14ce824 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -40,6 +40,7 @@ IMergeTreeReader::IMergeTreeReader( , storage(data_part_->storage) , metadata_snapshot(metadata_snapshot_) , all_mark_ranges(all_mark_ranges_) + , serializations(data_part_->getSerializations()) , alter_conversions(storage.getAlterConversionsForPart(data_part)) { if (isWidePart(data_part)) diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index ddf9218d534..74e34e24eb0 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -87,8 +87,7 @@ protected: using ColumnPosition = std::optional; ColumnPosition findColumnForOffsets(const String & column_name) const; - using Serializations = std::map; - Serializations serializations; + const SerializationByName & serializations; friend class MergeTreeRangeReader::DelayedStream; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 0f5da77fb9b..25a21afbfa4 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -8,14 +8,16 @@ namespace DB IMergedBlockOutputStream::IMergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, const StorageMetadataPtr & metadata_snapshot_, - const SerializationInfoPtr & input_serialization_info_) + const NamesAndTypesList & columns_list, + bool reset_columns_) : storage(data_part->storage) , metadata_snapshot(metadata_snapshot_) , volume(data_part->volume) , part_path(data_part->isStoredOnDisk() ? data_part->getFullRelativePath() : "") - , input_serialization_info(input_serialization_info_) - , new_serialization_info(data_part->storage.getSettings()->ratio_of_defaults_for_sparse_serialization) + , reset_columns(reset_columns_) { + if (reset_columns) + new_serialization_infos = SerializationInfoByName(columns_list, {}); } NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( @@ -32,18 +34,14 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. std::map stream_counts; - std::unordered_map serialziations; - for (const NameAndTypePair & column : columns) + const auto & serializations = data_part->getSerializations(); + for (const auto & column : columns) { - auto serialization = IDataType::getSerialization(column, *data_part->serialization_info); - - serialization->enumerateStreams( + serializations.at(column.name)->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; }); - - serialziations[column.name] = std::move(serialization); } NameSet remove_files; @@ -65,7 +63,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( } }; - serialziations[column_name]->enumerateStreams(callback); + serializations.at(column_name)->enumerateStreams(callback); } /// Remove files on disk and checksums diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index c124b588e73..3a65de511fa 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -14,7 +14,8 @@ public: IMergedBlockOutputStream( const MergeTreeDataPartPtr & data_part, const StorageMetadataPtr & metadata_snapshot_, - const SerializationInfoPtr & input_serialization_info_); + const NamesAndTypesList & columns_list, + bool reset_columns_); virtual ~IMergedBlockOutputStream() = default; @@ -46,8 +47,9 @@ protected: String part_path; IMergeTreeDataPart::MergeTreeWriterPtr writer; - SerializationInfoPtr input_serialization_info; - SerializationInfoBuilder new_serialization_info; + + bool reset_columns = false; + SerializationInfoByName new_serialization_infos; }; using IMergedBlockOutputStreamPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index c2f36865d00..6066d3e8dfa 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -158,15 +158,19 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->parent_part); global_ctx->new_data_part->uuid = global_ctx->future_part->uuid; - global_ctx->new_data_part->setColumns(global_ctx->storage_columns); global_ctx->new_data_part->partition.assign(global_ctx->future_part->getPartition()); global_ctx->new_data_part->is_temp = global_ctx->parent_part == nullptr; ctx->need_remove_expired_values = false; ctx->force_ttl = false; - SerializationInfoBuilder serialization_info_builder( - global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization); + SerializationInfo::Settings info_settings = + { + .ratio_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, + .choose_kind = true, + }; + + SerializationInfoByName infos(global_ctx->storage_columns, info_settings); for (const auto & part : global_ctx->future_part->parts) { @@ -178,10 +182,10 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() ctx->force_ttl = true; } - serialization_info_builder.add(*part->serialization_info); + infos.add(part->getSerializationInfos()); } - global_ctx->input_serialization_info = std::move(serialization_info_builder).build(); + global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos); const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; if (local_part_min_ttl && local_part_min_ttl <= global_ctx->time_of_merge) @@ -256,7 +260,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->merging_columns, MergeTreeIndexFactory::instance().getMany(global_ctx->metadata_snapshot->getSecondaryIndices()), ctx->compression_codec, - global_ctx->input_serialization_info, + /*reset_columns=*/ true, ctx->blocks_are_granules_size); global_ctx->rows_written = 0; @@ -435,7 +439,6 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const global_ctx->metadata_snapshot, ctx->executor->getHeader(), ctx->compression_codec, - global_ctx->input_serialization_info, /// we don't need to recalc indices here /// because all of them were already recalculated and written /// as key part of vertical merge diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index dcd4fe0e543..090f2c28508 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -157,7 +157,6 @@ private: SyncGuardPtr sync_guard{nullptr}; MergeTreeData::MutableDataPartPtr new_data_part{nullptr}; - SerializationInfoPtr input_serialization_info{nullptr}; size_t rows_written{0}; UInt64 watch_prev_elapsed{0}; diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index 1990d622868..ad1c2abeee7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -59,7 +59,6 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const { @@ -74,7 +73,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter( return std::make_unique( shared_from_this(), ordered_columns_list, metadata_snapshot, indices_to_recalc, index_granularity_info.marks_file_extension, - default_codec_, serialization_info_, writer_settings, computed_index_granularity); + default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 8f2f27e0a91..2fcc7b7034b 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -51,7 +51,6 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 6c28749d171..4ec53d88339 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -60,7 +60,6 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartInMemory::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & /* indices_to_recalc */, const CompressionCodecPtr & /* default_codec */, - const SerializationInfoPtr & /* serialization_info */, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & /* computed_index_granularity */) const { @@ -92,7 +91,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri auto compression_codec = storage.getContext()->chooseCompressionCodec(0, 0); auto indices = MergeTreeIndexFactory::instance().getMany(metadata_snapshot->getSecondaryIndices()); - MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec, new_data_part->serialization_info); + MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, indices, compression_codec); out.write(block); const auto & projections = metadata_snapshot->getProjections(); for (const auto & [projection_name, projection] : projection_parts) @@ -123,7 +122,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri auto projection_indices = MergeTreeIndexFactory::instance().getMany(desc.metadata->getSecondaryIndices()); MergedBlockOutputStream projection_out( projection_data_part, desc.metadata, projection_part->columns, projection_indices, - projection_compression_codec, new_data_part->serialization_info); + projection_compression_codec); projection_out.write(projection_part->block); projection_out.writeSuffixAndFinalizePart(projection_data_part); diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index 3a767a5f4dc..118340f0233 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -40,7 +40,6 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 9a146107587..33ed08ff1d3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -61,14 +61,13 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const { return std::make_unique( shared_from_this(), columns_list, metadata_snapshot, indices_to_recalc, index_granularity_info.marks_file_extension, - default_codec_, serialization_info_, writer_settings, computed_index_granularity); + default_codec_, writer_settings, computed_index_granularity); } @@ -81,8 +80,7 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( if (checksums.empty()) return size; - auto serialization = getSerializationForColumn(column); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + serializations.at(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); @@ -159,8 +157,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const { for (const NameAndTypePair & name_type : columns) { - auto serialization = getSerializationForColumn(name_type); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + serializations.at(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(name_type, substream_path); String mrk_file_name = file_name + index_granularity_info.marks_file_extension; @@ -174,7 +171,6 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const }); } } - } else { @@ -182,8 +178,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const std::optional marks_size; for (const NameAndTypePair & name_type : columns) { - auto serialization = IDataType::getSerialization(name_type, *serialization_info); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + serializations.at(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { auto file_path = path + ISerialization::getFileNameForStream(name_type, substream_path) + index_granularity_info.marks_file_extension; @@ -218,7 +213,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const }; bool res = true; - auto serialization = IDataType::getSerialization(column, *serialization_info); + auto serialization = getSerialization(column.name); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); @@ -232,8 +227,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const { String filename; - auto serialization = getSerializationForColumn(column); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + serializations.at(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (filename.empty()) filename = ISerialization::getFileNameForStream(column, substream_path); @@ -255,7 +249,7 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col if (rows_count != 0 && column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && getSerializationForColumn(column)->getKind() == ISerialization::Kind::DEFAULT) + && serializations.at(column.name)->getKind() == ISerialization::Kind::DEFAULT) { size_t rows_in_column = size.data_uncompressed / column.type->getSizeOfValueInMemory(); if (rows_in_column != rows_count) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index a3a170ddd80..a43396f8cd5 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -45,7 +45,6 @@ public: const StorageMetadataPtr & metadata_snapshot, const std::vector & indices_to_recalc, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & writer_settings, const MergeTreeIndexGranularity & computed_index_granularity) const override; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 0f9b7a40da5..3d40b7cfda7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -16,12 +16,11 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, - default_codec_, serialization_info_, settings_, index_granularity_) + default_codec_, settings_, index_granularity_) , plain_file(data_part->volume->getDisk()->writeFile( part_path + MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, settings.max_compress_block_size, @@ -34,12 +33,8 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( , marks(*marks_file) { const auto & storage_columns = metadata_snapshot->getColumns(); - serializations.reserve(columns_list.size()); for (const auto & column : columns_list) - { - serializations.emplace(column.name, column.type->getSerialization(column.name, *serialization_info)); addStreams(column, storage_columns.getCodecDescOrDefault(column.name, default_codec)); - } } void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, const ASTPtr & effective_codec_desc) @@ -71,7 +66,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, }; ISerialization::SubstreamPath path; - serializations[column.name]->enumerateStreams(path, callback, column.type, nullptr); + serializations.at(column.name)->enumerateStreams(path, callback, column.type); } namespace @@ -212,7 +207,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeIntBinary(UInt64(0), marks); writeColumnSingleGranule( - block.getByName(name_and_type->name), serializations[name_and_type->name], + block.getByName(name_and_type->name), serializations.at(name_and_type->name), stream_getter, granule.start_row, granule.rows_to_write); /// Each type always have at least one substream diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index 0acf5d243ad..8b86a9701c9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -15,7 +15,6 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index b9237822b2d..cd8d2590095 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -68,7 +68,6 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( const MergeTreeIndices & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : IMergeTreeDataPartWriter(data_part_, @@ -76,8 +75,8 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( , skip_indices(indices_to_recalc_) , part_path(data_part_->getFullRelativePath()) , marks_file_extension(marks_file_extension_) + , serializations(data_part_->getSerializations()) , default_codec(default_codec_) - , serialization_info(serialization_info_) , compute_granularity(index_granularity.empty()) { if (settings.blocks_are_granules_size && !index_granularity.empty()) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index eaada9f24d6..fa0c08b81aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -87,7 +87,6 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); @@ -124,8 +123,8 @@ protected: const String part_path; const String marks_file_extension; + const SerializationByName & serializations; const CompressionCodecPtr default_codec; - const SerializationInfoPtr serialization_info; const bool compute_granularity; @@ -133,9 +132,6 @@ protected: MergeTreeIndexAggregators skip_indices_aggregators; std::vector skip_index_accumulated_marks; - using SerializationsMap = std::unordered_map; - SerializationsMap serializations; - std::unique_ptr index_file_stream; std::unique_ptr index_stream; DataTypes index_types; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 1648a6bc7de..78afcc5972e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -75,22 +75,17 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( const std::vector & indices_to_recalc_, const String & marks_file_extension_, const CompressionCodecPtr & default_codec_, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings_, const MergeTreeIndexGranularity & index_granularity_) : MergeTreeDataPartWriterOnDisk(data_part_, columns_list_, metadata_snapshot_, indices_to_recalc_, marks_file_extension_, - default_codec_, serialization_info_, settings_, index_granularity_) + default_codec_, settings_, index_granularity_) { const auto & columns = metadata_snapshot->getColumns(); for (const auto & it : columns_list) - { - serializations.emplace(it.name, it.type->getSerialization(it.name, *serialization_info)); addStreams(it, columns.getCodecDescOrDefault(it.name, default_codec)); - } } - void MergeTreeDataPartWriterWide::addStreams( const NameAndTypePair & column, const ASTPtr & effective_codec_desc) @@ -123,7 +118,7 @@ void MergeTreeDataPartWriterWide::addStreams( }; ISerialization::SubstreamPath path; - serializations[column.name]->enumerateStreams(path, callback, column.type, nullptr); + serializations.at(column.name)->enumerateStreams(path, callback, column.type); } @@ -218,7 +213,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm { auto & column = block_to_write.getByName(it->name); - if (serializations[column.name]->getKind() != ISerialization::Kind::SPARSE) + if (serializations.at(column.name)->getKind() != ISerialization::Kind::SPARSE) column.column = recursiveRemoveSparse(column.column); if (permutation) @@ -280,7 +275,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( ISerialization::SubstreamPath & path) { StreamsWithMarks result; - serializations[column.name]->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + serializations.at(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; @@ -315,7 +310,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( ISerialization::SerializeBinaryBulkSettings & serialize_settings, const Granule & granule) { - const auto & serialization = serializations[name_and_type.name]; + const auto & serialization = serializations.at(name_and_type.name); serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, 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. @@ -350,7 +345,7 @@ void MergeTreeDataPartWriterWide::writeColumn( { ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); - serializations[name]->serializeBinaryBulkStatePrefix(serialize_settings, it->second); + serializations.at(name)->serializeBinaryBulkStatePrefix(serialize_settings, it->second); } const auto & global_settings = storage.getContext()->getSettingsRef(); @@ -391,7 +386,7 @@ void MergeTreeDataPartWriterWide::writeColumn( } } - serializations[name]->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + serializations.at(name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) @@ -405,7 +400,7 @@ void MergeTreeDataPartWriterWide::writeColumn( void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, const IDataType & type) { - const auto & serialization = serializations[name]; + const auto & serialization = serializations.at(name); if (!type.isValueRepresentedByNumber() || type.haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName()); @@ -543,7 +538,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch if (!serialization_states.empty()) { serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns); - serializations[it->name]->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); + serializations.at(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); } if (write_final_mark) @@ -568,7 +563,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch { if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && serializations[column.name]->getKind() == ISerialization::Kind::DEFAULT) + && serializations.at(column.name)->getKind() == ISerialization::Kind::DEFAULT) { validateColumnOfFixedSize(column.name, *column.type); } @@ -596,7 +591,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark( { writeSingleMark(column, offset_columns, 0, path); /// Memoize information about offsets - serializations[column.name]->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + serializations.at(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 7190c4c2655..5eaaa0c1bbe 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -24,7 +24,6 @@ public: const std::vector & indices_to_recalc, const String & marks_file_extension, const CompressionCodecPtr & default_codec, - const SerializationInfoPtr & serialization_info_, const MergeTreeWriterSettings & settings, const MergeTreeIndexGranularity & index_granularity); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 95d8b775537..858287dbc41 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -359,7 +359,13 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart( if (data.storage_settings.get()->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); - new_data_part->setColumns(columns); + const auto & data_settings = data.getSettings(); + + SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; + SerializationInfoByName infos(columns, settings); + infos.add(block); + + new_data_part->setColumns(columns, infos); new_data_part->rows_count = block.rows(); new_data_part->partition = std::move(partition); new_data_part->minmax_idx = std::move(minmax_idx); @@ -406,15 +412,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); - const auto & data_settings = data.getSettings(); - - SerializationInfoBuilder serialization_info(data_settings->ratio_of_defaults_for_sparse_serialization); - serialization_info.add(block); - const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot,columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), - compression_codec, std::move(serialization_info).build()); + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); bool sync_on_insert = data_settings->fsync_after_insert; @@ -458,7 +458,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl( new_data_part->is_temp = is_temp; NamesAndTypesList columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); - new_data_part->setColumns(columns); + SerializationInfo::Settings settings{data.getSettings()->ratio_of_defaults_for_sparse_serialization, true}; + SerializationInfoByName infos(columns, settings); + infos.add(block); + + new_data_part->setColumns(columns, infos); if (new_data_part->isStoredOnDisk()) { @@ -506,16 +510,12 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeProjectionPartImpl( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); - SerializationInfoBuilder serialization_info(data.getSettings()->ratio_of_defaults_for_sparse_serialization); - serialization_info.add(block); - MergedBlockOutputStream out( new_data_part, metadata_snapshot, columns, {}, - compression_codec, - std::move(serialization_info).build()); + compression_codec); out.writeWithPermutation(block, perm_ptr); out.writeSuffixAndFinalizePart(new_data_part); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index c25462aeef4..f1dfecdbb0d 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -64,16 +64,6 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( } column_positions[i] = std::move(position); - - if (column_from_part.isSubcolumn()) - { - auto name_in_storage = column_from_part.getNameInStorage(); - /// We have to read whole column and extract subcolumn. - serializations.emplace(name_in_storage, data_part->getSerializationForColumn( - {name_in_storage, column_from_part.getTypeInStorage()})); - } - - serializations.emplace(column_from_part.name, data_part->getSerializationForColumn(column_from_part)); } /// Do not use max_read_buffer_size, but try to lower buffer size with maximal size of granule to avoid reading much data. diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 24a1a8a0285..cb1db9ebd22 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -109,7 +109,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si /// 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(*serializations[name]); + res_columns[pos] = type->createColumn(*serializations.at(name)); auto & column = res_columns[pos]; try @@ -188,9 +188,7 @@ void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type, profile_callback, clock_type)); }; - auto serialization = data_part->getSerializationForColumn(name_and_type); - serialization->enumerateStreams(callback); - serializations.emplace(name_and_type.name, std::move(serialization)); + serializations.at(name_and_type.name)->enumerateStreams(callback); } @@ -231,7 +229,7 @@ void MergeTreeReaderWide::prefetch( std::unordered_set & prefetched_streams) { const auto & name = name_and_type.name; - auto & serialization = serializations[name]; + auto & serialization = serializations.at(name); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { @@ -259,7 +257,7 @@ void MergeTreeReaderWide::readData( deserialize_settings.avg_value_size_hint = avg_value_size_hint; const auto & name = name_and_type.name; - auto & serialization = serializations[name]; + auto & serialization = serializations.at(name); if (deserialize_binary_bulk_state_map.count(name) == 0) { diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 382d5789fe1..28f25296a7b 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -195,8 +195,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor metadata_snapshot, block.getNamesAndTypesList(), {}, - CompressionCodecFactory::instance().get("NONE", {}), - part->serialization_info); + CompressionCodecFactory::instance().get("NONE", {})); part->minmax_idx->update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0, context); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 546ef945fad..2f09ba18b76 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -18,9 +18,9 @@ MergedBlockOutputStream::MergedBlockOutputStream( const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, - const SerializationInfoPtr & input_serialization_info_, + bool reset_columns_, bool blocks_are_granules_size) - : IMergedBlockOutputStream(data_part, metadata_snapshot_, input_serialization_info_) + : IMergedBlockOutputStream(data_part, metadata_snapshot_, columns_list_, reset_columns_) , columns_list(columns_list_) , default_codec(default_codec_) { @@ -34,7 +34,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( if (!part_path.empty()) volume->getDisk()->createDirectories(part_path); - writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, input_serialization_info, writer_settings); + writer = data_part->getWriter(columns_list, metadata_snapshot, skip_indices, default_codec, writer_settings); } /// If data is pre-sorted. @@ -78,12 +78,12 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( else part_columns = *total_columns_list; - new_part->serialization_info = std::move(new_serialization_info).buildFrom(*input_serialization_info); + if (reset_columns) + new_part->setColumns(part_columns, new_serialization_infos); if (new_part->isStoredOnDisk()) finalizePartOnDisk(new_part, part_columns, checksums, sync); - new_part->setColumns(part_columns); new_part->rows_count = rows_count; new_part->modification_time = time(nullptr); new_part->index = writer->releaseIndexColumns(); @@ -168,11 +168,12 @@ void MergedBlockOutputStream::finalizePartOnDisk( removeEmptyColumnsFromPart(new_part, part_columns, checksums); - if (new_part->serialization_info->getNumberOfRows() > 0) + const auto & serialization_infos = new_part->getSerializationInfos(); + if (!serialization_infos.empty()) { auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); HashingWriteBuffer out_hashing(*out); - new_part->serialization_info->writeText(out_hashing); + serialization_infos.writeText(out_hashing); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash(); out->finalize(); @@ -219,7 +220,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm return; writer->write(block, permutation); - new_serialization_info.add(block); + if (reset_columns) + new_serialization_infos.add(block); rows_count += rows; } diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 2a7c2570450..64f74a0c645 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -19,7 +19,7 @@ public: const NamesAndTypesList & columns_list_, const MergeTreeIndices & skip_indices, CompressionCodecPtr default_codec_, - const SerializationInfoPtr & input_serialization_info_, + bool reset_columns_ = false, bool blocks_are_granules_size = false); Block getHeader() const { return metadata_snapshot->getSampleBlock(); } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 6cbdc363380..71ea9458dde 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -14,12 +14,11 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec, - const SerializationInfoPtr & input_serialization_info_, const MergeTreeIndices & indices_to_recalc, WrittenOffsetColumns * offset_columns_, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) - : IMergedBlockOutputStream(data_part, metadata_snapshot_, input_serialization_info_) + : IMergedBlockOutputStream(data_part, metadata_snapshot_, header_.getNamesAndTypesList(), /*reset_columns=*/ true) , header(header_) { const auto & global_settings = data_part->storage.getContext()->getSettings(); @@ -36,7 +35,6 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( metadata_snapshot_, indices_to_recalc, default_codec, - input_serialization_info, std::move(writer_settings), index_granularity); @@ -53,7 +51,7 @@ void MergedColumnOnlyOutputStream::write(const Block & block) return; writer->write(block, nullptr); - new_serialization_info.add(block); + new_serialization_infos.add(block); } MergeTreeData::DataPart::Checksums @@ -79,8 +77,7 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( if (all_checksums.files.count(removed_file)) all_checksums.files.erase(removed_file); - new_part->setColumns(columns); - new_part->serialization_info = std::move(new_serialization_info).buildFrom(*input_serialization_info); + new_part->setColumns(columns, new_serialization_infos); return checksums; } diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 67e98e97698..4b75bc52f72 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -18,7 +18,6 @@ public: const StorageMetadataPtr & metadata_snapshot_, const Block & header_, CompressionCodecPtr default_codec_, - const SerializationInfoPtr & input_serialization_info_, const MergeTreeIndices & indices_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, const MergeTreeIndexGranularity & index_granularity = {}, diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 156d4ffcce4..598b946dada 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -313,8 +313,7 @@ NameSet collectFilesToSkip( files_to_skip.insert(stream_name + mrk_extension); }; - auto serialization = source_part->getSerializationForColumn({entry.name, entry.type}); - serialization->enumerateStreams(callback); + source_part->getSerialization(entry.name)->enumerateStreams(callback); } for (const auto & index : indices_to_recalc) { @@ -339,8 +338,7 @@ static NameToNameVector collectFilesForRenames( std::map stream_counts; for (const auto & column : source_part->getColumns()) { - auto serialization = source_part->getSerializationForColumn(column); - serialization->enumerateStreams( + source_part->getSerialization(column.name)->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; @@ -384,10 +382,7 @@ static NameToNameVector collectFilesForRenames( auto column = source_part->getColumns().tryGetByName(command.column_name); if (column) - { - auto serialization = source_part->getSerializationForColumn(*column); - serialization->enumerateStreams(callback); - } + source_part->getSerialization(column->name)->enumerateStreams(callback); } else if (command.type == MutationCommand::Type::RENAME_COLUMN) { @@ -409,10 +404,7 @@ static NameToNameVector collectFilesForRenames( auto column = source_part->getColumns().tryGetByName(command.column_name); if (column) - { - auto serialization = source_part->getSerializationForColumn(*column); - serialization->enumerateStreams(callback); - } + source_part->getSerialization(column->name)->enumerateStreams(callback); } } @@ -968,8 +960,7 @@ private: ctx->metadata_snapshot, ctx->new_data_part->getColumns(), skip_part_indices, - ctx->compression_codec, - ctx->new_data_part->serialization_info); + ctx->compression_codec); ctx->mutating_pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); ctx->mutating_executor = std::make_unique(ctx->mutating_pipeline); @@ -1118,7 +1109,6 @@ private: ctx->metadata_snapshot, ctx->updated_header, ctx->compression_codec, - ctx->source_part->serialization_info, std::vector(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), nullptr, ctx->source_part->index_granularity, @@ -1298,11 +1288,13 @@ bool MutateTask::prepare() ctx->new_data_part->uuid = ctx->future_part->uuid; ctx->new_data_part->is_temp = true; ctx->new_data_part->ttl_infos = ctx->source_part->ttl_infos; - ctx->new_data_part->serialization_info = ctx->source_part->serialization_info; /// It shouldn't be changed by mutation. ctx->new_data_part->index_granularity_info = ctx->source_part->index_granularity_info; - ctx->new_data_part->setColumns(MergeTreeDataMergerMutator::getColumnsForNewDataPart(ctx->source_part, ctx->updated_header, ctx->storage_columns, ctx->for_file_renames)); + ctx->new_data_part->setColumns( + MergeTreeDataMergerMutator::getColumnsForNewDataPart(ctx->source_part, ctx->updated_header, ctx->storage_columns, ctx->for_file_renames), + ctx->source_part->getSerializationInfos()); + ctx->new_data_part->partition.assign(ctx->source_part->partition); ctx->disk = ctx->new_data_part->volume->getDisk(); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 2e57ebab8c4..2733edfde77 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -98,13 +98,13 @@ IMergeTreeDataPart::Checksums checkDataPart( }; }; - auto serialization_info = std::make_shared(); + SerializationInfoByName serialization_infos(columns_txt, {}); auto serialization_path = path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME; if (disk->exists(serialization_path)) { auto serialization_file = disk->readFile(serialization_path); - serialization_info->readText(*serialization_file); + serialization_infos.readText(*serialization_file); } /// This function calculates only checksum of file content (compressed or uncompressed). @@ -141,7 +141,7 @@ IMergeTreeDataPart::Checksums checkDataPart( const NamesAndTypesList & projection_columns_list = projection->getColumns(); for (const auto & projection_column : projection_columns_list) { - auto serialization = IDataType::getSerialization(projection_column, *serialization_info); + auto serialization = projection_column.type->getSerialization(*serialization_infos.at(projection_column.name)); serialization->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { @@ -214,7 +214,7 @@ IMergeTreeDataPart::Checksums checkDataPart( { for (const auto & column : columns_list) { - auto serialization = IDataType::getSerialization(column, *serialization_info); + auto serialization = column.type->getSerialization(*serialization_infos.at(column .name)); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path) + ".bin"; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 0bb76b10b5d..e0ee86dad03 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -166,7 +166,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = IDataType::getSerialization(name_and_type, {}); + auto serialization = type->getDefaultSerialization(); auto create_stream_getter = [&](bool stream_for_prefix) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5bb8de1f925..8ae4f74bcb5 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7109,10 +7109,6 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP new_data_part->minmax_idx = std::move(minmax_idx); new_data_part->is_temp = true; - /// Create empty serialization_info. - auto ratio = getSettings()->ratio_of_defaults_for_sparse_serialization; - new_data_part->serialization_info = SerializationInfoBuilder(ratio).build(); - SyncGuardPtr sync_guard; if (new_data_part->isStoredOnDisk()) { @@ -7138,8 +7134,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP const auto & index_factory = MergeTreeIndexFactory::instance(); MergedBlockOutputStream out(new_data_part, metadata_snapshot, columns, - index_factory.getMany(metadata_snapshot->getSecondaryIndices()), - compression_codec, new_data_part->serialization_info); + index_factory.getMany(metadata_snapshot->getSecondaryIndices()), compression_codec); bool sync_on_insert = settings->fsync_after_insert; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index d5c13582f3f..485c6295e07 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -168,7 +168,7 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = IDataType::getSerialization(name_and_type, {}); + auto serialization = name_and_type.type->getDefaultSerialization(); settings.getter = [&] (const ISerialization::SubstreamPath & path) -> ReadBuffer * { diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 8401ab321d1..0a99c626d48 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -222,7 +222,7 @@ void StorageSystemPartsColumns::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.marks); - auto serialization = part->getSerializationForColumn(column); + auto serialization = part->getSerialization(column.name); if (columns_mask[src_index++]) columns[res_index++]->insert(ISerialization::kindToString(serialization->getKind())); @@ -235,7 +235,7 @@ void StorageSystemPartsColumns::processNextStorage( subcolumn_names.push_back(name); subcolumn_types.push_back(data.type->getName()); subcolumn_sers.push_back(ISerialization::kindToString(data.serialization->getKind())); - }, serialization, column.type, nullptr); + }, { serialization, column.type, nullptr, nullptr }); if (columns_mask[src_index++]) columns[res_index++]->insert(subcolumn_names); diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.reference b/tests/queries/0_stateless/01780_column_sparse_tuple.reference index 394fc148ef6..22337838cff 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.reference +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.reference @@ -57,3 +57,10 @@ a aaaaaa a aaaaaa +id [] [] [] +t ['a','b','b.u','b.s'] ['UInt64','Tuple(u UInt32, s String)','UInt32','String'] ['Sparse','Default','Sparse','Default'] +aaaaaa +a +aaaaaa +a +aaaaaa diff --git a/tests/queries/0_stateless/01780_column_sparse_tuple.sql b/tests/queries/0_stateless/01780_column_sparse_tuple.sql index 847e95d1fd8..da679f2c7eb 100644 --- a/tests/queries/0_stateless/01780_column_sparse_tuple.sql +++ b/tests/queries/0_stateless/01780_column_sparse_tuple.sql @@ -40,4 +40,14 @@ SELECT t.a FROM sparse_tuple WHERE t.b.u != 0 ORDER BY id LIMIT 5; SELECT t.b.s FROM sparse_tuple ORDER BY id LIMIT 5; SELECT t.b.s FROM sparse_tuple WHERE t.b.u != 0 ORDER BY id LIMIT 5; +DETACH TABLE sparse_tuple; +ATTACH TABLE sparse_tuple; + +SELECT column, subcolumns.names, subcolumns.types, subcolumns.serializations +FROM system.parts_columns +WHERE table = 'sparse_tuple' AND database = currentDatabase() +ORDER BY column; + +SELECT t.b.s FROM sparse_tuple WHERE t.b.u != 0 ORDER BY id LIMIT 5; + DROP TABLE IF EXISTS sparse_tuple; From 0a010e7d3813f8bbe0757deae62c5ae222bb80f2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 30 Oct 2021 01:09:50 +0300 Subject: [PATCH 060/358] fix serialization info --- src/DataTypes/Serializations/ISerialization.cpp | 1 + src/DataTypes/Serializations/SerializationInfo.cpp | 13 ++++++------- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index b09e606080f..13b57a3ca04 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -34,6 +34,7 @@ String ISerialization::kindToString(Kind kind) case Kind::SPARSE: return "Sparse"; } + __builtin_unreachable(); } ISerialization::Kind ISerialization::stringToKind(const String & str) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index ab94127dc0c..da258b0448c 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -92,7 +92,7 @@ void SerializationInfo::deserializeFromKindsBinary(ReadBuffer & in) readBinary(kind_num, in); auto maybe_kind = magic_enum::enum_cast(kind_num); if (!maybe_kind) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown serialization kind " + std::to_string(kind_num)); + throw Exception(ErrorCodes::CORRUPTED_DATA, "Unknown serialization kind " + std::to_string(kind_num)); kind = *maybe_kind; } @@ -109,7 +109,7 @@ Poco::JSON::Object SerializationInfo::toJSON() const void SerializationInfo::fromJSON(const Poco::JSON::Object & object) { if (!object.has(KEY_KIND) || !object.has(KEY_NUM_DEFAULTS) || !object.has(KEY_NUM_ROWS)) - throw Exception(ErrorCodes::LOGICAL_ERROR, + throw Exception(ErrorCodes::CORRUPTED_DATA, "Missed field '{}' or '{}' or '{}' in SerializationInfo of columns", KEY_KIND, KEY_NUM_DEFAULTS, KEY_NUM_ROWS); @@ -129,7 +129,8 @@ SerializationInfoByName::SerializationInfoByName( const SerializationInfo::Settings & settings) { for (const auto & column : columns) - emplace(column.name, column.type->createSerializationInfo(settings)); + if (column.type->supportsSparseSerialization()) + emplace(column.name, column.type->createSerializationInfo(settings)); } void SerializationInfoByName::add(const Block & block) @@ -138,8 +139,7 @@ void SerializationInfoByName::add(const Block & block) { auto it = find(column.name); if (it == end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Not found column {} in serialization infos", column.name); + continue; it->second->add(*column.column); } @@ -151,8 +151,7 @@ void SerializationInfoByName::add(const SerializationInfoByName & other) { auto it = find(name); if (it == end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Not found column {} in serialization infos", name); + continue; it->second->add(*info); } From c66207f606a1a19eeff351e31b52d0acd8039abb Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 1 Nov 2021 05:13:07 +0300 Subject: [PATCH 061/358] fix nested --- src/DataTypes/IDataType.cpp | 13 +++++++++++++ src/DataTypes/IDataType.h | 2 ++ src/DataTypes/Serializations/SerializationInfo.cpp | 3 +++ src/Storages/MergeTree/IMergeTreeReader.cpp | 8 ++++++++ src/Storages/MergeTree/IMergeTreeReader.h | 4 +++- src/Storages/MergeTree/MergeTreeReaderCompact.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 9 ++++----- src/Storages/StorageLog.cpp | 2 +- src/Storages/StorageTinyLog.cpp | 2 +- 9 files changed, 38 insertions(+), 11 deletions(-) diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index 57199f7188f..edc9e4159f4 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -211,4 +211,17 @@ SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, con return column.type->getSerialization(info); } +// static +SerializationPtr IDataType::getSerialization(const NameAndTypePair & column) +{ + if (column.isSubcolumn()) + { + const auto & type_in_storage = column.getTypeInStorage(); + auto serialization = type_in_storage->getDefaultSerialization(); + return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), serialization); + } + + return column.type->getDefaultSerialization(); +} + } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 41b624c5655..a978fb79bfd 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -117,6 +117,8 @@ public: /// This method typically should be used to get serialization for reading column or subcolumn. static SerializationPtr getSerialization(const NameAndTypePair & column, const SerializationInfo & info); + static SerializationPtr getSerialization(const NameAndTypePair & column); + protected: virtual String doGetName() const { return getFamilyName(); } virtual SerializationPtr doGetDefaultSerialization() const = 0; diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index da258b0448c..3be512624bf 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -128,6 +128,9 @@ SerializationInfoByName::SerializationInfoByName( const NamesAndTypesList & columns, const SerializationInfo::Settings & settings) { + if (settings.ratio_for_sparse >= 1.0) + return; + for (const auto & column : columns) if (column.type->supportsSparseSerialization()) emplace(column.name, column.type->createSerializationInfo(settings)); diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index c52d14ce824..8510f971fe7 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -307,6 +307,14 @@ IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const St return {}; } +SerializationPtr IMergeTreeReader::getSerialization(const NameAndTypePair & column) const +{ + auto it = serializations.find(column.name); + if (it != serializations.end()) + return it->second; + return column.type->getDefaultSerialization(); +} + void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const { if (num_columns_to_read != columns.size()) diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 74e34e24eb0..b513c326166 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -87,11 +87,13 @@ protected: using ColumnPosition = std::optional; ColumnPosition findColumnForOffsets(const String & column_name) const; - const SerializationByName & serializations; + SerializationPtr getSerialization(const NameAndTypePair & column) const; friend class MergeTreeRangeReader::DelayedStream; private: + const SerializationByName & serializations; + /// 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 f1dfecdbb0d..63233b2bd6f 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -139,7 +139,7 @@ size_t MergeTreeReaderCompact::readRows(size_t from_mark, bool continue_reading, auto column_from_part = getColumnFromPart(*column_it); if (res_columns[i] == nullptr) - res_columns[i] = column_from_part.type->createColumn(*serializations.at(column_from_part.name)); + res_columns[i] = column_from_part.type->createColumn(*getSerialization(column_from_part)); } while (read_rows < max_rows_to_read) @@ -217,7 +217,7 @@ void MergeTreeReaderCompact::readData( const auto & type_in_storage = name_and_type.getTypeInStorage(); const auto & name_in_storage = name_and_type.getNameInStorage(); - const auto & serialization = serializations.at(name_in_storage); + auto serialization = getSerialization(NameAndTypePair{name_in_storage, type_in_storage}); ColumnPtr temp_column = type_in_storage->createColumn(*serialization); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); @@ -233,7 +233,7 @@ void MergeTreeReaderCompact::readData( } else { - const auto & serialization = serializations.at(name); + auto serialization = getSerialization(name_and_type); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr); } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index cb1db9ebd22..dc0d62d85d8 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -109,7 +109,7 @@ size_t MergeTreeReaderWide::readRows(size_t from_mark, bool continue_reading, si /// 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(*serializations.at(name)); + res_columns[pos] = type->createColumn(*getSerialization(column_from_part)); auto & column = res_columns[pos]; try @@ -188,7 +188,7 @@ void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type, profile_callback, clock_type)); }; - serializations.at(name_and_type.name)->enumerateStreams(callback); + getSerialization(name_and_type)->enumerateStreams(callback); } @@ -228,8 +228,7 @@ void MergeTreeReaderWide::prefetch( ISerialization::SubstreamsCache & cache, std::unordered_set & prefetched_streams) { - const auto & name = name_and_type.name; - auto & serialization = serializations.at(name); + auto serialization = getSerialization(name_and_type); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { @@ -257,7 +256,7 @@ void MergeTreeReaderWide::readData( deserialize_settings.avg_value_size_hint = avg_value_size_hint; const auto & name = name_and_type.name; - auto & serialization = serializations.at(name); + auto serialization = getSerialization(name_and_type); if (deserialize_binary_bulk_state_map.count(name) == 0) { diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index e0ee86dad03..b87eed6beed 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -166,7 +166,7 @@ void LogSource::readData(const NameAndTypePair & name_and_type, ColumnPtr & colu { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = type->getDefaultSerialization(); + auto serialization = IDataType::getSerialization(name_and_type); auto create_stream_getter = [&](bool stream_for_prefix) { diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 485c6295e07..19f246a90e0 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -168,7 +168,7 @@ void TinyLogSource::readData(const NameAndTypePair & name_and_type, { ISerialization::DeserializeBinaryBulkSettings settings; /// TODO Use avg_value_size_hint. const auto & [name, type] = name_and_type; - auto serialization = name_and_type.type->getDefaultSerialization(); + auto serialization = IDataType::getSerialization(name_and_type); settings.getter = [&] (const ISerialization::SubstreamPath & path) -> ReadBuffer * { From 8fe3de16c6a54159ae7fc84dd85190bde1ce2383 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 1 Nov 2021 05:40:43 +0300 Subject: [PATCH 062/358] fix nested --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 8 ++++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 3 +-- 3 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index c1f7adef620..b29d57fb890 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -451,6 +451,14 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const } } +SerializationPtr IMergeTreeDataPart::getSerializationOrDefault(const NameAndTypePair & column) const +{ + auto it = serializations.find(column.name); + return it == serializations.end() + ? column.type->getDefaultSerialization() + : it->second; +} + void IMergeTreeDataPart::removeIfNeeded() { if (state == State::DeleteOnDestroy || is_temp) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 1695988ea32..3a05c804566 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -132,6 +132,7 @@ public: const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } const SerializationByName & getSerializations() const { return serializations; } const SerializationPtr & getSerialization(const String & column_name) const { return serializations.at(column_name); } + SerializationPtr getSerializationOrDefault(const NameAndTypePair & column) const; /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 33ed08ff1d3..8029b9d55cc 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -213,8 +213,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const }; bool res = true; - auto serialization = getSerialization(column.name); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerializationOrDefault(column)->enumerateStreams([&](const auto & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); if (!check_stream_exists(file_name)) From a2b98f253e86b8ed8ba12ab6159f9a648ba40d03 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 1 Nov 2021 16:52:13 +0300 Subject: [PATCH 063/358] fix nested --- src/Storages/MergeTree/IMergeTreeReader.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 8510f971fe7..08f57ec009c 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -43,14 +43,6 @@ IMergeTreeReader::IMergeTreeReader( , serializations(data_part_->getSerializations()) , alter_conversions(storage.getAlterConversionsForPart(data_part)) { - if (isWidePart(data_part)) - { - /// For wide parts convert plain arrays of Nested to subcolumns - /// to allow to use shared offset column from cache. - columns = Nested::convertToSubcolumns(columns); - part_columns = Nested::collect(part_columns); - } - for (const auto & column_from_part : part_columns) columns_from_part[column_from_part.name] = &column_from_part.type; } From 9823f28855495f3439a01113b5faa17c87422cf8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 2 Nov 2021 06:03:52 +0300 Subject: [PATCH 064/358] fix nested --- .../Serializations/SerializationInfo.cpp | 4 +-- .../Serializations/SerializationInfo.h | 6 ++-- .../SerializationLowCardinality.cpp | 8 +++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 35 +++++------------- src/Storages/MergeTree/IMergeTreeDataPart.h | 12 +++---- src/Storages/MergeTree/IMergeTreeReader.cpp | 19 +++++----- src/Storages/MergeTree/IMergeTreeReader.h | 4 --- .../MergeTree/IMergedBlockOutputStream.cpp | 15 +++++--- .../MergeTree/MergeTreeDataPartWide.cpp | 12 +++---- .../MergeTreeDataPartWriterCompact.cpp | 4 +-- .../MergeTreeDataPartWriterOnDisk.cpp | 1 - .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 1 - .../MergeTree/MergeTreeDataPartWriterWide.cpp | 36 ++++++++++--------- .../MergeTree/MergeTreeDataPartWriterWide.h | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 11 +++--- .../MergeTree/MergeTreeReaderWide.cpp | 11 +++--- .../MergeTree/MergeTreeWriteAheadLog.cpp | 1 + src/Storages/MergeTree/MutateTask.cpp | 8 ++--- src/Storages/MergeTree/checkDataPart.cpp | 14 +++++--- .../System/StorageSystemPartsColumns.cpp | 2 +- 20 files changed, 101 insertions(+), 105 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 3be512624bf..837b648748b 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -128,7 +128,7 @@ SerializationInfoByName::SerializationInfoByName( const NamesAndTypesList & columns, const SerializationInfo::Settings & settings) { - if (settings.ratio_for_sparse >= 1.0) + if (settings.isAlwaysDefault()) return; for (const auto & column : columns) @@ -206,7 +206,7 @@ void SerializationInfoByName::readText(ReadBuffer & in) if (it == end()) throw Exception(ErrorCodes::CORRUPTED_DATA, - "There is not column {} in serialization infos", name); + "There is no column {} in serialization infos", name); it->second->fromJSON(*elem_object); } diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 0635bfc9cbf..55faa965145 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -40,6 +40,8 @@ public: { const double ratio_for_sparse = 1.0; const bool choose_kind = false; + + bool isAlwaysDefault() const { return ratio_for_sparse >= 1.0; } }; SerializationInfo(ISerialization::Kind kind_, const Settings & settings_); @@ -82,9 +84,7 @@ class SerializationInfoByName : public std::unordered_mapenumerateStreams(path, callback, path.back().data); + path.push_back(Substream::DictionaryKeys); + path.back().data = dict_data; + + dict_inner_serialization->enumerateStreams(path, callback, dict_data); path.back() = Substream::DictionaryIndexes; path.back().data = data; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index b29d57fb890..5c18e874943 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -416,8 +416,7 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const for (const auto & column : columns) { - auto & serialization = serializations[column.name]; - column_name_to_position.emplace(column.name, pos); + column_name_to_position.emplace(column.name, pos++); auto it = new_infos.find(column.name); if (it != new_infos.end()) @@ -426,37 +425,19 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const const auto & new_info = it->second; if (old_info) - { old_info->replaceData(*new_info); - } else - { old_info = new_info->clone(); - serialization = column.type->getSerialization(*old_info); - } } - else - { - serialization = column.type->getDefaultSerialization(); - } - - IDataType::forEachSubcolumn([&](const auto &, const auto & subname, const auto & subdata) - { - auto subcolumn_name = Nested::concatenateName(column.name, subname); - column_name_to_position.emplace(subcolumn_name, pos); - serializations.emplace(subcolumn_name, subdata.serialization); - }, {serialization, column.type, nullptr, nullptr}); - - ++pos; } } -SerializationPtr IMergeTreeDataPart::getSerializationOrDefault(const NameAndTypePair & column) const +SerializationPtr IMergeTreeDataPart::getSerialization(const NameAndTypePair & column) const { - auto it = serializations.find(column.name); - return it == serializations.end() - ? column.type->getDefaultSerialization() - : it->second; + auto it = serialization_infos.find(column.getNameInStorage()); + return it == serialization_infos.end() + ? IDataType::getSerialization(column) + : IDataType::getSerialization(column, *it->second); } void IMergeTreeDataPart::removeIfNeeded() @@ -788,7 +769,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const if (column_size.data_compressed != 0 && !storage_columns.hasCompressionCodec(part_column.name)) { String path_to_data_file; - serializations.at(part_column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(part_column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (path_to_data_file.empty()) { @@ -974,7 +955,7 @@ void IMergeTreeDataPart::loadRowsCount() for (const NameAndTypePair & column : columns) { - ColumnPtr column_col = column.type->createColumn(*serializations.at(column.name)); + ColumnPtr column_col = column.type->createColumn(*getSerialization(column)); if (!column_col->isFixedAndContiguous() || column_col->lowCardinality()) continue; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 096f3c9474b..2e88a5f7a20 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -132,9 +132,7 @@ public: const NamesAndTypesList & getColumns() const { return columns; } const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } - const SerializationByName & getSerializations() const { return serializations; } - const SerializationPtr & getSerialization(const String & column_name) const { return serializations.at(column_name); } - SerializationPtr getSerializationOrDefault(const NameAndTypePair & column) const; + SerializationPtr getSerialization(const NameAndTypePair & column) const; /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; @@ -427,10 +425,6 @@ protected: /// Columns description. Cannot be changed, after part initialization. NamesAndTypesList columns; - SerializationInfoByName serialization_infos; - - SerializationByName serializations; - const Type part_type; /// Not null when it's a projection part. @@ -455,6 +449,10 @@ private: /// In compact parts order of columns is necessary NameToNumber column_name_to_position; + SerializationInfoByName serialization_infos; + + SerializationByName serializations; + /// Reads part unique identifier (if exists) from uuid.txt void loadUUID(); diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 08f57ec009c..79186402027 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -40,9 +40,16 @@ IMergeTreeReader::IMergeTreeReader( , storage(data_part_->storage) , metadata_snapshot(metadata_snapshot_) , all_mark_ranges(all_mark_ranges_) - , serializations(data_part_->getSerializations()) , alter_conversions(storage.getAlterConversionsForPart(data_part)) { + if (isWidePart(data_part)) + { + /// For wide parts convert plain arrays of Nested to subcolumns + /// to allow to use shared offset column from cache. + columns = Nested::convertToSubcolumns(columns); + part_columns = Nested::collect(part_columns); + } + for (const auto & column_from_part : part_columns) columns_from_part[column_from_part.name] = &column_from_part.type; } @@ -290,7 +297,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.getNameInStorage()); if (position && Nested::extractTableName(part_column.name) == table_name) return position; } @@ -299,14 +306,6 @@ IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const St return {}; } -SerializationPtr IMergeTreeReader::getSerialization(const NameAndTypePair & column) const -{ - auto it = serializations.find(column.name); - if (it != serializations.end()) - return it->second; - return column.type->getDefaultSerialization(); -} - void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const { if (num_columns_to_read != columns.size()) diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index dc6a0c1fa2b..28334b9a8bb 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -89,13 +89,9 @@ protected: using ColumnPosition = std::optional; ColumnPosition findColumnForOffsets(const String & column_name) const; - SerializationPtr getSerialization(const NameAndTypePair & column) const; - friend class MergeTreeRangeReader::DelayedStream; private: - const SerializationByName & serializations; - /// Alter conversions, which must be applied on fly if required MergeTreeData::AlterConversions alter_conversions; diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 25a21afbfa4..b563e142cf4 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -17,7 +17,15 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( , reset_columns(reset_columns_) { if (reset_columns) - new_serialization_infos = SerializationInfoByName(columns_list, {}); + { + SerializationInfo::Settings info_settings = + { + .ratio_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .choose_kind = false, + }; + + new_serialization_infos = SerializationInfoByName(columns_list, info_settings); + } } NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( @@ -34,10 +42,9 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. std::map stream_counts; - const auto & serializations = data_part->getSerializations(); for (const auto & column : columns) { - serializations.at(column.name)->enumerateStreams( + data_part->getSerialization(column)->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; @@ -63,7 +70,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( } }; - serializations.at(column_name)->enumerateStreams(callback); + data_part->getSerialization(*column_with_type)->enumerateStreams(callback); } /// Remove files on disk and checksums diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 501babd1348..b279c1aba6a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -80,7 +80,7 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( if (checksums.empty()) return size; - serializations.at(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); @@ -162,7 +162,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const { for (const NameAndTypePair & name_type : columns) { - serializations.at(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(name_type)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(name_type, substream_path); String mrk_file_name = file_name + index_granularity_info.marks_file_extension; @@ -183,7 +183,7 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const std::optional marks_size; for (const NameAndTypePair & name_type : columns) { - serializations.at(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(name_type)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { auto file_path = path + ISerialization::getFileNameForStream(name_type, substream_path) + index_granularity_info.marks_file_extension; @@ -218,7 +218,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const }; bool res = true; - getSerializationOrDefault(column)->enumerateStreams([&](const auto & substream_path) + getSerialization(column)->enumerateStreams([&](const auto & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); if (!check_stream_exists(file_name)) @@ -231,7 +231,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const { String filename; - serializations.at(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (filename.empty()) filename = ISerialization::getFileNameForStream(column, substream_path); @@ -253,7 +253,7 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col if (rows_count != 0 && column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && serializations.at(column.name)->getKind() == ISerialization::Kind::DEFAULT) + && getSerialization(column)->getKind() == ISerialization::Kind::DEFAULT) { size_t rows_in_column = size.data_uncompressed / column.type->getSizeOfValueInMemory(); if (rows_in_column != rows_count) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 3d40b7cfda7..ce85bc75c80 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -66,7 +66,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, }; ISerialization::SubstreamPath path; - serializations.at(column.name)->enumerateStreams(path, callback, column.type); + data_part->getSerialization(column)->enumerateStreams(path, callback, column.type); } namespace @@ -207,7 +207,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeIntBinary(UInt64(0), marks); writeColumnSingleGranule( - block.getByName(name_and_type->name), serializations.at(name_and_type->name), + block.getByName(name_and_type->name), data_part->getSerialization(*name_and_type), stream_getter, granule.start_row, granule.rows_to_write); /// Each type always have at least one substream diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index cd8d2590095..03ae6688beb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -75,7 +75,6 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( , skip_indices(indices_to_recalc_) , part_path(data_part_->getFullRelativePath()) , marks_file_extension(marks_file_extension_) - , serializations(data_part_->getSerializations()) , default_codec(default_codec_) , compute_granularity(index_granularity.empty()) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index fa0c08b81aa..fb46175c2aa 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -123,7 +123,6 @@ protected: const String part_path; const String marks_file_extension; - const SerializationByName & serializations; const CompressionCodecPtr default_codec; const bool compute_granularity; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 78afcc5972e..b620bf8130e 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -118,7 +118,7 @@ void MergeTreeDataPartWriterWide::addStreams( }; ISerialization::SubstreamPath path; - serializations.at(column.name)->enumerateStreams(path, callback, column.type); + data_part->getSerialization(column)->enumerateStreams(path, callback, column.type); } @@ -213,7 +213,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm { auto & column = block_to_write.getByName(it->name); - if (serializations.at(column.name)->getKind() != ISerialization::Kind::SPARSE) + if (data_part->getSerialization(*it)->getKind() != ISerialization::Kind::SPARSE) column.column = recursiveRemoveSparse(column.column); if (permutation) @@ -275,7 +275,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( ISerialization::SubstreamPath & path) { StreamsWithMarks result; - serializations.at(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + data_part->getSerialization(column)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; @@ -310,7 +310,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( ISerialization::SerializeBinaryBulkSettings & serialize_settings, const Granule & granule) { - const auto & serialization = serializations.at(name_and_type.name); + const auto & serialization = data_part->getSerialization(name_and_type); serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, 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. @@ -340,12 +340,13 @@ void MergeTreeDataPartWriterWide::writeColumn( const auto & [name, type] = name_and_type; auto [it, inserted] = serialization_states.emplace(name, nullptr); + auto serialization = data_part->getSerialization(name_and_type); if (inserted) { ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); - serializations.at(name)->serializeBinaryBulkStatePrefix(serialize_settings, it->second); + serialization->serializeBinaryBulkStatePrefix(serialize_settings, it->second); } const auto & global_settings = storage.getContext()->getSettingsRef(); @@ -386,7 +387,7 @@ void MergeTreeDataPartWriterWide::writeColumn( } } - serializations.at(name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) @@ -398,12 +399,13 @@ void MergeTreeDataPartWriterWide::writeColumn( } -void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, const IDataType & type) +void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePair & name_type) { - const auto & serialization = serializations.at(name); + const auto & [name, type] = name_type; + const auto & serialization = data_part->getSerialization(name_type); - if (!type.isValueRepresentedByNumber() || type.haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type.getName()); + if (!type->isValueRepresentedByNumber() || type->haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type->getName()); auto disk = data_part->volume->getDisk(); String escaped_name = escapeForFileName(name); @@ -443,7 +445,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, if (index_granularity_rows == 0) { - auto column = type.createColumn(); + auto column = type->createColumn(); serialization->deserializeBinaryBulk(*column, bin_in, 1000000000, 0.0); @@ -463,7 +465,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, ErrorCodes::LOGICAL_ERROR, "Incorrect mark rows for part {} for mark #{} (compressed offset {}, decompressed offset {}), in-memory {}, on disk {}, total marks {}", data_part->getFullPath(), mark_num, offset_in_compressed_file, offset_in_decompressed_block, index_granularity.getMarkRows(mark_num), index_granularity_rows, index_granularity.getMarksCount()); - auto column = type.createColumn(); + auto column = type->createColumn(); serialization->deserializeBinaryBulk(*column, bin_in, index_granularity_rows, 0.0); @@ -502,7 +504,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const String & name, "Still have something in marks stream, last mark #{} index granularity size {}, last rows {}", mark_num, index_granularity.getMarksCount(), index_granularity_rows); if (!bin_in.eof()) { - auto column = type.createColumn(); + auto column = type->createColumn(); serialization->deserializeBinaryBulk(*column, bin_in, 1000000000, 0.0); @@ -538,7 +540,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch if (!serialization_states.empty()) { serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns); - serializations.at(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); + data_part->getSerialization(*it)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); } if (write_final_mark) @@ -563,9 +565,9 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(IMergeTreeDataPart::Ch { if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && serializations.at(column.name)->getKind() == ISerialization::Kind::DEFAULT) + && data_part->getSerialization(column)->getKind() == ISerialization::Kind::DEFAULT) { - validateColumnOfFixedSize(column.name, *column.type); + validateColumnOfFixedSize(column); } } #endif @@ -591,7 +593,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark( { writeSingleMark(column, offset_columns, 0, path); /// Memoize information about offsets - serializations.at(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + data_part->getSerialization(column)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index 5eaaa0c1bbe..6303fbbac0d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -84,7 +84,7 @@ private: /// Method for self check (used in debug-build only). Checks that written /// data and corresponding marks are consistent. Otherwise throws logical /// errors. - void validateColumnOfFixedSize(const String & name, const IDataType & type); + void validateColumnOfFixedSize(const NameAndTypePair & name_type); void fillIndexGranularity(size_t index_granularity_for_block, size_t rows_in_block) override; diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 8c4e4388877..c17e1196c4e 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -54,7 +54,7 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { auto column_from_part = getColumnFromPart(*name_and_type); - auto position = data_part->getColumnPosition(column_from_part.name); + auto position = data_part->getColumnPosition(column_from_part.getNameInStorage()); if (!position && typeid_cast(column_from_part.type.get())) { /// If array of Nested column is missing in part, @@ -139,7 +139,10 @@ size_t MergeTreeReaderCompact::readRows( auto column_from_part = getColumnFromPart(*column_it); if (res_columns[i] == nullptr) - res_columns[i] = column_from_part.type->createColumn(*getSerialization(column_from_part)); + { + auto serialization = data_part->getSerialization(column_from_part); + res_columns[i] = column_from_part.type->createColumn(*serialization); + } } while (read_rows < max_rows_to_read) @@ -220,7 +223,7 @@ void MergeTreeReaderCompact::readData( const auto & type_in_storage = name_and_type.getTypeInStorage(); const auto & name_in_storage = name_and_type.getNameInStorage(); - auto serialization = getSerialization(NameAndTypePair{name_in_storage, type_in_storage}); + auto serialization = data_part->getSerialization(NameAndTypePair{name_in_storage, type_in_storage}); ColumnPtr temp_column = type_in_storage->createColumn(*serialization); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); @@ -236,7 +239,7 @@ void MergeTreeReaderCompact::readData( } else { - auto serialization = getSerialization(name_and_type); + auto serialization = data_part->getSerialization(name_and_type); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr); } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index e4f9ae4d336..ab2551c8591 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -106,7 +106,10 @@ size_t MergeTreeReaderWide::readRows( /// 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(*getSerialization(column_from_part)); + { + auto serialization = data_part->getSerialization(column_from_part); + res_columns[pos] = type->createColumn(*serialization); + } auto & column = res_columns[pos]; try @@ -185,7 +188,7 @@ void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type, profile_callback, clock_type)); }; - getSerialization(name_and_type)->enumerateStreams(callback); + data_part->getSerialization(name_and_type)->enumerateStreams(callback); } @@ -228,7 +231,7 @@ void MergeTreeReaderWide::prefetch( ISerialization::SubstreamsCache & cache, std::unordered_set & prefetched_streams) { - auto serialization = getSerialization(name_and_type); + auto serialization = data_part->getSerialization(name_and_type); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { @@ -256,7 +259,7 @@ void MergeTreeReaderWide::readData( deserialize_settings.avg_value_size_hint = avg_value_size_hint; const auto & name = name_and_type.name; - auto serialization = getSerialization(name_and_type); + auto serialization = data_part->getSerialization(name_and_type); if (deserialize_binary_bulk_state_map.count(name) == 0) { diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 9323249946a..694357ab0c2 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -199,6 +199,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor part->minmax_idx->update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0, context); + part->setColumns(block.getNamesAndTypesList()); if (metadata_snapshot->hasSortingKey()) metadata_snapshot->getSortingKey().expression->execute(block); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b6efdf83300..b94e7620c12 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -313,7 +313,7 @@ NameSet collectFilesToSkip( files_to_skip.insert(stream_name + mrk_extension); }; - source_part->getSerialization(entry.name)->enumerateStreams(callback); + source_part->getSerialization({entry.name, entry.type})->enumerateStreams(callback); } for (const auto & index : indices_to_recalc) { @@ -338,7 +338,7 @@ static NameToNameVector collectFilesForRenames( std::map stream_counts; for (const auto & column : source_part->getColumns()) { - source_part->getSerialization(column.name)->enumerateStreams( + source_part->getSerialization(column)->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; @@ -382,7 +382,7 @@ static NameToNameVector collectFilesForRenames( auto column = source_part->getColumns().tryGetByName(command.column_name); if (column) - source_part->getSerialization(column->name)->enumerateStreams(callback); + source_part->getSerialization(*column)->enumerateStreams(callback); } else if (command.type == MutationCommand::Type::RENAME_COLUMN) { @@ -404,7 +404,7 @@ static NameToNameVector collectFilesForRenames( auto column = source_part->getColumns().tryGetByName(command.column_name); if (column) - source_part->getSerialization(column->name)->enumerateStreams(callback); + source_part->getSerialization(*column)->enumerateStreams(callback); } } diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 2733edfde77..6b46e65e2c1 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -107,6 +107,14 @@ IMergeTreeDataPart::Checksums checkDataPart( serialization_infos.readText(*serialization_file); } + auto get_serialization = [&serialization_infos](const auto & column) + { + auto it = serialization_infos.find(column.name); + return it == serialization_infos.end() + ? column.type->getDefaultSerialization() + : column.type->getSerialization(*it->second); + }; + /// This function calculates only checksum of file content (compressed or uncompressed). /// It also calculates checksum of projections. auto checksum_file = [&](const String & file_path, const String & file_name) @@ -141,8 +149,7 @@ IMergeTreeDataPart::Checksums checkDataPart( const NamesAndTypesList & projection_columns_list = projection->getColumns(); for (const auto & projection_column : projection_columns_list) { - auto serialization = projection_column.type->getSerialization(*serialization_infos.at(projection_column.name)); - serialization->enumerateStreams( + get_serialization(projection_column)->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin"; @@ -214,8 +221,7 @@ IMergeTreeDataPart::Checksums checkDataPart( { for (const auto & column : columns_list) { - auto serialization = column.type->getSerialization(*serialization_infos.at(column .name)); - serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + get_serialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path) + ".bin"; checksums_data.files[file_name] = checksum_compressed_file(disk, path + file_name); diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 0a99c626d48..7985e712d39 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -222,7 +222,7 @@ void StorageSystemPartsColumns::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.marks); - auto serialization = part->getSerialization(column.name); + auto serialization = part->getSerialization(column); if (columns_mask[src_index++]) columns[res_index++]->insert(ISerialization::kindToString(serialization->getKind())); From e1ab0ad383efc8b55b2a30f7970af021a55b5391 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 2 Nov 2021 20:44:15 +0300 Subject: [PATCH 065/358] fix tests --- src/Storages/MergeTree/MergeTreeDataPartCompact.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../00804_test_delta_codec_compression.reference | 2 +- ...0961_checksums_in_system_parts_columns_table.reference | 2 +- .../0_stateless/01780_column_sparse_full.reference | 5 +++++ tests/queries/0_stateless/01780_column_sparse_full.sql | 8 ++++++++ 6 files changed, 17 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index c4c2e65547b..f4da730b1f0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -125,7 +125,7 @@ void MergeTreeDataPartCompact::loadIndexGranularity() bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const { - if (!getColumnPosition(column.name)) + if (!getColumnPosition(column.getNameInStorage())) 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 d1a0344859d..c5ee9ebd01f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -45,7 +45,7 @@ public: bool isStoredOnDisk() const override { return false; } bool isStoredOnRemoteDisk() 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.getNameInStorage()); } 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/00804_test_delta_codec_compression.reference b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference index 79302586e92..949d37ed27a 100644 --- a/tests/queries/0_stateless/00804_test_delta_codec_compression.reference +++ b/tests/queries/0_stateless/00804_test_delta_codec_compression.reference @@ -1,4 +1,4 @@ -83 +84 1 46 1 diff --git a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference index 30ab6c68aaa..186f2feab79 100644 --- a/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference +++ b/tests/queries/0_stateless/00961_checksums_in_system_parts_columns_table.reference @@ -1 +1 @@ -20000101_1_1_0 test_00961 21d49af89b4905a5d1c76d4f1b02e20e d816caf0abffc3f9aabdb4f92bcf7bfb da96ff1e527a8a1f908ddf2b1d0af239 +20000101_1_1_0 test_00961 b5fce9c4ef1ca42ce4ed027389c208d2 fc3b062b646cd23d4c23d7f5920f89ae da96ff1e527a8a1f908ddf2b1d0af239 diff --git a/tests/queries/0_stateless/01780_column_sparse_full.reference b/tests/queries/0_stateless/01780_column_sparse_full.reference index eedc070dd37..4d2d0a58798 100644 --- a/tests/queries/0_stateless/01780_column_sparse_full.reference +++ b/tests/queries/0_stateless/01780_column_sparse_full.reference @@ -100,6 +100,11 @@ all_2_2_0 u Default 0 0 0 0 0 0 0 0 0 +====== id Default s Sparse u Sparse +====== +990 990 +980 980 980 +970 970 diff --git a/tests/queries/0_stateless/01780_column_sparse_full.sql b/tests/queries/0_stateless/01780_column_sparse_full.sql index b51f8f7ce4e..af6fde116d9 100644 --- a/tests/queries/0_stateless/01780_column_sparse_full.sql +++ b/tests/queries/0_stateless/01780_column_sparse_full.sql @@ -92,6 +92,14 @@ SYSTEM START MERGES t_sparse_full; OPTIMIZE TABLE t_sparse_full FINAL; +SELECT '======'; + SELECT column, serialization_kind FROM system.parts_columns WHERE table = 't_sparse_full' AND database = currentDatabase() AND active ORDER BY name, column; + +SELECT '======'; + +SELECT id, u, s FROM t_sparse_full ORDER BY u DESC LIMIT 3; + +DROP TABLE t_sparse_full; From 46fa062a81219c0fdddba93ae461938b87cfb95c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 2 Nov 2021 23:30:28 +0300 Subject: [PATCH 066/358] fix tests --- src/Columns/ColumnSparse.cpp | 2 +- src/Columns/ColumnUnique.h | 4 ++-- src/Columns/IColumnImpl.h | 1 + src/DataTypes/IDataType.h | 12 ++++++------ src/DataTypes/Serializations/SerializationInfo.cpp | 1 - .../Serializations/SerializationInfoTuple.cpp | 3 +-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- ...3_system_columns_and_system_tables_long.reference | 2 +- 8 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 5a0087af719..5f8c9538de6 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -34,7 +34,7 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs const ColumnUInt64 * offsets_concrete = typeid_cast(offsets.get()); if (!offsets_concrete) - throw Exception( ErrorCodes::LOGICAL_ERROR, "'offsets' column must be a ColumnUInt64, got: {}", offsets->getName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "'offsets' column must be a ColumnUInt64, got: {}", offsets->getName()); /// 'values' should contain one extra element: default value at 0 position. if (offsets->size() + 1 != values->size()) diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index b04eaad2a25..59eca547852 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -125,12 +125,12 @@ public: double getRatioOfDefaultRows(double) const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getRatioOfDefaultRows' not implemeted for ColumnUnique"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getRatioOfDefaultRows' not implemented for ColumnUnique"); } void getIndicesOfNonDefaultRows(IColumn::Offsets &, size_t, size_t) const override { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getIndicesOfNonDefaultRows' not implemeted for ColumnUnique"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method 'getIndicesOfNonDefaultRows' not implemented for ColumnUnique"); } const UInt64 * tryGetSavedHash() const override { return reverse_index.tryGetSavedHash(); } diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 4be143c8574..5b35171f2dd 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -16,6 +16,7 @@ namespace DB namespace ErrorCodes { extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int LOGICAL_ERROR; } template diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index c88690ae02f..894571e114e 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -303,12 +303,12 @@ public: const ISerialization * getCustomSerialization() const { return custom_serialization.get(); } private: - template - Ptr getForSubcolumn( - const String & subcolumn_name, - const SubstreamData & data, - Ptr SubstreamData::*member, - bool throw_if_null = true) const; + template + Ptr getForSubcolumn( + const String & subcolumn_name, + const SubstreamData & data, + Ptr SubstreamData::*member, + bool throw_if_null = true) const; }; diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 837b648748b..5176e68f3ca 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -17,7 +17,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int INCORRECT_DATA; extern const int CORRUPTED_DATA; } diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index bb41fb8a30c..6cc70496599 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -8,7 +8,6 @@ namespace DB namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int CORRUPTED_DATA; extern const int THERE_IS_NO_COLUMN; } @@ -106,7 +105,7 @@ void SerializationInfoTuple::fromJSON(const Poco::JSON::Object & object) auto subcolumns = object.getArray("subcolumns"); if (elems.size() != subcolumns->size()) throw Exception(ErrorCodes::THERE_IS_NO_COLUMN, - "Mismatched number of subcolumns beetween JSON and SerializationInfoTuple." + "Mismatched number of subcolumns between JSON and SerializationInfoTuple." "Expected: {}, got: {}", elems.size(), subcolumns->size()); for (size_t i = 0; i < elems.size(); ++i) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 5c18e874943..d86eed4f447 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -903,7 +903,7 @@ void IMergeTreeDataPart::loadRowsCount() /// Most trivial types if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) + && getSerialization(column)->getKind() == ISerialization::Kind::DEFAULT) { auto size = getColumnSize(column.name, *column.type); diff --git a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference index 2b05c417fae..12af231d18c 100644 --- a/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference +++ b/tests/queries/0_stateless/00753_system_columns_and_system_tables_long.reference @@ -6,7 +6,7 @@ │ name2 │ 1 │ 0 │ 0 │ 0 │ │ name3 │ 0 │ 0 │ 0 │ 0 │ └───────┴─────────────────────┴───────────────────┴───────────────────┴────────────────────┘ -276 1 +231 1 ┌─name────────────────┬─partition_key─┬─sorting_key───┬─primary_key─┬─sampling_key─┠│ check_system_tables │ date │ date, version │ date │ │ └─────────────────────┴───────────────┴───────────────┴─────────────┴──────────────┘ From f37d12d16e48a713ac66d3974d7d5ad2f0f2be79 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Nov 2021 23:29:48 +0300 Subject: [PATCH 067/358] keep serialization infos after drops and renames --- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + .../MergeTree/IMergedBlockOutputStream.cpp | 2 + .../MergeTree/IMergedBlockOutputStream.h | 1 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 51 ++++++++++++------- .../MergeTree/MergeTreeDataMergerMutator.h | 3 +- .../MergeTree/MergedBlockOutputStream.cpp | 14 +++-- .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 8 +-- .../01780_column_sparse_alter.reference | 7 +++ .../0_stateless/01780_column_sparse_alter.sql | 26 ++++++++++ 11 files changed, 87 insertions(+), 29 deletions(-) create mode 100644 tests/queries/0_stateless/01780_column_sparse_alter.reference create mode 100644 tests/queries/0_stateless/01780_column_sparse_alter.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 2e88a5f7a20..3c82184af48 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -132,6 +132,7 @@ public: const NamesAndTypesList & getColumns() const { return columns; } const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } + SerializationInfoByName & getSerializationInfos() { return serialization_infos; } SerializationPtr getSerialization(const NameAndTypePair & column) const; /// Throws an exception if part is not stored in on-disk format. diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index b563e142cf4..6e00a8d6a6d 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -31,6 +31,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( const MergeTreeDataPartPtr & data_part, NamesAndTypesList & columns, + SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums) { const NameSet & empty_columns = data_part->expired_columns; @@ -71,6 +72,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( }; data_part->getSerialization(*column_with_type)->enumerateStreams(callback); + serialization_infos.erase(column_name); } /// Remove files on disk and checksums diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index 3a65de511fa..a7c25edabd4 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -38,6 +38,7 @@ protected: static NameSet removeEmptyColumnsFromPart( const MergeTreeDataPartPtr & data_part, NamesAndTypesList & columns, + SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums); const MergeTreeData & storage; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c1637ab538b..8c8775b9444 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -673,27 +673,50 @@ void MergeTreeDataMergerMutator::splitMutationCommands( } -NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart( +std::pair +MergeTreeDataMergerMutator::getColumnsForNewDataPart( MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList storage_columns, + const SerializationInfoByName & serialization_infos, const MutationCommands & commands_for_removes) { - /// In compact parts we read all columns, because they all stored in a - /// single file - if (!isWidePart(source_part)) - return updated_header.getNamesAndTypesList(); - NameSet removed_columns; NameToNameMap renamed_columns_to_from; + NameToNameMap renamed_columns_from_to; + /// All commands are validated in AlterCommand so we don't care about order for (const auto & command : commands_for_removes) { if (command.type == MutationCommand::DROP_COLUMN) removed_columns.insert(command.column_name); + if (command.type == MutationCommand::RENAME_COLUMN) + { renamed_columns_to_from.emplace(command.rename_to, command.column_name); + renamed_columns_from_to.emplace(command.column_name, command.rename_to); + } } + + bool is_wide_part = isWidePart(source_part); + SerializationInfoByName new_serialization_infos; + for (const auto & [name, info] : serialization_infos) + { + if (is_wide_part && removed_columns.count(name)) + continue; + + auto it = renamed_columns_from_to.find(name); + if (it != renamed_columns_from_to.end()) + new_serialization_infos.emplace(it->second, info); + else + new_serialization_infos.emplace(name, info); + } + + /// In compact parts we read all columns, because they all stored in a + /// single file + if (!is_wide_part) + return {updated_header.getNamesAndTypesList(), new_serialization_infos}; + Names source_column_names = source_part->getColumns().getNames(); NameSet source_columns_name_set(source_column_names.begin(), source_column_names.end()); for (auto it = storage_columns.begin(); it != storage_columns.end();) @@ -720,18 +743,9 @@ NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart( } else { - bool was_renamed = false; - bool was_removed = removed_columns.count(it->name); - /// Check that this column was renamed to some other name - for (const auto & [rename_to, rename_from] : renamed_columns_to_from) - { - if (rename_from == it->name) - { - was_renamed = true; - break; - } - } + bool was_renamed = renamed_columns_from_to.count(it->name); + bool was_removed = removed_columns.count(it->name); /// If we want to rename this column to some other name, than it /// should it's previous version should be dropped or removed @@ -740,7 +754,6 @@ NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart( ErrorCodes::LOGICAL_ERROR, "Incorrect mutation commands, trying to rename column {} to {}, but part {} already has column {}", renamed_columns_to_from[it->name], it->name, source_part->name, it->name); - /// Column was renamed and no other column renamed to it's name /// or column is dropped. if (!renamed_columns_to_from.count(it->name) && (was_renamed || was_removed)) @@ -751,7 +764,7 @@ NamesAndTypesList MergeTreeDataMergerMutator::getColumnsForNewDataPart( } } - return storage_columns; + return {storage_columns, new_serialization_infos}; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index e5c8a4d8285..bcac642eb16 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -150,10 +150,11 @@ private: MutationCommands & for_file_renames); /// Get the columns list of the resulting part in the same order as storage_columns. - static NamesAndTypesList getColumnsForNewDataPart( + static std::pair getColumnsForNewDataPart( MergeTreeData::DataPartPtr source_part, const Block & updated_header, NamesAndTypesList storage_columns, + const SerializationInfoByName & serialization_infos, const MutationCommands & commands_for_removes); static ExecuteTTLType shouldExecuteTTL( diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 2f09ba18b76..a78c840fdac 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -78,11 +78,15 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( else part_columns = *total_columns_list; - if (reset_columns) - new_part->setColumns(part_columns, new_serialization_infos); + auto & serialization_infos = reset_columns + ? new_serialization_infos + : new_part->getSerializationInfos(); if (new_part->isStoredOnDisk()) - finalizePartOnDisk(new_part, part_columns, checksums, sync); + finalizePartOnDisk(new_part, part_columns, serialization_infos, checksums, sync); + + if (reset_columns) + new_part->setColumns(part_columns, serialization_infos); new_part->rows_count = rows_count; new_part->modification_time = time(nullptr); @@ -100,6 +104,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( void MergedBlockOutputStream::finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, + SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums, bool sync) { @@ -166,9 +171,8 @@ void MergedBlockOutputStream::finalizePartOnDisk( out->sync(); } - removeEmptyColumnsFromPart(new_part, part_columns, checksums); + removeEmptyColumnsFromPart(new_part, part_columns, serialization_infos, checksums); - const auto & serialization_infos = new_part->getSerializationInfos(); if (!serialization_infos.empty()) { auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index 64f74a0c645..ffc740bf410 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -49,6 +49,7 @@ private: void finalizePartOnDisk( const MergeTreeData::MutableDataPartPtr & new_part, NamesAndTypesList & part_columns, + SerializationInfoByName & serialization_infos, MergeTreeData::DataPart::Checksums & checksums, bool sync); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 71ea9458dde..ff79a187490 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -72,7 +72,7 @@ MergedColumnOnlyOutputStream::writeSuffixAndGetChecksums( auto columns = new_part->getColumns(); - auto removed_files = removeEmptyColumnsFromPart(new_part, columns, checksums); + auto removed_files = removeEmptyColumnsFromPart(new_part, columns, new_serialization_infos, checksums); for (const String & removed_file : removed_files) if (all_checksums.files.count(removed_file)) all_checksums.files.erase(removed_file); diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index b94e7620c12..54af0f19121 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1291,10 +1291,12 @@ bool MutateTask::prepare() /// It shouldn't be changed by mutation. ctx->new_data_part->index_granularity_info = ctx->source_part->index_granularity_info; - ctx->new_data_part->setColumns( - MergeTreeDataMergerMutator::getColumnsForNewDataPart(ctx->source_part, ctx->updated_header, ctx->storage_columns, ctx->for_file_renames), - ctx->source_part->getSerializationInfos()); + auto [new_columns, new_infos] = MergeTreeDataMergerMutator::getColumnsForNewDataPart( + ctx->source_part, ctx->updated_header, ctx->storage_columns, + ctx->source_part->getSerializationInfos(), ctx->commands_for_part); + + ctx->new_data_part->setColumns(new_columns, new_infos); ctx->new_data_part->partition.assign(ctx->source_part->partition); ctx->disk = ctx->new_data_part->volume->getDisk(); diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.reference b/tests/queries/0_stateless/01780_column_sparse_alter.reference new file mode 100644 index 00000000000..61e6ee38c93 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_alter.reference @@ -0,0 +1,7 @@ +id Default +u Sparse +s Sparse +20 11 +id Default +t Sparse +20 diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.sql b/tests/queries/0_stateless/01780_column_sparse_alter.sql new file mode 100644 index 00000000000..d350086a598 --- /dev/null +++ b/tests/queries/0_stateless/01780_column_sparse_alter.sql @@ -0,0 +1,26 @@ +SET mutations_sync = 2; + +DROP TABLE IF EXISTS t_sparse_alter; + +CREATE TABLE t_sparse_alter (id UInt64, u UInt64, s String) +ENGINE = MergeTree ORDER BY id +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; + +INSERT INTO t_sparse_alter SELECT + number, + if (number % 10 = 0, number, 0), + if (number % 20 = 0, toString(number), '') +FROM numbers(200); + +SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY name; + +SELECT uniqExact(u), uniqExact(s) FROM t_sparse_alter; + +ALTER TABLE t_sparse_alter DROP COLUMN s, RENAME COLUMN u TO t; +ALTER TABLE t_sparse_alter MODIFY COLUMN t UInt16; + +SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY name; + +SELECT uniqExact(t) FROM t_sparse_alter; + +DROP TABLE t_sparse_alter; From 67852e9134dc13b2f0564fb14c5f36d42bc81f03 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 3 Nov 2021 22:32:29 +0200 Subject: [PATCH 068/358] Implemented `CREATE [MATERIALIZED|LIVE] VIEW ... COMMENT` --- src/Parsers/ASTCreateQuery.cpp | 5 ++++- src/Parsers/ParserCreateQuery.cpp | 4 ++++ src/Storages/LiveView/StorageLiveView.cpp | 8 ++++++-- src/Storages/LiveView/StorageLiveView.h | 2 +- src/Storages/StorageFactory.cpp | 6 +++--- src/Storages/StorageMaterializedView.cpp | 8 ++++++-- src/Storages/StorageMaterializedView.h | 3 ++- 7 files changed, 26 insertions(+), 10 deletions(-) diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 2c048886247..d7fbb3c1fb1 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -396,8 +396,11 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (select) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" + << (comment ? "(" : "") + << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); select->formatImpl(settings, state, frame); + settings.ostr << (comment ? ")" : ""); } if (tables) diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index 07e321c3376..899c5a04f1d 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -727,6 +727,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (!select_p.parse(pos, select, expected)) return false; + auto comment = parseComment(pos, expected); auto query = std::make_shared(); node = query; @@ -756,6 +757,9 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (live_view_periodic_refresh) query->live_view_periodic_refresh.emplace(live_view_periodic_refresh->as().value.safeGet()); + if (comment) + query->set(query->comment, comment); + return true; } diff --git a/src/Storages/LiveView/StorageLiveView.cpp b/src/Storages/LiveView/StorageLiveView.cpp index 4641a1631f2..8cc5d3d8aa1 100644 --- a/src/Storages/LiveView/StorageLiveView.cpp +++ b/src/Storages/LiveView/StorageLiveView.cpp @@ -279,7 +279,8 @@ StorageLiveView::StorageLiveView( const StorageID & table_id_, ContextPtr context_, const ASTCreateQuery & query, - const ColumnsDescription & columns_) + const ColumnsDescription & columns_, + const String & comment) : IStorage(table_id_) , WithContext(context_->getGlobalContext()) { @@ -290,6 +291,9 @@ StorageLiveView::StorageLiveView( StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(columns_); + if (!comment.empty()) + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); if (!query.select) @@ -617,7 +621,7 @@ void registerStorageLiveView(StorageFactory & factory) "Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED); - return StorageLiveView::create(args.table_id, args.getLocalContext(), args.query, args.columns); + return StorageLiveView::create(args.table_id, args.getLocalContext(), args.query, args.columns, args.comment); }); } diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 265f5d60ec4..115897b65c0 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -233,7 +233,7 @@ private: ContextPtr context_, const ASTCreateQuery & query, const ColumnsDescription & columns - ); + , const String & comment); }; } diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index ba2ac3f72a3..2e2a9cab331 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -146,9 +146,6 @@ StoragePtr StorageFactory::get( throw Exception("Unknown table engine " + name, ErrorCodes::UNKNOWN_STORAGE); } - if (query.comment) - comment = query.comment->as().value.get(); - auto check_feature = [&](String feature_description, FeatureMatcherFn feature_matcher_fn) { if (!feature_matcher_fn(it->second.features)) @@ -194,6 +191,9 @@ StoragePtr StorageFactory::get( } } + if (query.comment) + comment = query.comment->as().value.get(); + ASTs empty_engine_args; Arguments arguments{ .engine_name = name, diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index d2493ff7c43..d076b690966 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -64,7 +64,8 @@ StorageMaterializedView::StorageMaterializedView( ContextPtr local_context, const ASTCreateQuery & query, const ColumnsDescription & columns_, - bool attach_) + bool attach_, + const String & comment) : IStorage(table_id_), WithMutableContext(local_context->getGlobalContext()) { StorageInMemoryMetadata storage_metadata; @@ -85,6 +86,9 @@ StorageMaterializedView::StorageMaterializedView( auto select = SelectQueryDescription::getSelectQueryFromASTForMatView(query.select->clone(), local_context); storage_metadata.setSelectQuery(select); + if (!comment.empty()) + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); bool point_to_itself_by_uuid = has_inner_table && query.to_inner_uuid != UUIDHelpers::Nil @@ -436,7 +440,7 @@ void registerStorageMaterializedView(StorageFactory & factory) /// Pass local_context here to convey setting for inner table return StorageMaterializedView::create( args.table_id, args.getLocalContext(), args.query, - args.columns, args.attach); + args.columns, args.attach, args.comment); }); } diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index b6f677e84bb..53d2ed75896 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -109,7 +109,8 @@ protected: ContextPtr local_context, const ASTCreateQuery & query, const ColumnsDescription & columns_, - bool attach_); + bool attach_, + const String & comment); }; } From 6532ede2f694accbd7c7cb1bc1d749817ce8e7f5 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Wed, 3 Nov 2021 22:52:32 +0200 Subject: [PATCH 069/358] Added tests for CREATE [MATERIALIZED|LIVE] VIEW ... COMMENT '' --- .../0_stateless/02048_views_with_comment.reference | 3 +++ tests/queries/0_stateless/02048_views_with_comment.sql | 10 ++++++++++ 2 files changed, 13 insertions(+) create mode 100644 tests/queries/0_stateless/02048_views_with_comment.reference create mode 100644 tests/queries/0_stateless/02048_views_with_comment.sql diff --git a/tests/queries/0_stateless/02048_views_with_comment.reference b/tests/queries/0_stateless/02048_views_with_comment.reference new file mode 100644 index 00000000000..ad9817ad120 --- /dev/null +++ b/tests/queries/0_stateless/02048_views_with_comment.reference @@ -0,0 +1,3 @@ +live_view_comment_test LiveView live view +materialized_view_comment_test MaterializedView materialized view +view_comment_test View simple view diff --git a/tests/queries/0_stateless/02048_views_with_comment.sql b/tests/queries/0_stateless/02048_views_with_comment.sql new file mode 100644 index 00000000000..9b501b76669 --- /dev/null +++ b/tests/queries/0_stateless/02048_views_with_comment.sql @@ -0,0 +1,10 @@ +# Make sure that any kind of `VIEW` can be created with a `COMMENT` clause +# and value of that clause is visible as `comment` column of `system.tables` table. + +CREATE VIEW view_comment_test AS (SELECT 1) COMMENT 'simple view'; +CREATE MATERIALIZED VIEW materialized_view_comment_test TO test1 (a UInt64) AS (SELECT 1) COMMENT 'materialized view'; + +SET allow_experimental_live_view=1; +CREATE LIVE VIEW live_view_comment_test AS (SELECT 1) COMMENT 'live view'; + +SELECT name, engine, comment FROM system.tables WHERE name LIKE '%view_comment_test' ORDER BY name; \ No newline at end of file From 84e914e05ada1bc4c1d59dc31bc3cad710357b55 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 5 Nov 2021 01:46:00 +0300 Subject: [PATCH 070/358] minor fixes near serializations --- src/Columns/ColumnArray.cpp | 2 +- src/Columns/ColumnSparse.cpp | 2 +- src/DataStreams/IBlockInputStream.cpp | 365 ------------------ src/DataStreams/IBlockInputStream.h | 275 ------------- src/DataStreams/materializeBlock.cpp | 29 -- .../Serializations/SerializationInfo.cpp | 1 - .../Serializations/SerializationInfoTuple.cpp | 3 +- src/Formats/NativeReader.cpp | 2 - .../PipelineExecutingBlockInputStream.h | 45 --- src/Processors/Transforms/FilterTransform.cpp | 2 - .../ConvertingBlockInputStream.h | 53 --- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 - src/Storages/MergeTree/MergeTask.cpp | 1 + src/Storages/MergeTree/MergeTask.h | 1 - src/Storages/StorageLog.cpp | 2 - .../integration/test_merge_tree_hdfs/test.py | 4 +- tests/integration/test_merge_tree_s3/test.py | 4 +- 17 files changed, 8 insertions(+), 786 deletions(-) delete mode 100644 src/DataStreams/IBlockInputStream.cpp delete mode 100644 src/DataStreams/IBlockInputStream.h delete mode 100644 src/DataStreams/materializeBlock.cpp delete mode 100644 src/Processors/Executors/PipelineExecutingBlockInputStream.h delete mode 100644 src/QueryPipeline/ConvertingBlockInputStream.h diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index e4e0f42ffdb..929c0153a0a 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -185,7 +185,7 @@ StringRef ColumnArray::getDataAt(size_t n) const bool ColumnArray::isDefaultAt(size_t n) const { const auto & offsets_data = getOffsets(); - return offsets_data[n] == offsets_data[n - 1]; + return offsets_data[n] == offsets_data[static_cast(n) - 1]; } diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 5f8c9538de6..a4b770f8f6b 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -47,7 +47,7 @@ ColumnSparse::ColumnSparse(MutableColumnPtr && values_, MutableColumnPtr && offs if (!offsets_concrete->empty() && _size <= offsets_concrete->getData().back()) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Size sparse columns ({}) should be greater than last position of non-default value ({})", + "Size of sparse column ({}) should be greater than last position of non-default value ({})", _size, offsets_concrete->getData().back()); #ifndef NDEBUG diff --git a/src/DataStreams/IBlockInputStream.cpp b/src/DataStreams/IBlockInputStream.cpp deleted file mode 100644 index 3c4494b69e1..00000000000 --- a/src/DataStreams/IBlockInputStream.cpp +++ /dev/null @@ -1,365 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include - -namespace ProfileEvents -{ - extern const Event ThrottlerSleepMicroseconds; - extern const Event SelectedRows; - extern const Event SelectedBytes; -} - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int QUERY_WAS_CANCELLED; - extern const int TOO_MANY_ROWS; - extern const int TOO_MANY_BYTES; - extern const int TOO_MANY_ROWS_OR_BYTES; - extern const int LOGICAL_ERROR; -} - - -/// It's safe to access children without mutex as long as these methods are called before first call to `read()` or `readPrefix()`. - - -Block IBlockInputStream::read() -{ - if (total_rows_approx) - { - progressImpl(Progress(0, 0, total_rows_approx)); - total_rows_approx = 0; - } - - if (!info.started) - { - info.total_stopwatch.start(); - info.started = true; - } - - Block res; - - if (isCancelledOrThrowIfKilled()) - return res; - - if (!checkTimeLimit()) - limit_exceeded_need_break = true; - - if (!limit_exceeded_need_break) - res = readImpl(); - - if (res) - { - info.update(res); - - if (enabled_extremes) - updateExtremes(res); - - if (limits.mode == LimitsMode::LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) - limit_exceeded_need_break = true; - - if (quota) - checkQuota(res); - } - else - { - /** If the stream is over, then we will ask all children to abort the execution. - * This makes sense when running a query with LIMIT - * - there is a situation when all the necessary data has already been read, - * but children sources are still working, - * herewith they can work in separate threads or even remotely. - */ - cancel(false); - } - - progress(Progress(res.rows(), res.bytes())); - -#ifndef NDEBUG - if (res) - { - Block header = getHeader(); - if (header) - { - if (columnsCanDifferInRepresentationAmongBlocks()) - assertCompatibleHeader(res, header, getName()); - else - assertBlocksHaveEqualStructure(res, header, getName()); - } - - } -#endif - - return res; -} - - -void IBlockInputStream::readPrefix() -{ -#ifndef NDEBUG - if (!read_prefix_is_called) - read_prefix_is_called = true; - else - throw Exception("readPrefix is called twice for " + getName() + " stream", ErrorCodes::LOGICAL_ERROR); -#endif - - readPrefixImpl(); - - forEachChild([&] (IBlockInputStream & child) - { - child.readPrefix(); - return false; - }); -} - - -void IBlockInputStream::readSuffix() -{ -#ifndef NDEBUG - if (!read_suffix_is_called) - read_suffix_is_called = true; - else - throw Exception("readSuffix is called twice for " + getName() + " stream", ErrorCodes::LOGICAL_ERROR); -#endif - - forEachChild([&] (IBlockInputStream & child) - { - child.readSuffix(); - return false; - }); - - readSuffixImpl(); -} - - -void IBlockInputStream::updateExtremes(Block & block) -{ - size_t num_columns = block.columns(); - - if (!extremes) - { - MutableColumns extremes_columns(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnPtr & src = block.safeGetByPosition(i).column; - - if (isColumnConst(*src)) - { - /// Equal min and max. - extremes_columns[i] = src->cloneResized(2); - } - else - { - Field min_value; - Field max_value; - - src->getExtremes(min_value, max_value); - - extremes_columns[i] = src->cloneEmpty(); - - extremes_columns[i]->insert(min_value); - extremes_columns[i]->insert(max_value); - } - } - - extremes = block.cloneWithColumns(std::move(extremes_columns)); - } - else - { - for (size_t i = 0; i < num_columns; ++i) - { - ColumnPtr & old_extremes = extremes.safeGetByPosition(i).column; - - if (isColumnConst(*old_extremes)) - continue; - - Field min_value = (*old_extremes)[0]; - Field max_value = (*old_extremes)[1]; - - Field cur_min_value; - Field cur_max_value; - - block.safeGetByPosition(i).column->getExtremes(cur_min_value, cur_max_value); - - if (cur_min_value < min_value) - min_value = cur_min_value; - if (cur_max_value > max_value) - max_value = cur_max_value; - - MutableColumnPtr new_extremes = old_extremes->cloneEmpty(); - - new_extremes->insert(min_value); - new_extremes->insert(max_value); - - old_extremes = std::move(new_extremes); - } - } -} - - -bool IBlockInputStream::checkTimeLimit() const -{ - return limits.speed_limits.checkTimeLimit(info.total_stopwatch, limits.timeout_overflow_mode); -} - - -void IBlockInputStream::checkQuota(Block & block) -{ - switch (limits.mode) - { - case LimitsMode::LIMITS_TOTAL: - /// Checked in `progress` method. - break; - - case LimitsMode::LIMITS_CURRENT: - { - UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds(); - quota->used({Quota::RESULT_ROWS, block.rows()}, {Quota::RESULT_BYTES, block.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); - prev_elapsed = total_elapsed; - break; - } - } -} - - -void IBlockInputStream::progressImpl(const Progress & value) -{ - if (progress_callback) - progress_callback(value); - - if (process_list_elem) - { - if (!process_list_elem->updateProgressIn(value)) - cancel(/* kill */ true); - - /// The total amount of data processed or intended for processing in all leaf sources, possibly on remote servers. - - ProgressValues progress = process_list_elem->getProgressIn(); - size_t total_rows_estimate = std::max(progress.read_rows, progress.total_rows_to_read); - - /** Check the restrictions on the amount of data to read, the speed of the query, the quota on the amount of data to read. - * NOTE: Maybe it makes sense to have them checked directly in ProcessList? - */ - if (limits.mode == LimitsMode::LIMITS_TOTAL) - { - if (!limits.size_limits.check(total_rows_estimate, progress.read_bytes, "rows to read", - ErrorCodes::TOO_MANY_ROWS, ErrorCodes::TOO_MANY_BYTES)) - cancel(false); - } - - size_t total_rows = progress.total_rows_to_read; - - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds - UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); - - if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) - { - CurrentThread::updatePerformanceCounters(); - last_profile_events_update_time = total_elapsed_microseconds; - } - - limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); - - if (quota && limits.mode == LimitsMode::LIMITS_TOTAL) - quota->used({Quota::READ_ROWS, value.read_rows}, {Quota::READ_BYTES, value.read_bytes}); - } - - ProfileEvents::increment(ProfileEvents::SelectedRows, value.read_rows); - ProfileEvents::increment(ProfileEvents::SelectedBytes, value.read_bytes); -} - - -void IBlockInputStream::cancel(bool kill) -{ - if (kill) - is_killed = true; - - bool old_val = false; - if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) - return; - - forEachChild([&] (IBlockInputStream & child) - { - child.cancel(kill); - return false; - }); -} - - -bool IBlockInputStream::isCancelled() const -{ - return is_cancelled; -} - -bool IBlockInputStream::isCancelledOrThrowIfKilled() const -{ - if (!is_cancelled) - return false; - if (is_killed) - throw Exception("Query was cancelled", ErrorCodes::QUERY_WAS_CANCELLED); - return true; -} - - -void IBlockInputStream::setProgressCallback(const ProgressCallback & callback) -{ - progress_callback = callback; - - forEachChild([&] (IBlockInputStream & child) - { - child.setProgressCallback(callback); - return false; - }); -} - - -void IBlockInputStream::setProcessListElement(QueryStatus * elem) -{ - process_list_elem = elem; - - forEachChild([&] (IBlockInputStream & child) - { - child.setProcessListElement(elem); - return false; - }); -} - - -Block IBlockInputStream::getTotals() -{ - if (totals) - return totals; - - Block res; - forEachChild([&] (IBlockInputStream & child) - { - res = child.getTotals(); - return bool(res); - }); - return res; -} - - -Block IBlockInputStream::getExtremes() -{ - if (extremes) - return extremes; - - Block res; - forEachChild([&] (IBlockInputStream & child) - { - res = child.getExtremes(); - return bool(res); - }); - return res; -} - -} diff --git a/src/DataStreams/IBlockInputStream.h b/src/DataStreams/IBlockInputStream.h deleted file mode 100644 index 0ab0aa33112..00000000000 --- a/src/DataStreams/IBlockInputStream.h +++ /dev/null @@ -1,275 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ -} - -class ProcessListElement; -class EnabledQuota; -class QueryStatus; - - -/** The stream interface for reading data by blocks from the database. - * Relational operations are supposed to be done also as implementations of this interface. - * Watches out at how the source of the blocks works. - * Lets you get information for profiling: rows per second, blocks per second, megabytes per second, etc. - * Allows you to stop reading data (in nested sources). - */ -class IBlockInputStream : public TypePromotion -{ - friend struct BlockStreamProfileInfo; - -public: - IBlockInputStream() { info.parent = this; } - virtual ~IBlockInputStream() = default; - - IBlockInputStream(const IBlockInputStream &) = delete; - IBlockInputStream & operator=(const IBlockInputStream &) = delete; - - /// To output the data stream transformation tree (query execution plan). - virtual String getName() const = 0; - - /** Get data structure of the stream in a form of "header" block (it is also called "sample block"). - * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. - * It is guaranteed that method "read" returns blocks of exactly that structure. - */ - virtual Block getHeader() const = 0; - - virtual const BlockMissingValues & getMissingValues() const - { - static const BlockMissingValues none; - return none; - } - - /** Read next block. - * If there are no more blocks, return an empty block (for which operator `bool` returns false). - * NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously. - * This also applies for readPrefix, readSuffix. - */ - Block read(); - - /** Read something before starting all data or after the end of all data. - * In the `readSuffix` function, you can implement a finalization that can lead to an exception. - * readPrefix() must be called before the first call to read(). - * readSuffix() should be called after read() returns an empty block, or after a call to cancel(), but not during read() execution. - */ - - /** The default implementation calls readPrefixImpl() on itself, and then readPrefix() recursively for all children. - * There are cases when you do not want `readPrefix` of children to be called synchronously, in this function, - * but you want them to be called, for example, in separate threads (for parallel initialization of children). - * Then overload `readPrefix` function. - */ - virtual void readPrefix(); - - /** The default implementation calls recursively readSuffix() on all children, and then readSuffixImpl() on itself. - * If this stream calls read() in children in a separate thread, this behavior is usually incorrect: - * readSuffix() of the child can not be called at the moment when the same child's read() is executed in another thread. - * In this case, you need to override this method so that readSuffix() in children is called, for example, after connecting streams. - */ - virtual void readSuffix(); - - /// Do not allow to change the table while the blocks stream and its children are alive. - void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } - - /// Get information about execution speed. - const BlockStreamProfileInfo & getProfileInfo() const { return info; } - - /** Get "total" values. - * The default implementation takes them from itself or from the first child source in which they are. - * The overridden method can perform some calculations. For example, apply an expression to the `totals` of the child source. - * There can be no total values - then an empty block is returned. - * - * Call this method only after all the data has been retrieved with `read`, - * otherwise there will be problems if any data at the same time is computed in another thread. - */ - virtual Block getTotals(); - - /// The same for minimums and maximums. - virtual Block getExtremes(); - - - /** Set the execution progress bar callback. - * The callback is passed to all child sources. - * By default, it is called for leaf sources, after each block. - * (But this can be overridden in the progress() method) - * The function takes the number of rows in the last block, the number of bytes in the last block. - * Note that the callback can be called from different threads. - */ - virtual void setProgressCallback(const ProgressCallback & callback); - - - /** In this method: - * - the progress callback is called; - * - the status of the query execution in ProcessList is updated; - * - checks restrictions and quotas that should be checked not within the same source, - * but over the total amount of resources spent in all sources at once (information in the ProcessList). - */ - virtual void progress(const Progress & value) - { - /// The data for progress is taken from leaf sources. - if (children.empty()) - progressImpl(value); - } - - void progressImpl(const Progress & value); - - - /** Set the pointer to the process list item. - * It is passed to all child sources. - * General information about the resources spent on the request will be written into it. - * Based on this information, the quota and some restrictions will be checked. - * This information will also be available in the SHOW PROCESSLIST request. - */ - virtual void setProcessListElement(QueryStatus * elem); - - /** Set the approximate total number of rows to read. - */ - void addTotalRowsApprox(size_t value) { total_rows_approx += value; } - - - /** Ask to abort the receipt of data as soon as possible. - * By default - just sets the flag is_cancelled and asks that all children be interrupted. - * This function can be called several times, including simultaneously from different threads. - * Have two modes: - * with kill = false only is_cancelled is set - streams will stop silently with returning some processed data. - * with kill = true also is_killed set - queries will stop with exception. - */ - virtual void cancel(bool kill); - - bool isCancelled() const; - bool isCancelledOrThrowIfKilled() const; - - /** Set limitations that checked on each block. */ - virtual void setLimits(const StreamLocalLimits & limits_) - { - limits = limits_; - } - - const StreamLocalLimits & getLimits() const - { - return limits; - } - - /** Set the quota. If you set a quota on the amount of raw data, - * then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. - */ - virtual void setQuota(const std::shared_ptr & new_quota) - { - quota = new_quota; - } - - /// If true, columns with same name and type may have different representation (normal, const, sparse) - /// in different blocks, that was read during stream execution. - virtual bool columnsCanDifferInRepresentationAmongBlocks() const { return false; } - - /// Enable calculation of minimums and maximums by the result columns. - void enableExtremes() { enabled_extremes = true; } - -protected: - /// Order is important: `table_locks` must be destroyed after `children` so that tables from - /// which child streams read are protected by the locks during the lifetime of the child streams. - std::vector table_locks; - - BlockInputStreams children; - std::shared_mutex children_mutex; - - BlockStreamProfileInfo info; - std::atomic is_cancelled{false}; - std::atomic is_killed{false}; - ProgressCallback progress_callback; - QueryStatus * process_list_elem = nullptr; - /// According to total_stopwatch in microseconds - UInt64 last_profile_events_update_time = 0; - - /// Additional information that can be generated during the work process. - - /// Total values during aggregation. - Block totals; - /// Minimums and maximums. The first row of the block - minimums, the second - the maximums. - Block extremes; - - - void addChild(const BlockInputStreamPtr & child) - { - std::unique_lock lock(children_mutex); - children.push_back(child); - } - - /** Check limits. - * But only those that can be checked within each separate stream. - */ - bool checkTimeLimit() const; - -#ifndef NDEBUG - bool read_prefix_is_called = false; - bool read_suffix_is_called = false; -#endif - -private: - bool enabled_extremes = false; - - /// The limit on the number of rows/bytes has been exceeded, and you need to stop execution on the next `read` call, as if the thread has run out. - bool limit_exceeded_need_break = false; - - /// Limitations and quotas. - - StreamLocalLimits limits; - - std::shared_ptr quota; /// If nullptr - the quota is not used. - UInt64 prev_elapsed = 0; - - /// The approximate total number of rows to read. For progress bar. - size_t total_rows_approx = 0; - - /// Derived classes must implement this function. - virtual Block readImpl() = 0; - - /// Here you can do a preliminary initialization. - virtual void readPrefixImpl() {} - - /// Here you need to do a finalization, which can lead to an exception. - virtual void readSuffixImpl() {} - - void updateExtremes(Block & block); - - /** Check quotas. - * But only those that can be checked within each separate stream. - */ - void checkQuota(Block & block); - - size_t checkDepthImpl(size_t max_depth, size_t level) const; - - template - void forEachChild(F && f) - { - /// NOTE: Acquire a read lock, therefore f() should be thread safe - std::shared_lock lock(children_mutex); - - // Reduce lock scope and avoid recursive locking since that is undefined for shared_mutex. - const auto children_copy = children; - lock.unlock(); - - for (auto & child : children_copy) - if (f(*child)) - return; - } - -}; - -} diff --git a/src/DataStreams/materializeBlock.cpp b/src/DataStreams/materializeBlock.cpp deleted file mode 100644 index 7c8756be1a1..00000000000 --- a/src/DataStreams/materializeBlock.cpp +++ /dev/null @@ -1,29 +0,0 @@ -#include -#include - -namespace DB -{ - -Block materializeBlock(const Block & block) -{ - if (!block) - return block; - - Block res = block; - size_t columns = res.columns(); - for (size_t i = 0; i < columns; ++i) - { - auto & element = res.getByPosition(i); - element.column = recursiveRemoveSparse(element.column->convertToFullColumnIfConst()); - } - - return res; -} - -void materializeBlockInplace(Block & block) -{ - for (size_t i = 0; i < block.columns(); ++i) - block.getByPosition(i).column = recursiveRemoveSparse(block.getByPosition(i).column->convertToFullColumnIfConst()); -} - -} diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 5176e68f3ca..9b683b6b0aa 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -17,7 +17,6 @@ namespace DB namespace ErrorCodes { - extern const int INCORRECT_DATA; extern const int CORRUPTED_DATA; } diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index 6cc70496599..378bed2af53 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -57,6 +57,7 @@ void SerializationInfoTuple::replaceData(const SerializationInfo & other) for (size_t i = 0; i < elems.size(); ++i) elems[i]->replaceData(*info_tuple.elems[i]); } + MutableSerializationInfoPtr SerializationInfoTuple::clone() const { MutableSerializationInfos elems_cloned; @@ -70,7 +71,6 @@ MutableSerializationInfoPtr SerializationInfoTuple::clone() const void SerializationInfoTuple::serialializeKindBinary(WriteBuffer & out) const { SerializationInfo::serialializeKindBinary(out); - for (const auto & elem : elems) elem->serialializeKindBinary(out); } @@ -78,7 +78,6 @@ void SerializationInfoTuple::serialializeKindBinary(WriteBuffer & out) const void SerializationInfoTuple::deserializeFromKindsBinary(ReadBuffer & in) { SerializationInfo::deserializeFromKindsBinary(in); - for (const auto & elem : elems) elem->deserializeFromKindsBinary(in); } diff --git a/src/Formats/NativeReader.cpp b/src/Formats/NativeReader.cpp index 112762277de..b8cddc233dd 100644 --- a/src/Formats/NativeReader.cpp +++ b/src/Formats/NativeReader.cpp @@ -13,8 +13,6 @@ #include #include -#include - namespace DB { diff --git a/src/Processors/Executors/PipelineExecutingBlockInputStream.h b/src/Processors/Executors/PipelineExecutingBlockInputStream.h deleted file mode 100644 index adfabc7ad4e..00000000000 --- a/src/Processors/Executors/PipelineExecutingBlockInputStream.h +++ /dev/null @@ -1,45 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -class QueryPipeline; -class PullingAsyncPipelineExecutor; -class PullingPipelineExecutor; - -/// Implement IBlockInputStream from QueryPipeline. -/// It's a temporary wrapper. -class PipelineExecutingBlockInputStream : public IBlockInputStream -{ -public: - explicit PipelineExecutingBlockInputStream(QueryPipeline pipeline_); - ~PipelineExecutingBlockInputStream() override; - - String getName() const override { return "PipelineExecuting"; } - Block getHeader() const override; - bool columnsCanDifferInRepresentationAmongBlocks() const override { return true; } - - void cancel(bool kill) override; - - /// Implement IBlockInputStream methods via QueryPipeline. - void setProgressCallback(const ProgressCallback & callback) final; - void setProcessListElement(QueryStatus * elem) final; - void setLimits(const StreamLocalLimits & limits_) final; - void setQuota(const std::shared_ptr & quota_) final; - -protected: - void readPrefixImpl() override; - Block readImpl() override; - -private: - std::unique_ptr pipeline; - /// One of executors is used. - std::unique_ptr executor; /// for single thread. - std::unique_ptr async_executor; /// for many threads. - bool is_execution_started = false; - - void createExecutor(); -}; - -} diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 8c0648fe40a..364fb8e1958 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -4,8 +4,6 @@ #include #include -#include - namespace DB { diff --git a/src/QueryPipeline/ConvertingBlockInputStream.h b/src/QueryPipeline/ConvertingBlockInputStream.h deleted file mode 100644 index 0e5d183ee32..00000000000 --- a/src/QueryPipeline/ConvertingBlockInputStream.h +++ /dev/null @@ -1,53 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/** Convert one block structure to another: - * - * Leaves only necessary columns; - * - * Columns are searched in source first by name; - * and if there is no column with same name, then by position. - * - * Converting types of matching columns (with CAST function). - * - * Materializing columns which are const in source and non-const in result, - * throw if they are const in result and non const in source, - * or if they are const and have different values. - */ -class ConvertingBlockInputStream : public IBlockInputStream -{ -public: - enum class MatchColumnsMode - { - /// Require same number of columns in source and result. Match columns by corresponding positions, regardless to names. - Position, - /// Find columns in source by their names. Allow excessive columns in source. - Name - }; - - ConvertingBlockInputStream( - const BlockInputStreamPtr & input, - const Block & result_header, - MatchColumnsMode mode); - - String getName() const override { return "Converting"; } - Block getHeader() const override { return header; } - bool columnsCanDifferInRepresentationAmongBlocks() const override { return true; } - -private: - Block readImpl() override; - - Block header; - - /// How to construct result block. Position in source block, where to get each column. - using Conversion = std::vector; - Conversion conversion; -}; - -} diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 3c82184af48..424fda31417 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -400,9 +400,6 @@ public: /// part creation (using alter query with materialize_ttl setting). bool checkAllTTLCalculated(const StorageMetadataPtr & metadata_snapshot) const; - // /// Returns serialization for column according to serialization_info. - // SerializationPtr getSerializationForColumn(const NameAndTypePair & column) const; - /// Return some uniq string for file /// Required for distinguish different copies of the same part on S3 String getUniqueId() const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index f8f19cdb3f3..4fada844cae 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 090f2c28508..80c8e7165f8 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 297c0397642..50974ca2009 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -17,8 +17,6 @@ #include #include -#include - #include #include diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index da55cd3df12..d26692a0d93 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -39,8 +39,8 @@ def create_table(cluster, table_name, additional_settings=None): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 def random_string(length): diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index a59540a22df..e99adf01ec5 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -67,8 +67,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 def random_string(length): From a7f219b3aacf9291e3c1921afb01ddf43ce01463 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 8 Nov 2021 21:24:38 +0300 Subject: [PATCH 071/358] better sampling in ColumnSparse and fix alter rename column --- src/Columns/IColumnImpl.h | 32 +++++++++++++------ .../MergeTree/MergeTreeDataMergerMutator.cpp | 5 +++ .../test_merge_tree_s3_failover/test.py | 2 +- tests/integration/test_partition/test.py | 6 ++-- .../test.py | 2 +- .../test_replicated_merge_tree_s3/test.py | 4 +-- .../test.py | 4 +-- 7 files changed, 35 insertions(+), 20 deletions(-) diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 5b35171f2dd..1be52087d11 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -149,22 +149,34 @@ double IColumn::getRatioOfDefaultRowsImpl(double sample_ratio) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Value of 'sample_ratio' must be in interval (0.0; 1.0], but got: {}", sample_ratio); + /// Randomize a little to avoid boundary effects. + std::uniform_int_distribution dist(1, static_cast(1.0 / sample_ratio)); + size_t num_rows = size(); size_t num_sampled_rows = static_cast(num_rows * sample_ratio); - if (num_sampled_rows == 0) - return 0.0; - - size_t step = num_rows / num_sampled_rows; - std::uniform_int_distribution dist(1, step); - + size_t num_checked_rows = dist(thread_local_rng); + num_sampled_rows = std::min(num_sampled_rows + dist(thread_local_rng), num_rows); size_t res = 0; - for (size_t i = 0; i < num_rows; i += step) + + if (num_sampled_rows == num_rows) { - size_t idx = std::min(i + dist(thread_local_rng), num_rows - 1); - res += static_cast(*this).isDefaultAt(idx); + for (size_t i = 0; i < num_rows; ++i) + res += static_cast(*this).isDefaultAt(i); + num_checked_rows = num_rows; + } + else if (num_sampled_rows != 0) + { + for (size_t i = num_checked_rows; i < num_rows; ++i) + { + if (num_checked_rows * num_rows <= i * num_sampled_rows) + { + res += static_cast(*this).isDefaultAt(i); + ++num_checked_rows; + } + } } - return static_cast(res) / num_sampled_rows; + return static_cast(res) / num_checked_rows; } template diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8c8775b9444..f08a70909f0 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -684,10 +684,15 @@ MergeTreeDataMergerMutator::getColumnsForNewDataPart( NameSet removed_columns; NameToNameMap renamed_columns_to_from; NameToNameMap renamed_columns_from_to; + ColumnsDescription part_columns(source_part->getColumns()); /// All commands are validated in AlterCommand so we don't care about order for (const auto & command : commands_for_removes) { + /// If we don't have this column in source part, than we don't need to materialize it + if (!part_columns.has(command.column_name)) + continue; + if (command.type == MutationCommand::DROP_COLUMN) removed_columns.insert(command.column_name); diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 9f00afc8f1f..b6b47417523 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -72,7 +72,7 @@ def drop_table(cluster): # S3 request will be failed for an appropriate part file write. -FILES_PER_PART_BASE = 6 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt, serialization.txt +FILES_PER_PART_BASE = 5 # partition.dat, default_compression_codec.txt, count.txt, columns.txt, checksums.txt FILES_PER_PART_WIDE = FILES_PER_PART_BASE + 1 + 1 + 3 * 2 # Primary index, MinMax, Mark and data file for column(s) # In debug build there are additional requests (from MergeTreeDataPartWriterWide.cpp:554 due to additional validation). diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index dc47d47c131..b43c85a4d48 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -61,8 +61,8 @@ def partition_complex_assert_checksums(): "13cae8e658e0ca4f75c56b1fc424e150\tshadow/1/data/test/partition_complex/19700102_2_2_0/minmax_p.idx\n" \ "25daad3d9e60b45043a70c4ab7d3b1c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/partition.dat\n" \ "3726312af62aec86b64a7708d5751787\tshadow/1/data/test/partition_complex/19700201_1_1_0/partition.dat\n" \ + "37855b06a39b79a67ea4e86e4a3299aa\tshadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt\n" \ "38e62ff37e1e5064e9a3f605dfe09d13\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.bin\n" \ - "43d32c3316e810e0231ee4f93dbf2875\tshadow/1/data/test/partition_complex/19700102_2_2_0/checksums.txt\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/k.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700102_2_2_0/v1.mrk\n" \ @@ -70,19 +70,17 @@ def partition_complex_assert_checksums(): "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.mrk\n" \ "4ae71336e44bf9bf79d2752e234818a5\tshadow/1/data/test/partition_complex/19700201_1_1_0/v1.mrk\n" \ "55a54008ad1ba589aa210d2629c1df41\tshadow/1/data/test/partition_complex/19700201_1_1_0/primary.idx\n" \ + "5f087cb3e7071bf9407e095821e2af8f\tshadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt\n" \ "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700102_2_2_0/columns.txt\n" \ "77d5af402ada101574f4da114f242e02\tshadow/1/data/test/partition_complex/19700201_1_1_0/columns.txt\n" \ "88cdc31ded355e7572d68d8cde525d3a\tshadow/1/data/test/partition_complex/19700201_1_1_0/p.bin\n" \ "9e688c58a5487b8eaf69c9e1005ad0bf\tshadow/1/data/test/partition_complex/19700102_2_2_0/primary.idx\n" \ - "b0f1c38fe8a3e0b38d75f4d9c142bc45\tshadow/1/data/test/partition_complex/19700201_1_1_0/checksums.txt\n" \ "c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700102_2_2_0/default_compression_codec.txt\n" \ "c0904274faa8f3f06f35666cc9c5bd2f\tshadow/1/data/test/partition_complex/19700201_1_1_0/default_compression_codec.txt\n" \ "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700102_2_2_0/count.txt\n" \ "c4ca4238a0b923820dcc509a6f75849b\tshadow/1/data/test/partition_complex/19700201_1_1_0/count.txt\n" \ "cfcb770c3ecd0990dcceb1bde129e6c6\tshadow/1/data/test/partition_complex/19700102_2_2_0/p.bin\n" \ "e2af3bef1fd129aea73a890ede1e7a30\tshadow/1/data/test/partition_complex/19700201_1_1_0/k.bin\n" \ - "e8490b8552f8b9b774db2f9eb1c90349\tshadow/1/data/test/partition_complex/19700102_2_2_0/serialization.txt\n" \ - "e8490b8552f8b9b774db2f9eb1c90349\tshadow/1/data/test/partition_complex/19700201_1_1_0/serialization.txt\n" \ "f2312862cc01adf34a93151377be2ddf\tshadow/1/data/test/partition_complex/19700201_1_1_0/minmax_p.idx\n" assert TSV(instance.exec_in_container(cmd).replace(' ', '\t')) == TSV(checksums) diff --git a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py index 89380f67ba7..f557a69569a 100644 --- a/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_hdfs_zero_copy/test.py @@ -9,7 +9,7 @@ from pyhdfs import HdfsClient SHARDS = 2 FILES_OVERHEAD_PER_TABLE = 1 # format_version.txt -FILES_OVERHEAD_PER_PART_COMPACT = 8 +FILES_OVERHEAD_PER_PART_COMPACT = 7 def wait_for_hdfs_objects(cluster, fp, expected, num_tries=30): diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 658fe80393b..d04bdae36e2 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -29,8 +29,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 def random_string(length): diff --git a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py index 6d00a94733b..793abc53566 100644 --- a/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_zero_copy/test.py @@ -32,8 +32,8 @@ def cluster(): FILES_OVERHEAD = 1 FILES_OVERHEAD_PER_COLUMN = 2 # Data and mark files -FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 + 1 -FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 + 1 +FILES_OVERHEAD_PER_PART_WIDE = FILES_OVERHEAD_PER_COLUMN * 3 + 2 + 6 + 1 +FILES_OVERHEAD_PER_PART_COMPACT = 10 + 1 def random_string(length): From 8b95b7d2713594ac5265d7648b9c7fc9ad3a2afb Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 21:40:10 +0300 Subject: [PATCH 072/358] init commit with parsing and BAD realisation --- .../data/system/asynchronous_metric_log | 1 + programs/server/data/system/crash_log | 1 + programs/server/data/system/metric_log | 1 + programs/server/data/system/part_log | 1 + programs/server/data/system/query_log | 1 + programs/server/data/system/query_thread_log | 1 + programs/server/data/system/text_log | 1 + programs/server/data/system/trace_log | 1 + programs/server/data/test/hits | 1 + programs/server/data/test/visits | 1 + programs/server/metadata/system | 1 + programs/server/metadata/test | 1 + .../201403_1_6_1/checksums.txt | Bin 0 -> 14305 bytes .../201403_1_6_1/columns.txt | 183 ++++++++++++ .../201403_1_6_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../201403_1_6_1/minmax_StartDate.idx | 1 + .../201403_1_6_1/partition.dat | Bin 0 -> 4 bytes .../201403_1_6_1/primary.idx | Bin 0 -> 3240 bytes .../201403_7_9_1/checksums.txt | Bin 0 -> 14273 bytes .../201403_7_9_1/columns.txt | 183 ++++++++++++ .../201403_7_9_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../201403_7_9_1/minmax_StartDate.idx | 1 + .../201403_7_9_1/partition.dat | Bin 0 -> 4 bytes .../201403_7_9_1/primary.idx | Bin 0 -> 1314 bytes .../format_version.txt | 1 + .../202010_1_28_6/checksums.txt | Bin 0 -> 262 bytes .../202010_1_28_6/columns.txt | 259 +++++++++++++++++ .../202010_1_28_6/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202010_1_28_6/minmax_event_date.idx | 1 + .../202010_1_28_6/partition.dat | Bin 0 -> 4 bytes .../202010_1_28_6/primary.idx | 1 + .../202012_29_2358_466/checksums.txt | Bin 0 -> 11371 bytes .../202012_29_2358_466/columns.txt | 259 +++++++++++++++++ .../202012_29_2358_466/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_29_2358_466/minmax_event_date.idx | 1 + .../202012_29_2358_466/partition.dat | Bin 0 -> 4 bytes .../202012_29_2358_466/primary.idx | 1 + .../202101_2359_3529_235/checksums.txt | Bin 0 -> 11145 bytes .../202101_2359_3529_235/columns.txt | 259 +++++++++++++++++ .../202101_2359_3529_235/count.txt | 1 + .../default_compression_codec.txt | 1 + .../minmax_event_date.idx | 1 + .../202101_2359_3529_235/partition.dat | Bin 0 -> 4 bytes .../202101_2359_3529_235/primary.idx | Bin 0 -> 18 bytes .../202102_3530_4606_216/checksums.txt | Bin 0 -> 10969 bytes .../202102_3530_4606_216/columns.txt | 259 +++++++++++++++++ .../202102_3530_4606_216/count.txt | 1 + .../default_compression_codec.txt | 1 + .../minmax_event_date.idx | 1 + .../202102_3530_4606_216/partition.dat | Bin 0 -> 4 bytes .../202102_3530_4606_216/primary.idx | 1 + .../202102_4607_4607_0/checksums.txt | Bin 0 -> 260 bytes .../202102_4607_4607_0/columns.txt | 259 +++++++++++++++++ .../202102_4607_4607_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_4607_4607_0/minmax_event_date.idx | 1 + .../202102_4607_4607_0/partition.dat | Bin 0 -> 4 bytes .../202102_4607_4607_0/primary.idx | 1 + .../202102_4608_4608_0/checksums.txt | Bin 0 -> 260 bytes .../202102_4608_4608_0/columns.txt | 259 +++++++++++++++++ .../202102_4608_4608_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_4608_4608_0/minmax_event_date.idx | 1 + .../202102_4608_4608_0/partition.dat | Bin 0 -> 4 bytes .../202102_4608_4608_0/primary.idx | 1 + .../format_version.txt | 1 + .../hits.sql | 141 ++++++++++ .../visits.sql | 189 +++++++++++++ .../202010_1_5_1/checksums.txt | Bin 0 -> 260 bytes .../202010_1_5_1/columns.txt | 47 ++++ .../202010_1_5_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202010_1_5_1/minmax_event_date.idx | 1 + .../202010_1_5_1/partition.dat | Bin 0 -> 4 bytes .../202010_1_5_1/primary.idx | 1 + .../202012_6_47_9/checksums.txt | Bin 0 -> 262 bytes .../202012_6_47_9/columns.txt | 47 ++++ .../202012_6_47_9/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_6_47_9/minmax_event_date.idx | 1 + .../202012_6_47_9/partition.dat | Bin 0 -> 4 bytes .../202012_6_47_9/primary.idx | 1 + .../202101_48_74_6/checksums.txt | Bin 0 -> 261 bytes .../202101_48_74_6/columns.txt | 47 ++++ .../202101_48_74_6/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202101_48_74_6/minmax_event_date.idx | 1 + .../202101_48_74_6/partition.dat | Bin 0 -> 4 bytes .../202101_48_74_6/primary.idx | 1 + .../202102_75_79_1/checksums.txt | Bin 0 -> 260 bytes .../202102_75_79_1/columns.txt | 47 ++++ .../202102_75_79_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_75_79_1/minmax_event_date.idx | 1 + .../202102_75_79_1/partition.dat | Bin 0 -> 4 bytes .../202102_75_79_1/primary.idx | 1 + .../202102_80_80_0/checksums.txt | Bin 0 -> 260 bytes .../202102_80_80_0/columns.txt | 47 ++++ .../202102_80_80_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_80_80_0/minmax_event_date.idx | 1 + .../202102_80_80_0/partition.dat | Bin 0 -> 4 bytes .../202102_80_80_0/primary.idx | 1 + .../format_version.txt | 1 + .../all_1_1_0/checksums.txt | Bin 0 -> 188 bytes .../all_1_1_0/columns.txt | 13 + .../all_1_1_0/count.txt | 1 + .../all_1_1_0/default_compression_codec.txt | 1 + .../all_1_1_0/primary.idx | 1 + .../all_2_2_0/checksums.txt | Bin 0 -> 188 bytes .../all_2_2_0/columns.txt | 13 + .../all_2_2_0/count.txt | 1 + .../all_2_2_0/default_compression_codec.txt | 1 + .../all_2_2_0/primary.idx | 1 + .../format_version.txt | 1 + .../202012_1_18_4/checksums.txt | Bin 0 -> 260 bytes .../202012_1_18_4/columns.txt | 20 ++ .../202012_1_18_4/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_1_18_4/minmax_event_date.idx | 1 + .../202012_1_18_4/partition.dat | Bin 0 -> 4 bytes .../202012_1_18_4/primary.idx | 1 + .../202101_19_22_1/checksums.txt | Bin 0 -> 260 bytes .../202101_19_22_1/columns.txt | 20 ++ .../202101_19_22_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202101_19_22_1/minmax_event_date.idx | 1 + .../202101_19_22_1/partition.dat | Bin 0 -> 4 bytes .../202101_19_22_1/primary.idx | 1 + .../format_version.txt | 1 + .../201403_1_29_2/checksums.txt | Bin 0 -> 10792 bytes .../201403_1_29_2/columns.txt | 135 +++++++++ .../201403_1_29_2/count.txt | 1 + .../default_compression_codec.txt | 1 + .../201403_1_29_2/minmax_EventDate.idx | 1 + .../201403_1_29_2/partition.dat | Bin 0 -> 4 bytes .../201403_1_29_2/primary.idx | Bin 0 -> 15330 bytes .../format_version.txt | 1 + .../202010_1_4_1/checksums.txt | Bin 0 -> 260 bytes .../202010_1_4_1/columns.txt | 12 + .../202010_1_4_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202010_1_4_1/minmax_event_date.idx | 1 + .../202010_1_4_1/partition.dat | Bin 0 -> 4 bytes .../202010_1_4_1/primary.idx | 1 + .../202012_35_35_0/checksums.txt | Bin 0 -> 260 bytes .../202012_35_35_0/columns.txt | 12 + .../202012_35_35_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_35_35_0/minmax_event_date.idx | 1 + .../202012_35_35_0/partition.dat | Bin 0 -> 4 bytes .../202012_35_35_0/primary.idx | 1 + .../202012_5_34_6/checksums.txt | Bin 0 -> 261 bytes .../202012_5_34_6/columns.txt | 12 + .../202012_5_34_6/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_5_34_6/minmax_event_date.idx | 1 + .../202012_5_34_6/partition.dat | Bin 0 -> 4 bytes .../202012_5_34_6/primary.idx | 1 + .../202101_36_115_17/checksums.txt | Bin 0 -> 262 bytes .../202101_36_115_17/columns.txt | 12 + .../202101_36_115_17/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202101_36_115_17/minmax_event_date.idx | 1 + .../202101_36_115_17/partition.dat | Bin 0 -> 4 bytes .../202101_36_115_17/primary.idx | 1 + .../202102_116_127_3/checksums.txt | Bin 0 -> 261 bytes .../202102_116_127_3/columns.txt | 12 + .../202102_116_127_3/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_116_127_3/minmax_event_date.idx | 1 + .../202102_116_127_3/partition.dat | Bin 0 -> 4 bytes .../202102_116_127_3/primary.idx | 1 + .../202102_128_128_0/checksums.txt | Bin 0 -> 260 bytes .../202102_128_128_0/columns.txt | 12 + .../202102_128_128_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_128_128_0/minmax_event_date.idx | 1 + .../202102_128_128_0/partition.dat | Bin 0 -> 4 bytes .../202102_128_128_0/primary.idx | 1 + .../202102_129_129_0/checksums.txt | Bin 0 -> 261 bytes .../202102_129_129_0/columns.txt | 12 + .../202102_129_129_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_129_129_0/minmax_event_date.idx | 1 + .../202102_129_129_0/partition.dat | Bin 0 -> 4 bytes .../202102_129_129_0/primary.idx | 1 + .../format_version.txt | 1 + .../202010_1_5_1/checksums.txt | Bin 0 -> 260 bytes .../202010_1_5_1/columns.txt | 41 +++ .../202010_1_5_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202010_1_5_1/minmax_event_date.idx | 1 + .../202010_1_5_1/partition.dat | Bin 0 -> 4 bytes .../202010_1_5_1/primary.idx | 1 + .../202012_6_45_8/checksums.txt | Bin 0 -> 262 bytes .../202012_6_45_8/columns.txt | 41 +++ .../202012_6_45_8/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_6_45_8/minmax_event_date.idx | 1 + .../202012_6_45_8/partition.dat | Bin 0 -> 4 bytes .../202012_6_45_8/primary.idx | 1 + .../202101_46_72_6/checksums.txt | Bin 0 -> 262 bytes .../202101_46_72_6/columns.txt | 41 +++ .../202101_46_72_6/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202101_46_72_6/minmax_event_date.idx | 1 + .../202101_46_72_6/partition.dat | Bin 0 -> 4 bytes .../202101_46_72_6/primary.idx | 1 + .../202102_73_77_1/checksums.txt | Bin 0 -> 261 bytes .../202102_73_77_1/columns.txt | 41 +++ .../202102_73_77_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_73_77_1/minmax_event_date.idx | 1 + .../202102_73_77_1/partition.dat | Bin 0 -> 4 bytes .../202102_73_77_1/primary.idx | 1 + .../202102_78_78_0/checksums.txt | Bin 0 -> 260 bytes .../202102_78_78_0/columns.txt | 41 +++ .../202102_78_78_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_78_78_0/minmax_event_date.idx | 1 + .../202102_78_78_0/partition.dat | Bin 0 -> 4 bytes .../202102_78_78_0/primary.idx | 1 + .../format_version.txt | 1 + .../202010_1_29_6/checksums.txt | Bin 0 -> 262 bytes .../202010_1_29_6/columns.txt | 15 + .../202010_1_29_6/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202010_1_29_6/minmax_event_date.idx | 1 + .../202010_1_29_6/partition.dat | Bin 0 -> 4 bytes .../202010_1_29_6/primary.idx | 1 + .../202012_2358_2361_1/checksums.txt | Bin 0 -> 260 bytes .../202012_2358_2361_1/columns.txt | 15 + .../202012_2358_2361_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_2358_2361_1/minmax_event_date.idx | 1 + .../202012_2358_2361_1/partition.dat | Bin 0 -> 4 bytes .../202012_2358_2361_1/primary.idx | 1 + .../202012_30_2357_1895/checksums.txt | Bin 0 -> 263 bytes .../202012_30_2357_1895/columns.txt | 15 + .../202012_30_2357_1895/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_30_2357_1895/minmax_event_date.idx | 1 + .../202012_30_2357_1895/partition.dat | Bin 0 -> 4 bytes .../202012_30_2357_1895/primary.idx | 1 + .../202101_2362_3536_470/checksums.txt | Bin 0 -> 263 bytes .../202101_2362_3536_470/columns.txt | 15 + .../202101_2362_3536_470/count.txt | 1 + .../default_compression_codec.txt | 1 + .../minmax_event_date.idx | 1 + .../202101_2362_3536_470/partition.dat | Bin 0 -> 4 bytes .../202101_2362_3536_470/primary.idx | Bin 0 -> 24 bytes .../202101_3537_3539_1/checksums.txt | Bin 0 -> 260 bytes .../202101_3537_3539_1/columns.txt | 15 + .../202101_3537_3539_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202101_3537_3539_1/minmax_event_date.idx | 1 + .../202101_3537_3539_1/partition.dat | Bin 0 -> 4 bytes .../202101_3537_3539_1/primary.idx | 1 + .../202102_3540_4637_415/checksums.txt | Bin 0 -> 263 bytes .../202102_3540_4637_415/columns.txt | 15 + .../202102_3540_4637_415/count.txt | 1 + .../default_compression_codec.txt | 1 + .../minmax_event_date.idx | 1 + .../202102_3540_4637_415/partition.dat | Bin 0 -> 4 bytes .../202102_3540_4637_415/primary.idx | 1 + .../202102_4638_4638_0/checksums.txt | Bin 0 -> 260 bytes .../202102_4638_4638_0/columns.txt | 15 + .../202102_4638_4638_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_4638_4638_0/minmax_event_date.idx | 1 + .../202102_4638_4638_0/partition.dat | Bin 0 -> 4 bytes .../202102_4638_4638_0/primary.idx | 1 + .../format_version.txt | 1 + .../asynchronous_metric_log.sql | 12 + .../crash_log.sql | 17 ++ .../metric_log.sql | 264 ++++++++++++++++++ .../part_log.sql | 25 ++ .../query_log.sql | 52 ++++ .../query_thread_log.sql | 46 +++ .../text_log.sql | 20 ++ .../trace_log.sql | 17 ++ .../202010_1_3_1/checksums.txt | Bin 0 -> 260 bytes .../202010_1_3_1/columns.txt | 7 + .../202010_1_3_1/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202010_1_3_1/minmax_event_date.idx | 1 + .../202010_1_3_1/partition.dat | Bin 0 -> 4 bytes .../202010_1_3_1/primary.idx | 1 + .../202012_4_295_59/checksums.txt | Bin 0 -> 262 bytes .../202012_4_295_59/columns.txt | 7 + .../202012_4_295_59/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202012_4_295_59/minmax_event_date.idx | 1 + .../202012_4_295_59/partition.dat | Bin 0 -> 4 bytes .../202012_4_295_59/primary.idx | 1 + .../202101_296_439_30/checksums.txt | Bin 0 -> 262 bytes .../202101_296_439_30/columns.txt | 7 + .../202101_296_439_30/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202101_296_439_30/minmax_event_date.idx | 1 + .../202101_296_439_30/partition.dat | Bin 0 -> 4 bytes .../202101_296_439_30/primary.idx | 1 + .../202102_440_574_27/checksums.txt | Bin 0 -> 262 bytes .../202102_440_574_27/columns.txt | 7 + .../202102_440_574_27/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_440_574_27/minmax_event_date.idx | 1 + .../202102_440_574_27/partition.dat | Bin 0 -> 4 bytes .../202102_440_574_27/primary.idx | 1 + .../202102_575_575_0/checksums.txt | Bin 0 -> 260 bytes .../202102_575_575_0/columns.txt | 7 + .../202102_575_575_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_575_575_0/minmax_event_date.idx | 1 + .../202102_575_575_0/partition.dat | Bin 0 -> 4 bytes .../202102_575_575_0/primary.idx | 1 + .../202102_576_576_0/checksums.txt | Bin 0 -> 260 bytes .../202102_576_576_0/columns.txt | 7 + .../202102_576_576_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_576_576_0/minmax_event_date.idx | 1 + .../202102_576_576_0/partition.dat | Bin 0 -> 4 bytes .../202102_576_576_0/primary.idx | 1 + .../202102_577_577_0/checksums.txt | Bin 0 -> 260 bytes .../202102_577_577_0/columns.txt | 7 + .../202102_577_577_0/count.txt | 1 + .../default_compression_codec.txt | 1 + .../202102_577_577_0/minmax_event_date.idx | 1 + .../202102_577_577_0/partition.dat | Bin 0 -> 4 bytes .../202102_577_577_0/primary.idx | 1 + .../format_version.txt | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 49 +++- src/Interpreters/InterpreterSelectQuery.h | 1 + src/Parsers/ASTSelectQuery.cpp | 3 + src/Parsers/ASTSelectQuery.h | 1 + src/Parsers/ParserSelectQuery.cpp | 11 +- src/Processors/QueryPlan/GroupingSetsStep.cpp | 46 +++ src/Processors/QueryPlan/GroupingSetsStep.h | 25 ++ .../Transforms/AggregatingTransform.h | 1 + src/Processors/Transforms/CubeTransform.cpp | 9 +- .../Transforms/GroupingSetsTransform.cpp | 81 ++++++ .../Transforms/GroupingSetsTransform.h | 35 +++ src/Processors/ya.make | 160 +++++++++++ 348 files changed, 4180 insertions(+), 15 deletions(-) create mode 120000 programs/server/data/system/asynchronous_metric_log create mode 120000 programs/server/data/system/crash_log create mode 120000 programs/server/data/system/metric_log create mode 120000 programs/server/data/system/part_log create mode 120000 programs/server/data/system/query_log create mode 120000 programs/server/data/system/query_thread_log create mode 120000 programs/server/data/system/text_log create mode 120000 programs/server/data/system/trace_log create mode 120000 programs/server/data/test/hits create mode 120000 programs/server/data/test/visits create mode 120000 programs/server/metadata/system create mode 120000 programs/server/metadata/test create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/checksums.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/partition.dat create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/primary.idx create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/checksums.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/partition.dat create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/primary.idx create mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/checksums.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/partition.dat create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/checksums.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/columns.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/count.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/default_compression_codec.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/minmax_event_date.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/partition.dat create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/primary.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/checksums.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/columns.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/count.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/default_compression_codec.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/minmax_event_date.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/partition.dat create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/primary.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/checksums.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/columns.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/count.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/default_compression_codec.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/minmax_event_date.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/partition.dat create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/primary.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/checksums.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/partition.dat create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/checksums.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/partition.dat create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx create mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt create mode 100644 programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql create mode 100644 programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/checksums.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/partition.dat create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/checksums.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/partition.dat create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/checksums.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/columns.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/count.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/default_compression_codec.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/minmax_event_date.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/partition.dat create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/primary.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/checksums.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/partition.dat create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/checksums.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/columns.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/count.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/default_compression_codec.txt create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/minmax_event_date.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/partition.dat create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/primary.idx create mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/format_version.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/checksums.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/checksums.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx create mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/checksums.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/partition.dat create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/checksums.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/partition.dat create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx create mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/checksums.txt create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/partition.dat create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/primary.idx create mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/checksums.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/partition.dat create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/checksums.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/partition.dat create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/checksums.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/partition.dat create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/checksums.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/partition.dat create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/checksums.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/partition.dat create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/checksums.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/partition.dat create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/checksums.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/columns.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/count.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/default_compression_codec.txt create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/minmax_event_date.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/partition.dat create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/primary.idx create mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/format_version.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/checksums.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/partition.dat create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/checksums.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/partition.dat create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/checksums.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/partition.dat create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/checksums.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/partition.dat create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/checksums.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/partition.dat create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx create mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/checksums.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/partition.dat create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/checksums.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/columns.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/count.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/default_compression_codec.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/minmax_event_date.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/partition.dat create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/primary.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/checksums.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/columns.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/count.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/default_compression_codec.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/minmax_event_date.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/partition.dat create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/primary.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/checksums.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/columns.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/count.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/default_compression_codec.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/minmax_event_date.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/partition.dat create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/primary.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/checksums.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/partition.dat create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/checksums.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/columns.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/count.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/default_compression_codec.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/minmax_event_date.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/partition.dat create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/primary.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/checksums.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/partition.dat create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx create mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql create mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/checksums.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/partition.dat create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/checksums.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/partition.dat create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/checksums.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/partition.dat create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/checksums.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/columns.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/count.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/default_compression_codec.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/minmax_event_date.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/partition.dat create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/primary.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/checksums.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/partition.dat create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/checksums.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/partition.dat create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/checksums.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/partition.dat create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx create mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt create mode 100644 src/Processors/QueryPlan/GroupingSetsStep.cpp create mode 100644 src/Processors/QueryPlan/GroupingSetsStep.h create mode 100644 src/Processors/Transforms/GroupingSetsTransform.cpp create mode 100644 src/Processors/Transforms/GroupingSetsTransform.h create mode 100644 src/Processors/ya.make diff --git a/programs/server/data/system/asynchronous_metric_log b/programs/server/data/system/asynchronous_metric_log new file mode 120000 index 00000000000..e065d7bf424 --- /dev/null +++ b/programs/server/data/system/asynchronous_metric_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1 \ No newline at end of file diff --git a/programs/server/data/system/crash_log b/programs/server/data/system/crash_log new file mode 120000 index 00000000000..7c05d325fb8 --- /dev/null +++ b/programs/server/data/system/crash_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665 \ No newline at end of file diff --git a/programs/server/data/system/metric_log b/programs/server/data/system/metric_log new file mode 120000 index 00000000000..cf9f177d49f --- /dev/null +++ b/programs/server/data/system/metric_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0 \ No newline at end of file diff --git a/programs/server/data/system/part_log b/programs/server/data/system/part_log new file mode 120000 index 00000000000..b19efce9efd --- /dev/null +++ b/programs/server/data/system/part_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d \ No newline at end of file diff --git a/programs/server/data/system/query_log b/programs/server/data/system/query_log new file mode 120000 index 00000000000..4222415ce96 --- /dev/null +++ b/programs/server/data/system/query_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df \ No newline at end of file diff --git a/programs/server/data/system/query_thread_log b/programs/server/data/system/query_thread_log new file mode 120000 index 00000000000..6108d6eab87 --- /dev/null +++ b/programs/server/data/system/query_thread_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc \ No newline at end of file diff --git a/programs/server/data/system/text_log b/programs/server/data/system/text_log new file mode 120000 index 00000000000..4ae519959fe --- /dev/null +++ b/programs/server/data/system/text_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500 \ No newline at end of file diff --git a/programs/server/data/system/trace_log b/programs/server/data/system/trace_log new file mode 120000 index 00000000000..6dfdc4802b3 --- /dev/null +++ b/programs/server/data/system/trace_log @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a \ No newline at end of file diff --git a/programs/server/data/test/hits b/programs/server/data/test/hits new file mode 120000 index 00000000000..28b16b0be67 --- /dev/null +++ b/programs/server/data/test/hits @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64 \ No newline at end of file diff --git a/programs/server/data/test/visits b/programs/server/data/test/visits new file mode 120000 index 00000000000..49e67e1c13b --- /dev/null +++ b/programs/server/data/test/visits @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f \ No newline at end of file diff --git a/programs/server/metadata/system b/programs/server/metadata/system new file mode 120000 index 00000000000..4ff5d3274d6 --- /dev/null +++ b/programs/server/metadata/system @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681 \ No newline at end of file diff --git a/programs/server/metadata/test b/programs/server/metadata/test new file mode 120000 index 00000000000..b4ed8633adf --- /dev/null +++ b/programs/server/metadata/test @@ -0,0 +1 @@ +/home/max/github/ClickHouse/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080 \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/checksums.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..da44b12b708c635f7165a2c8b8437d5fd40cbb12 GIT binary patch literal 14305 zcmX|IcOaGj_kTaneJ-xOcVuLhl@W=oGD2h*nHM+X%9e~oMo5Svqe3!DMOId1WklJO zQpzaz+GPBm^7;P$z6V{1KAsMK2Ltdvz67Gz@q_Zz z?VqOPubs%3R#X^Fo*H1>j;uTez!g0JHV#K&7&M#%PILz-SGmo243KC&ecX>u2_S-^7rXgRoMniX%9$dE9AAjJQ^BMy58h(CwFJ}iYr*mMP=l9({B-tiDE1Y5JYY;_7 zk4cc?B$St(gyTDrKvAQ3?5TC9HG9&IxhK7vpiEqa0V& zehL99ErTYA%-t4EU-;_2!DnhiYwsSj%fq=EjZjhhBUnt3QiFrSas7IPPGO|8F%?c0 zz(#<~dA#s!_;tI3=I*%fo8r{Tt$E6aix6o zF9EFIU#-6SHR#^{{#Q4&2XnBHZ*YXNODHGgHxAc$BRm=afxW>JCVbrY;)5_(PTx?a zouvF09AK0N_UOHMdizD9){3NGm7V37OZa5mJS#VDpQ~-_0yg2=2d~R3wgJpEZF-7L_(4?;P_v#CGL8|8^GG> z*FCs!T>#pFySQfi;=5ZjSc7f!9cCgp4)oCI-S`xmKlqGZ5x`dHR;9%` zowhcRpTFFu--iX$)E$5vwTYe%1g}ukR(~_HPDAa$q4Axfwp6|I*%R}J=y5|a`ja2Z z`^S09;zc)|al-2dp#L4*m64nW)ZvDu0?I0NL0bYe%J9Kc0aqcY5g{Y%o!ASI!Df)k z?Cp=0k}8@XY&7jqV|{{=QjzgLIM$DJrqqzFrrBXc8Ne6OTkX3YP1w_Ooc1Z`@Vb2D zY9ojO&>E`*kAKoV+E{a%_Dra9w)gI|Bhb(x5}gkNpoZET>A<8uJyQLCERz+wCd5{5 zwA!z<}9rwX#<7DHU>jI8o-y9k9@tO^lQ;@B6@rhy8Q~fe(fISYF>-Z4; zy}MvCC7SUz9zy3PxW2&y6Ka2W{unW1EpNMkJvh%`9HY0>W7L18ui*ZV(|nzQsSuvOpv}&8D#x5CA=V)PU&f2*}(M$vBHQH<+*IyAF9EWg640)3>1aN$rSM z%1PNSZpT_jE#*gtwGQDK9081m#Cbr*oK-eixS4>ze@n2#v+#^b6W|g8Ukx36-0^-| z4t{u7qE9d$Fth_UxF4a@1ZYgsbpf~mp_T#+0L#PA`vL$6eeMJ3AS`?fMt@*3K7}xh zp>O2F`}a*Z0AL(~hyW-aFmz+tVNDo^`lkz^YXeM-Yry-gw;zfXzTNN=XU{WC@7=rN zk(aWJU%ybap}d`lsZ=7HygI- zwM$$01bm!#Rdz~lNl>J0>|O>KbXhj&vK}JGKmSg%l%q?2;bx3y@x@|+FhGK#CC-9} zn7PWkW1+EZ>`I?fC|G(vm_T3p&kiMih)y46_Feh1$CUJ``qu=6MVIA@TY+Szq`;Z$ zf<+5rsp2#f-^c8LmYa`*@81klnT1V0EiegdRZnYw)*{?<)(WUcKb%iL4r0^%LViyZ zuzDguMcE!8|7ENy@uWeK*ItS}YD(Jefm)A(AuVGM2PZq!)@&Q5=H3x5zWcqh#%s4# z9-mIC`Wtt3Yr!5&Gi2?w$cf1Kw>vw*(EsA@AcKi1L)DexlSjF}JrT}-3fhryV;=(1 zhu}Aa;e~l9DgU7i-7bt`jq^FYFVWxUEnu+6u%W-Be`9b4)aIsm54=+$B?-Cx2%i0| zX|^)d`0GNiTy>ZA1;IkfCFt~9vyJ6Q1sUtG#vr#4I~ylJiN>iD8VBiC`Qx1vadlDe z0sp^MxeeN9|r5&Dm zb#1aEKj_$(3hE`Ogk|IpgGJzUYNb)DQh7kZ`@j1_cef0g%zos!q@?;jjw!e*dMc>R z4+Aa&xaY51zQfet*0!In$awXr?a`RNo?fmMyae@TkDnY&a;Hv-*Qpav60PyW?7I>f zW0QRNbLmSpOwWfu*L?UUya4S2SmHN7XYiLVHC=-$ISxpQ_?i}O(|F0ZQ%6QYwLN;D zxELIFKhHYPQN(^btXpG<#Q}K$n*%bGf@C(i9J%*v^T@EO>0&?u8zhSYRtY7-L`BA4 z>wE+g=YDw8?T!*Ml#pzRTjO-! zhT6w>&gam5lL}I*1=usxWbCi}J?K&1J*BJKUAf0=f8g{RZ;4+-4bAC14asPGlcnq- zX!oP>;{0m>qX1)x8lnwNj9=p_d>G;b>)yYcEUOa|Niz|)bZ|H;xGyNIVzj_A(*zHgsJnX0KDFC z7enmv$-vgBH;lZ?R8t|Hfg{d!yg3;%3JRZwD9?=;b4> z^AlL14VX;2t>x{N5WRLZXvik+;}qk+m4|SC{lgd7O*6jEDw6#5^e>sqh zv39yH`7U|Td59{ju5EHw4$zs=3*vn~qj;4#$NrU&ZNX0XshpTF+yCKhkyI55^A~9O zo}LQ1ID74QxoqatQQm!+Cs7t2i+&mBNqzI8^#p~rjrSU$)%N#sz*7Qx^d7qJ--N;* z543KVy}FB(HeUL%8&P5fR$<-+qlKCKdxu@F4f^lMQmoMZyHXOxJ%?b6&p#&nH%25! zh~2jKfclh|^Dey1hx+ZO#9hWmGj`7lMN;p+HG5mb^7$1vM1IgE5@ySNjT0%0iYx)l zl~}36G^pR2qV)W|k)Fysoh`kR$N2nGy5ND$2mi5iy3D3#^Ybb#LU;B!5^J5d!%=e{ z4!&-N4wr~Nlqk=Tr(KZ zty+T>h~e0oxck;uT=ebwZ_S^QF}DVS1TWMRk@hW)S{CM4JsS5j1GWoV4lK%bTBGTq z-&w)%ahgG&sA9MiN1yJTHGuwEB4hf&*;4B=l3ebuV`73SZmodnD!_EYWSrdJa9__6 z#k)=Gu1cPtuIx{@3Uj>II*lW&W4$x1hR+q`uG0;&1iy zzAN<3|2DP^>T*x<7n2RW%3#ks%$rQm$Bi&Cg>`3yLnLx7Tn&813>PtSCD9a z&hsl=DPOFL3uRVUwwh)DtU7p<&O3PMg<@p!zJ3YRJ3LP&5-^Q?yZp3;aXMkEHZ|(W zWIZT3^A~ngErzRaojQEXK+TwARPG&{8Va~z2U8O5c4x|O;*LVDrLF2WH)_Bw4vc*X zSMV~O=yt0ghtzaknn~#r#wiE7dKA6f(BnfZ7~FG<5}mY6I{~lHx8u7CCU~^a3dRSJ z8Tq~)nBa`Aew5W+IhhGpdje)1qJzgdyn~Y)-dDq`l$neyK4F~V)_vRlhY;2qv^#j4 zbKLD5<`1l1{_-pNX^;NLdm`&tvPst>b6iDU=;-#+IVlj z1(b__$)fm_buk9cau_UsACphWUQKicJ1|CFqT{RN1^>aU8;z|j2D)r${zgkBDAQf1 z#Uf<~|CRjk%$+=z(;WcggDhc+mzX$C-o#aG-jtDHe`PLo;VU0@0}76A2{(4N_Me$L z+XPSd-Rl5gKSN?>-Bb?=HHyoP$7=-p{iYrO>^H$n-$O@FY5Dv3;JuvAQ}1!zxm?XF zt)L-x{kBT6&)VhPCQGPe{WYX%c|7jr#Ob>z(IT;LQ3R z|K9xVWt<9F0VvzWv7NT5VSai*xQ;JtV+=}O+=SO-ZEFIy zp31|`^GDB%nD;Hg0Oj)kqJ0Ts_%239*xTLTF9#2vscfh)hJgsMH^uu996Ta1WaNFL zu<{G{FO@qxdd^vrl9AGdmk&a#Q0m=b>dqg`OfRZWw9Uuas_X#Fzgt~`&0hbKp=V4? z&z+MLeljFu0(2mXFQ}nf5~ucwyfF2q0KXw8j}M+UVV}{*Fli%?aPq9|*zt`o>Ssv$ z_ZheTqr)qd!?>QoQ}tt-kMvn25~!mu09rJA=rpmBkdjh}Z6n%b)ggv2K)u$g5 zL^OLyxKU;uJ}Yqk=-k0CTF-wTru#=RNHXfoE;qf6pH@@CECTZ_^4BSQ_3)8yUv1&bxxTW{HI->1LHAsyn< z5SY}U0}Oh`MtHwKqK|tROvVmx)IpWiw*^hIwt*q9$RKl-%$x<-fMrSAUcq%XhS2tb z7K3wDsf;K^N9zE}=G~mGr67sWyNq4T$B|%nK&|KHM6_ceArtq7`@k1ewyLr8QhkO? zx>+l*i)u1+>=J!BTH$h;507TJa%F1J90jlm*u;crPiS4YIv>%Tqmf(wOYbY7+e9g2 zNv?ike%|3D?hG#J_j46g4Sph6$~D-x>|WLqUYTF^tsxSwa!~NUTAk{g%A?DVRqLg= zs8c58p5x;It)8zY(LailgnWE*KuP7o8H(S`9H~czsh+E(NtU1p9++|Qp^D?9DBI3a z&u`uSxA#w*MAwxtfV;Tu_8~A-`r6B@d+C5)&sUom=;c9laIS_)$cHVsfNDj~{l)LE zm%MhcrlBp7mT2&hz3C}veH4mrVLj!(^NPpqqa+DB%cE!f>LfM8dj_jWA_slMQ=n-` zbR-N@laQ|;ndj4w56WJP0;3KgtO-YvJkF(*uyXW(395OfYL7dAy=*?bgB z)MB;gF681kcd1e7>MY7gH4v?VaY7xV^mszq^5#9Yd|+?w;N^@*Us~lHHVb5X;ZRZZ z`-`)=i$KkJ`t`HA-GO z#)TY{M-MC!%P)wY!5pC1cL;D0r6nOZ+jx(Lv(=sMf*zC~n6!G*67snWJ#o@PiF>Kl zoWJb6oa7C4!t8%1j-yMn}Jr05AJLpF{Fbax6VCLJ>15~}l>a3U;R?=8 zoEV52jk|2sL;7-MlNRL@=BPqw2z$}7y?1%t^k~5Eljm#_;F)8)iz1IU!3Xc;_nK>u zqX9BAdU2M(aZgo!sqp!qrNwM0?}*TkWU&}Atok9V-;N8`XB>Go02aOs{$Qke8t>-Y zF<=^O4woD{HNXS7WdL%7b(;%a+$)IgcSZ8Bx^Dck@6&$VrP*!NQbzQ39^|tT&k3hD z!e`5KI$$6IRz#5H-&?mJiq_X0mG*wu@XbNg$b!2c`j#AD$4%_Jv^RWu(?LbP z!35%eZy{_fp8!apfH^SR93vDyp9Nlhkat~hkm-AHB^+r$-#ld2=)Wn~eEByKs^}cl zWMNeX3=V*ig&2g{VBe!}2c~wz1LX;N%HKOYhnF^?FY;^zKKs)TRA{$_J{>M^)0J}o z5SqA9n8V~5sD>>i-;(6n4yL?DIk^ki;(##GIi6#ebLg;q{+q~^N85CX#c5iL@|}y& z{Yt2~@1gIJ^YIGrFYjbl@cyi4|0m>Z!9l4b;Bq`c?Q3`z;VyaD!SjI=X zwXiL2d-=?J%@=g-FUFuIt93=U%cpKz#i}*5STPxBU*K>7FlxsYO@QxT9#dTHw4LnE zpX>e3_I4d`9R?95{tjM#1ixT_U1QI{^ti-I|NiB02=S@d271!Kz{8WX4HVx2xzT`Y@KeOKmq+@{VvBb6#hTWJDp)3eX*b*qDR?Fk=Pr z&=$uOBy->UwQERq%{ZpL?p=wbFBD5dk<60((=ajKU!9mz+!QawjFJ^p8-wT9suc%5 zOg$wo+7;*tdKrUj00UG16GzA$usc`1{x@Gx@A-Ip4EO4%<8kU`=q1NT@}BFpfWxlO zHiN0tRMP(^j$!n278l05?n~gSVw*W%=HLLsS4v@Hw<|0pObpA388-g7S*z;3cQ_N} zPnc3AiVfKBk-(*qk=b(x1Y>e%JRZBEEgQ5ivW}ro{yZ?@!^#tPXSBe4nHmFFg@K0Q z-+G#6hx>k-Kx)kxvwLfo2xl8h2}#%fDfH{}EvWG{Rq>TK(!X7zw+ELi+Ku<=>nerq z^!N1UFoPx@c8%yS-^8i_EsEl;Oe7>@g0fzpU7XvK?sea@Lo9LYSqu#*o}4KcGvn{k zhdiV;gN;8dkpH6=EzU~cnB>c8Vwgx zoII`$RLxvn{XLFEQJyu;WL}}Q+q07T$l;pxG+SL~Cj9YDz~;HMEzvB{YW zUv=A_S3VJcfl9i`>)|AyJ9Mo@UybR(`L^R0ruThz(k;@oF8`H83c&{ERh!R?citHU zW)Cp19;C_$SPuat8m*7Ml98=4@#7cep3^e0zEBbXdgaV$1EqK>>vufW8;!d~hZjT= zj%@5Z*b1i9cIO8!xBHUh?xCl9%(kBayEs5D3&Bq)_qe7upaD^lx`(qr)tWi`7Kn#m z4?x|;`i7w=Jk(xnoUIztZSQw~JNjQivy18e@lc=7l1;2qJ*v5fAU+7V4+CTnEh)!8 zAYB~SK(k0r0ayBo#e_p z)AJX7>3Bdk@Iz^S0Kp00Ou51}Rt>r4DKHnwILTOY)k%r>QJse76AYepq+WW$z^C}Y zajc>E3C$v>7ob8@q&pfGKTQPGU*A8vUQt>b6W`GE5M%;MRQddbuC09)!`2XS94E&4 zVdI#uj5}Qe;S;(x>@2B?aZx{|eljyru&U_{uQ%EUK;6L)tU*`mzu<@q$7BAOON=Kq zNGTnJIZW+%S2*bD6rL@Bq$-(U{2UMCGA$h?|a(4S@r@Ol>G4D z_9Py4xr(=M9!*_Lyy6|xtXID6nGCQH1`nVV@Gph^E+i3p`(IJ!wVe^_l1Yh(Twqv(JHqcOG+UG^&EhgfW^F|+_G$JCRlDu) z?cf~64G!ufb>rtAHlvLX2OkH|R2Ea68MtqSg2ue4B}CP zK&@45?GWZ!pXEJ{m3^sZ6T_e#bI*gpcOV*wNdE%W^DEQ8K+fk0Pinf7$s8@{0S`{$sF%u%XLwT=G4 z15qr-Q_2s@#dhsXaL&NWK z!LuJ&3NtodpiKvL4$Jeex7yUBc=de3dIfo&{jceO!*tFtaA^U@Yj~yc19e*hfCD{r zU%az3*gJ%hzBBk>*}pnj_d8xGEA~^pDgGbYU$6VuQ|5a>Z*DHb-9JsU>}US8E?GaB zTgSNIIn@D#O82Sqh=I)(ACc!0yosZ6(~N7ATYN|QJ|Kf z3a&lQU*)tG0lD-Rv8_d-LDVMvwbnY&EwTwKn{Qtr&=VedlkL+}aje%J0HlU%58`Ai zAHf|~a?Sew@nLrjUeg^l(}mB`d zcoSmG8EukOjGm>EO>jEs@vLP4#-Qoz5~`%!egE)0AM2Vlw^xX6s?&xzpt;KMFGaj* z#~d%PvW*G7DRg6M>Ntplh_#k|zm^h%SfQ0Z&CS|yR)H!mIeFA&e3dU`{BJbP;S z`ufPHHf@~J^AvOaxnJQdV?6z~IhUJ%Nl!HRpI04&1fXNjV zAmHC1yWG8%_C}Fu!3r0QQ;Gh9M#7_Cu~5^>N6I3e_JRiU!YZFn9ieJ1MO*Bsy6FaX zF;juZDm*?ER$miz*v@Y~?E-jEsy>Ax)e&}q$^Gm(cj#sUpBz_5s?^sxx2TU>mPLu%0pe^o(mXJ74 z!V_y^siO9zP3CQ!nkQ8MrU9#R=9!F-S8hbd={rwk%07GiJ*?(Ks}Km3TYhpBcKcp+ zvufQTrv3zk4p<1U2{KBz6K(Xjz@HEwO_rqVaLf{sVKZkdHD7wr&nW5+#oYxNDuRyF^_Q`6=d;E8>o4^u0ZvhX zy^kK(e7aqo&%3pv88`q-4>tLb@atmo~Yc9x&XpdW55mx0wu?Gd8C5)@a zy49fZ9C7&5soKdy_5FZ03BW%o$;iv%urn96b(mLt(j<|R;fXrcp>WJ9d?(g3N&t6f zYV$|gpVz9+X8-Bu&^ZGTDls#>w*H~3#9O#R#sX-168-Sql1oTs(jv4eRA9EsaV4Kc zvb#4a?a2^o2`*)4(K(p&O6YJ!-3EU_#(sZh6;4|n8Ckfy!^9+Z3^e6GwM%PheRq4$oZRCrS@wi(^~-+MFN(DY0a(3 z$tzZH>ORxcKxz8PAFyPL9Q5#K6vVzbKtg(>1Py<4tfZCTY_g@|m2+}(6HV#1;rVga zlOjyKh;qW^;tOJaJbw}FHOfvW8{xoMCH}Kf2_I#$f9q`mF<_#LC%C$$qdH${mELWr zF+q4jam0}B-sz|KLaTIS4ZfwedD{LYzWK8cyQt8+CrfO2fcd9zZ$Q@$P_EX}rbYO^ z?-8OceH`2gj#?6&L-@!@Tc!S-o<*sXkiZC$zTtk&#up7d8liKJ;f^w`D`ECupNSfX zew2FnuQ+uEWSZyt$#bsK(;OlV6Mb{fv^4UKg+^>9dHGNR=u@?M+6KEIwWAdcGHt+}I#00ii_8S6A zn9D!jZG8MHuF)1E{vtH2{{&zZ1@ML_h(b1b))zNm-Pqx~QX>qRZf%{#*~PcsLt9NN z*e5zO&chR(_1`qgdtc~XDq=!I(GF$KxhcmwSiA;|+-=;bRHwE4{xRq4=+XB!6Nv(( zlbdC4R2COWGASZd6qT4A_;D5?ZhrUX}d5fbK$=1#?!zQ3s9d849*5Q}pHP3RCG+^kJJAc?I4 zRR%%}ch3Ig#G%7|27p9uoP8IOmKxuB=zKzAYnJjA;4me4pfiddV=yLH2x>tU$h2vG z9zC7Ql}OI%Gsy)x%?*EoZP0kW{H}CnH(<)wmpk)l zr~F@pZl>OYf*qx9Pn~wL6DHh0u+(_-&|jc$rbg$WI#f@!o*MdfuU(I4wqz`h!&zP& zy9fQs2(LGmWIGf-Hs>W6Hc9=LcgRH{d(dQ@sm7I0<5mcQ*@KdGYB_%;@^4z_eAN`P^e#hJ!-6hp2`4lWKpn7redaFf z$w^UqaXjEMB|4!E1X%{$9X=A$e;teexl<=zME~F>InSV2Sb9%;dymd zu`|@lVl*L8>PKTcGdAA1m~$E}o-3wLyf07eYma1%$N9>Ez;9cJ6wlk&gdl3QfKG}< zT^k)yqD2|Q${(gzBFbMDT#0x$FUT=@&bmht4Lz1`x}q1U(|;5ll^{;wJBI!@%qBY* zrx#}w%vY*EZrwW@FnlFd>4 zK(Rz2v10(*ne97t)V(p_38Ph@zthKlN25<{KLGgGEnmSL9E?9<&h)cE+=6Mn)>x(s zFk1q+n+lz)`oq2W({VSWuplDaN_U`{jQdG;rGi#Jr{hd+)SNT1r3>q#!MR}f7<%>- z!WMe;?UbRLwEjW%%y)O|YEA=oQGm>$F&wjU)`Gz+O1{=Cu>6gbX+q)EzByzMp82Ix zVVD2@Y}sbcz2U^PjsJ$t)@Y~=Kcp05l_C>x91S5o*S`a1GorTvJ^=60jn;-8A9??* zX4Iy9!Y$sln{tKB4Z2*=G=5w}PjHy4)}9qdBNc-L(U_{@A9TnJ7s7&o*IK&Of)DfYP6A5b*5dSF&# zwA#F#9b$0v8dA`cp;;(Jwph+aAPCSz95MV(VI{_|gL2X zHBra+MlD%?mEz*P5uIQ z;d9M-T&=`jf6xq%CqPi;#|K^2>U|Llx7C?M zVf#O6Nau<3KtLMM=cC18{Fdse8)39R6n|CE9{SBZMe-yDquDQdLC`IRU+9N}kIe3F z!-j>RKA?dkz#fKOMJBO_Yg>Aps!PmCsOKBtN8Om>4b>jcaO(u-jgdj9*>CcVWub1r znl-mT=u!0swQ|B4>3emek7RZ}j!P^)pggDtaBwKt#(bUMc> zRXMTwZ!*-4)XSZ)sx&Dvb_PPKfC@23J)t>?{x@%jcVWrRhc?4DY&uUz&OqN@$H3)v zs?EK%IfgatMs#BL5d~obKnF*mGsJq{T+Ek9qTP9}0yRS=n^K?z#tbw5t17bEU9W$4 zetKf2htsrV{MP1{x$qri1Gdus-qm|VEiNiMX6Ne#<_V5KuwS5TU?ka|1#~?o$CeHa zQIr(^+6K&r0c?yaWDNxZ?bLA(^J~rBRch;QHF96T-lzNnFBN~j@=QjL>)^cMb%n}F zI~Ek19l#dF&N*Wlwodt9PV$rnLpK!c024X}c8PI|yLCl+)$ncKO7b>O4_jAt-Odsh zl%KFbshF|tWc@lzTGlv4Kni3M{u>OzV520Wn?A;V7*#LZk-Qv-0URO#+eS}DqN>F2 z?da8*eLYpSIQ}@n+45}A5%k1G?}e7)S6{!J+rV3?c1zs(uWnwVAWL@EN2)k$XJvmF z4~Wi_0rcpKQt6>o+I8-qEdtq9*^~rn9{wb%odblfc?3$Ujm4-TOun@rlLXvq`h1z_ zx#9iZ`a0&bRpzrUzXN1F1)}spxq*MpDr^gyXKJTK<8GN=x-K;Zr9+JjKI zs~O9gfCyVs8AOV;zpbJ?S^SLl=+#@xqhk)=PyA0xA5H1rH0o={E(*Mzyi%Y^7E>GIdIK=T9$&@-D)WG04p>0^C$zF|g-{_l-C{x74%ZhfasLtk$j^wDg zy0tWTrPYd+GIQN05k>$#O1b}zoqHuPkU8tx5SEWWc;Bb{P!X7Dy`BBB!T9T~a|2Snme6pM^@EEEqp=aV~s}@HZMu?(fg#`+ExklO=#c_XWjwU9^nn0=`<1b*t(JHBZ-Y z^q#9mJtIEi;A6SNpF`xIgg&*pmO8ckcfuF?H*-OTgnopkaZQ~k-TU-eg{e$KwFQ~-&_U)?r}1^8ZjRJ z=AaxT*!oExs=RCH91&;ET&8PM#O_ofxY1!l`)|&Q3=JIHJer?%JH5(Ie*J?(tG)^b zm;&TEdZ#t(*xwn!+Y^`qL$mS%@A6ye3D41m^;BdWH7$(boCPh*)~<@eCotH5!;j0j z%}u{M6aMz#_`&*P7ri%x1rOaeJMK|0{_#*p*5u zO>Ai6?T-{n!WY*S%l!&!6|*ybQR3AE{8~(z%JaPXgal?xRNnH>Ri zTJT?{|Ck|iN`owa#ySmG6G@Y){%dr}5v>sW%<%&(#hVG)^K31H;dP_v3;W)RIr(T| z!(Pr}o>ijEGU2-bEufAAuq^cwvYZ`@=@-t`_qT5ycqjHG(;8HkPL zjpoP}v_mU!vR`a^D2-})`X5h-0A2mDs%Ym0D6`XkgamR7Z zC9-~3;ITw@hdB$HaZr_-vGV2b8snjU7p}FH3Yizz)B&p{-Vr5&L>~<&KSDsdD1-P3 zwgdYKolZ=Hl?83O`YSOypG6ofPO)I~{tdJ{0=n3fAJ{DkUe3h8{V(45vHiQZ%ETa_ zp1ig5iBIGdJNKT>>%7rThWB9Z#E|zV95>B=GE6VnMNc;VM{_qY&mwhWHq0vSXkIRO z#kO~w53r&59LxsZ(~>d0u6=L(dDJq4AK51@EqoR+#S+lDIoVjLDay?I9tVI7jCAIwswG*MrH&g_4 z-e`Z15Wqnak#ctML$;5-)l!U#J;l+&h>2}@E(U14eF&ZoKEYDx+eKWnZGXh}%008% Na{YAY`X7sc{{tR9x848% literal 0 HcmV?d00001 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt new file mode 100644 index 00000000000..0d87e8d0e8d --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt @@ -0,0 +1,183 @@ +columns format version: 1 +181 columns: +`CounterID` UInt32 +`StartDate` Date +`Sign` Int8 +`IsNew` UInt8 +`VisitID` UInt64 +`UserID` UInt64 +`StartTime` DateTime +`Duration` UInt32 +`UTCStartTime` DateTime +`PageViews` Int32 +`Hits` Int32 +`IsBounce` UInt8 +`Referer` String +`StartURL` String +`RefererDomain` String +`StartURLDomain` String +`EndURL` String +`LinkURL` String +`IsDownload` UInt8 +`TraficSourceID` Int8 +`SearchEngineID` UInt16 +`SearchPhrase` String +`AdvEngineID` UInt8 +`PlaceID` Int32 +`RefererCategories` Array(UInt16) +`URLCategories` Array(UInt16) +`URLRegions` Array(UInt32) +`RefererRegions` Array(UInt32) +`IsYandex` UInt8 +`GoalReachesDepth` Int32 +`GoalReachesURL` Int32 +`GoalReachesAny` Int32 +`SocialSourceNetworkID` UInt8 +`SocialSourcePage` String +`MobilePhoneModel` String +`ClientEventTime` DateTime +`RegionID` UInt32 +`ClientIP` UInt32 +`ClientIP6` FixedString(16) +`RemoteIP` UInt32 +`RemoteIP6` FixedString(16) +`IPNetworkID` UInt32 +`SilverlightVersion3` UInt32 +`CodeVersion` UInt32 +`ResolutionWidth` UInt16 +`ResolutionHeight` UInt16 +`UserAgentMajor` UInt16 +`UserAgentMinor` UInt16 +`WindowClientWidth` UInt16 +`WindowClientHeight` UInt16 +`SilverlightVersion2` UInt8 +`SilverlightVersion4` UInt16 +`FlashVersion3` UInt16 +`FlashVersion4` UInt16 +`ClientTimeZone` Int16 +`OS` UInt8 +`UserAgent` UInt8 +`ResolutionDepth` UInt8 +`FlashMajor` UInt8 +`FlashMinor` UInt8 +`NetMajor` UInt8 +`NetMinor` UInt8 +`MobilePhone` UInt8 +`SilverlightVersion1` UInt8 +`Age` UInt8 +`Sex` UInt8 +`Income` UInt8 +`JavaEnable` UInt8 +`CookieEnable` UInt8 +`JavascriptEnable` UInt8 +`IsMobile` UInt8 +`BrowserLanguage` UInt16 +`BrowserCountry` UInt16 +`Interests` UInt16 +`Robotness` UInt8 +`GeneralInterests` Array(UInt16) +`Params` Array(String) +`Goals.ID` Array(UInt32) +`Goals.Serial` Array(UInt32) +`Goals.EventTime` Array(DateTime) +`Goals.Price` Array(Int64) +`Goals.OrderID` Array(String) +`Goals.CurrencyID` Array(UInt32) +`WatchIDs` Array(UInt64) +`ParamSumPrice` Int64 +`ParamCurrency` FixedString(3) +`ParamCurrencyID` UInt16 +`ClickLogID` UInt64 +`ClickEventID` Int32 +`ClickGoodEvent` Int32 +`ClickEventTime` DateTime +`ClickPriorityID` Int32 +`ClickPhraseID` Int32 +`ClickPageID` Int32 +`ClickPlaceID` Int32 +`ClickTypeID` Int32 +`ClickResourceID` Int32 +`ClickCost` UInt32 +`ClickClientIP` UInt32 +`ClickDomainID` UInt32 +`ClickURL` String +`ClickAttempt` UInt8 +`ClickOrderID` UInt32 +`ClickBannerID` UInt32 +`ClickMarketCategoryID` UInt32 +`ClickMarketPP` UInt32 +`ClickMarketCategoryName` String +`ClickMarketPPName` String +`ClickAWAPSCampaignName` String +`ClickPageName` String +`ClickTargetType` UInt16 +`ClickTargetPhraseID` UInt64 +`ClickContextType` UInt8 +`ClickSelectType` Int8 +`ClickOptions` String +`ClickGroupBannerID` Int32 +`OpenstatServiceName` String +`OpenstatCampaignID` String +`OpenstatAdID` String +`OpenstatSourceID` String +`UTMSource` String +`UTMMedium` String +`UTMCampaign` String +`UTMContent` String +`UTMTerm` String +`FromTag` String +`HasGCLID` UInt8 +`FirstVisit` DateTime +`PredLastVisit` Date +`LastVisit` Date +`TotalVisits` UInt32 +`TraficSource.ID` Array(Int8) +`TraficSource.SearchEngineID` Array(UInt16) +`TraficSource.AdvEngineID` Array(UInt8) +`TraficSource.PlaceID` Array(UInt16) +`TraficSource.SocialSourceNetworkID` Array(UInt8) +`TraficSource.Domain` Array(String) +`TraficSource.SearchPhrase` Array(String) +`TraficSource.SocialSourcePage` Array(String) +`Attendance` FixedString(16) +`CLID` UInt32 +`YCLID` UInt64 +`NormalizedRefererHash` UInt64 +`SearchPhraseHash` UInt64 +`RefererDomainHash` UInt64 +`NormalizedStartURLHash` UInt64 +`StartURLDomainHash` UInt64 +`NormalizedEndURLHash` UInt64 +`TopLevelDomain` UInt64 +`URLScheme` UInt64 +`OpenstatServiceNameHash` UInt64 +`OpenstatCampaignIDHash` UInt64 +`OpenstatAdIDHash` UInt64 +`OpenstatSourceIDHash` UInt64 +`UTMSourceHash` UInt64 +`UTMMediumHash` UInt64 +`UTMCampaignHash` UInt64 +`UTMContentHash` UInt64 +`UTMTermHash` UInt64 +`FromHash` UInt64 +`WebVisorEnabled` UInt8 +`WebVisorActivity` UInt32 +`ParsedParams.Key1` Array(String) +`ParsedParams.Key2` Array(String) +`ParsedParams.Key3` Array(String) +`ParsedParams.Key4` Array(String) +`ParsedParams.Key5` Array(String) +`ParsedParams.ValueDouble` Array(Float64) +`Market.Type` Array(UInt8) +`Market.GoalID` Array(UInt32) +`Market.OrderID` Array(String) +`Market.OrderPrice` Array(Int64) +`Market.PP` Array(UInt32) +`Market.DirectPlaceID` Array(UInt32) +`Market.DirectOrderID` Array(UInt32) +`Market.DirectBannerID` Array(UInt32) +`Market.GoodID` Array(String) +`Market.GoodName` Array(String) +`Market.GoodQuantity` Array(Int32) +`Market.GoodPrice` Array(Int64) +`IslandID` FixedString(16) diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt new file mode 100644 index 00000000000..7821086ced1 --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt @@ -0,0 +1 @@ +1194218 \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx new file mode 100644 index 00000000000..0cb239d932a --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx @@ -0,0 +1 @@ +?? \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/partition.dat b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..9c176950d3f897f5cc9714d7011107a21e47a229 GIT binary patch literal 4 Lcmdlj#LNHy11td1 literal 0 HcmV?d00001 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/primary.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/primary.idx new file mode 100644 index 0000000000000000000000000000000000000000..4731cba6d463bb6755b51e8dc1e552175657502e GIT binary patch literal 3240 zcmXApc|4SR8^#TXtXXG}B_i1)yOZ-+24R?CG`6uPOGWl!?5||YQXwKsgpjgK2qC-3 z64{AJmV}cm#ru1Xf4`sWzOL)Of6wQUp`xPVfI_j^Ob7Yz8obxM9TZ|mW;$%TuYSIHA}GJR&WA=e{u6-x?-dP}2dM7}E>c4a3290!)z5`(S0Gda0ixqV+<0 zFhM~DtxX}yI=4DnrNeH)ga;zIK43b49!HIsrDJ3=;jBBkwo~{!AI~3Y$1;NY2!t z&(kvwQ*BS$lA5k&hqISb10P~t;a9Sy@BK%@rI>?%sew-hk-w?w^GFQseW|cWM;buB z_hzXw#gGaG8~5!Gb6}Eku-xNuQA92Fm8V7A4h?YLp2qbI9N(c?{3ph_bN?@}DL6sn*VmH|5JYWJ|Cs%|w>5C%1|E9d%3He{+>1f&k-qbw_qSFaM z^@OaAEslNNV4^aC2^aLhtBw%d6Wy09izlBDm6vku%OeZxQBi{=m>{9%M|KKh9oiOd z#pm2l2s3@^v-urvj(2qYelS5nX+LU^-U^ks`0=mIPKW`aCe{LjMWH8e2A+`8=JsFV z)yrM3lXI3Z;fDTM?+rbzF&!@cR_f{rS$7FL+!%^c_Vp_Wh6xW8Z%q|(v!5qeIW5EW zgb0f(9Zjrv_{$c&CBCs|0$W^OJdm$lWny8Apf~HDWCqu`@F)0ECO3x9$o3yJOpwrP z6W!vvdV|KZ?L#D(@Ibi=T6x~Qp4&_d!8XieEMRJAy1ybIXe7DCTH-ectPnLBnw zzPF%s`v2Ns0a;J^>~l!3MRlWZ_}R)H!&79;<=0W@B+W3K%Ur@?zkRQC|)dhX>4 z??&02kSv4l_NeA9+xwGujbOqFnXymFX1VK1R|keUoe=gE>nw|AUy&wne>_aMAl=z3 zh%dN`CyUv}rYA&nNzyCo;r`Mk5hJS;@_udN)0tC?rAY`UO_(5|FTs*%IeJwm&y-*G zCuHT=q*{q;PpMMff(#QB6fA@kdQ9ntWb(}Zn?gZ4#3_gZB z-+`}jErt1d?;8c_9bYy>-6eQ0MssOvhtKDSuSgAEq4cr>&C6mX#OL$-u1gmldt7_L z4zgCJjB)a@wfkDS7cF-5IKb3xmz7y}2CREjT@wZ-JkaO2^hGJudigujN``T?C@|%( zSg(jo&aSjCkqB+0;{h7`IVoY>B7d}E^h&6f3_r+^sp+x=ai-aFz({LO<+Td+GWqn0tpzGXgS zf?~~0xDGV>XJ+5bRho>2hQe?TaO5WwHw~+PjB$iMT5>CRydq&@xSjD~U%C85;Oz0HG{N8UYbG1{nxPUh z6=?SP7Z4LRrud+&4Y#SiG|+jW%=Och9~oonkR2w8N(UO#&^({RY*~h6QiT6i*DUZ} zNOXwHs&WptgMQP_At)OVPxQd@G-&0+6QWLaN8!0k17KJgmA-zXH4e&cMHLH9tqXmcHeR(xL#;EEn!gNpnj? zl9mi}0~-Kw-tSQf_ggg6++FfE5o!Wi%c)i8gxhbsvr+V}XlA_wq&)S~az;yToyxZ7 z?cVd9;9~Wsgmc!=1{p%3xp+m6KCqV;eyXZ_RV|#(n9VM(Y5=_8eqG&O%SIQ4#$Vh? zh;0~TDUu0U>@Rz=$X7^S{hc2{XS`q6ajV8|8w14(Dzc-FcFnJCl{a&Kl++4tQ8hQj5#JZY5 z(bpF((bgKkTSqYO`f_C%ln2L_X&RglsKUsM7vSeP`ZlQU8E zt#8)#+pOY?lmL+mUz^gd__>A0?PCQy)Iev$mQntIyvx;Q#=Zf&NG;%8Ozov>*;h0^ z^XP$rFXt7|kPzprZXQ7*&+Cp^^Vwm6ro@Tw>%9^tvL8O#!{iT4y4#8vrUaYn94;Jc ztH$EN?hzqpnSRK~?74Lf>Bx;*Avhs=_UTEalho_@k|KYzEj!S8WyA>`qlR2)pe?v5 zsNw*89Ql5isS!ELgIU|Hq&HkaL*{kcnvWcY73SqpL9yYUz&U|)T%;l|B5QIS5!P^I@$UsAUP3L>a-xB?^#1uU)5DBtesz*y@A8ejkcImwOJ2DoK zSaCzgI?L3=>V_)a^MOfV$^}K)B05@cUrSNNw-GXc^S?&*e@ymNvQ%+DEQ0Pm0aJR~ zXMPd*coR6X#jmzo#!EQBcWH zZ(8$`C=>MaX$5$T!g-9lnDqD?;cnt>lf_rSc};WXB2xMoQ!)^UN4vfT7t1TPOivDt zk~21T44FS{1e(O0#2u}MWZ#GSyFmuaZ6G~~Z^9UDy}r0{Dj@W?O9vnn#=Y?|TSm1< zGnd_0^}0d2BC|cMmmMj(UprDqL2Cf`AjB}M;*2Xx?$S9%Aw45NgJ*bzx>}%qIh_lM zA#_gw&5ProqD@_m{k)ebK}T$#L7xjaAL?U|2$i4jZ)1(tR=|5v+~VH4B9j-ex{Nvp zKHtG!qMJK~60EZF`|B?LqaE8o!$2vf>XS@Aqa2c@EiJtd2yco;qFOdDZd@}SV?%ok j(w$po#558Imm2ua8h8CU23d;MR=5E3ApT7w51;=7A{bPx literal 0 HcmV?d00001 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/checksums.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..ab58ad06ceeb0d11e4723395c0f1b415399d055b GIT binary patch literal 14273 zcmX|IcRbbq_kX?L?|bdNHz6e>Nk;Z6$|l-%ac!5b9hH%pB1(2e6j_m3vLcm~vOXj- zl9cYXvi)At_xFzn*L|ORUgMnS^E}VF4lYCocOO4bA2BB{vM0e;%%4d1arGh{6jNfr z$A+4pV9tNsH|Rtd_#W6Cep>|qR$~B`WUgYYx{m&aBxhF=(bz!V-j%eHfLa;ptFG`r zRWu$*OH3UcN=6di33DebTc)d3(ma_{vRqbR_{F9IC`_JYcg22YR&_RRnv+UjldwiP z&%DM_K&$Iav<4L1LN2?fgqp|6+X;^y*OB_{=7x3nGTNeFI2E<{+VRtF7G@p%ZaccY z06hw*OnaG1Cz#je+L+nPzdy9#V?s4(z8K*^?f0lT{UuL z{9~L40ZIV#sK=G5x8Pv!e*7NZ%b7{H*}yJfZ$cnB`w=)13T~~OVPr|A__YqUKbr3Y z5zlh$ls%Y34I1rvFO3GjcOOaX&};ntn#&&G)dA`NX7N97qeGlH@vs+OWhT-HTR))F zH+g|j`CmjJZF5seRusW9+LzZgS+v=uk>^kS&r;gouP-IJdtbd+!c6#GNUs1;Q$XdD zYqm$V@1W#{*t24=r;2DTeuo!lqzSWjU&|R>+ zFN#~6>zVayWD_-NsGQci`e}&p=Hk=Sj50R_bpa0b2D8}5?s99|hWqSCSAL2Ph*`7bvt>-`}TsM*A zYi#ixUcc3w*bVESPc-Q0R@8Bi)l~T>Kb!-zNEgTHsN`wOPxf>(sq4(|wMb+M(} ziVnq8(Nnz7a5(!R^87yQRc~$$fD*wRM(VIkx@%-Tk?%fgZ|wg5XjsR*+`0Rc?Cw%vjI$SM71CPsjx5 zU6#{9os=SV$Kj;5{C$r;hY-NFIVzS}m(s^k;lWWF{5+bk+W!vVi2=tBlfC@BU!oKm znDFBCOoWZhnG-{H{Hk&t-Fy7^q9fQut(wz>O}mqm7^!dVdH1HZ^J)Tsy1>;AZL*#B z=f7(CX=MoCmzJFh=ukxyFXu6YDv%q6zg9}+V16T}_J-p(b8?E$7_#*JCo(oSNq20e ztKne+J?<4YNyBOnP#6*?0Ts)ga(9f|DNO&#*>i->F251*OMp>R0@faEc1(01PDo=5~Nq;221NbpahnW%%+jv83SFetX)0aLt|P z0PbJD%Have-sg8J8b@D#bG_;-L?16dax-8(j&Z`j;s4+rWH@F`^dLGU)6VjJC_;EC7O#cHOVZ;m zn3stAT`<^6Yb-J}Uxw~pA1$&@sy7SfWp)6x+rWhuQU8#=4)z{;TmMpAH1ZV<2*nd^ z2xMoXZy7?Rg;7=X5P6nrsgRrZ8z1tp`(b46-{EpTz9C-rZ4F=R=cithjG(drXHPfFTS&gleCA8?J>;EWu-Ho`p`9jM z7#)E3dP#I(B5#g69XG9gQQEn+EV#HiB&i~zLHjfH65W$R+X})g)g1$;0os74V$;jW ziO3R{;-|l!-U8?42Sepn zdIN)(XXwCwfZ6~m(z%?R%$$4q>n$-S%@w9f5q4xBgsN0p+&V1A74In+K9xk9?2}KMg2|SB6r6iHhmWuIQ*qHV%5r8h|PR^ceK& z(&K|vMuzL>4$64ES86xVCMoAyQ&VAJ>Va(5*fk_h%5b&!~vI}Wmi zJe_K-o2F&CK90F{6=2Kgl@eA~J5fr+jjlK*u7kbJ+hm4{&C~IW?Ugo6nAUAvuS)As z0L=SXa9MDje>z!Al7nx@vU@d!e7UKrt1)%45&}CO*FJ`P7(JL5XVFnnM0Vc>b_(s~ zq(9t`0-7A3 zCB7#r`gi8?pDN)Vht@m@J}#yNH!m`V{NXQ%X4&JY zlsn9=Nmo>BZ}l?gX604}dL|oM>9aVnBUp;Bs(Qvnf7{$VGOz1qdqlR6PI_T6&105^ z<^<CBe_ZP31G(zy|JP{`hoxkzlLEyBtSPz?2}#_2WU=;oBW%7gc*+pRW==0g%6r zQL*P59+o@x!qrRcI(hpGnu-B?2*8?Aw*8gR+GHDxn3h>jU*N!~bs#p1TA%jVfvupo zOv-%B4=Pgg66?RO-l|^U_?zHYkb<@(b5~zqgMM3|ugg`o+UEe9(m80RqmAbHhQI#d z>9XRueQw_#--y;a2bPg?Q3}3HC#X)&)#lM9KOeDkfOG5ZjCU4_g;G9QgwMGP5UR*( zZvoRGvX`e3+%8BNa3gQfb}hMcvPm?RN7k~eK|deGfo1`{9+I6cVN9z#qIZZf=L}mQ zGbE0-fPyhl7COIeFfj=%9U2;_P89)+JAi@>!5IcWUv3_9j_4RM2g;7!Mt?icyw)v3 zUbl(fFR#r9zPuKRoK~vzT%lgEU@M2g(v30EoqRv6J+iC8c;T9vjc1t-eN5FxtX{?W^U3(6-o3|WfMP$cwi1zN8Dl~QQ_7C zs54l{E^|klC%LWWI)xKE6IuCZfPgfx@NvCJRCo=STUMx)ejuz~_mTkThx*FNGaZ)d zBCuk~YMb-s#4QDWKFINL)JkwtP#;z-WI$!J3N2l@siAt9rGAKRPrn-r;5h6>@US8h z99)P#x}LwF~GP5WRh;VT8^OVqug&)lm=b@wLj7&J4J%U2_2I z=#SaqOPg79ankj|t#)rFFK!nE%Scg=GuiTWw5}$v1$TUcIH?U_9*}c)=g~ck9?`q> z@^iI`(|mPrg9@DkSVEd^Hv9~Fe#3r0>&DFQGV)9t0Nc3{YyNTP^rqC|oQ|Fei=^^T zjesW(RQT*vG}QMalZhk;U4~7*CK>BJ!jBYMrRa1t6PNV$lY5cI_l9f^eY@1Xj(*2a z>7Op{vJU)5Y^i8}aQhqEV>&ZOtR99u6MG9;H2^Mv9tR27V)K5r&TgmOx#GR-e#80l zM_zHdfCc1m)JwauA$l*);a9%?N$kd-S%V^-WB9s8^rf!G!#Bw{6Ro$yoQn25!Dy#` zFx7rKHkJ4zNn9D2BcKt`YA{oI=Y`K?h@(tNFDYer@wbhie4|0G_OnYjT2MP!P{Tu7ENY$RU5p}=jYZtWqreeet zHIJY`3Y@nll3fWN7cmO%>2)^4)%>QxWE+K=AU&@c5L_yPEFcBD%~DoumfJ#>qQ%~i z&3LW??C+7zqCLk?3);JdC`g6U`1Ez8<^V&4rXQo?B4U~Lmv72T3Mj~&pR(c0q~+A_ z$Cgl$$U#raiI;O6uKKrY6BE?fAdfx`=4mfyPq!_;2qgpWZB9a%9jq_@16Nx86EmDu01udkNGcMnZ@fqXJ1({Hx(Z; za+^Dl21k1xmRTg(hc*^Z2AT3;L-n@|i z>ah)2LcPyaXaYO44++7`4gK{-tp6aKHer>=BumZ9F*{#BA&fu70x%ep9J~}bAyTfh z^mPqs?kxT`n_6^M1JQonxeB!x+zyUe&HU~%`;eRP#OE+GsMXMDoEYp zUrkOoIZ$iPksYjis^(e>$=W9xFd6&kdHI>}K}$7E=5rs(E4-Xcua4evW8YXQEQG*) zk|Z45|NbFkk1xUUzNP2bKj3EW99ZorXYsmJ$a#$S!H+%@z+&uU;1xjf@FF;t!)lHJu|LjVDo~f6;YZD}h-(u;tT5PlPzL~xdK#$_9Dtd|Pb$k*Kr`v7VVDX2P^yRT$`ho%+(nLHzr#xoDV`4a?^BQekl zg49EayHfG;ha5t+g)~1mzxQ`lwS-O$3bI~d{aDtgOtN8%=+Cp7+-pA@^sT9) z`e8m7SN}&X{Wm{eHUFjgcB0MUUG3WfSHNiEN^-wQJ1-3R;PYhT)VCWF-cDBP-WM6r zwNHVH$Z$snLgzKbea^bvz3-3sVdj^o028ExHc$z?GvH}SKM*CU6rykm?C%6XP?swe z0*-+zndsnqm4A~**Jb?F4{7P?H+HeitLHU>QjmG>=BnkW-G`KAZ2IOsViY+Rg>(QO z3X7}Qs`~Po47qUY6Zc-8cQ7bV0RjqucAaL8TYL9$u8`=Lx3yeQ8zcRD2*=I&8R%`10%x$syMfU>NNbW8&GoeWz!w%Zqk<| z_roL6{8~~z>0D}nl>pc{Ayr-CiJ)svJtKYZmxEaxk8+XRK2rvyRDiK|l#8A2l;WEw z|M4dGTE8Ix1mXk%{K2YsvUiVy(yE7}28V@jDyRbPSRm=;c#?O6w_bR3b~nu`ioTtZ zebCw)rW%1fbUEzC6U65nbNcj{&<&}P26-NMCx~DLZL~=VWced@pry=6BzeDWIM@Rv zWV&SmxcVu&xUOe0GLN;Q@5vThC0|Yf$ak4tS?}Me%qzTOYYABIeZKgE*9o9d_Hl;E z@4u8I-_pBINPQ`|6+NIxP5}He;DV(ef#mDzTgAA@la8LerZg8;E$Z<&GoQ*o5)%aX zWUh|s)K15Bb<&BUBef;pb`_woLcwbT)DtlGp)>>pmxM0zLvO}~QrCYdGh8|=uh8`E z_FcJAO{ChJqkN^s-XKuU#55}Wu(rS+9|Q+s&jHp&4di}N*(WsjOqZk@@X15GU=<+J z09^wMTT7e6oA7+4cD?;3RVvghd|UEt5?mU3%)~t72UTE?#1ouhV~9E*GeQfAhbv%K zL%3OG0ZQegAo2tI6kDJ7`5kYNs216MTSgvMhQ2i)%ZfH*jGb;UNLNbEGqSyy1!T^mN0~J>gg{(TKC4T2o3Hof>>_&eh;;0m+kStz`Sx4}f)mR>H#NB!sqSR65VM zDsi$R>2%`~2Tt4yHfi5SvxFuJXUo$EFedlf7*FkdT^PHKk$Mph@Iw|;Epc^8YW^B)hY!`mQZOB!+yX9a#HY!WJl?+hI$ z(`;$?K zY!A`M_07eVRqui=Rif?((;u~o`OovW2539ROBNKl}{IqirwUaXFp?Q+dO|4EE%r$1@{i7F44cP zH2-bfzmt_83a6+S%;`@~4aa(a^SV8PS4hRzq}>N{z}JfCL?jc*-hhg)DT_riFSNw)ug8H*DVk^a8Ya$V+umpQqWHO}ERB zcG(aE^1#m8mq7L%h4%i7NJj!p!|gd(;?CZvA`yzrSpvz>-hT13^rmm=ShAHK*nDCb zJ|##7^acR+0Tg`ZP317*$DB{syOx$z&i2v)8gpw;NHSl%;+Qv#G&~eE;H4drsFTmY z=#zuY;U1+jIp(rFTOv(N4wEaxb!pA5BRJ-m!gdCIx~!a+cOh3TO|w$e8L*jq6G=Y4 z1Yg|{nq~f%caT4Y<)51)kKYyamh`&Bz|4pIekO9*{Mj9*Dsc<5*XyK$7riv>kZ9(E zCG2CE{qy+Y_Jc%qyBXs8zy-jy6W}5dDsH|2lU&Rz_baOP*a{>LzQ~t>xi!APe|HZtw5vKzuh6x^w=@zj^d;j zlDa9vUKS;qvdL33ub|yT)Ojz&=LzUwjq3&jqhQp8tJ&*=yp}IK) zU8@*FCizkmXEt^&w7iXRm}R_lVhFm1DXzwbo2;%J#MWJd(^RkgmHOdz@CA#=?fgqo z!Vhii_xn6@-&LVpCW`V}?-yF&o%AmE8o=AW?#~NM$2QfLl3kIkLeZ16*REa-tsJ=J z@?YrjYs$&O=vMpD5#BhZV~$N0YV^DctmEn9cfzXy4#6A&aufWOaMLys@61GsOwGFe zfdi?nxm7~8J#Z3~^ZMAz1p7_ftTkw3guT#mP-TIP`x>lr`)HNf6Ay4BNUQ4jun)7s(C=*3jCFwIN1 zx5>*qU4d-<-FmCGwVTy8Yc;Qr|ALJT0-4|$!U+Lvh3~U6@ibG+5%pAF_M+R}XbWpH zQapKFgX13i;A1llIo0`7+CB|#9Pp~1h3`*QQZ9_8#BvVs9kO!ZN%{>qr2wjlQLz4< zj$FQzQ&Qh}d3Mzh9^18x4KuBwPIr$Nn#> z=rIMRcxZ`jKl}$DB@E$tAf9QhH+lETK&)NqOE*eee+AJ<~>@NLj`Ud3w6gE#D z+x7c-ju369&<}wa6eP~-)tRJB+?$)Ml-X={gJur6K{IQ#=~@LFJN$!r zCWSsfCjMy&Mc#eGj=#y&;9qEd_&DpU?UibGHE4$OfHty6o)S^IrG-=(or+xd?dn18hIz--y2c4C7!iN&VK0-U*~o z`Ua9!wPw*}q)t;aZrx^nkz*x&vL@=qQ?9kJSm>=SG~_wu}0m2dGg+zP5u(#R=} z&ksZ_F58!!po$6 zEoLk@YPa1F6Z;gyaeWO^ew7QSCXj)bF<9`^xmvp0G4uWGb9=w} zo}S==Dc%KQPe7fK<;7Lai?ywbvPs5{89;Ckp!tBKVuK3hR*?+R0T!t2KIcn$8%p6{ zAR7^gsVFZa7-U{V=s1izeeVbvz{2ixJG*m1@*|l=D`6HJu;La32%kekQEFfp7g~jH= zIbUq!uNBXYmh);LF#X^pJW~8JHvELtpPYje z*U}KpF&hnp_V4=UQK!DFFDLi=*dcZ*6nv12Ft`DftccEpU32{5VFF({6U48l@^|8^ zN7$Z_tppIt$1CaDoL7{c9&bZUO^r=ES7+XLo%yNbi z9p)7}?!gCo2)3L5gbt)wuV#~zcURg|YQXkCJ zgPG#PaibdEYOTj05?&C%>N`1^LKzpNj7i7xdbVvXph3DJzcY8-z135@>-ifN85ytx zASM!2{JrlHh?YkU&vce1d(T9E<1lUKxe%UtRB>nE!K*E#_f<63pLqhoXY3% z3(W0{pp)XG^y%=ub*2s3U!H(-tC8#Bx7DzZ5nkiJnZv^Q+v}nP@wez>PnJIuIhCMTczw^Tal4>~J0_>s?3CAsINE`tovWjT2;8qV(TvdI1H5_Mc-fbk>ZwP%?m$@sF^=x?Xlq<} zQ7grBbXa2X!99RN&l~`iQN-rCbp{iz#gzp%g+$H@0A?#MdoN!S(I*YMmQl~>zH+zA ze0sSebjIuyNpKhQPGk`sE;y=JD3ffgUc_D;=1kDsW)4gAMhn7xo`L~fA%E&dauO_0 z0S;>-f$ZQis|Y7$zOw0S3puqn+>m>t_E~@+eZHa=tfFT`JRAkj>YuzE*~c8~dX8mV z`k%!V_9*s>v&uf^JDy1&5tg3~*meO+i>(2G!<@^44_aO$?;qLTbftRi@dz-}+8vEB zAe(8i*ie_=Re=u*7i)(5X}=2=|8`_$#DR)WC> zxOU0m{SK~x*7^-U)b_5je!jfxzhE{WhP6F48<#QyrUMWP+Hm^$V<&xEZbP<**h5U( zaeLm5B0L@a?k2li^X0c((t2_{zSDrt8qSGrWts6^4g$Y(-nKJYHI25FdD13*hVyx- zR>z)=w9U_2otMW|b?9>J>EYa-37{}EB=#Q{i2m`EzI&F!bW_q2@LIci!0|(mJ`~$0 zg*=6-<3b+uLCb?}Z_3#C&Iv>t`+Q+fKMkXrFRP2exJ%G~pU8Kc!kDkQ|4$pb%3r1L z2|G2Zw|mif9yg*{&%D3_sdgY9@`D5)*^H)3J z`}GUj`(!UlH>dw+zEauq`0y5Kc*?Megzg-z%xY9w3KYVgW_F!a{pSxBDSlwD^yo}R zHNOClll{-!qE*)xViNJyv&W`my=JbcJ_c zm;k@Emjmn<2(Z$7!?1bPhMI>M4l8Ta20T`|bOq@XP}hS<@yfzLxVBx){8d$uK-8rT z83A252~-N9{-#sT&HED5yBzP~=3=1cri9vP(L2cxXoUb~Zq9uM1mAvG zWHw|ld|;LhxmB6q{eg7$zCHPC#sL_9Y1Qk$cSrDaE)v*FV|Ec1L#PC<40Kp&o z>!rieEZZ7J>;TRcM&6xye-$gkgZ*Z|eRcROs`@2`lougI&P?FJ7sfo;#VDhCeZ%5+ zyB$FOPP|dEc%WEQwBOsgOYCXuYl}e(z!U_qUcp&`rN#=HFKu}x@kdy1-07;4#MjKi zq0NdE*4y27H!Q=ev+ev-6fMgLQsV%?Ca)djF`@RIp9klfE%JAlPd)f7;tV+K0NO|g zT#!|PwNe${f!vFaMqb=IX+*i_Atk>9N%{5`3+GCq8DSiZ_`Wab2$vE)4JXW1_Fx60 zZ*EVyZ|1)9f&V+#bfx~6fM*xL$+E*v(8@uaVB}>PpOtGBY={9IHeTK)M1P`32P_Nw+8E^{ zSLG9u);kU<%8kf<_4wrkGgYUULRE8=t!bie6_+OOZq;qCsC2UxF==WEE=%)~&UbFt zM0o&K8!umihm++v2S)4|mZCC|N3D$VrsWG<_+1Rk9;g9}HOM@WbMN1N{48 zw-DJBSi7sR?mo5Ym?oUZGp9pZvMRShW~X1E0>k7#3~uW_*=vdfi^E@11@z#p-%^ef zZ?YyG#x_MAR7zx7N_%Pnw>&^~VUoG_Lp1!mZF7{<(B2w}5mQ#vK>rc2f$rQl`pn!7 zD~`Px6hv3=vc{tc*66k;mz96!w>xr;k_`vn9DiSXx(VC{N^l}ePhnNyQ4iYmS&`)Y zK=NU|#%pV7X2D=&aV-ylvnDEy{AV|m(QL4;eaUSQf?m?B2=yzwy$~ z_gC+!1v~ZvmJC=OmJEr~dG<57e(mTNtLZyEe>MGi5X^psjyp?4ySVQ3JG<9^Ff*o0FNP6z%6T{+GEwjYSO)Tj= zeaOT5QaoqC>?}dI_vy#-}3At6SzupJ2b-;Wqj3B-pJ4 zSQThs!)q}Kucq_j)vxmC4?Y1LtO%{ zyMm|hk`$Q-yix!!#!bb2=;M&n z`q}XTRc4MD(&4SnN)wQU8-6El!o9F}kLewnMib)hb9ec4VEya?W@(1Cd#>*+aLN8? z`9(f!eRB}7K##7nAdUDx0=Yj}<{!yt53@b;_N#nV(Xy*YADMp|i3AFz-DvtM-j>UH z=AfRQiU%vys!zgBL?Z`vtVw%WH0zzi?t)9HK3gp+d<#r9na?hSlB+$=@R)W)iAU{$BV5zvexq-`(O~0aD1$DW?-q8Tw=m$175H zQE{uO=sgDKV`yG>Y@A2YRBqC@BXFbzG5SoebP!1%kUt_7$+nU5U=d(2H6I65nwO1- zhzlc52E9bi>Ce-3vw-cGjlL@_5<&O3I9#0bk7v)co=RCfU2{(ToV1=F?*Pp*`naIZ z|A6Owzgw~GKB7w;vL#BM0rNCE#4jm2ye>^2DxdePGJF{TtjBCjgSCJ%?Gk@vD^1oL z|A4O&0hM+I^uB=i3Wj7g@6pTUrZ_o{2s-`*gVQcjp8wO)vwV6EocT7KKa+VekwN`# z2jGM=q$goG*W9LS4({1&tl=20)y4eoh(>=H53+{Nb;K26&R=+%j(k*$9(8)k1!=M) z6vf<$uaqLc$`{^4)~@)g^i=^CNV~U&%1M`4seDxww3l2HIXocs%OO_H-LTil8JQUs z%ZPXSdE^ep+FH=XEqYhFW#R9BodAniwYJ?pmWUu(Z_cO<+T}Bifa3(fJ%^Q0<`dk+ zYhKUP>$X~g4-`EMr^)^tR3y7xLM@~6)b|fqt#ar!C5xpkwFmhNWfZ2LI+2S}SBV2x zB(wf_{2CB}Cuw79N_2Gf+xo5mUqwHz==4c`ZL5iY5@;9SCw@xE&%M?D zL&>(=uW-p1p&+FV?(Y~rVf#KB6OQg5GIXBXnqokU2y{Ps^Gv^u8lkbtmw49SlCK}t)2nDUMIjZE;=UStdZK@qCJyQV= zp}-p8p1`M|^nXV3$-l>N_oe^b&zO4x-7b8=#0SQBpQA$}EwgFC^lsksJS7)P6D2s? zu#!K5Ha>T5u|0UjMmB}z#5Rjtz*=H6vPbuch0v{8oD#{A=>Z(#z~b+;UNfW!%}<|jQiymmkMH1jH&MDh%qWbOIS^To7mzXbQI?Yz2#7~2HW)cE!>IK+~} z&CI&zV0Tnp{BL-dyk|`x{3`Wijm}_j@ZmMrcQ^sqG(`g53B7f%pHqFNMtOTOgwY#5 z@K-pXV41RwDvCwzQn7zT@TWvw&Ta{(s5KuIU!7Ql(6Wl$KHR<@*deFVdF*g|$^~Q& zJxonM_;Z3F{yMb4mObga_P_B)Ea_Lc{rJb($t2VP-aMD_ki`mVyxB|df5Jnh)2B<=M&%%% zCvB=0^IxXAfW2Je4@*u*_ElO{lqT;#PzE@50DL*`)^kz=wlusOnIN3@5tn_gsH z2VYlz(k|9_I-J7W0|(4+b+|ZT&-H~2n_~>hN-^NXXa7}d;QaVrZ8DFYE6LF-fJY4@ zaQ{8D8=v;q{-j}=?tJ#4YGk-9Utpj<9^FKj^}^dJc(KNDvqQFhpU?CC>&h%)5((gw zOBctC+Ruph88YwE0XKwar)^uMGqPqMC4WYhkG{|s$&Svx5I5xlMR$hi%%UebPwfzsYx=++e73Tv<|vT?P5@Z|s#LS~Tj_u|_rzcCi{jY3ds7wA zJ8XR|Ox`!pm*%iyd_N!$e;14XTr~Zxa}#iRx{^Ezf#-bbkO+1PldEGOPCrjAKd^+- zHhJls6x{<;Wx(tW`+KhEc_~uzjs#!ano6s_^3@na-WE12=EalUfYF=m>Pa96$-}2} X7;-Zk*B`ohbhoWo-401f;RXK(?Rkf` literal 0 HcmV?d00001 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt new file mode 100644 index 00000000000..0d87e8d0e8d --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt @@ -0,0 +1,183 @@ +columns format version: 1 +181 columns: +`CounterID` UInt32 +`StartDate` Date +`Sign` Int8 +`IsNew` UInt8 +`VisitID` UInt64 +`UserID` UInt64 +`StartTime` DateTime +`Duration` UInt32 +`UTCStartTime` DateTime +`PageViews` Int32 +`Hits` Int32 +`IsBounce` UInt8 +`Referer` String +`StartURL` String +`RefererDomain` String +`StartURLDomain` String +`EndURL` String +`LinkURL` String +`IsDownload` UInt8 +`TraficSourceID` Int8 +`SearchEngineID` UInt16 +`SearchPhrase` String +`AdvEngineID` UInt8 +`PlaceID` Int32 +`RefererCategories` Array(UInt16) +`URLCategories` Array(UInt16) +`URLRegions` Array(UInt32) +`RefererRegions` Array(UInt32) +`IsYandex` UInt8 +`GoalReachesDepth` Int32 +`GoalReachesURL` Int32 +`GoalReachesAny` Int32 +`SocialSourceNetworkID` UInt8 +`SocialSourcePage` String +`MobilePhoneModel` String +`ClientEventTime` DateTime +`RegionID` UInt32 +`ClientIP` UInt32 +`ClientIP6` FixedString(16) +`RemoteIP` UInt32 +`RemoteIP6` FixedString(16) +`IPNetworkID` UInt32 +`SilverlightVersion3` UInt32 +`CodeVersion` UInt32 +`ResolutionWidth` UInt16 +`ResolutionHeight` UInt16 +`UserAgentMajor` UInt16 +`UserAgentMinor` UInt16 +`WindowClientWidth` UInt16 +`WindowClientHeight` UInt16 +`SilverlightVersion2` UInt8 +`SilverlightVersion4` UInt16 +`FlashVersion3` UInt16 +`FlashVersion4` UInt16 +`ClientTimeZone` Int16 +`OS` UInt8 +`UserAgent` UInt8 +`ResolutionDepth` UInt8 +`FlashMajor` UInt8 +`FlashMinor` UInt8 +`NetMajor` UInt8 +`NetMinor` UInt8 +`MobilePhone` UInt8 +`SilverlightVersion1` UInt8 +`Age` UInt8 +`Sex` UInt8 +`Income` UInt8 +`JavaEnable` UInt8 +`CookieEnable` UInt8 +`JavascriptEnable` UInt8 +`IsMobile` UInt8 +`BrowserLanguage` UInt16 +`BrowserCountry` UInt16 +`Interests` UInt16 +`Robotness` UInt8 +`GeneralInterests` Array(UInt16) +`Params` Array(String) +`Goals.ID` Array(UInt32) +`Goals.Serial` Array(UInt32) +`Goals.EventTime` Array(DateTime) +`Goals.Price` Array(Int64) +`Goals.OrderID` Array(String) +`Goals.CurrencyID` Array(UInt32) +`WatchIDs` Array(UInt64) +`ParamSumPrice` Int64 +`ParamCurrency` FixedString(3) +`ParamCurrencyID` UInt16 +`ClickLogID` UInt64 +`ClickEventID` Int32 +`ClickGoodEvent` Int32 +`ClickEventTime` DateTime +`ClickPriorityID` Int32 +`ClickPhraseID` Int32 +`ClickPageID` Int32 +`ClickPlaceID` Int32 +`ClickTypeID` Int32 +`ClickResourceID` Int32 +`ClickCost` UInt32 +`ClickClientIP` UInt32 +`ClickDomainID` UInt32 +`ClickURL` String +`ClickAttempt` UInt8 +`ClickOrderID` UInt32 +`ClickBannerID` UInt32 +`ClickMarketCategoryID` UInt32 +`ClickMarketPP` UInt32 +`ClickMarketCategoryName` String +`ClickMarketPPName` String +`ClickAWAPSCampaignName` String +`ClickPageName` String +`ClickTargetType` UInt16 +`ClickTargetPhraseID` UInt64 +`ClickContextType` UInt8 +`ClickSelectType` Int8 +`ClickOptions` String +`ClickGroupBannerID` Int32 +`OpenstatServiceName` String +`OpenstatCampaignID` String +`OpenstatAdID` String +`OpenstatSourceID` String +`UTMSource` String +`UTMMedium` String +`UTMCampaign` String +`UTMContent` String +`UTMTerm` String +`FromTag` String +`HasGCLID` UInt8 +`FirstVisit` DateTime +`PredLastVisit` Date +`LastVisit` Date +`TotalVisits` UInt32 +`TraficSource.ID` Array(Int8) +`TraficSource.SearchEngineID` Array(UInt16) +`TraficSource.AdvEngineID` Array(UInt8) +`TraficSource.PlaceID` Array(UInt16) +`TraficSource.SocialSourceNetworkID` Array(UInt8) +`TraficSource.Domain` Array(String) +`TraficSource.SearchPhrase` Array(String) +`TraficSource.SocialSourcePage` Array(String) +`Attendance` FixedString(16) +`CLID` UInt32 +`YCLID` UInt64 +`NormalizedRefererHash` UInt64 +`SearchPhraseHash` UInt64 +`RefererDomainHash` UInt64 +`NormalizedStartURLHash` UInt64 +`StartURLDomainHash` UInt64 +`NormalizedEndURLHash` UInt64 +`TopLevelDomain` UInt64 +`URLScheme` UInt64 +`OpenstatServiceNameHash` UInt64 +`OpenstatCampaignIDHash` UInt64 +`OpenstatAdIDHash` UInt64 +`OpenstatSourceIDHash` UInt64 +`UTMSourceHash` UInt64 +`UTMMediumHash` UInt64 +`UTMCampaignHash` UInt64 +`UTMContentHash` UInt64 +`UTMTermHash` UInt64 +`FromHash` UInt64 +`WebVisorEnabled` UInt8 +`WebVisorActivity` UInt32 +`ParsedParams.Key1` Array(String) +`ParsedParams.Key2` Array(String) +`ParsedParams.Key3` Array(String) +`ParsedParams.Key4` Array(String) +`ParsedParams.Key5` Array(String) +`ParsedParams.ValueDouble` Array(Float64) +`Market.Type` Array(UInt8) +`Market.GoalID` Array(UInt32) +`Market.OrderID` Array(String) +`Market.OrderPrice` Array(Int64) +`Market.PP` Array(UInt32) +`Market.DirectPlaceID` Array(UInt32) +`Market.DirectOrderID` Array(UInt32) +`Market.DirectBannerID` Array(UInt32) +`Market.GoodID` Array(String) +`Market.GoodName` Array(String) +`Market.GoodQuantity` Array(Int32) +`Market.GoodPrice` Array(Int64) +`IslandID` FixedString(16) diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt new file mode 100644 index 00000000000..91d679bb3ab --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt @@ -0,0 +1 @@ +485007 \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx new file mode 100644 index 00000000000..0cb239d932a --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx @@ -0,0 +1 @@ +?? \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/partition.dat b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..9c176950d3f897f5cc9714d7011107a21e47a229 GIT binary patch literal 4 Lcmdlj#LNHy11td1 literal 0 HcmV?d00001 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/primary.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/primary.idx new file mode 100644 index 0000000000000000000000000000000000000000..6765990e38d151d57b7f18ee97cef776d10afa02 GIT binary patch literal 1314 zcmWlWeLT|%9LIkfbtfq=jO5`;VIDKrgED^QVO-ecan`Y3+vPFyba~p2$@RqD@@i2{ zj>>tskZy@YSmSc#p(3L^OkGSfgc6qf_V4HQ{k%V)@AqqiAc!t{Fjmofqgz$l&^?9GGcP7X*~wPp&bP(`jJIVv)0UyaSh*M zkuB(XdTF)(i?wCF@H^}&##iOQF2_o~-?g#M!Im&&1`Dn|yKj^p)NMXm>R#*m@vsVz zZPu-jeCFLQ^*t=4wO&zy^zU)wnbq%T1!(hoy2-Dqh&DPQy4R{(Jf|lS`+48Rtq1Z% z+p6j7Aj-yjDKgiGgKD6p7;jXsEbNaG5W+dm#7&^^dvi-JGbrjCHOfQztD2DYU%(d! z_km{*?D<8$u8rD=7HVi!`$;+)YUcXREOR_e7t*5_Sy7x$jSSCB9R@B@AJIWIMzef< z3Uw7OG_zz{t^v53DhRdu7Ts9UVjjbIt1;~5oEx$&ZnkY`XEKg6Q1=~l9;uuQSab-U z#&&&*lslThj3&V`5YsPdF+2MuVBabpIL9b?mi=3wdnT~g2Ro7Np>s~>|6Bs(AUElxC6TdRbjifCdaVjLpWd!AZw5AV&PY5$@T}XA z^(4s1ON}ZHvdGCGNg9RVBH+)R8dqiZA=7xW-8B*1Fucm!@4hc{>u-malLxhWAZu@w z>3}+RKS%!N^eBM4i}qK|Q#IO!>bB`~e)>_iKo*9*A?|!p zXVNE9x)4w7!RVl$OP|C#G)NWF;ge|*8blx;7~?4tsMQkBBDQZviaU_!V*cXFjvM8x z)Y4c6VHDsXJ%ht38|aCSp+a{%E)`~wq?dU2+D(Mbt!XU3xiIML(%*IN=>qKrH$H-# z+{DD-(9xe%Vkx|(>6@RWjC?c}vW707wIzx0JeK%rQn5n{C_33M>P`}RWBddyB&^9P zAYW=9S!vp$h>sXJkV(z`9n#0rXCkeFy>DF)%Am-9xB^-A7uuAI`$;vt9$XA9rx^CC z^v>v3dOAO+mFLBZLrTEKWf^;w`U`FDdE5%?W0!)WUE{P=rKmK}F40f7XsrU5@xd(f zyy@1>lcGKTq?J3M9P+zky~BBxX~_A*+b+EUvTDWzf`uoxMQ-jhNn-v(n8^5p;em`cqBLGPl2ap)i&2)(v|-Qn7p;U%sQbWzw7yEO`m8-r)J^xyXs) zEPHDF#dT6}c_cgvT|2Jy4E0w{MMkC|i|L@K-t{5Ec11Xp*e;)ipUxMUZo|{*nv>t~ zJ*=%i!xs75-@o78c!U$0_#yQ&bs0F+Y{rsMi*sH!>>AObT!YI?zVnalJ{)GSWU3eb EKZSBM)c^nh literal 0 HcmV?d00001 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/checksums.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..1200b25bd52ed0332f4e976e5e109661007e72bb GIT binary patch literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$axV}lqE!_QlD*Y>FAiIEP1 z+5i9m&;S4c@jnI$V{dhCbS`vwbOV+!t186UB-eP+=laeWci#XAWMOn+E@EkJtA+vp zC@FDdb-WItZml__r5>08+N>Jd_3*EI_1(J_Im&g zaA9(EX>@6CZZ2eDbOiRJGt`jAL&ph4JQlLni7WsMaB^vFVRCscX=Hc|qQj~+&$gS+ M9BY!1)str&05BDACIA2c literal 0 HcmV?d00001 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt new file mode 100644 index 00000000000..4835061786d --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt @@ -0,0 +1,259 @@ +columns format version: 1 +257 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`milliseconds` UInt64 +`ProfileEvent_Query` UInt64 +`ProfileEvent_SelectQuery` UInt64 +`ProfileEvent_InsertQuery` UInt64 +`ProfileEvent_FailedQuery` UInt64 +`ProfileEvent_FailedSelectQuery` UInt64 +`ProfileEvent_FailedInsertQuery` UInt64 +`ProfileEvent_QueryTimeMicroseconds` UInt64 +`ProfileEvent_SelectQueryTimeMicroseconds` UInt64 +`ProfileEvent_InsertQueryTimeMicroseconds` UInt64 +`ProfileEvent_FileOpen` UInt64 +`ProfileEvent_Seek` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64 +`ProfileEvent_ReadBufferAIORead` UInt64 +`ProfileEvent_ReadBufferAIOReadBytes` UInt64 +`ProfileEvent_WriteBufferAIOWrite` UInt64 +`ProfileEvent_WriteBufferAIOWriteBytes` UInt64 +`ProfileEvent_ReadCompressedBytes` UInt64 +`ProfileEvent_CompressedReadBufferBlocks` UInt64 +`ProfileEvent_CompressedReadBufferBytes` UInt64 +`ProfileEvent_UncompressedCacheHits` UInt64 +`ProfileEvent_UncompressedCacheMisses` UInt64 +`ProfileEvent_UncompressedCacheWeightLost` UInt64 +`ProfileEvent_IOBufferAllocs` UInt64 +`ProfileEvent_IOBufferAllocBytes` UInt64 +`ProfileEvent_ArenaAllocChunks` UInt64 +`ProfileEvent_ArenaAllocBytes` UInt64 +`ProfileEvent_FunctionExecute` UInt64 +`ProfileEvent_TableFunctionExecute` UInt64 +`ProfileEvent_MarkCacheHits` UInt64 +`ProfileEvent_MarkCacheMisses` UInt64 +`ProfileEvent_CreatedReadBufferOrdinary` UInt64 +`ProfileEvent_CreatedReadBufferAIO` UInt64 +`ProfileEvent_CreatedReadBufferAIOFailed` UInt64 +`ProfileEvent_CreatedReadBufferMMap` UInt64 +`ProfileEvent_CreatedReadBufferMMapFailed` UInt64 +`ProfileEvent_CreatedWriteBufferOrdinary` UInt64 +`ProfileEvent_CreatedWriteBufferAIO` UInt64 +`ProfileEvent_CreatedWriteBufferAIOFailed` UInt64 +`ProfileEvent_DiskReadElapsedMicroseconds` UInt64 +`ProfileEvent_DiskWriteElapsedMicroseconds` UInt64 +`ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64 +`ProfileEvent_NetworkSendElapsedMicroseconds` UInt64 +`ProfileEvent_ThrottlerSleepMicroseconds` UInt64 +`ProfileEvent_QueryMaskingRulesMatch` UInt64 +`ProfileEvent_ReplicatedPartFetches` UInt64 +`ProfileEvent_ReplicatedPartFailedFetches` UInt64 +`ProfileEvent_ObsoleteReplicatedParts` UInt64 +`ProfileEvent_ReplicatedPartMerges` UInt64 +`ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64 +`ProfileEvent_ReplicatedPartMutations` UInt64 +`ProfileEvent_ReplicatedPartChecks` UInt64 +`ProfileEvent_ReplicatedPartChecksFailed` UInt64 +`ProfileEvent_ReplicatedDataLoss` UInt64 +`ProfileEvent_InsertedRows` UInt64 +`ProfileEvent_InsertedBytes` UInt64 +`ProfileEvent_DelayedInserts` UInt64 +`ProfileEvent_RejectedInserts` UInt64 +`ProfileEvent_DelayedInsertsMilliseconds` UInt64 +`ProfileEvent_DuplicatedInsertedBlocks` UInt64 +`ProfileEvent_ZooKeeperInit` UInt64 +`ProfileEvent_ZooKeeperTransactions` UInt64 +`ProfileEvent_ZooKeeperList` UInt64 +`ProfileEvent_ZooKeeperCreate` UInt64 +`ProfileEvent_ZooKeeperRemove` UInt64 +`ProfileEvent_ZooKeeperExists` UInt64 +`ProfileEvent_ZooKeeperGet` UInt64 +`ProfileEvent_ZooKeeperSet` UInt64 +`ProfileEvent_ZooKeeperMulti` UInt64 +`ProfileEvent_ZooKeeperCheck` UInt64 +`ProfileEvent_ZooKeeperClose` UInt64 +`ProfileEvent_ZooKeeperWatchResponse` UInt64 +`ProfileEvent_ZooKeeperUserExceptions` UInt64 +`ProfileEvent_ZooKeeperHardwareExceptions` UInt64 +`ProfileEvent_ZooKeeperOtherExceptions` UInt64 +`ProfileEvent_ZooKeeperWaitMicroseconds` UInt64 +`ProfileEvent_ZooKeeperBytesSent` UInt64 +`ProfileEvent_ZooKeeperBytesReceived` UInt64 +`ProfileEvent_DistributedConnectionFailTry` UInt64 +`ProfileEvent_DistributedConnectionMissingTable` UInt64 +`ProfileEvent_DistributedConnectionStaleReplica` UInt64 +`ProfileEvent_DistributedConnectionFailAtAll` UInt64 +`ProfileEvent_CompileAttempt` UInt64 +`ProfileEvent_CompileSuccess` UInt64 +`ProfileEvent_CompileFunction` UInt64 +`ProfileEvent_CompiledFunctionExecute` UInt64 +`ProfileEvent_CompileExpressionsMicroseconds` UInt64 +`ProfileEvent_CompileExpressionsBytes` UInt64 +`ProfileEvent_ExternalSortWritePart` UInt64 +`ProfileEvent_ExternalSortMerge` UInt64 +`ProfileEvent_ExternalAggregationWritePart` UInt64 +`ProfileEvent_ExternalAggregationMerge` UInt64 +`ProfileEvent_ExternalAggregationCompressedBytes` UInt64 +`ProfileEvent_ExternalAggregationUncompressedBytes` UInt64 +`ProfileEvent_SlowRead` UInt64 +`ProfileEvent_ReadBackoff` UInt64 +`ProfileEvent_ReplicaPartialShutdown` UInt64 +`ProfileEvent_SelectedParts` UInt64 +`ProfileEvent_SelectedRanges` UInt64 +`ProfileEvent_SelectedMarks` UInt64 +`ProfileEvent_SelectedRows` UInt64 +`ProfileEvent_SelectedBytes` UInt64 +`ProfileEvent_Merge` UInt64 +`ProfileEvent_MergedRows` UInt64 +`ProfileEvent_MergedUncompressedBytes` UInt64 +`ProfileEvent_MergesTimeMilliseconds` UInt64 +`ProfileEvent_MergeTreeDataWriterRows` UInt64 +`ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64 +`ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64 +`ProfileEvent_MergeTreeDataWriterBlocks` UInt64 +`ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64 +`ProfileEvent_CannotRemoveEphemeralNode` UInt64 +`ProfileEvent_RegexpCreated` UInt64 +`ProfileEvent_ContextLock` UInt64 +`ProfileEvent_StorageBufferFlush` UInt64 +`ProfileEvent_StorageBufferErrorOnFlush` UInt64 +`ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64 +`ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64 +`ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64 +`ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64 +`ProfileEvent_DictCacheKeysRequested` UInt64 +`ProfileEvent_DictCacheKeysRequestedMiss` UInt64 +`ProfileEvent_DictCacheKeysRequestedFound` UInt64 +`ProfileEvent_DictCacheKeysExpired` UInt64 +`ProfileEvent_DictCacheKeysNotFound` UInt64 +`ProfileEvent_DictCacheKeysHit` UInt64 +`ProfileEvent_DictCacheRequestTimeNs` UInt64 +`ProfileEvent_DictCacheRequests` UInt64 +`ProfileEvent_DictCacheLockWriteNs` UInt64 +`ProfileEvent_DictCacheLockReadNs` UInt64 +`ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64 +`ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64 +`ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64 +`ProfileEvent_PolygonsAddedToPool` UInt64 +`ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64 +`ProfileEvent_RWLockAcquiredReadLocks` UInt64 +`ProfileEvent_RWLockAcquiredWriteLocks` UInt64 +`ProfileEvent_RWLockReadersWaitMilliseconds` UInt64 +`ProfileEvent_RWLockWritersWaitMilliseconds` UInt64 +`ProfileEvent_DNSError` UInt64 +`ProfileEvent_RealTimeMicroseconds` UInt64 +`ProfileEvent_UserTimeMicroseconds` UInt64 +`ProfileEvent_SystemTimeMicroseconds` UInt64 +`ProfileEvent_SoftPageFaults` UInt64 +`ProfileEvent_HardPageFaults` UInt64 +`ProfileEvent_VoluntaryContextSwitches` UInt64 +`ProfileEvent_InvoluntaryContextSwitches` UInt64 +`ProfileEvent_OSIOWaitMicroseconds` UInt64 +`ProfileEvent_OSCPUWaitMicroseconds` UInt64 +`ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64 +`ProfileEvent_OSReadBytes` UInt64 +`ProfileEvent_OSWriteBytes` UInt64 +`ProfileEvent_OSReadChars` UInt64 +`ProfileEvent_OSWriteChars` UInt64 +`ProfileEvent_PerfCpuCycles` UInt64 +`ProfileEvent_PerfInstructions` UInt64 +`ProfileEvent_PerfCacheReferences` UInt64 +`ProfileEvent_PerfCacheMisses` UInt64 +`ProfileEvent_PerfBranchInstructions` UInt64 +`ProfileEvent_PerfBranchMisses` UInt64 +`ProfileEvent_PerfBusCycles` UInt64 +`ProfileEvent_PerfStalledCyclesFrontend` UInt64 +`ProfileEvent_PerfStalledCyclesBackend` UInt64 +`ProfileEvent_PerfRefCpuCycles` UInt64 +`ProfileEvent_PerfCpuClock` UInt64 +`ProfileEvent_PerfTaskClock` UInt64 +`ProfileEvent_PerfContextSwitches` UInt64 +`ProfileEvent_PerfCpuMigrations` UInt64 +`ProfileEvent_PerfAlignmentFaults` UInt64 +`ProfileEvent_PerfEmulationFaults` UInt64 +`ProfileEvent_PerfMinEnabledTime` UInt64 +`ProfileEvent_PerfMinEnabledRunningTime` UInt64 +`ProfileEvent_PerfDataTLBReferences` UInt64 +`ProfileEvent_PerfDataTLBMisses` UInt64 +`ProfileEvent_PerfInstructionTLBReferences` UInt64 +`ProfileEvent_PerfInstructionTLBMisses` UInt64 +`ProfileEvent_PerfLocalMemoryReferences` UInt64 +`ProfileEvent_PerfLocalMemoryMisses` UInt64 +`ProfileEvent_CreatedHTTPConnections` UInt64 +`ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64 +`ProfileEvent_QueryProfilerSignalOverruns` UInt64 +`ProfileEvent_CreatedLogEntryForMerge` UInt64 +`ProfileEvent_NotCreatedLogEntryForMerge` UInt64 +`ProfileEvent_CreatedLogEntryForMutation` UInt64 +`ProfileEvent_NotCreatedLogEntryForMutation` UInt64 +`ProfileEvent_S3ReadMicroseconds` UInt64 +`ProfileEvent_S3ReadBytes` UInt64 +`ProfileEvent_S3ReadRequestsCount` UInt64 +`ProfileEvent_S3ReadRequestsErrors` UInt64 +`ProfileEvent_S3ReadRequestsThrottling` UInt64 +`ProfileEvent_S3ReadRequestsRedirects` UInt64 +`ProfileEvent_S3WriteMicroseconds` UInt64 +`ProfileEvent_S3WriteBytes` UInt64 +`ProfileEvent_S3WriteRequestsCount` UInt64 +`ProfileEvent_S3WriteRequestsErrors` UInt64 +`ProfileEvent_S3WriteRequestsThrottling` UInt64 +`ProfileEvent_S3WriteRequestsRedirects` UInt64 +`ProfileEvent_QueryMemoryLimitExceeded` UInt64 +`CurrentMetric_Query` Int64 +`CurrentMetric_Merge` Int64 +`CurrentMetric_PartMutation` Int64 +`CurrentMetric_ReplicatedFetch` Int64 +`CurrentMetric_ReplicatedSend` Int64 +`CurrentMetric_ReplicatedChecks` Int64 +`CurrentMetric_BackgroundPoolTask` Int64 +`CurrentMetric_BackgroundMovePoolTask` Int64 +`CurrentMetric_BackgroundSchedulePoolTask` Int64 +`CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64 +`CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64 +`CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64 +`CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64 +`CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64 +`CurrentMetric_DiskSpaceReservedForMerge` Int64 +`CurrentMetric_DistributedSend` Int64 +`CurrentMetric_QueryPreempted` Int64 +`CurrentMetric_TCPConnection` Int64 +`CurrentMetric_MySQLConnection` Int64 +`CurrentMetric_HTTPConnection` Int64 +`CurrentMetric_InterserverConnection` Int64 +`CurrentMetric_PostgreSQLConnection` Int64 +`CurrentMetric_OpenFileForRead` Int64 +`CurrentMetric_OpenFileForWrite` Int64 +`CurrentMetric_Read` Int64 +`CurrentMetric_Write` Int64 +`CurrentMetric_SendScalars` Int64 +`CurrentMetric_SendExternalTables` Int64 +`CurrentMetric_QueryThread` Int64 +`CurrentMetric_ReadonlyReplica` Int64 +`CurrentMetric_MemoryTracking` Int64 +`CurrentMetric_EphemeralNode` Int64 +`CurrentMetric_ZooKeeperSession` Int64 +`CurrentMetric_ZooKeeperWatch` Int64 +`CurrentMetric_ZooKeeperRequest` Int64 +`CurrentMetric_DelayedInserts` Int64 +`CurrentMetric_ContextLockWait` Int64 +`CurrentMetric_StorageBufferRows` Int64 +`CurrentMetric_StorageBufferBytes` Int64 +`CurrentMetric_DictCacheRequests` Int64 +`CurrentMetric_Revision` Int64 +`CurrentMetric_VersionInteger` Int64 +`CurrentMetric_RWLockWaitingReaders` Int64 +`CurrentMetric_RWLockWaitingWriters` Int64 +`CurrentMetric_RWLockActiveReaders` Int64 +`CurrentMetric_RWLockActiveWriters` Int64 +`CurrentMetric_GlobalThread` Int64 +`CurrentMetric_GlobalThreadActive` Int64 +`CurrentMetric_LocalThread` Int64 +`CurrentMetric_LocalThreadActive` Int64 +`CurrentMetric_DistributedFilesToInsert` Int64 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt new file mode 100644 index 00000000000..1e3852840b5 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt @@ -0,0 +1 @@ +211 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx new file mode 100644 index 00000000000..4b6c49f8d50 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx @@ -0,0 +1 @@ +…H…H \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/partition.dat b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac GIT binary patch literal 4 Lcmb0VWo7^X0I>iw literal 0 HcmV?d00001 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx new file mode 100644 index 00000000000..714ba773856 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx @@ -0,0 +1 @@ +…H®-œ_…H€.œ_ \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/checksums.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..8031891da7f5bd9e7a744c4e46bca92f86a61006 GIT binary patch literal 11371 zcmYLPWk8f!7ryt6iFe{6BOODTASt1MgoHtZf{DX0z`!trGlPJLEP`Mwirs;Yxng&& z3brDyg|)^?n1I4}2Xw!GZ{9lRp65BwxzaS5G((e}sj*5?sWT;7t0^+IMy^s0wi?K6 zW%(_Ax9Uw%Cs*1leY>)FhMxlff0qK#8BQ+=2hf}xqKRzfe1@63$QiXG9$%}iEg9`GF^A|RoKKy+X18a0Xvu(BUq@n9ZMMb6< zY+Oa67{G?*;3s+CVc^~M{y{EFG(kPZ$}tU~e>tGsSn3eDLZ*pRt$<(=Ko40WVFN(X zfP@QyU?9oA2U$ZvWUNe?TndbMD+$hS78nM)W+>*vv%y(uvP_v;qKHr>mqP@hi1(CW zLIBFcOJJD?_J=A|NfJd|np!6L1e%Gt^z_R;y?6X=vSH1r4j1u?0bHUTR!QDpda}aK zn%|2%vtaC`gQoyD7{DCRPALCY@2;|c(d*dzr76Sh_8$Nj2#za=4#1Y1@&z=JGw3m6 zU!*u09qYa77Wdc$w<4EP+}`Z>*TvjP)umhdmafS#a9x$aLL&jdp4CopE?68{BAWf; zyUW7QuV?991_NOEu(-IWp(>?Pc9}+v(1_LPnWR`f16cE% zI4A;;Xf~dn#cbe(P5%?3Y(m5fBsf6;WdoAO&}^3N`4L%E(WvbM9 zacaDu<;v7H*hF1=4?}(P=&k)KTYAnM&yKvgzhNq#LELSv&GElAZXGKew^}56ZEc6V zVE|6U4sEb#jEgfm{49UNi zC_e7300^EFyaM1KnI%)+rH5kYiy1>Tkw-bDDTiDy-wQ7-SbyI9ai8O=CeRM8pG-gR z^z}*4_4V;SlY21I0?^$?C$Vrq=dp2_Yl2(-gQ`?6vtI&tzz&F4%e7;1V{M!XLW|2m zwGTcgZfqVgd0Obo>A0~ef_6|)1%95ry{t!JN!9or1baFlh67lLO}I3(S6d6~a^B3Q zquO~z-q(O0`A3vQtqsrCN-P*Ko8eq^0PR6hlv*as%+kt^vydD+|IYt>2l6~Vuw3eyQFiqqF#^R!=X4cnrz+ms=g+AM5>8ZHyFA@6`(@pIQOY_uIJlo!Eav)F` zHs4Ip2^}SFuU*O)E|;9LU48Q1(tlty(&AnjV5qYIRrD~#YsjU#z7hzbso{4Lj}By!O=KxeN9}o2=^HxT z;^4~{wGZj79oVdBYNuP=@KsCmFHc)|YxVWCJtXu6FctiOOFunL3Gp0napdWfo#lL| z4UsI+AM8^p6?rkTEQMSuQ3Lc9FYj9VmuJ2N0(Stj!9omDLv=8lilggoS}+`-LTpz% zisbMhrmC3GGwQ@{eS$#U?brY8A92?2p9< zG7>`Xe66{kNOs4{A8?7+t+tmt>OLJF)N|3LYsZ)I`0x!4)Oceprqjcgxo=O7Ngp3N za9*z6is^o=UACW0kQd=s9#=SK+uIiFMf>y zhASgp1MJ3L>I)}ItxCbD7d!=Xti%A!v?Kuvg-UuKPy0K)<<$K3n`ck(sXfwq{^*0- zr{BX)VuZPVc0z;OI#1L2gEkWVp>-%=FhX<^;TM7jd~QC&u_(KA_{hD+bzmT%1Ba$% zD_z-5WD9+ca$g%P-FEDS`rFOMgA3);@GqCG(eGE@wSze zz5C&2RBcAA|f&T8`>QISNsnW(^?3f~3qL0K}L$Q4{=&1wfB0}su1VfuM zX&t9x17ft93bQx7UA+1KXDX)RAh1Z4nT2N@pw(XHqcwz)*eV3mIt*)~ME@snPCkHtI$SPQt28pHN|`*HWz72miLsFV9B@Yiv=T#MJR~ARvX#>7 z$c&jwl5+svy{(WjL8xN0rBa!Og2w19AgFF$pnK6Xc@KyQ&Xq~Cx3chkUqK|$F-G1O z(pr_dmavi~59AXENw+|Ss*~45md9R{WqldCp{gTntH&a6zV==X^^=f!zmm3|&Gfx6 zY1W~fy|s-F=>%vbFyR51@ce)W91a@!ta`f1!)?-!8Nyu?9#E(pKx!|yi8@>XE4L0w zpJsjZ#WRcf6K*w?O)JNd?q&Va%Tvr=M{{x)7V1w=B5@!ik4Q2`IWcfoYV2Ls2Qq&<~VpH4Zx~W%> z>N{Q--*SC4)d}VI={-r4MY*3|<>kyi8J+h7^66-Ry|@#G^Y;%cT<17u*0Le5seiwv z?SLeECl{R|@N6N`jYEPYN_WHHG3A3)?jI#Ug6$9JF|dh+6QIp`&&-;xDHD#) zo8jj>bd%-ZHalS_+{c`7PJDGZIO}oawVL{ve_jCrjVD|VTFEAdV7uU4O?yaR`u`1kO_-hxcNSnp03Dg-MH8H0 zNP7e94d%=;Hd+8k;+Kg%NW9X7Q@C7DXo6mW9(YT1d6=D?Ai->NFn}YD2&Rlt5lOMT z0SkEn4rH@Y2{1t^QFX0y_&n&}f6ux7+`)AgQ=p=%~{V#;ZV%vI1 z(%nIvA4&Aku!*Q(g(OQej9{vJG?722*k^6tDA-$0Z0#_3@os%%+(OO|IIm~PoPVc1 zTVGkyZXO-Y;|?UTldy3X*g;ITT(RiR)xB?D4)w`9mJ{X<`~kcMQ3b%8cg0~yOKC{i!1scaCkCuoSaQ(XH6tFYj0#cyw&SC3eE$T1QKWM&Vg4!_YM+s_t-1UCjsI zDt)!g)XAqopRM1QRc;GLFo^+L3?)-&Bc?$gjRt(U5(r|r0w+iva$;5j{kR`I6u}h< z93hGWY_t@w{6a1Y01#ACE|{VjnIjiyWbSqa3ReN{7=YG;-ifhVi9%M!SmGPK?q{Uf z>lFr?*gU25or5#ZVk@cjs%&j=u2d#VmakJGkLuU;KZ zNV?bv+F(S7+>|&Y`E>1oQIr`$agQoYt%02sO*_)f+Es?J& z9^bt@X-?wsF^&c0O9I8GdMAE2z&HTe+sRLy6>%TjV~YW=VQ71=X=_N#xt&qAkk!R@Pyqb!Ut^ZmUW;ynO?`@r@;wD3? zSDorT^}x9;TTFHYK0LYRr5=tmnDVs|F}4S?QH|5gEBp#W4y*7WSPpH5N1m}|8pTc2 z$+eLA2kvq5z5ZbCsEU*CH*Om{9k+Ji`dnVp+tJti1A)iQ^dmdd1YJaS2A*l}e~wjO zbUr@*pJT?GQ%?3NYzJIV0D1f-@{eho23)gn@o#zMI;eN6)x?mu1-L0^^6rj>yX^yM zd*>PDRUS{nW~V;&5pGLKw9og6uYGJA4#)b@=3}Y1*sbJ#1f+&)f6)Y-bRL zLyasS537^vv(zZI<(Omh*7Mfd>_a?mlb*+c9=FiTVxG}>k z`WD3(8{3Y~+haVId9&vT`P90&-Ps56!h zvKc1-ikWUFJ$SWICF^v+;K-8c&CTmpZ)3wYINP?`-79}#ab4@N&GDW#Sp^W!fxub} zICFKux{A98wkz&U@3kQz`7K4e0uaI{4)@XNFg`QOqHDgTZCG3zYS&WCS%_T@a*8kf zyp=oH`em5(>+{-UkKR(<{w*lCH!)NZv=66={1o+9{gpmG17v;C%7n1_KEBqIF?GtB zv)TU2S?f!lGs{LUXrHyl2*(+mD7M2*ZkZK(SLXSd{|w9v7==teTjIw>aX1MN!ocjp zF;rJ#x;QJ>_;5zQhY8Due*6#yg_#rY=Hua1hMn`-eyHWVW`~6H_nBn2H-U97HYOk) zWVGezF)Js3Fx?&{<=?7&@)~#^#YhC8Re<8GhhE-jF7X(<(3~8v@Dhuo{oO(j-Cp

tE*-EM!D=JNt|@+#bc$NKO04cs5&wmf+Gu>W_lI>|ngkKO=) zBKe?+y1fey%C5RKs^DYPnIRirzL<6X|;J~dwz#j+T zCcJ@voL_Y@sQ~1DuGzQiqT4wc$kj)i@UmbK(@~#_w9_8K+X{~d4dU_TIhn?VWy|oK zp9MelYm8emy$2Z>e`KRC2%%%cW_&I(}!ANkVFv8=^n!(`991K?XEigeByruJ zy}wz*<3L}N!DddGG?C9Hid;KwlsO@zr(EWU!)}Ku=iv6@kB48n?02b5Qa;#K8`V&s zPXJ*Az-+u7ZVaol+{)Wld!((^Xx06my)5*fM#!`|Ds@JTOe&L4c`d?oKEYJ%bLi3E z*>eArdHOC9$rp1f@SGPly?lQDX6tE}#FMDJ+v5}o$KGW6|FtJay5jkT z#Iy0`??hxsx9GYCTbFEPnLoZ%ddYn-J8?pw9{X;O{%df18oyQN{2sJ(BS`j0n>%-} zz7zc-9PKcnPA}bj&Ex1n6YjS@|M+qE0k#|FgU4+-qim=SU*xYRVh*r%JX04M4zQWw z2>Ah$^szu|%#9ei0cJWpv|NY1%(=+ep;4no%hlR!iK3BU?in_bE7|E29`1OrPJDaQ zwDE$dwJ#X+6{}T+=Fmq0LICBW; z7y%Zf+F=3lETUj-+@;lxa~7AR3l|lbL`KFAk1Pcku{Y2czD)Cm&` zD@Jcg&sed%dURyVR?j&aK-dmVGjPShgKdz#Yv$69aO(|M%s)ClN%C&Qu2!%hxyQ+U z1-ILh#!d;fx$;f}AkoE9eh`B#LI+U_ip(5Z6pn^-k=~7fyj%modIs5A% z5ff+TcsS#+j!q=f%43^`>|js2le4kxx~*$Z9=8iGI^pK5CEl~1Y-Ak>lPdgj*jZSm z#)PmdY;>}o_fC@y`;pMP>hQsph0EXCWrJ3t;zPlqpcnD)nsV&@1gEvemc7|zHXwe% z&+=KXtr_l!L!-*8{Eq1`E+A zU?KE|oNjeOm``AJc1Da0*VAN5X}A!np$I?ZcV#e!#F|i4gT#1HfY+VVFkdNH1f8?8 zhrSwsNRE-vhoHL!f8>IhY9`?brU0V5PoW7pd1zL4xI9&zX^LKv#um2K6w^g84GMKY z6&Duxk+2XJg6@YT)5!+!Ac@Iu3t=sEr-j|wnlOh92+qt_%z&tccIqA)$?Rw0ZN>gT zV>WEX&=&vE@eW4l3(z`f&2%uoskShRX4^p2k2DslNpAO233m`MC4~48j2uV{eDk}r z`;d*3>t*==G7(q?bwwbIfd*0Ga%He`tM4DlG1*Eb&QY8MsG2qsF6=53;V(#G35AP9 zXcjON_Mmv(+N5wZVI3no!6FD2I>|D`x4(N2q|~~EMj=af<$@t`ATJDOhK51~fhR)f z@(P?(@U24BTnBiN5Y-%nRe{*lL%6dW|N%fYos2bO4u`gQMUMOx?PIAbYwll}H znl;RJOYwZKL9=T92sl?yN+uS{(dvQK|FjKqirW~y7NcbH^qq**@DZ$yN{|gzz36QmOecn zpg&0w<3=#^06Pl6`9N@nV46w1idau@#bxn!5p%)?YsKuAhzTL<3ktW?a7(7A31LN(zT-#1II_ZKS~~>cWA}Krfr?uG7l|y7HwF zGn9*mPoA|Qe`SUmQh+yTBFmhS)Bi;yGY=e%fSw1avMUXeX{2g-mbNIWiM%l3H3E#H7ru-7lg~yKf z_uHM9I*P=&S}%gzF|0ZE^jr2pRB2ySJceI=E}c6;j79(fI^}J_;LC|vKHdQYa*|2p zf{}p#$+C%!!EVk1n0>?S!^|0nr3)>V9w@E%j`hRKZ`B%peQcJ~+T9+9JO8)||C+?@ zsza3kL7SJr3m|J#Oc~d*Gi)^)N8(YnYecHY_@r&Z5?p57GNn3wVA7)WMZUzn=X;l! z^-dv#U6rYJxFctIcHE*B_ZM~S7mS;AYUNW#8Y4@Uf%Z=UcSaOr$w*(J`pn?BN~Om~Xa4*${%bqpNYROePv*%9NWo%qcL zSJu;Uejhr{n)eMb-6h?2$X8RaFap>a?~_Xuv1!@bWL1u`9iY#&2*1L1C3GmmccGy- z(1BULzEhZLnY#(UmJoD@HhvL7c>*AS$T-ph-{8gQ61SLj0$gl^s1Idf=3qbwv7X(n zlwoqkCGeZfNEB^k!n|Ub#enG+*U$vW#WBwkVMHeTnBiiYPV?ZobGc(6IT%<+rchRB z9WusGupcvBetntX!UX`<(C7v!Lg|o8{u0+X6hV%L)X8`pit$+k9@h}Ex^;CXn6Zj= zTg;+G9K^=38O;24{A(294EV(YG!-N9H3~aP*YN{{lX=Qm5;+u~Ix$KzW+Dk<24MbE z_qgz`KkCG|Tt+Y@;8*{;n?Yb5_U#INj+=8pY#{g7Xtzy6gWQt{| zGGTW#(8YZIEdnkVAOq5zvFBpiqAR5Hml8PJMMk78e**EV6o!>Wf~h6YeFHk`2H zn)@KYkT(ca3JjUkEM6%fyH}E5PH@ivWIBVI=#wmxo7qCo;SxWF@Qb* z%J7D>WPDk6smHQe;i0OQTW7zpdU4bPyC_yA3;Y=FpRmU-kryz_F4DaCoC1H(Hs7h) z_17T|wQTn|^z7!gDRa zu2bSgwLm9f3^Qt?o>V~p_`L?RdPtvLdnZu0md^TD$CSTA!%^WCPwmH)9I-y`CgtWORAgr?oAH< z!MZ7s82ElpVW7o__;m-%pJFnG`4K+X(FsG>a97n?^&MRyA929M%fT}Tw@wEZQuqfG=PD+e^|vgNWOcu0LPD7E-=y0=q64$C;a zI;kBxapdAlZi-P2b|);j{qA^Xu;3el#ABX<&B9r3;bXgb?Hh;Gp6k;YZSY!8|5$}8 z=PMUnAkhT2$kY^2RF@7Y2t>REaIh*xYvb8Wd}M833XW#4o4M*FH*D60ygtUG+hO## z`Z3`et1U8NHN2lBo$oG$WZ&g;+lb=JgmU^V1;f=)V0qS45bu_BE*obC*cCQYUI z8E3*oyKplg8i%(7E=!nW>604yB+5&)ZJkTk*Xc!cNzg+PXTLhp` zFb?3K?BFdH3WYZK^4AlRw~8B`r_stXy#a4}d^7Qx^>NO%>ZTV%w~aEs$|@%3TnTQ6 zn{#tF9(#1J>hp~vCcW6I{3;(uTwS579ZoErSo5rZ#w*{qAFL~P*A;wW^y1=dC~Z6{ z9Bs**V18w60t<>6N0CL`LYX>Nfs4Wv-#+2;gf#UTQB6y5-}g(e@b#4 z6kvkeqn8D9kM}z+G0{-q0~W^1N3l>rS$qFvBQY=uZIj8 zds__+uoRD82uq9e`;p9MFvz zJsr>LR^C%Q*uFrx!yH-wupDoiD`|-@NkXI&+WjoT)~5X<^BKH1hrgcRSuOr4}2xgMOZ_}lmFVzAW=>->oJ#gyaTRd{F; zF|umc#hUGtoyJADKfkvP$A_NNK-m&?&z{?sFPgJvh$6^;7@7nO;BHL*{Jz}}o~C-A z;aZpFxa!J+pCAlapabR>CWAL}(QbgqNdQ&>0Yfrcnl6z-e@LG3Lb*o9%&hZlal8iB7qv5aevKziE9)#09 zBPOLk>k}ZlGCtGD;9R+8q@f8umPL`>Ngxx3=)J%xR3;^inUUxh?l6WN+6DZf6Wnto zYFTrbURLc$coU&?3|fYq-_C(gkWhZZm2xgFZo88d>`F&Wox|FD0X7o7nVL8oG*HbOb%@4@OW#j!c0 zGw6m#W)DLTfGT{NOsSC^_Od|sY>L~BddP@PfjK-x64oe|pm^OR0QnIV;n|M~8G_1i zZoWiFOQE0}`A{6B^x}B~FlEwYsgxAk1f{{>wK;39^p-_4N1Lmq%a)vAfi-&buF0x{ zga3?N0fe%C8Njay*pc6ikHZ%I58~OiLe1;%+x%*01RY~yDrFM@ta|7dnNW0kelkv<-s@P}yqg?lCRrP$Aw2wFT z`~&MYQ3qGJfBzo7E60sEzj;Mf%8~>CJpfDqE#$`x!PQC67W|uK8D+h}bC@qMn+kvp zuushF)1#Sudzy9UPTIgyX6}!_FZjpXEswD`>?CYzx{XgCuFFRHM^fJ}<_*f8Ub4<@8sZ0Kzn>fVeNiQTHmCb7Fv;i zUG8LXEw4Jb<500;eH4F9kBU?mgsx0Ns5yQT(=)hGr&DSS(Mp3(m6;x-$jr;pX$v)3 zL511bN?n+`P@kKSnXAkyR4Ze(T6LmApVvJ@rP)VaUpe&4oV2iCldZ+KJ{gu~;wkHy z5j%rCC0kCbHQGo#ek$$Yn}!^ldHK3L&xjFk<1UY%l)Ld^^UKpi#=0k?TEsmPqSC{( z8HEOARxJu3X-Gh75iRc(k2#N`m3qA*M;WBk<|%cXQ73ut;iv=I3V(NwMqD>-Q5|-W zcZoq}WIEh)@XS{?ad{W%3?FksVnD>v0c9=KceerOT0}+#?xT zlO>3@aSTBpwHkx6#1N&;%u80N3|+BI-aY~e$rwaiJ9wiKy6kvCKyYFk0>?_ zky1DU`%<(sT&>Mes1tK_N<}@YqfgN{t)@I4-cG!3ynjW|o}*-TiS;-Rouv4lC@v5TSM&wO+ep*VG`Gu@ zIpY1%HIruYhIAi;ns9R2+r7`9f0@)d{oI@>uP3?;=8^#jfr~Uz!-}@qwO{18GNsqJ zmj^A+6(d(%9g&zA8?4o6lxH~PBbEt!QeY)qhQxW4b*Dlc7YSC@Rnl@@1#%R+@yS3$ zlc}&jyQ#<*Z2uFJLP{zOqWB?*6e3#4a(Ety+l0Z;QehY+X|<@Z5kHPr=4*AOi8=^O zl_sYV%(%MXT^C;;wTv1W>3Ku{bYGgL6b|HGQskm}Ha`YzF7+oQ2#-PH z$CPjk3WzCCYX0T~gY(4?X6opDTtl@evCLZh?_};O&-L`^cXe>&=F`9N`kbxv_uewe zQfkqDQA1*`t$-M|+n-W$Gt?n`waz8DdD zT&zN8h%PiJY*{pnB6q*bM;od6dWO(Ab!#H ze0+J>%JodC$Dd_x8_yi(U>7t#R;N_v7Z{WWxI_iPu<3uUgoWN5+yikvd3E$F%0)qj zZ}a=U_gL75Uk`?5oOlDf*6+qOLtd=5J5cEUF>3hP1s#nJg$rTJfnPTpmT;e{cYe-y z^qWz*1pUi(h)?zuQtlj-NINQZN7y=g6TeOr8M(=QJO1_Ki)h01vJQ?QNvdH4-O6K&+Wg{XNKi$>t#AL)W#}JuoY5{f-FfC7sCD_{H8x5;N z1#}&~l0Lco?&Z_rj#c|TwysrLu5tba`$=l{SU3kw!Z@=_^s2{P7#(fda0ed)~ zB|cf>e%8cds;1&)A8rg6bwxY08g*&BvOulMROk@-4@|O*)8^gJk=WaKzOw+Zf{l=y z0~z`{M;wVrIrxSX!}USwVM>65AE+~P22rhfPq>B8^+sgA$cCImT-=(^ZLNnJ{Ti0YT$`?cMQ*Lm|UG{&-1W6q@mXbnFr4UKL3a5BMpjLxZ#g0cxj4A#QBvllX2 zi$5C9$h)oTs~wX*MJhry=t^h_cjwLOafc)dN#Tm3Exdf(e%Sv^8m&D-d z;yQX6eP_>))f4~vF~ncW56t;@oMQoGqCp|wmml7%mOfo^YL2^4T|+k-vO6w?b-%xO zrjtp(Za3F`TFsmvefdN<-+ho$_l!cpV)q=S?mn3CF!f~Ig&$ni@ke(Kz3dwP$4lE$ zV8Y$Q{mFMlKM#)uCG>!=_Q~mVYjXR6+5QFoHbazqw*E1;zQf>}P@xFF&hSBol)W<_#K?fMVbTj)(w@TlJZ6c z&Z2Lc3g-^hY6mEl1xnp0Gh{&IFOF2$i2`FoQV}Q7!owVn4{}dX>eU#`@5Z!wfjPw~ zM<&Tj%mMOb!l%YO3~8PD!jIKyvjI6ni-2#X2LkI-1gh2A%v-SSdis2aB?qc4qsp|E z6_<7?*(q;vGj(l&)AtQqUr#7{l=^Xm@19!6aKr-0G*iXm5zE!u{Qu5p)E%mwB^rc! zAWl$lZlT7QDxT9Teq9P)n{s3Rjfz)q@7vc0j)qqXv|HbM42^4da_^_f<4m?BwjYfc zx+Q0Sr&oq%=477<^*cXp^=SVNx4NRP-spU=LZi_d;sJh&lyl68Two!0i9_v-AU~dx zyY~Zhbf+B?wLC=SakRkXAYA&0{-5nw%R|s?ZGHjlIM84?BO*&FncN`^C>{X4yt5a= zL7^oDy7{!Zeho#e8R`F%IX6EfIufnQ)M@p~OsytsD%Vo@1ye&Y?+FqNLZk-JG#OJd zVTGE^i^PVVPSev6r!}mIB}Hiy3Ntg6dInpPV~Ao}xrOnMc3Im{d}xU>vv55Zt~-lk zkdZwKztE1_{AHAbGOLG(+C}#WQfr%qb(GDX3g^eH_ml=hN@(uKQ|!h|%unj{(y(nq zH&uTf@cfu-$xfFg-9}L8BL$oZ0cZMw%_7D+ueXAr<^9u|$lz^Tp2oktSU4N<-0fF41}!aGGpdR-52NPB3!fqrAT4mrP4pNilYnr^ zj8MraHPIGpeUa^D#EGy$y#NNE6_t6&Cp2Db+Oe#3PNl&x7LNxj`yawX*Jt%EFfSBI zj#LDXwhM-QqeqRj--x?;cS?S~j{nx+H|8(LEvS<_8UYLo64_@9vVyV{swpE&!Bdg= zZw~2$s3}p}oKTHHS9;ExY@pgx)6nnJI)x>Ei2Rh=3pW_|&4Ufna_4mzbG>Qzz{nU! zidbQ(ykit1fC)6Qz?7g8#e^wT>P=EB<0ZU>jc<|jNE8ztttgOT8R63ucLu`I$cVG# zG!miS2nMo~yROATUIL_ku+%mIrKPx+fMV#fRJRv|NWyXD)Hd}YeP4m&tW6W zdJSTlan+?o@r7p^#fi~Yi;sCNPW*vIoR)KK#x@$!`0|J?cDGz)#JqWnV)oqAHyM1gexSRo&Yt1+t;R)i=Fiok4xa+kxd5S6ha(}(G_`HoVwje@h4 zT6+Ta4n|_K3{21?6hit3hpa?h670b@+JklEar`?Vp>sNfrb@SfYsGG30R4?~(X53?&euM={jcR0gFWt9cMQuz0SI5fWFCUhy_8}hv${OGsqW91 zx3(*%Htf1S4;b5pujfq{@7TO~^S*TtB?)c?-R&?*giXx{)VgaIp$WF5t+k1N)n0j8 zoAStvC+WVP8I^gsikNFhhYJ~AVNkOyG#M;A!N$WjY?)w5TCZSZ5 zInc!&O5nB33T>ewv?Noh%*x)((2$|xViDFu3Q zgi2|qxmzYwI_eJwxE`Opif_(u#I7Ao?u7ik_e-q9tJ}eU`Q`h{_`wJga5MgxvO!vP z)9)OLwT<0$bhpE21TdvkD>DtH$d-IYnBTUdjizXt3%b}91_?G-RT?L;A}G9dx3iX&2}TY)aBkZz&QJ^6poGFcQ$O!)2744g0wsbB5J8(oA6kf zVD2p>(!+$I&R*V0NBAXXo*Pp} z!H_m%f)=mwSBkY$FFv9)m;IlaK+*y7*GLuL>;J@UTfF-}<6LUDNLI{13vv6%m=)IS zJ9!;FJGt6n`ID$M-ob~KT^}*6e4Nb~7|i`}gMR*QRu(sX#fFr5-j3N~(jN>w9yXdi z;nA!fogRsPoZI>N;@7N4jY!ZFVS%WQu8n@vv|*3c>upO?Q|`>(l>XOg<4`BRjSP;n zvw3(pdrT#L@Qy_zYF$?oo+G}x?fW06)(uOW%DLU%YI>zL^%swGgCaAyQZ|}ywV&9~ zn=KXb+!4CN7nO>D9!ptQE=0S)hzeHrd$?2p;xz%4U+Z`ESa#LQ)k{~0jz9e2%l7m65&iwsYfN@UYKpY#LX81Ls|s8Hgkn{LudV1BhMdm@OcI+=>~RzrtHeE~MQiB?@Hn%jM}ZlXzMu%)>6Vh|7Z5V9?;tSKP5@6Qn_1&O`l2adt5duYPC zTjrarUVXW0`$ziYgw;&+D~98VP$*ETPR1gkfwFMm+6MOSWjo}3QH^C+@CK~>^de{d zp#Gm<@JH8GEY1#i#k9Jfcy~uD3oXUz!>^;q+h~#&oqsa1{SxyBg)93X^4K*JC=TN- z*E)>cy8lbFKxrFN8X|*)0M%S0)hy+Um+bf=og5K5ZLG@%8acCYKOhmx>oOU3=UeF3 z3hU^{JulhRE!tnZt@fgiZJIQj#FoKwqS;lQo(ul8dB+)d!7ss(-iHD~iw7j^7yWH% zyXWiTf84I>mV0&9Zl43F=YD_;gdGCahkQ&*lLXWQ;I>FAH;SD``H~$dA}Vv={k8<6_&D%eGM_{xz& z<<8;LsomSdJ+c+3(9D<%aWmcG5J!6Jcnh6-=+3!)JFzG6>JvlQ0$6@($B6ikeX=H< z=_c4R{LQT%%S0pwaqx5qlrugu)5-g{ol`hI-oL){oU2hH;s=Y15`k8_-|z2~$F&Vx zi{@mVd9?Z7*1`jeV6bA%tbxqikcUo3UoIQf>yIjU12QaL2WPQ+eC+DIFPAhP(@)+r zKG7Ky% zi3&m(?1*o9-n_Bi?@5xY?)ikhlQU9RVJy!QU|7}!*S_VKE~S*!NOGp_nK0c{M2&^# zCz^#>%=++(%*r1!;-_`${Z$yjCQ-{KS1V$734V63ksW{Q;(7hq^?FBq0Ga7kJobQ9 z9bF;+cXjCX!;|NBSmbieYeo41i)k?4wfpx^1-{%B@M`=Mb?ON7@=+9$3`Epa^qsoq zGJIi#>3>-^r#f7{v7LuJEAgX&N<*<$mlvCTMKT z@Y_xk09CvKOrct;nD@z^8o50ni#jmVa?MLA9fr_P zSMVO@6LpCR>-@w1%g!HXbJpj~5j5g|ZeObn5cF+Wx_*|lz8k_XrEM7Bg!6io z?WJs|e+`PbZ!UF3mDi_x%g3g!q{4{J9tJ`XN>S2JT{CFe*ia+p7z#m_7wRS zs-h%B7J{=f?D3d{;Mk-=DxINFq5ep**DO9!rsYJK9?*?%1PD1LwUMyD-eM;wUIvL2!;h=F-b z^-s5F%0~)skDr~byP%=WYpMx_AL(9io@{Tu#4e+8M2_wPZG>M?ewuK3+)qPPv5Uv+ zQ{fK1lxyafTE)a9M8-@)0EnCD;{%u9*Sgj&Tzkgj;buSehe=?t6MLhBCkDKw1KB-2 zVm6##+;I~&vR0!MHB!Kaa2x>nEh&diEg|%Y9&_*Psq$+thol@?y7QHS7M;t6sE!{Pa? zNPV#XQo!AvaZ`sjphuW$p#e=8M}6vlwuh?s!RStQ zELN&=u;t0e`Y&b9qodGDc?4R*SfO5cjT#fI7%Fn2B3eFgBFJ-YC6L zA6%OG%iEg0C#qj)h6Xc0-u!PhjJ4qkRZ>{7dWfZ8N74b@_dX$>X0TQCaQn)E< zVf+?UbT@K=$YD`P*jfXy;~6|2yB8D&KXN4t`7((g#c9>oi3d1;KGhu+A*yvwi4{FE zxS%jvm7^PBOTX~vt}OJjc8ztpD9O)H%HQ1 z-bjk+z|j0cbp>C%c7tP>?Qw_ba7n0XBAB0E^6Hqg9$v&C0&x zvf>Lh8esKqVzP%Llgw)A%fJ(~WFiAy88QXgNK6@FtB@yILpjDP4{<4qOa9_oz~%4R zgBYE$1W+rpJO!v{BGN@cDPRDU1>rHOdyX@dAhTMGldgu?Ww# zMZ3L8yu?;`fdi)`Ffw>6i?2LRds`4b=r60v8-!WU9#7uAg_ZBzeLRFT;?8-4Q}6AM z6z!><_=9cy1MRY;T14I^96r(P-m0|A)q6I)IR57Lf%VT!a989Wt_ z3jDGVU+B2}^2-0Np;<*u2OOtCDr^3F+W!~9^PWywEqXts>G)S;{lvjDfIP`B`J~1! zZMVp4*NLqgyX0X?ht4#7FPLP3Dr`w8XguC9}jZD_$e%_V3F@i>kyc zz7Lf{jUy)e``EGYGYtgSS|rY;*)#!yEH^+7c)G0Hon!y5n45U!XSCX9-JHpjH}wM> z2;`sk9UoDibSAZ;?&0CWlWE4#!qCC|NgZe(&}l{Ns^xvDC*hXz9ycI^6S*}ott-sM zkrx_zt^9S2_vy``N&oykdLpbg!YBH~fWP_AXUq!h-fUkd3Mx`v`WYaT=^>f1wedju54{f0RTw)5I*f<`PVV6auL_fnj2*j{{Vjw6TM|&y5mv^K7?~COjAiE_wpE-WsaOCh z(4K%?MLXL-Bg&{52)zW<&NxsD9b>ZZIFhBr67GO!vqq9d&QJne1?_1exk5|H2rQu6 z#3?>Q(YBb`oeY`TQg~b_u)y3_O_~j69TL3ea5<)Y!UGr$W`Bpm1qS$sXuOz=0%X3x z7)xSf`=3Dl%M|&>1jE-Gk_<~WhN3DE+lq{!jtTJP0H6$(>|~0ah}z~iUi3n`cx9GK zr_8+Al~ln@lPM!1FzVt^DG2n?h}RknYNu2(1Lmugv}On+2#Bsx_~Ed z%UHDlf90=Vue~HNVh2cvO}I#Wq+6+zc5$PVe5YuSwRD~g?4SmcCYi-be{b~MP1D*=s@ou z4BQOpQp-@Kue;@W&wR5p<)rtrjIkBYFq9{>&?kf9v5x+dwK*;2)VWvl=jAfZ zz}zeOM~3V#>AXpL!a3vW=$+!$v5a(&cp};>UZIh&qy#+bQU@j*Lm^~r3z1b6$go~5 zCivR9+;U;G?;Ebw?weh_SNu8c(MM282Pe-}mvwcWW%GJ_?~`=JL;IOD`llz|#(Mko zJhNEbZcstGTjYwjYPXmDA_EfC+TsQQIVF)3_KE0=B)^FMKi7O6fHWw>B}G@UgLxN0V+0ncOw0 zPp9LY?k7S$1w)78Zyd)M&E*2}6ll53`3>_#CsB9{7OmS9jz!?hsa@2BF)UnT7)Di z$S8P{2rNM-J>gCVg0Mt948coI_zIVV<`k{NE{+J&*WQ4}7l+ z^URk_7fac9_xrSd@pq?&5E^Q|3|P6QZY&J zg;Nj>5q6LX^6*p27jq7Uz1jG#!r3V<2)=77c|S?NHqpQK<;?B>)Snz2{K;4n8b4zY zm-k>3v3$_HKHb?jA|tUg(wzvJwc+6mG1b?`12~2R5qGYPO~t_WewP;}7%~~cxm-xr zG8V3WtI`=hyuL^hZHw&?S_l9=!FOrWvytIlwkLU(zvP{FuGY91t65pSBoE%k=ATa?OFU5-g6}<6N18!MA`C1f!BEC zM1PePSvL#IW_3wDTT?%1N7*m;JNWoid^d4see-QgH)j{cw0keMcOX3w)}aPGa-Zq> z6z%CzlXgf`g4dC#K}c$t;aP9FaUM11+{4AzmD4v$MsFPF`gwy2Pn=6-2`G9X zVprs68QsrmFv30v4@RHqwdE53u01!)GcM$IJ$_`k9NCOQs6BE?&v!PhV}7ln-@MOS z7W^D_%b58pQ+!tGB@x0`nL~o#6~}xzzGqY4j#Spd%eKMbEbx^E&7UrNq2-JvQuX{i;_@X^&08 zH;19m>_6SNh^BpslDz!<_18*?<0NFBuhQfzO46%QovHET&F-qK66T`y(datM>Z!=; z*>+xD!f7NffG@*TLA*NB9Ufv>1{fJ;aAp9pP!tuhyA|WvEjG5# zXFt2U8;qX{DBm6i-}m1=H&5)d*4k^|%v?ogo~|%o=a8+@=F9aCMGCD>sZn=x=*rgq zqnTf@_WN~A*;aRiE=@wHDSi^iFQibbRjBn*3cXgDnciETnU|y06sog& z7iMQGwBf2kU2Z~Vt|F^YrHIvNREct3Ugr#@`Y$5ZtclyBaTr}!Id|1h7yskfo~( zg?dHSMifNTkdSUfjD0{n=01v2=ydWNMQ^PpPodq6Tye^ueTI^uo zJ_ec6N;ucH2jU8fNiDNW7@l&+qW`$RPN2N!rrnx64;GUqGXFHsV#?8H;iK;F)AkQZ zRJ!y-Qak}QF{6$==SYe^zFSq)^TKKOTXm?TKjPAD7}qk}$g?$eGcOG}bEsX})Uihz|r-v7X-U#^%v$$Jp;4nj?) zA##|!P-Uhb7DKKrNh-*a>lJZ@ib6$ixgIX*mN6FgT?ZpOT!kcy7<-QhWER*@QL+MA z+IuD=Yq}IMHjbg_u|};|jL`SjWacHym3nvR-zM^P-;>3!KOX24gcJ6X_Jif8a1)76 zKA1K5N838zv;X`GZ!@RHsSgt2G0?xDuF`Jd$p6X|)q=6RvKreX2Osn(RH2fWD6%5e zI)zp*z?Py@NECtaC?uJSZJb9TPw2Tn(Znl;7b@MZ5xDQKZNU*Jz^@+8o6M=;0aBQ6+Eq`Md2R^ea902V|LCoq!u~_Jhcd*=m3=VHm>2sDWfJ%yeLZ8}v8tzLO!^dbiU_<=TrjM8 z(KmSCUJ`S|@#31%Hi2O*)GUX9#*Fqc@3DKb~XY93Q z$WhcqK%)^uD`0qb704fA<`<@gL?-G@1fhr)B1Xh=`JRZ^faxeC6NMA0aY03{%;P9U zzD8S;s0AQUs&ndL6(7nkU%55lXVcmqAv9vP{iu2MB`_cDhq7}PS2r8Kj}c*Wzm z9FE_S4@Z99DNad`nLY*?-_b;7j6Swi=yt4LwM28G`he~v@Vxq zMCfE#sK+E0&_Y}C9Rg`5nNn4XF6`gE@UidxgKbj_``5yDGC%jZdec8Xn>wBRw@O@O z?f;k%m_e9899kiqKYdY5*7d_FkEYoMUp~Uc9nkn#twNDspjRB^Q6+4JP5(s_wzkW| zxGUm$@oSmQB$H43Yc1^n*5uRV{)mB+13SvN9oH4cyBiB_;C|`Le5?-k_A7wIqx(b+jbj+dm@|z z6Wi)W(Thv(E&G=4{Z*SDe#k$DhuqN)japR_uP9I{Gv!)D3t(=Sb6cYHGnV)m=4mDn zU5J6Ia{y|uawU<7mck;pQ#moGrw|WEAlR{t zt#u0VE>i0L<{1{MF>3DzkCEe|A6N89t(ju}S32DpJG(W0YZlnMME?V|m-a(+0t~{E5&EORcujEzV@WTwT$XKSvgIWB{6R3C!}RA?sx`ZuB^d zVr^qL9jkVT5O@qwXrB=jBJs#kXz#(oZ6)qY@^Ut`J`{4#zx}s?cb4863GuXc!p9F8 z!#ZrKII||bW8d>>>5LJY8i+L>8+fKFWkOzzV@*Wc2kX%oE`G&DTG(8hNJ;J!y4Z)V zi0J$E%-r{j!?dWLv}zrHx@K35h2r_?RL^g723rncjJr$n%~TY~ojX{g>8DT>D6}I@ zksi?+u1w@gU|9hNBuNY`F4rSLp;KWv>cVMRZ8O5HMkdKi&48gSL{AO435&}4CWzH) zvVrKria?L02ZJh;2dh+?%$qRKD{h zeC|=p^X85bhz0I$A_py-Mqd~@|I53_W75w}nI}M95w~|pZlT(+wf`ko7dYiPpX%wK zld&nCDLQg$v_aPdzBO^G4;iO_%A^L zkoahnGE=M3DKa(ctO}l`=o==3G5-k?4nTA<5MVMUG2w;k%nQ_ptz?*~h-=(F)RGWQ zLSbg6LdU_D^f)3MqZ%+=Xq~kU#fObhWETF#gSML_1{nmV=o{mx$zM(!6j@!xWH;m8 zTcv3d!DL#l9@<`>%6g71O>6MBwOMuTBIg%*R{zCHS|GQU`8pJSsM@dTIEjoxcLriE=zfR%XWGiPE&b*X~YkrX;4Kv~s0&AKV@3IbKo(~fk92v6)TiCbjk0?MnL(MS} zmZO+(xl*-BW@Tu?BiQf=J%_|GQBm>&3v59J4B<@!R5SoPL;pr%)D2N9Y-itTEf(<; z0O(=4wh4)b@GgQh&}J!bI-oWnWY*bCeY2}yxyLtK{cu``4tBP?#c4uy|5)sPQvC2M zx475h@-z1uv4q<^vrYJ?#C}JgO4Kv0a#rQZg)^EqVurs#NRJU~x)Bp#7CaeAGB}0? z3tiZVy=Vf^Hhi8A#DpJ&xPx#V7m}r4eHCMlN59#fc57N<{qxglKRjs@Zc|Y3KSe7S z`7!;FwTW*&6}&(aE(~`z`pF!)b7P6i%(yFy#=0-qQ}OR8WHpiu4~E8@dKDYyvCV?L?S+||S;mk&8vMuT)u#XPoSkmzfhi_yl>W>7N&!-cS6=fI> znutI^GzVWb;V^C_Fj1L58L}>DvBpl&dWOTm_h({Dd*{2D;Tlm~979^^_#`$=B`?tR zA?%)3wafz774bJK+u8l>srFCweD3qMZvfkf^Y;vP-_*ma{LMUDTCR9CvMU2o!iSXc zH#reHqO$no&{-!W*S02-6OAYc;q#czL!e?yONO-}w>^T?Dhb>q!&Mv-iK zupX$SkZZv{GNl8M7%xVWIH9C7=5%V$E=Hz_4I(;&Cxa94Ha?vXb3a2!gAqn+Ii8@X zTH8`jY+>vD50)-L!a<0xMD5ZO^m3JA659qECdPA&z5fd!nuHQ{W^@M+$avRsR%#0M zVIwjXimVL2m3=@ua-y3dN_4H-h8BwL$zQ0QBI~}4j)YkGTS{XQk4}fnV=Dk7jC5>Gtt@O=D={66OlTF{1f7jQh#TofJ7dTH9-SkmJrVLrQfJzkQ4efu z<(o<95O4|B!6OejG!iiQ3>OV@rJ)=P!-+)18V1bn=5vA}i_~cK#~{>p^QFdxX1@vq zzKBY&11k`MEN2X;!}2it#p8TuFyEEV!WPi}bGyF-o;iOSn?D%p80J3xw^;0-?@7^!g?H>cY5V6sWRPZ z)`dOqXN5X%XvKvn$%pk`H6?9c(01+EuQ^FOCFUW%-`db_)}UnO_g(%^gIGIiN61WX zp~EYS>h^k6kCWZE?T{9CU9jH|-+Q-PGs=OAn-l#8ti8FX^Bnwj$moO8jZasf%X(&V zqK{mg6$|D|xV%u6?p;UzAdD77(Ur0#lM`1?cRb!%boV7%h@B#1R$8;~?Q5CxRGDqY*^$z%^9Rh# z+wM5m(d0XH&+~r+MxS~+b4ibm58kDjoypFY(7v!z<6)eo^_l@0Sv}Wx8(Y?Q$!Ff@ zdL;BiSSYR|=i+Y4mshTGzwF_X_GtBxa%)n37W{f4kItFElq!TZ|waIy+rI_)D2@tMOKbWGeKPFpH{TX_4TpckK?u_e5%~9 z`ih%yszn;tP4M-9pZyRX4+$HCjYDX_b1ROB_WM^J#J77bcMfI9v47#7p z{n6uw3%)xPTun)JHJ@X(4Vh~|7W>sQd#o>*oEMx*nmOU^zTcOt`<3d3g8=d^TrpY~ zaV{j&HG8Vw!q3tI_Bf(7D3NC$HZfyTrA7Wx$KzMi;N^rr;^@K~w!^?nGwJ zn}7RF=)=`zXDh?!O+kJKfB_J82v!~TH7Q9FlKW!14D_hM;51m4Y_yoZLa0qh8~EJs zL3Ia~w>vN-_f}E*qy1NFA_oM+#5R3g;$uGOQevl?7xQ)xAKM5|0Fr_2S5Gb-+-|Mh zvu5ql>qkCZ+2MQ=6`C5L4L32hJdh7lqFR^6uN#~{(0T5P*E^-EHNO@>_uXE%UK-G- zS*^O$Nw?PJ%iQH+8iTmF5+-Hr#W9<*^P38Gr6%XgR($Q)Urc)-MEi@uAH5rG5;Eo2 z%Fll_9+}d2TG`%)VT<5p?ss|HKR#cW9j^~Ke!FhVb$9|W6R(50*j4vQzH8TezwIyD z_Bxd1+YQ;+5^grwITIY)=~`7DC|RsKwDRQE1<3=FD&b9yxy1o@Z}mKLd~tT?dE1Ya z=vn9z>5VWfBi?Z4-y5lHC9K*@zC8e|D_Mwq=Cr-5@6)^2(v)n6-8Yg}9(vkM09$Hv$IB-3+uJ?3cDn3>iDW>n@6DZo4ebn0YQM5u|tkLGhD>465fjTHI^O!wXVADn@rjB4PSg- z$(V42eszN20gk^;Jj+Fc`@c{PjhAE`yVdlmcp3UhdUXnyR+-6G9oAZAO!jh%SV9<*+AW5<+5=1}L@qLb>V_VXv4Q#<#t*Y5tt}{3oG#FRIrc z3x9nSBG;sBcg6~F(4`ea=M6{?e)#?!GLR-{4#@z*?xHvtq2*o0Gk|(7kh-=H#vDue z@cM-Fjx6WJoP{%Bez-rX=Dr^n(4%sqHB%cz@s3Z8?fcq@5^&OSN#fm3ng<|N3SPDSOb+Lm`46;SpbJ>BKD z_!`7DghV;K%>00fMZvj}dDYBK^EcZb ztVlE;4L#$B4!N`F*0B#eZC-zvv@C6&r^v_yO?cUj_&YD26nl-BK2_OqOp!>3h;v9T z^{OK`Nljgw@UG4Q%UsS65bzoc-uN})8AIXtcZ+QoANY9R|31dR-Ng8GDrUtzeN2W8%D4XN-xxflw|() zv1TtwfIVn2aHm5KA9*1hBBfuDowPd^7;|yyC<2yaS-b*@O$v2pKPlB>8JrY1>veN7 zKZ2g8_RuXPGAej>JO_`*9tDLVk2=wX0t;ym!Zn)O)DxsuKIx2#5HWsKYQ+o>DJYCm z=4gjm(-({-ThyBI!Te_RWrnWMz{LK&KN34>DPmg`SrZ>sBL)=QXW}UJ;Yy7einN#s z4$Ci8m0|js+r`*7hTX}7UBG|O1qJB961)E-=xEWbPIPJ*TLF3QIZel*9lMCSaav55H4V4ZcAG6pfr8-Q#s%LCge4$#s z*s+T;KT2vTo!MLpfg2d+66$-kjv*ut6sWlPKK#&_Zf9e4rxg^m9 zfvvPJ%QMclC<2T}jjAL^qt*pyWht@}HBd2p=3*)8FEm(-%`V}2F5{0JBGqFNo@ez~@>eDJSq=rhT|R{zd`MJab!X?2bI5h6 zOXH=Kx?VygH}F3Qq;I-Q_XOAt7X7{Q_{+5g6-ocebD#X94 zq@9*uk$#w}t*txVe=QnsOT)mI@c}w|+Ohm?FaHIZS+dXVb5v^&OzX57$hrsmToW_y zS^7(_;JEQ$=MMGz#-XW5hSvihz4Z17>4bL=sg}v&`Zr#z7-SDQjfC#+ZpXr58hE7} zktCO4Q-uhyt*<>0=*o6|^fIAv!|e39`<~BBj|FV(1v7u>;*CURTDi08?2Zg(mt#|^ zVLxH075*ZBUEZ&JyD_xyl+CjTnVopH3y_<(u?C%Yi!4m?zvGtuTqp0B`1^mS7A>6! zgg11Bm2Xy>K)9(Pta}xi7{%cmjGJcmT<&mh+=K_~Z;l%OZeeUKZ}NRMGhUIS7*TN3 z1}rH<^@*h1ZZ%=}n@LMbMnxC6YaPeJ3)k;68E13g(%%#B4mV5Fei(9}HKq$vU7Scu zLWe=ld$2w2F0iz3F{dT!Ll|4k*^^ACAWP8+k=zpOOD%={qUgexhr@{BirA8Pq zK)v`1!Sf~5jIm`q2tkh+YQ*CC1k?z%V4TINq@~CMUnlG(LI;|g8!k^oY|H2a5DO9u z%5LWDQJ!cLVvM{go=t?uz&2(a;0TbF)50q(A8xk9=4>CKXa?dNImOB>QskMSP;gYW z%o>}HTn8^-ZR7ZB)ot&5-Q>{ax2PU}vi>4oz1Fn;(Syq$BQoryv&?W0gcYC)UzW8W zca<}s?$o~3SLH>q%wir}-IX3SlZpOe_{P-!v>AI*$bZz8I<?bd89{I6*Qm(eN-2cMXV-+BUnsA2> zGj6D^4J*8UUxFq)>|NZY@sn~lJZO6L_n@&J$;F6+GQ>LaX zsNmELVRx@#U6V6CO49W%|A0?fMsG?IL`5mM znbtcP5T)B`w{v?$w#b1^awb2re?GWA?XlgFbHWM zEJY(3x)%O4LRczELGVype3?fhuoE1%oMA+ZgmEQ0C|Y$#B3*JVbB@fLKG#qGzn*CYSyb;}cM=4`V4M-13zsGKz5%o8D#Km{MV>LYtt1LL(77v*!eVfQ>x|xn5U%r#$%pnq!1pn~# zW`OK&6OHwtIl}V;RwWIJkA0bP=0tC%0rx)ieZOmUv3TU+m!kY8-qSh^{WKKnC2n49 zBDNnew|i%HAKxIM4C0~>!`kq1hJ>4F3)4zHcou9b22u4#$(vxn6NGblu$C4qwEoz? za$kN=Bwc5V?GRcBtav53(C4*s#KOTA5zZG&x;g!ndH|1emyWr-)6F9?vrWnT>oM(j z@%Zp=VvoUA;WpbUYjt(LF5EHmm>|uBtihp^(Ha}DRVcS73~*Dn`DE*?ii^EpPA;vU zQj#gZ{ui*>2+uA*PE~B~et!HDmxwc0NBH%j>BtOk1_x?;NBj7|UZ%e`cm?+)`;Xs8 z{gE9>3q|q3He8uiw=f~y0zY~TMR+Tc2*Kgza=kNfAZ8Bw(?SaX>;IQ4-&wYM^Z!bE z2MECX%pcrm$BDTB{X4AA6q#2 zCl=se15TGn^}Qr3>cenwv%a_P(qGw1Xa(TXa>9Tyob;$dm0tOdSWqv9Lt08lulU^c z*ds=tyMfT}*q(jW91_jge++TR&uq$vI$kjpR*aoA9NH%oGakrDJ?vJN+X-vnCnU8_ zzSG~1cHnb_Mn<6)d=9D*QHD6u1i}#+DV7Ej4)JQw2KAyd1TuDxl$(hqMpmJTn1g=~ zgJ0Nm1sXMnZ$)$QpW2S2F!PdKoj^ZA&v~{e**Z_TebKai9hUijt*k>*g_Bg!$;hX*2Cf^eubFp+-yZu)(?TWA#eZ{3~O_y|V zuSk1%^-lcnB5FGT$@G<8Uo0<_k)ug^+sRX^!=)ArtEVk__C1wWu62&84{1O5W$=pydeKkva!Tswv%GI9yN4WS(7%J0w}Pll zL7&+#GuJA6EwQR-bIPXQY_A@#ky*Y{oi87e4%O>2Tq{;)jo>f=i>=b@c7EEqH}$k# zq5n-JDS%&}l)d@2E}h}uM^3O+c1c-$Bk#?NfvV>iAT$cR><@3KuhWpPvmL#83`$Lc z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOB(Jyhd}l+lM_DLQ|R1S9JggWMOn+E@EkJ+)pEt zS0B@B+5na71A~^(h?oKJe_7^kUF)~R705&JjF@Ba(+MsB@NR8#YcsGwevk!s&HdgP zC{Zt|h7q9Z02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnnTemQio;UGet* Kde0cxq6Ywpes2{3 literal 0 HcmV?d00001 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt new file mode 100644 index 00000000000..4835061786d --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt @@ -0,0 +1,259 @@ +columns format version: 1 +257 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`milliseconds` UInt64 +`ProfileEvent_Query` UInt64 +`ProfileEvent_SelectQuery` UInt64 +`ProfileEvent_InsertQuery` UInt64 +`ProfileEvent_FailedQuery` UInt64 +`ProfileEvent_FailedSelectQuery` UInt64 +`ProfileEvent_FailedInsertQuery` UInt64 +`ProfileEvent_QueryTimeMicroseconds` UInt64 +`ProfileEvent_SelectQueryTimeMicroseconds` UInt64 +`ProfileEvent_InsertQueryTimeMicroseconds` UInt64 +`ProfileEvent_FileOpen` UInt64 +`ProfileEvent_Seek` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64 +`ProfileEvent_ReadBufferAIORead` UInt64 +`ProfileEvent_ReadBufferAIOReadBytes` UInt64 +`ProfileEvent_WriteBufferAIOWrite` UInt64 +`ProfileEvent_WriteBufferAIOWriteBytes` UInt64 +`ProfileEvent_ReadCompressedBytes` UInt64 +`ProfileEvent_CompressedReadBufferBlocks` UInt64 +`ProfileEvent_CompressedReadBufferBytes` UInt64 +`ProfileEvent_UncompressedCacheHits` UInt64 +`ProfileEvent_UncompressedCacheMisses` UInt64 +`ProfileEvent_UncompressedCacheWeightLost` UInt64 +`ProfileEvent_IOBufferAllocs` UInt64 +`ProfileEvent_IOBufferAllocBytes` UInt64 +`ProfileEvent_ArenaAllocChunks` UInt64 +`ProfileEvent_ArenaAllocBytes` UInt64 +`ProfileEvent_FunctionExecute` UInt64 +`ProfileEvent_TableFunctionExecute` UInt64 +`ProfileEvent_MarkCacheHits` UInt64 +`ProfileEvent_MarkCacheMisses` UInt64 +`ProfileEvent_CreatedReadBufferOrdinary` UInt64 +`ProfileEvent_CreatedReadBufferAIO` UInt64 +`ProfileEvent_CreatedReadBufferAIOFailed` UInt64 +`ProfileEvent_CreatedReadBufferMMap` UInt64 +`ProfileEvent_CreatedReadBufferMMapFailed` UInt64 +`ProfileEvent_CreatedWriteBufferOrdinary` UInt64 +`ProfileEvent_CreatedWriteBufferAIO` UInt64 +`ProfileEvent_CreatedWriteBufferAIOFailed` UInt64 +`ProfileEvent_DiskReadElapsedMicroseconds` UInt64 +`ProfileEvent_DiskWriteElapsedMicroseconds` UInt64 +`ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64 +`ProfileEvent_NetworkSendElapsedMicroseconds` UInt64 +`ProfileEvent_ThrottlerSleepMicroseconds` UInt64 +`ProfileEvent_QueryMaskingRulesMatch` UInt64 +`ProfileEvent_ReplicatedPartFetches` UInt64 +`ProfileEvent_ReplicatedPartFailedFetches` UInt64 +`ProfileEvent_ObsoleteReplicatedParts` UInt64 +`ProfileEvent_ReplicatedPartMerges` UInt64 +`ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64 +`ProfileEvent_ReplicatedPartMutations` UInt64 +`ProfileEvent_ReplicatedPartChecks` UInt64 +`ProfileEvent_ReplicatedPartChecksFailed` UInt64 +`ProfileEvent_ReplicatedDataLoss` UInt64 +`ProfileEvent_InsertedRows` UInt64 +`ProfileEvent_InsertedBytes` UInt64 +`ProfileEvent_DelayedInserts` UInt64 +`ProfileEvent_RejectedInserts` UInt64 +`ProfileEvent_DelayedInsertsMilliseconds` UInt64 +`ProfileEvent_DuplicatedInsertedBlocks` UInt64 +`ProfileEvent_ZooKeeperInit` UInt64 +`ProfileEvent_ZooKeeperTransactions` UInt64 +`ProfileEvent_ZooKeeperList` UInt64 +`ProfileEvent_ZooKeeperCreate` UInt64 +`ProfileEvent_ZooKeeperRemove` UInt64 +`ProfileEvent_ZooKeeperExists` UInt64 +`ProfileEvent_ZooKeeperGet` UInt64 +`ProfileEvent_ZooKeeperSet` UInt64 +`ProfileEvent_ZooKeeperMulti` UInt64 +`ProfileEvent_ZooKeeperCheck` UInt64 +`ProfileEvent_ZooKeeperClose` UInt64 +`ProfileEvent_ZooKeeperWatchResponse` UInt64 +`ProfileEvent_ZooKeeperUserExceptions` UInt64 +`ProfileEvent_ZooKeeperHardwareExceptions` UInt64 +`ProfileEvent_ZooKeeperOtherExceptions` UInt64 +`ProfileEvent_ZooKeeperWaitMicroseconds` UInt64 +`ProfileEvent_ZooKeeperBytesSent` UInt64 +`ProfileEvent_ZooKeeperBytesReceived` UInt64 +`ProfileEvent_DistributedConnectionFailTry` UInt64 +`ProfileEvent_DistributedConnectionMissingTable` UInt64 +`ProfileEvent_DistributedConnectionStaleReplica` UInt64 +`ProfileEvent_DistributedConnectionFailAtAll` UInt64 +`ProfileEvent_CompileAttempt` UInt64 +`ProfileEvent_CompileSuccess` UInt64 +`ProfileEvent_CompileFunction` UInt64 +`ProfileEvent_CompiledFunctionExecute` UInt64 +`ProfileEvent_CompileExpressionsMicroseconds` UInt64 +`ProfileEvent_CompileExpressionsBytes` UInt64 +`ProfileEvent_ExternalSortWritePart` UInt64 +`ProfileEvent_ExternalSortMerge` UInt64 +`ProfileEvent_ExternalAggregationWritePart` UInt64 +`ProfileEvent_ExternalAggregationMerge` UInt64 +`ProfileEvent_ExternalAggregationCompressedBytes` UInt64 +`ProfileEvent_ExternalAggregationUncompressedBytes` UInt64 +`ProfileEvent_SlowRead` UInt64 +`ProfileEvent_ReadBackoff` UInt64 +`ProfileEvent_ReplicaPartialShutdown` UInt64 +`ProfileEvent_SelectedParts` UInt64 +`ProfileEvent_SelectedRanges` UInt64 +`ProfileEvent_SelectedMarks` UInt64 +`ProfileEvent_SelectedRows` UInt64 +`ProfileEvent_SelectedBytes` UInt64 +`ProfileEvent_Merge` UInt64 +`ProfileEvent_MergedRows` UInt64 +`ProfileEvent_MergedUncompressedBytes` UInt64 +`ProfileEvent_MergesTimeMilliseconds` UInt64 +`ProfileEvent_MergeTreeDataWriterRows` UInt64 +`ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64 +`ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64 +`ProfileEvent_MergeTreeDataWriterBlocks` UInt64 +`ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64 +`ProfileEvent_CannotRemoveEphemeralNode` UInt64 +`ProfileEvent_RegexpCreated` UInt64 +`ProfileEvent_ContextLock` UInt64 +`ProfileEvent_StorageBufferFlush` UInt64 +`ProfileEvent_StorageBufferErrorOnFlush` UInt64 +`ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64 +`ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64 +`ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64 +`ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64 +`ProfileEvent_DictCacheKeysRequested` UInt64 +`ProfileEvent_DictCacheKeysRequestedMiss` UInt64 +`ProfileEvent_DictCacheKeysRequestedFound` UInt64 +`ProfileEvent_DictCacheKeysExpired` UInt64 +`ProfileEvent_DictCacheKeysNotFound` UInt64 +`ProfileEvent_DictCacheKeysHit` UInt64 +`ProfileEvent_DictCacheRequestTimeNs` UInt64 +`ProfileEvent_DictCacheRequests` UInt64 +`ProfileEvent_DictCacheLockWriteNs` UInt64 +`ProfileEvent_DictCacheLockReadNs` UInt64 +`ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64 +`ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64 +`ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64 +`ProfileEvent_PolygonsAddedToPool` UInt64 +`ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64 +`ProfileEvent_RWLockAcquiredReadLocks` UInt64 +`ProfileEvent_RWLockAcquiredWriteLocks` UInt64 +`ProfileEvent_RWLockReadersWaitMilliseconds` UInt64 +`ProfileEvent_RWLockWritersWaitMilliseconds` UInt64 +`ProfileEvent_DNSError` UInt64 +`ProfileEvent_RealTimeMicroseconds` UInt64 +`ProfileEvent_UserTimeMicroseconds` UInt64 +`ProfileEvent_SystemTimeMicroseconds` UInt64 +`ProfileEvent_SoftPageFaults` UInt64 +`ProfileEvent_HardPageFaults` UInt64 +`ProfileEvent_VoluntaryContextSwitches` UInt64 +`ProfileEvent_InvoluntaryContextSwitches` UInt64 +`ProfileEvent_OSIOWaitMicroseconds` UInt64 +`ProfileEvent_OSCPUWaitMicroseconds` UInt64 +`ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64 +`ProfileEvent_OSReadBytes` UInt64 +`ProfileEvent_OSWriteBytes` UInt64 +`ProfileEvent_OSReadChars` UInt64 +`ProfileEvent_OSWriteChars` UInt64 +`ProfileEvent_PerfCpuCycles` UInt64 +`ProfileEvent_PerfInstructions` UInt64 +`ProfileEvent_PerfCacheReferences` UInt64 +`ProfileEvent_PerfCacheMisses` UInt64 +`ProfileEvent_PerfBranchInstructions` UInt64 +`ProfileEvent_PerfBranchMisses` UInt64 +`ProfileEvent_PerfBusCycles` UInt64 +`ProfileEvent_PerfStalledCyclesFrontend` UInt64 +`ProfileEvent_PerfStalledCyclesBackend` UInt64 +`ProfileEvent_PerfRefCpuCycles` UInt64 +`ProfileEvent_PerfCpuClock` UInt64 +`ProfileEvent_PerfTaskClock` UInt64 +`ProfileEvent_PerfContextSwitches` UInt64 +`ProfileEvent_PerfCpuMigrations` UInt64 +`ProfileEvent_PerfAlignmentFaults` UInt64 +`ProfileEvent_PerfEmulationFaults` UInt64 +`ProfileEvent_PerfMinEnabledTime` UInt64 +`ProfileEvent_PerfMinEnabledRunningTime` UInt64 +`ProfileEvent_PerfDataTLBReferences` UInt64 +`ProfileEvent_PerfDataTLBMisses` UInt64 +`ProfileEvent_PerfInstructionTLBReferences` UInt64 +`ProfileEvent_PerfInstructionTLBMisses` UInt64 +`ProfileEvent_PerfLocalMemoryReferences` UInt64 +`ProfileEvent_PerfLocalMemoryMisses` UInt64 +`ProfileEvent_CreatedHTTPConnections` UInt64 +`ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64 +`ProfileEvent_QueryProfilerSignalOverruns` UInt64 +`ProfileEvent_CreatedLogEntryForMerge` UInt64 +`ProfileEvent_NotCreatedLogEntryForMerge` UInt64 +`ProfileEvent_CreatedLogEntryForMutation` UInt64 +`ProfileEvent_NotCreatedLogEntryForMutation` UInt64 +`ProfileEvent_S3ReadMicroseconds` UInt64 +`ProfileEvent_S3ReadBytes` UInt64 +`ProfileEvent_S3ReadRequestsCount` UInt64 +`ProfileEvent_S3ReadRequestsErrors` UInt64 +`ProfileEvent_S3ReadRequestsThrottling` UInt64 +`ProfileEvent_S3ReadRequestsRedirects` UInt64 +`ProfileEvent_S3WriteMicroseconds` UInt64 +`ProfileEvent_S3WriteBytes` UInt64 +`ProfileEvent_S3WriteRequestsCount` UInt64 +`ProfileEvent_S3WriteRequestsErrors` UInt64 +`ProfileEvent_S3WriteRequestsThrottling` UInt64 +`ProfileEvent_S3WriteRequestsRedirects` UInt64 +`ProfileEvent_QueryMemoryLimitExceeded` UInt64 +`CurrentMetric_Query` Int64 +`CurrentMetric_Merge` Int64 +`CurrentMetric_PartMutation` Int64 +`CurrentMetric_ReplicatedFetch` Int64 +`CurrentMetric_ReplicatedSend` Int64 +`CurrentMetric_ReplicatedChecks` Int64 +`CurrentMetric_BackgroundPoolTask` Int64 +`CurrentMetric_BackgroundMovePoolTask` Int64 +`CurrentMetric_BackgroundSchedulePoolTask` Int64 +`CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64 +`CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64 +`CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64 +`CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64 +`CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64 +`CurrentMetric_DiskSpaceReservedForMerge` Int64 +`CurrentMetric_DistributedSend` Int64 +`CurrentMetric_QueryPreempted` Int64 +`CurrentMetric_TCPConnection` Int64 +`CurrentMetric_MySQLConnection` Int64 +`CurrentMetric_HTTPConnection` Int64 +`CurrentMetric_InterserverConnection` Int64 +`CurrentMetric_PostgreSQLConnection` Int64 +`CurrentMetric_OpenFileForRead` Int64 +`CurrentMetric_OpenFileForWrite` Int64 +`CurrentMetric_Read` Int64 +`CurrentMetric_Write` Int64 +`CurrentMetric_SendScalars` Int64 +`CurrentMetric_SendExternalTables` Int64 +`CurrentMetric_QueryThread` Int64 +`CurrentMetric_ReadonlyReplica` Int64 +`CurrentMetric_MemoryTracking` Int64 +`CurrentMetric_EphemeralNode` Int64 +`CurrentMetric_ZooKeeperSession` Int64 +`CurrentMetric_ZooKeeperWatch` Int64 +`CurrentMetric_ZooKeeperRequest` Int64 +`CurrentMetric_DelayedInserts` Int64 +`CurrentMetric_ContextLockWait` Int64 +`CurrentMetric_StorageBufferRows` Int64 +`CurrentMetric_StorageBufferBytes` Int64 +`CurrentMetric_DictCacheRequests` Int64 +`CurrentMetric_Revision` Int64 +`CurrentMetric_VersionInteger` Int64 +`CurrentMetric_RWLockWaitingReaders` Int64 +`CurrentMetric_RWLockWaitingWriters` Int64 +`CurrentMetric_RWLockActiveReaders` Int64 +`CurrentMetric_RWLockActiveWriters` Int64 +`CurrentMetric_GlobalThread` Int64 +`CurrentMetric_GlobalThreadActive` Int64 +`CurrentMetric_LocalThread` Int64 +`CurrentMetric_LocalThreadActive` Int64 +`CurrentMetric_DistributedFilesToInsert` Int64 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt new file mode 100644 index 00000000000..301160a9306 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt @@ -0,0 +1 @@ +8 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/partition.dat b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx new file mode 100644 index 00000000000..3709b3e4f82 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx @@ -0,0 +1 @@ +ëHª©"`ëH±©"` \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/checksums.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..df2f868f3e238b0545f87bcd10382a47e43e9b21 GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$c3z@`!4L&OV%^7r2tD!?&< z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOEL%%`Kt3xZ#Bpu>(3pY7Tm^@j&dUd`w1=p@NR8#YcsGwZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnsy^i!!b(-JKN- K=7_>$hb#bNOKuqe literal 0 HcmV?d00001 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt new file mode 100644 index 00000000000..4835061786d --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt @@ -0,0 +1,259 @@ +columns format version: 1 +257 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`milliseconds` UInt64 +`ProfileEvent_Query` UInt64 +`ProfileEvent_SelectQuery` UInt64 +`ProfileEvent_InsertQuery` UInt64 +`ProfileEvent_FailedQuery` UInt64 +`ProfileEvent_FailedSelectQuery` UInt64 +`ProfileEvent_FailedInsertQuery` UInt64 +`ProfileEvent_QueryTimeMicroseconds` UInt64 +`ProfileEvent_SelectQueryTimeMicroseconds` UInt64 +`ProfileEvent_InsertQueryTimeMicroseconds` UInt64 +`ProfileEvent_FileOpen` UInt64 +`ProfileEvent_Seek` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64 +`ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64 +`ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64 +`ProfileEvent_ReadBufferAIORead` UInt64 +`ProfileEvent_ReadBufferAIOReadBytes` UInt64 +`ProfileEvent_WriteBufferAIOWrite` UInt64 +`ProfileEvent_WriteBufferAIOWriteBytes` UInt64 +`ProfileEvent_ReadCompressedBytes` UInt64 +`ProfileEvent_CompressedReadBufferBlocks` UInt64 +`ProfileEvent_CompressedReadBufferBytes` UInt64 +`ProfileEvent_UncompressedCacheHits` UInt64 +`ProfileEvent_UncompressedCacheMisses` UInt64 +`ProfileEvent_UncompressedCacheWeightLost` UInt64 +`ProfileEvent_IOBufferAllocs` UInt64 +`ProfileEvent_IOBufferAllocBytes` UInt64 +`ProfileEvent_ArenaAllocChunks` UInt64 +`ProfileEvent_ArenaAllocBytes` UInt64 +`ProfileEvent_FunctionExecute` UInt64 +`ProfileEvent_TableFunctionExecute` UInt64 +`ProfileEvent_MarkCacheHits` UInt64 +`ProfileEvent_MarkCacheMisses` UInt64 +`ProfileEvent_CreatedReadBufferOrdinary` UInt64 +`ProfileEvent_CreatedReadBufferAIO` UInt64 +`ProfileEvent_CreatedReadBufferAIOFailed` UInt64 +`ProfileEvent_CreatedReadBufferMMap` UInt64 +`ProfileEvent_CreatedReadBufferMMapFailed` UInt64 +`ProfileEvent_CreatedWriteBufferOrdinary` UInt64 +`ProfileEvent_CreatedWriteBufferAIO` UInt64 +`ProfileEvent_CreatedWriteBufferAIOFailed` UInt64 +`ProfileEvent_DiskReadElapsedMicroseconds` UInt64 +`ProfileEvent_DiskWriteElapsedMicroseconds` UInt64 +`ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64 +`ProfileEvent_NetworkSendElapsedMicroseconds` UInt64 +`ProfileEvent_ThrottlerSleepMicroseconds` UInt64 +`ProfileEvent_QueryMaskingRulesMatch` UInt64 +`ProfileEvent_ReplicatedPartFetches` UInt64 +`ProfileEvent_ReplicatedPartFailedFetches` UInt64 +`ProfileEvent_ObsoleteReplicatedParts` UInt64 +`ProfileEvent_ReplicatedPartMerges` UInt64 +`ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64 +`ProfileEvent_ReplicatedPartMutations` UInt64 +`ProfileEvent_ReplicatedPartChecks` UInt64 +`ProfileEvent_ReplicatedPartChecksFailed` UInt64 +`ProfileEvent_ReplicatedDataLoss` UInt64 +`ProfileEvent_InsertedRows` UInt64 +`ProfileEvent_InsertedBytes` UInt64 +`ProfileEvent_DelayedInserts` UInt64 +`ProfileEvent_RejectedInserts` UInt64 +`ProfileEvent_DelayedInsertsMilliseconds` UInt64 +`ProfileEvent_DuplicatedInsertedBlocks` UInt64 +`ProfileEvent_ZooKeeperInit` UInt64 +`ProfileEvent_ZooKeeperTransactions` UInt64 +`ProfileEvent_ZooKeeperList` UInt64 +`ProfileEvent_ZooKeeperCreate` UInt64 +`ProfileEvent_ZooKeeperRemove` UInt64 +`ProfileEvent_ZooKeeperExists` UInt64 +`ProfileEvent_ZooKeeperGet` UInt64 +`ProfileEvent_ZooKeeperSet` UInt64 +`ProfileEvent_ZooKeeperMulti` UInt64 +`ProfileEvent_ZooKeeperCheck` UInt64 +`ProfileEvent_ZooKeeperClose` UInt64 +`ProfileEvent_ZooKeeperWatchResponse` UInt64 +`ProfileEvent_ZooKeeperUserExceptions` UInt64 +`ProfileEvent_ZooKeeperHardwareExceptions` UInt64 +`ProfileEvent_ZooKeeperOtherExceptions` UInt64 +`ProfileEvent_ZooKeeperWaitMicroseconds` UInt64 +`ProfileEvent_ZooKeeperBytesSent` UInt64 +`ProfileEvent_ZooKeeperBytesReceived` UInt64 +`ProfileEvent_DistributedConnectionFailTry` UInt64 +`ProfileEvent_DistributedConnectionMissingTable` UInt64 +`ProfileEvent_DistributedConnectionStaleReplica` UInt64 +`ProfileEvent_DistributedConnectionFailAtAll` UInt64 +`ProfileEvent_CompileAttempt` UInt64 +`ProfileEvent_CompileSuccess` UInt64 +`ProfileEvent_CompileFunction` UInt64 +`ProfileEvent_CompiledFunctionExecute` UInt64 +`ProfileEvent_CompileExpressionsMicroseconds` UInt64 +`ProfileEvent_CompileExpressionsBytes` UInt64 +`ProfileEvent_ExternalSortWritePart` UInt64 +`ProfileEvent_ExternalSortMerge` UInt64 +`ProfileEvent_ExternalAggregationWritePart` UInt64 +`ProfileEvent_ExternalAggregationMerge` UInt64 +`ProfileEvent_ExternalAggregationCompressedBytes` UInt64 +`ProfileEvent_ExternalAggregationUncompressedBytes` UInt64 +`ProfileEvent_SlowRead` UInt64 +`ProfileEvent_ReadBackoff` UInt64 +`ProfileEvent_ReplicaPartialShutdown` UInt64 +`ProfileEvent_SelectedParts` UInt64 +`ProfileEvent_SelectedRanges` UInt64 +`ProfileEvent_SelectedMarks` UInt64 +`ProfileEvent_SelectedRows` UInt64 +`ProfileEvent_SelectedBytes` UInt64 +`ProfileEvent_Merge` UInt64 +`ProfileEvent_MergedRows` UInt64 +`ProfileEvent_MergedUncompressedBytes` UInt64 +`ProfileEvent_MergesTimeMilliseconds` UInt64 +`ProfileEvent_MergeTreeDataWriterRows` UInt64 +`ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64 +`ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64 +`ProfileEvent_MergeTreeDataWriterBlocks` UInt64 +`ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64 +`ProfileEvent_CannotRemoveEphemeralNode` UInt64 +`ProfileEvent_RegexpCreated` UInt64 +`ProfileEvent_ContextLock` UInt64 +`ProfileEvent_StorageBufferFlush` UInt64 +`ProfileEvent_StorageBufferErrorOnFlush` UInt64 +`ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64 +`ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64 +`ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64 +`ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64 +`ProfileEvent_DictCacheKeysRequested` UInt64 +`ProfileEvent_DictCacheKeysRequestedMiss` UInt64 +`ProfileEvent_DictCacheKeysRequestedFound` UInt64 +`ProfileEvent_DictCacheKeysExpired` UInt64 +`ProfileEvent_DictCacheKeysNotFound` UInt64 +`ProfileEvent_DictCacheKeysHit` UInt64 +`ProfileEvent_DictCacheRequestTimeNs` UInt64 +`ProfileEvent_DictCacheRequests` UInt64 +`ProfileEvent_DictCacheLockWriteNs` UInt64 +`ProfileEvent_DictCacheLockReadNs` UInt64 +`ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64 +`ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64 +`ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64 +`ProfileEvent_PolygonsAddedToPool` UInt64 +`ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64 +`ProfileEvent_RWLockAcquiredReadLocks` UInt64 +`ProfileEvent_RWLockAcquiredWriteLocks` UInt64 +`ProfileEvent_RWLockReadersWaitMilliseconds` UInt64 +`ProfileEvent_RWLockWritersWaitMilliseconds` UInt64 +`ProfileEvent_DNSError` UInt64 +`ProfileEvent_RealTimeMicroseconds` UInt64 +`ProfileEvent_UserTimeMicroseconds` UInt64 +`ProfileEvent_SystemTimeMicroseconds` UInt64 +`ProfileEvent_SoftPageFaults` UInt64 +`ProfileEvent_HardPageFaults` UInt64 +`ProfileEvent_VoluntaryContextSwitches` UInt64 +`ProfileEvent_InvoluntaryContextSwitches` UInt64 +`ProfileEvent_OSIOWaitMicroseconds` UInt64 +`ProfileEvent_OSCPUWaitMicroseconds` UInt64 +`ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64 +`ProfileEvent_OSReadBytes` UInt64 +`ProfileEvent_OSWriteBytes` UInt64 +`ProfileEvent_OSReadChars` UInt64 +`ProfileEvent_OSWriteChars` UInt64 +`ProfileEvent_PerfCpuCycles` UInt64 +`ProfileEvent_PerfInstructions` UInt64 +`ProfileEvent_PerfCacheReferences` UInt64 +`ProfileEvent_PerfCacheMisses` UInt64 +`ProfileEvent_PerfBranchInstructions` UInt64 +`ProfileEvent_PerfBranchMisses` UInt64 +`ProfileEvent_PerfBusCycles` UInt64 +`ProfileEvent_PerfStalledCyclesFrontend` UInt64 +`ProfileEvent_PerfStalledCyclesBackend` UInt64 +`ProfileEvent_PerfRefCpuCycles` UInt64 +`ProfileEvent_PerfCpuClock` UInt64 +`ProfileEvent_PerfTaskClock` UInt64 +`ProfileEvent_PerfContextSwitches` UInt64 +`ProfileEvent_PerfCpuMigrations` UInt64 +`ProfileEvent_PerfAlignmentFaults` UInt64 +`ProfileEvent_PerfEmulationFaults` UInt64 +`ProfileEvent_PerfMinEnabledTime` UInt64 +`ProfileEvent_PerfMinEnabledRunningTime` UInt64 +`ProfileEvent_PerfDataTLBReferences` UInt64 +`ProfileEvent_PerfDataTLBMisses` UInt64 +`ProfileEvent_PerfInstructionTLBReferences` UInt64 +`ProfileEvent_PerfInstructionTLBMisses` UInt64 +`ProfileEvent_PerfLocalMemoryReferences` UInt64 +`ProfileEvent_PerfLocalMemoryMisses` UInt64 +`ProfileEvent_CreatedHTTPConnections` UInt64 +`ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64 +`ProfileEvent_QueryProfilerSignalOverruns` UInt64 +`ProfileEvent_CreatedLogEntryForMerge` UInt64 +`ProfileEvent_NotCreatedLogEntryForMerge` UInt64 +`ProfileEvent_CreatedLogEntryForMutation` UInt64 +`ProfileEvent_NotCreatedLogEntryForMutation` UInt64 +`ProfileEvent_S3ReadMicroseconds` UInt64 +`ProfileEvent_S3ReadBytes` UInt64 +`ProfileEvent_S3ReadRequestsCount` UInt64 +`ProfileEvent_S3ReadRequestsErrors` UInt64 +`ProfileEvent_S3ReadRequestsThrottling` UInt64 +`ProfileEvent_S3ReadRequestsRedirects` UInt64 +`ProfileEvent_S3WriteMicroseconds` UInt64 +`ProfileEvent_S3WriteBytes` UInt64 +`ProfileEvent_S3WriteRequestsCount` UInt64 +`ProfileEvent_S3WriteRequestsErrors` UInt64 +`ProfileEvent_S3WriteRequestsThrottling` UInt64 +`ProfileEvent_S3WriteRequestsRedirects` UInt64 +`ProfileEvent_QueryMemoryLimitExceeded` UInt64 +`CurrentMetric_Query` Int64 +`CurrentMetric_Merge` Int64 +`CurrentMetric_PartMutation` Int64 +`CurrentMetric_ReplicatedFetch` Int64 +`CurrentMetric_ReplicatedSend` Int64 +`CurrentMetric_ReplicatedChecks` Int64 +`CurrentMetric_BackgroundPoolTask` Int64 +`CurrentMetric_BackgroundMovePoolTask` Int64 +`CurrentMetric_BackgroundSchedulePoolTask` Int64 +`CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64 +`CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64 +`CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64 +`CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64 +`CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64 +`CurrentMetric_DiskSpaceReservedForMerge` Int64 +`CurrentMetric_DistributedSend` Int64 +`CurrentMetric_QueryPreempted` Int64 +`CurrentMetric_TCPConnection` Int64 +`CurrentMetric_MySQLConnection` Int64 +`CurrentMetric_HTTPConnection` Int64 +`CurrentMetric_InterserverConnection` Int64 +`CurrentMetric_PostgreSQLConnection` Int64 +`CurrentMetric_OpenFileForRead` Int64 +`CurrentMetric_OpenFileForWrite` Int64 +`CurrentMetric_Read` Int64 +`CurrentMetric_Write` Int64 +`CurrentMetric_SendScalars` Int64 +`CurrentMetric_SendExternalTables` Int64 +`CurrentMetric_QueryThread` Int64 +`CurrentMetric_ReadonlyReplica` Int64 +`CurrentMetric_MemoryTracking` Int64 +`CurrentMetric_EphemeralNode` Int64 +`CurrentMetric_ZooKeeperSession` Int64 +`CurrentMetric_ZooKeeperWatch` Int64 +`CurrentMetric_ZooKeeperRequest` Int64 +`CurrentMetric_DelayedInserts` Int64 +`CurrentMetric_ContextLockWait` Int64 +`CurrentMetric_StorageBufferRows` Int64 +`CurrentMetric_StorageBufferBytes` Int64 +`CurrentMetric_DictCacheRequests` Int64 +`CurrentMetric_Revision` Int64 +`CurrentMetric_VersionInteger` Int64 +`CurrentMetric_RWLockWaitingReaders` Int64 +`CurrentMetric_RWLockWaitingWriters` Int64 +`CurrentMetric_RWLockActiveReaders` Int64 +`CurrentMetric_RWLockActiveWriters` Int64 +`CurrentMetric_GlobalThread` Int64 +`CurrentMetric_GlobalThreadActive` Int64 +`CurrentMetric_LocalThread` Int64 +`CurrentMetric_LocalThreadActive` Int64 +`CurrentMetric_DistributedFilesToInsert` Int64 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt new file mode 100644 index 00000000000..c7930257dfe --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt @@ -0,0 +1 @@ +7 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/partition.dat b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx new file mode 100644 index 00000000000..dc6e4026178 --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx @@ -0,0 +1 @@ +ëH²©"`ëH¸©"` \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql b/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql new file mode 100644 index 00000000000..81ffc1acf48 --- /dev/null +++ b/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql @@ -0,0 +1,141 @@ +ATTACH TABLE _ UUID '72667623-6371-4258-be57-799522ee4e64' +( + `WatchID` UInt64, + `JavaEnable` UInt8, + `Title` String, + `GoodEvent` Int16, + `EventTime` DateTime, + `EventDate` Date, + `CounterID` UInt32, + `ClientIP` UInt32, + `ClientIP6` FixedString(16), + `RegionID` UInt32, + `UserID` UInt64, + `CounterClass` Int8, + `OS` UInt8, + `UserAgent` UInt8, + `URL` String, + `Referer` String, + `URLDomain` String, + `RefererDomain` String, + `Refresh` UInt8, + `IsRobot` UInt8, + `RefererCategories` Array(UInt16), + `URLCategories` Array(UInt16), + `URLRegions` Array(UInt32), + `RefererRegions` Array(UInt32), + `ResolutionWidth` UInt16, + `ResolutionHeight` UInt16, + `ResolutionDepth` UInt8, + `FlashMajor` UInt8, + `FlashMinor` UInt8, + `FlashMinor2` String, + `NetMajor` UInt8, + `NetMinor` UInt8, + `UserAgentMajor` UInt16, + `UserAgentMinor` FixedString(2), + `CookieEnable` UInt8, + `JavascriptEnable` UInt8, + `IsMobile` UInt8, + `MobilePhone` UInt8, + `MobilePhoneModel` String, + `Params` String, + `IPNetworkID` UInt32, + `TraficSourceID` Int8, + `SearchEngineID` UInt16, + `SearchPhrase` String, + `AdvEngineID` UInt8, + `IsArtifical` UInt8, + `WindowClientWidth` UInt16, + `WindowClientHeight` UInt16, + `ClientTimeZone` Int16, + `ClientEventTime` DateTime, + `SilverlightVersion1` UInt8, + `SilverlightVersion2` UInt8, + `SilverlightVersion3` UInt32, + `SilverlightVersion4` UInt16, + `PageCharset` String, + `CodeVersion` UInt32, + `IsLink` UInt8, + `IsDownload` UInt8, + `IsNotBounce` UInt8, + `FUniqID` UInt64, + `HID` UInt32, + `IsOldCounter` UInt8, + `IsEvent` UInt8, + `IsParameter` UInt8, + `DontCountHits` UInt8, + `WithHash` UInt8, + `HitColor` FixedString(1), + `UTCEventTime` DateTime, + `Age` UInt8, + `Sex` UInt8, + `Income` UInt8, + `Interests` UInt16, + `Robotness` UInt8, + `GeneralInterests` Array(UInt16), + `RemoteIP` UInt32, + `RemoteIP6` FixedString(16), + `WindowName` Int32, + `OpenerName` Int32, + `HistoryLength` Int16, + `BrowserLanguage` FixedString(2), + `BrowserCountry` FixedString(2), + `SocialNetwork` String, + `SocialAction` String, + `HTTPError` UInt16, + `SendTiming` Int32, + `DNSTiming` Int32, + `ConnectTiming` Int32, + `ResponseStartTiming` Int32, + `ResponseEndTiming` Int32, + `FetchTiming` Int32, + `RedirectTiming` Int32, + `DOMInteractiveTiming` Int32, + `DOMContentLoadedTiming` Int32, + `DOMCompleteTiming` Int32, + `LoadEventStartTiming` Int32, + `LoadEventEndTiming` Int32, + `NSToDOMContentLoadedTiming` Int32, + `FirstPaintTiming` Int32, + `RedirectCount` Int8, + `SocialSourceNetworkID` UInt8, + `SocialSourcePage` String, + `ParamPrice` Int64, + `ParamOrderID` String, + `ParamCurrency` FixedString(3), + `ParamCurrencyID` UInt16, + `GoalsReached` Array(UInt32), + `OpenstatServiceName` String, + `OpenstatCampaignID` String, + `OpenstatAdID` String, + `OpenstatSourceID` String, + `UTMSource` String, + `UTMMedium` String, + `UTMCampaign` String, + `UTMContent` String, + `UTMTerm` String, + `FromTag` String, + `HasGCLID` UInt8, + `RefererHash` UInt64, + `URLHash` UInt64, + `CLID` UInt32, + `YCLID` UInt64, + `ShareService` String, + `ShareURL` String, + `ShareTitle` String, + `ParsedParams.Key1` Array(String), + `ParsedParams.Key2` Array(String), + `ParsedParams.Key3` Array(String), + `ParsedParams.Key4` Array(String), + `ParsedParams.Key5` Array(String), + `ParsedParams.ValueDouble` Array(Float64), + `IslandID` FixedString(16), + `RequestNum` UInt32, + `RequestTry` UInt8 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql b/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql new file mode 100644 index 00000000000..cd2d3509c4e --- /dev/null +++ b/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql @@ -0,0 +1,189 @@ +ATTACH TABLE _ UUID '044d7000-c91c-482d-9f0f-719f226eff0f' +( + `CounterID` UInt32, + `StartDate` Date, + `Sign` Int8, + `IsNew` UInt8, + `VisitID` UInt64, + `UserID` UInt64, + `StartTime` DateTime, + `Duration` UInt32, + `UTCStartTime` DateTime, + `PageViews` Int32, + `Hits` Int32, + `IsBounce` UInt8, + `Referer` String, + `StartURL` String, + `RefererDomain` String, + `StartURLDomain` String, + `EndURL` String, + `LinkURL` String, + `IsDownload` UInt8, + `TraficSourceID` Int8, + `SearchEngineID` UInt16, + `SearchPhrase` String, + `AdvEngineID` UInt8, + `PlaceID` Int32, + `RefererCategories` Array(UInt16), + `URLCategories` Array(UInt16), + `URLRegions` Array(UInt32), + `RefererRegions` Array(UInt32), + `IsYandex` UInt8, + `GoalReachesDepth` Int32, + `GoalReachesURL` Int32, + `GoalReachesAny` Int32, + `SocialSourceNetworkID` UInt8, + `SocialSourcePage` String, + `MobilePhoneModel` String, + `ClientEventTime` DateTime, + `RegionID` UInt32, + `ClientIP` UInt32, + `ClientIP6` FixedString(16), + `RemoteIP` UInt32, + `RemoteIP6` FixedString(16), + `IPNetworkID` UInt32, + `SilverlightVersion3` UInt32, + `CodeVersion` UInt32, + `ResolutionWidth` UInt16, + `ResolutionHeight` UInt16, + `UserAgentMajor` UInt16, + `UserAgentMinor` UInt16, + `WindowClientWidth` UInt16, + `WindowClientHeight` UInt16, + `SilverlightVersion2` UInt8, + `SilverlightVersion4` UInt16, + `FlashVersion3` UInt16, + `FlashVersion4` UInt16, + `ClientTimeZone` Int16, + `OS` UInt8, + `UserAgent` UInt8, + `ResolutionDepth` UInt8, + `FlashMajor` UInt8, + `FlashMinor` UInt8, + `NetMajor` UInt8, + `NetMinor` UInt8, + `MobilePhone` UInt8, + `SilverlightVersion1` UInt8, + `Age` UInt8, + `Sex` UInt8, + `Income` UInt8, + `JavaEnable` UInt8, + `CookieEnable` UInt8, + `JavascriptEnable` UInt8, + `IsMobile` UInt8, + `BrowserLanguage` UInt16, + `BrowserCountry` UInt16, + `Interests` UInt16, + `Robotness` UInt8, + `GeneralInterests` Array(UInt16), + `Params` Array(String), + `Goals.ID` Array(UInt32), + `Goals.Serial` Array(UInt32), + `Goals.EventTime` Array(DateTime), + `Goals.Price` Array(Int64), + `Goals.OrderID` Array(String), + `Goals.CurrencyID` Array(UInt32), + `WatchIDs` Array(UInt64), + `ParamSumPrice` Int64, + `ParamCurrency` FixedString(3), + `ParamCurrencyID` UInt16, + `ClickLogID` UInt64, + `ClickEventID` Int32, + `ClickGoodEvent` Int32, + `ClickEventTime` DateTime, + `ClickPriorityID` Int32, + `ClickPhraseID` Int32, + `ClickPageID` Int32, + `ClickPlaceID` Int32, + `ClickTypeID` Int32, + `ClickResourceID` Int32, + `ClickCost` UInt32, + `ClickClientIP` UInt32, + `ClickDomainID` UInt32, + `ClickURL` String, + `ClickAttempt` UInt8, + `ClickOrderID` UInt32, + `ClickBannerID` UInt32, + `ClickMarketCategoryID` UInt32, + `ClickMarketPP` UInt32, + `ClickMarketCategoryName` String, + `ClickMarketPPName` String, + `ClickAWAPSCampaignName` String, + `ClickPageName` String, + `ClickTargetType` UInt16, + `ClickTargetPhraseID` UInt64, + `ClickContextType` UInt8, + `ClickSelectType` Int8, + `ClickOptions` String, + `ClickGroupBannerID` Int32, + `OpenstatServiceName` String, + `OpenstatCampaignID` String, + `OpenstatAdID` String, + `OpenstatSourceID` String, + `UTMSource` String, + `UTMMedium` String, + `UTMCampaign` String, + `UTMContent` String, + `UTMTerm` String, + `FromTag` String, + `HasGCLID` UInt8, + `FirstVisit` DateTime, + `PredLastVisit` Date, + `LastVisit` Date, + `TotalVisits` UInt32, + `TraficSource.ID` Array(Int8), + `TraficSource.SearchEngineID` Array(UInt16), + `TraficSource.AdvEngineID` Array(UInt8), + `TraficSource.PlaceID` Array(UInt16), + `TraficSource.SocialSourceNetworkID` Array(UInt8), + `TraficSource.Domain` Array(String), + `TraficSource.SearchPhrase` Array(String), + `TraficSource.SocialSourcePage` Array(String), + `Attendance` FixedString(16), + `CLID` UInt32, + `YCLID` UInt64, + `NormalizedRefererHash` UInt64, + `SearchPhraseHash` UInt64, + `RefererDomainHash` UInt64, + `NormalizedStartURLHash` UInt64, + `StartURLDomainHash` UInt64, + `NormalizedEndURLHash` UInt64, + `TopLevelDomain` UInt64, + `URLScheme` UInt64, + `OpenstatServiceNameHash` UInt64, + `OpenstatCampaignIDHash` UInt64, + `OpenstatAdIDHash` UInt64, + `OpenstatSourceIDHash` UInt64, + `UTMSourceHash` UInt64, + `UTMMediumHash` UInt64, + `UTMCampaignHash` UInt64, + `UTMContentHash` UInt64, + `UTMTermHash` UInt64, + `FromHash` UInt64, + `WebVisorEnabled` UInt8, + `WebVisorActivity` UInt32, + `ParsedParams.Key1` Array(String), + `ParsedParams.Key2` Array(String), + `ParsedParams.Key3` Array(String), + `ParsedParams.Key4` Array(String), + `ParsedParams.Key5` Array(String), + `ParsedParams.ValueDouble` Array(Float64), + `Market.Type` Array(UInt8), + `Market.GoalID` Array(UInt32), + `Market.OrderID` Array(String), + `Market.OrderPrice` Array(Int64), + `Market.PP` Array(UInt32), + `Market.DirectPlaceID` Array(UInt32), + `Market.DirectOrderID` Array(UInt32), + `Market.DirectBannerID` Array(UInt32), + `Market.GoodID` Array(String), + `Market.GoodName` Array(String), + `Market.GoodQuantity` Array(Int32), + `Market.GoodPrice` Array(Int64), + `IslandID` FixedString(16) +) +ENGINE = CollapsingMergeTree(Sign) +PARTITION BY toYYYYMM(StartDate) +ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..939ea6ccf187b34a8db6e4def524f9dd13691092 GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bhh(-qXp!7uuz9v-dm@^cD z*Z=?k&Hw-a@jV6!V{dhCbS`vwbON#VDF3f}{FN>b+}S`mO_BfzWMOn+E@EkJ`W*io z<|Oi02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcns^T1OQK`i$9(N KipbM-Mdtv4FmBEO literal 0 HcmV?d00001 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt new file mode 100644 index 00000000000..5ea415161bf --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt @@ -0,0 +1,47 @@ +columns format version: 1 +45 columns: +`type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4) +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`result_rows` UInt64 +`result_bytes` UInt64 +`memory_usage` UInt64 +`current_database` String +`query` String +`exception_code` Int32 +`exception` String +`stack_trace` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`thread_ids` Array(UInt64) +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) +`Settings.Names` Array(String) +`Settings.Values` Array(String) diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt new file mode 100644 index 00000000000..9a037142aa3 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx new file mode 100644 index 00000000000..4b6c49f8d50 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx @@ -0,0 +1 @@ +…H…H \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/partition.dat b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac GIT binary patch literal 4 Lcmb0VWo7^X0I>iw literal 0 HcmV?d00001 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx new file mode 100644 index 00000000000..d599340cc82 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx @@ -0,0 +1 @@ +…H!.œ_…Hm.œ_ \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..747078ab876a6913b8e0a9a92d3d0a3e3a1deb0f GIT binary patch literal 262 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyWHkv{Wolw154dF1>;$$|v}nZZR-0 zTma&a_H3NV`K5U!dL%eM=a3btktj z6fTJR@=T@IyyV0s#)et!Y;O}d4yN1KT?$>ea}nQfR!#$k4|%yo*~S~V4GOsV4O}$! zZrdrRHA`0hWDw2G%*#!zh)*p`%`1rq8l0+^nNq=`zaq;cGh)rekNa9x-^dl4GVm58 z7L{a{09~#JRK)U=M_>H!p{7$hOFDXtB}~#8xC@Fha}$dyLArT1wS3n4_i1|9>D}5g J%+@!V7yzW3XQcoD literal 0 HcmV?d00001 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt new file mode 100644 index 00000000000..5ea415161bf --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt @@ -0,0 +1,47 @@ +columns format version: 1 +45 columns: +`type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4) +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`result_rows` UInt64 +`result_bytes` UInt64 +`memory_usage` UInt64 +`current_database` String +`query` String +`exception_code` Int32 +`exception` String +`stack_trace` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`thread_ids` Array(UInt64) +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) +`Settings.Names` Array(String) +`Settings.Values` Array(String) diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt new file mode 100644 index 00000000000..780fea92d29 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt @@ -0,0 +1 @@ +77 \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx new file mode 100644 index 00000000000..4fedf1265b2 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx @@ -0,0 +1 @@ +¾H¿H \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/partition.dat b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..7af4421fcb5ebc50aa06a5f4dee6e454020552a6 GIT binary patch literal 4 Lcmb0UWo7^X0J#7) literal 0 HcmV?d00001 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx new file mode 100644 index 00000000000..7f33001d913 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx @@ -0,0 +1 @@ +¾Hb˜ç_¿H»©ç_ \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..eec9f2828ad60100f834ee2ec7c2ecdd572e4d2f GIT binary patch literal 261 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyUI``tQA+<)x6f|4kELHWvPFy2-%6 za2|+1+Ocsa=a=S{=#^BIFika>Ad;2)__e!oc(_Su$x;T6l*E!my`;>%-kfLCXPdd* ze|I8LNK`6r-p_c(P3^3@vsGTkiX2IichpIIeqjR>r#{1nyxgK};|<&n9adSpqodSI zI&*}wm1=%7h~{SI(O)RPe>EV>iAT$c8rTFS^0FlRNAYdvChb+}S`mO_BfzWMOn+E@EkJqa$s) z<(i%4_5Su;$|AM?s{R3`Wi?hP8GT=Tu0P7G#$xv>0SPVu@NR8#YcsG56t3`xr|pUp z;{9}LW*LD002OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=138%)U-7Z;v9?{s9d$U?01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnn{AlpFuovTY~I Kn?YGm%iaJK*Kw-= literal 0 HcmV?d00001 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt new file mode 100644 index 00000000000..5ea415161bf --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt @@ -0,0 +1,47 @@ +columns format version: 1 +45 columns: +`type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4) +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`result_rows` UInt64 +`result_bytes` UInt64 +`memory_usage` UInt64 +`current_database` String +`query` String +`exception_code` Int32 +`exception` String +`stack_trace` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`thread_ids` Array(UInt64) +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) +`Settings.Names` Array(String) +`Settings.Values` Array(String) diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt new file mode 100644 index 00000000000..9a037142aa3 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx new file mode 100644 index 00000000000..0e3b7fb75b8 --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx @@ -0,0 +1 @@ +êHëH \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/partition.dat b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx new file mode 100644 index 00000000000..0b5fcd2fdea --- /dev/null +++ b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx @@ -0,0 +1 @@ +êHÌC!`ëHG£"` \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..a466972c338851bb9e376b5fec0b27c54f763a01 GIT binary patch literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyUI6hQ;Hcxb2IaD6D#6V%Tn`7;(^Ad>Sd->up}O2Td;F;<|11GQ{zVyvz!@t3lfV; zGE0E&)&nYHagV>iAT$cnNbtE_VA_ox+gYO}M76zw zkN^Mxh5!Hn@jV0yV{dhCbS`vwbOCIfRlVt&lYFUDR9N&B1^)mDWMOn+E@EkJ&k!F~ zY}HzkJ8Te4L?36TX=VY?FDFqniJ!N25;D3H*NTaDs0l6r@Fs0?Ycucyl#~%aH>Tie q3B6siBLUYk01I$(X>DP0c`j*WcnlPm#22y_7RAN-7E)Vm7x4gDE=jup literal 0 HcmV?d00001 diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt new file mode 100644 index 00000000000..725a526b957 --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt @@ -0,0 +1,13 @@ +columns format version: 1 +11 columns: +`event_date` Date +`event_time` DateTime +`timestamp_ns` UInt64 +`signal` Int32 +`thread_id` UInt64 +`query_id` String +`trace` Array(UInt64) +`trace_full` Array(String) +`version` String +`revision` UInt32 +`build_id` String diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx new file mode 100644 index 00000000000..b3b53069cfb --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx @@ -0,0 +1 @@ +ØH9Ö `ØH9Ö ` \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/checksums.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..41a34e68660fcf30f2ebf3886af70965d1c50d97 GIT binary patch literal 188 zcmV;t07L&{Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cC)=R{W#rbM&U?0f^HAjYm zkN^Mxh5!Hn@jV0yV{dhCbS`vwbOCIfRlVt&lYFUDR9N&B1^)mDWMOn+E@EkJ&=4}7 zAlkfU=Me)DyD->zPV@oLFCA^8bN;KGdEy@&Pv!bL6$vf?@Fs0?Ycucy8(uAC5Vito q|AbpmqG_6t01I$(X>DP0c`j*WcnnVC4s8oxdoa!Z4je{MT>k*Ano9!! literal 0 HcmV?d00001 diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt new file mode 100644 index 00000000000..725a526b957 --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt @@ -0,0 +1,13 @@ +columns format version: 1 +11 columns: +`event_date` Date +`event_time` DateTime +`timestamp_ns` UInt64 +`signal` Int32 +`thread_id` UInt64 +`query_id` String +`trace` Array(UInt64) +`trace_full` Array(String) +`version` String +`revision` UInt32 +`build_id` String diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx new file mode 100644 index 00000000000..24c83f6002c --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx @@ -0,0 +1 @@ +ØHÏØ `ØHÏØ ` \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/checksums.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..94103f5a76ccb7f25b7a74030c04dbc2413b0242 GIT binary patch literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyY$U>CqQtR5+I_knLZ-OHz2#4F(2= zb3pvjmW?wxzcjBzucV@cX`=X>wR6M7)Yk+*GdQ7`_n(0yC9xz?FDWyxQ#5noG1-=x zMe7?wHfev7dG?oaNwm=1jm!7WVfXwmTE9_VNPjb@9>a&c+@fsb3oO0Y#oSLXwp>@( z{bpjsq?khtqPdxQxrr6=sb#5oCGkLGQ}r@aDp&%G|HSG1ZQ1rGFrD>==$*L?yakCx zC7C5ack2NavHaxG7yo;x>6FfrjviwPlXM2|f}+ga#G*=&Zk~`GM+A);B?QWr7Vt)1 I5q`=5051<|M*si- literal 0 HcmV?d00001 diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt new file mode 100644 index 00000000000..7775d6f8eb4 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt @@ -0,0 +1,20 @@ +columns format version: 1 +18 columns: +`event_type` Enum8('NewPart' = 1, 'MergeParts' = 2, 'DownloadPart' = 3, 'RemovePart' = 4, 'MutatePart' = 5, 'MovePart' = 6) +`event_date` Date +`event_time` DateTime +`duration_ms` UInt64 +`database` String +`table` String +`part_name` String +`partition_id` String +`path_on_disk` String +`rows` UInt64 +`size_in_bytes` UInt64 +`merged_from` Array(String) +`bytes_uncompressed` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`peak_memory_usage` UInt64 +`error` UInt16 +`exception` String diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt new file mode 100644 index 00000000000..eb13855b7d7 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt @@ -0,0 +1 @@ +79 \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx new file mode 100644 index 00000000000..07abc658bdb --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx @@ -0,0 +1 @@ +¾HÁH \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/partition.dat b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..7af4421fcb5ebc50aa06a5f4dee6e454020552a6 GIT binary patch literal 4 Lcmb0UWo7^X0J#7) literal 0 HcmV?d00001 diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx new file mode 100644 index 00000000000..16d6a0eea67 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx @@ -0,0 +1 @@ +¾He¢ç_ÁH(|ê_ \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/checksums.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..2b7fff5ccb9418c7ab7aa25bd1fb7d93eceec517 GIT binary patch literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyZ>tGIK5Cl6r%;##xVQ61m%&ZZI$~ zoCD&Iwrrfq`K5U!dLfa6k3NUy@yiSHP=3yLyx6N@TAx_KfGyWeaxDL=3MNbdhr I+Xd$s04~;Rw*UYD literal 0 HcmV?d00001 diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt new file mode 100644 index 00000000000..7775d6f8eb4 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt @@ -0,0 +1,20 @@ +columns format version: 1 +18 columns: +`event_type` Enum8('NewPart' = 1, 'MergeParts' = 2, 'DownloadPart' = 3, 'RemovePart' = 4, 'MutatePart' = 5, 'MovePart' = 6) +`event_date` Date +`event_time` DateTime +`duration_ms` UInt64 +`database` String +`table` String +`part_name` String +`partition_id` String +`path_on_disk` String +`rows` UInt64 +`size_in_bytes` UInt64 +`merged_from` Array(String) +`bytes_uncompressed` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`peak_memory_usage` UInt64 +`error` UInt16 +`exception` String diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt new file mode 100644 index 00000000000..9a037142aa3 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx new file mode 100644 index 00000000000..7d66cd0aae2 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx @@ -0,0 +1 @@ +ËHËH \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/partition.dat b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..a713aecd075daae7071deceaa93147871ef28fbf GIT binary patch literal 4 LcmXRFWo7^X0v!O3 literal 0 HcmV?d00001 diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx new file mode 100644 index 00000000000..3e5042c4bf2 --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx @@ -0,0 +1 @@ +ËHÝÕ÷_ËHEÙ÷_ \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/checksums.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..322c6666b3efd30007e1786d71ebf7dbb6bc46f1 GIT binary patch literal 10792 zcmX|HcRbbq_kX?b`(FFno3hEsY|xNB%F0MWT<*m+?&Z2hG;B&%s8l3WDkCF18Ym-F zMwCdUVcmP}mEXI3zQ2F`aqoQ`uh)5<^E}V3N(6qi0e)R^Q=BoWMwR6U5K%rs`c zqebZ+C)gsghP0ebQHp1vhrPLI*c<=ZJ<`Lhf@PpljHCTEw>F?*11Nr)#cv=Ph4Sk{ zQLLo`hh*=scf4W)%=+F02SCT9rCf)2dF=i;=;5C>o-L#63GI$Q0ca@JhfltIz~o-- zVUp^>;#xklXaLv>(0EdQs&h*!P=04R%#`ObPt5>c0}44Tm_RWkhmfe0GfrS#Y_lu| zx|E`M@cx-EORtQYclOXES)l&BMwX0*vC@&#Tjz%O_zQrt*9PAq0OP>|%EJ6XqEKe|$sBK0qv> zd*fHs$uDQ1;-^d8BM$~`#{cxhaCw|pof<5$WkoJ)4d^_lS6&$lo*2oEywRSzc&(=y z@EQ6O2_&jfV}_%mZaSgmq`#_nod)CXsPIh1ssY%(%dE-G4-{53w~=WN`J~iO0qLO zTj>9Ihk_eeM7^mtzDyk%|8P&J&}n|p%YE2=06vL)t_WF_v&c$`FU?zELT>H33Zwz| zJ}0msRuQ!h*9loRNR=OjpIhTU2Gxu!L-~uthHtNoO-Y_qdX{85ruZvQPleM7@b5c? zoT@RrLus9trQ*%V{WXS!*a+a20e}zEI7KE2Atz%LK7ddQrDqnX52l{lGtE%9xZrMR z9xB*XMq81Zw$y2IKR4X|Nyi@g`viDEYVL9h_4M z;)hwLE=NifG>O4)0goY>L?U<&Lh3}4w>x67G^PnXG@XOz=$xF&3dtdE9P)b{uhSIro;(5EhGeoIkzh>1dH731 zbaZ73T8oE#RZblwzm_49+*bYxBs|NQn%t^Ej(yc<|WRl5jM z#-GooSiP z8d=$eAg`MK@;#jOM~4xwCm1NPN9p^qpgWnTz!J)~t$xqtfD!Fl`^RB}AwkKU`vHu6 z&e_Jh@cd2Ny4V5b;<`yAZv`M=WNm3k4hZxoPzi`fF;7>cVMl+zh2V!t2Xud&@MBNC zT<8J*pjxlKC1^{RKTLeyeQrNVieK;ufPDcSX6^UrU@OR-h+x}nm=XUu2}G4a4dOBq z)q;$}6Yylj=CAA(G}gaQ|A^G$fru9xc6G`>9LZn{H5}fKHPzkR*>Lz&-OHaNLt;h% z4O0P)c}>l%zT~(AbYs9E-HB@eXQ=n z*P3_L6vX6a5c+Gf!_F#Z`^LrVFLRPa3;+ra1axL?Z&3?1p3~L)U7q*cPz-AYJVs;^ zwHMMbBT{iFI%Y2Z38cVoi~W>_3s}V-AMUteAmM>pLyaVDU3$t=YZ*@5`Nz-xoaKL! z2Q$iy8su7D3N73@Wa#DARSvL@7!4yFbtm$;Mc4LoYVELtIHZ(oQ8|HqgI|Y_pqEe= zv%G7#&1Y-{&7Z!k=%^jve%yIza>8T2FN2A*t=`#2*Y$tHti!|2lHF+Mk1uABukB zn!Oxc2-r;=NW>t1ctLFR_6sz4dvB~&tng@kiXu91QI)~2O~x5O!qaTB3xx+pm3N!$wHG9geseT)!qb*6WqhacSj>6-94Yne;1 z$>-py+)A*5lFIA!d9(98d&TH-qu!#B*>>jB*)^AvV)y0NYW^@e^P%N*av9wI}np#%G1i#?H+fJ6J_wGv>DMX{2Qmq12TQjL3&mM+g0d-0QHuewu-)K^%JkTm|TCw+YqL^;O*`Hde34 z1@7NG1^7(~BmxEJ&xw{M1n0t=Vk2`gOdL|tV7QrBP@*}Ywz~%N6H2XaReBO7sRqZD zP*i@Yx#6N?uW!G?CM>fNmRssD9?uuGS?T?BEEb3=0h?fA1VLRM!KU&D7g3ClCIw7d zPL+Ae_O+|dVVxVOE!2Itf~au$^CC*m7yi2WJCr zQ!>s!*p`6v^daDV!K!H9oJzFeNbr6Ez1N8ey>InPx@Dn_8y#pone#{0Z(fz1^${`= zji-#a@pl!rcO5ZUVosefC?p*c8N9_01myvy7kSsx+6!KRrv_LmY5r~cyVZrT31@o{ z_()9g@07lGXD&}A>G{eBAS!eO0N4+K4?cEPde%_+)Qfqxl){o{VE}7NCgXM3z$?TU z<2U-kMbqz{`DOWVqkzSdrsb{ANWSNvW&W7Sr+G6V6AIzX*EzeL?+mo-FAtAW#)?#| z;-_*hyPR3TY!-#2z+7xUc4Pi>YqT%^T(s@=1F0clN-tp=%KnMxgsjEo9+|1IlfAFr z*KN}{2HW+#yP>?(VskE>&y9{xYnuS9nZ3P@F@y7y~kO8T5@KRYjd=9g7Z_4er)9Dlr2@-Ak7@xOa4-_9n+ zGYD{*+gK5(VPuM53v#lh%4O8Vk|!#)_$)qQ%!gC!=CE)JLTScwBMkIe><)7%svcBO zzU;9LYz*diyn6OTTl|YQE4BUm-D%T+&79;(rZLgc?_Qp}LeB3fo={v)6&mP9<>vd(<0X?c0jGJe5jl+HPsZt^>FB6xEQoX|&R;9>7Wgq3 zMjW)`a6v#Y(B$xH*4=%+zu0$pnjS4XybZI97@1`HSDKdAw_g%+tHCpbw}9O|_#G0h zKdVu092dXw?XiAJZF}VlP+sp&B0Yt4(^TQ9JG@Ogb-^r(hrav#MKTA3M-~+5=zuQ@ z;TQahm~P(c%tD+VY(XUXu_ED6)2=f-zW3YXyXVKNWi#Wwonq(g5w|zHPlbFIs_?#X zC~wTAE-ZZ8?F`n_yXE@XBRw7&a35QGszxXQ9Ol86WDjB(2OV9WyXTp?2$NrIcrpX8 z;iWy=tmS}2*2&k+lh^4<)26%5>ucM{wEh?OkiSp*o%UNxwT>g7ZpV1dTGT<&R%EIH z(o=a>vC=Uw<}#q%>c_J`Bfq|>3AHpbmhF1tjClT5gvnyP`A)~f*)Jf%JuQ|0JkPSf z^XMz>>BwhKyq9q;cMQ)1Zu4Mke>{bahhV@$N5?!t#hGj_|7Kfu%T_XS_E2hOMk!BQjMM(1Ok<{3a7k1WMb}5Zp zdGMVqg0itCdyuJZi2J`Z@uoko>8a2V`HuOO+}ilbDgxFtn#KaPY~LqaR%nVgiaMR>g*hxq32_xGt1u7OXv z7Ey*Pav?!F=Y#cx7mr>X^K{?#{t~lWf7}hV3;8{IF0ASEw!__k^C&J9ca@WluFG|J zS5l#L`)ZlP)R!;GfguVx&|+uiea0mGes((M;$j1e+eO`RWmfcP zWpaH-0qgEtwwNu{H`2V1sP}lN)&BeB`>%_42NJv#1x@MbL=`LgbwY9r8=G@Lh z|B+$_?9pz1@f)0v303inEI4@V7~rahWo&#Lc^GhRtS0StQa{%nnxUo<(}#>QfoP@` zRLL-LfA!%N;EF|&6jd^;qTVT02tWevTp7KOquZj}jymjZq{pl^wEi{~+$qCF`# z1N(Yb7c!G?a{5fIzPZ8o4(eLG%tE#-@y4XM=mq%=Tm7P6#j1x^~L8{TzMh|8{^5AP#vtW@atKR*{>hRf(%W0wfcrmk|fCIt%I3&LZCwSni2Ni@ZrWJTM zk1su4%aV#jEuz{R$4!|t%9$M4yEOdv$5a2S98e~QW=x1!)puc2VXl?(Y9~j4jzX+# z6?4aF{0c?yc-9;x-m$o2pz&@G3mqCUTbw4$K3g6fS8J$kiG1h?pzZpIi8UOrHLP8K zgcL9^Nofd-iKa!&$bYGy7DCy#7AJ(-mKM#sMvIPrhnY33qqcP;^myYK~ z6%T}>n62$B01abK;_OX-{Z=)F*s^pm+d~_0S_cYqNL%5U5p+oGW33)>$ZnxFUA18u z2;x~?t0h3oDEa*Q!`Wluwn7meRKp{pG5?GC`A?dkKU+n1ucXLa$5O9!0B%`e7fi)b z^;6NSqD^DWLG`^~(@N@!(G{&Xa*NSfP(@-9x+JAbU}ba90f|>H_#aT|@HW;gz@tWq zyIXzc4=)Nl6<2I1l=w@!VhnKsfjFW!i2$}lD}FF__0--B`1SH?U&l=EmokkT&{kq0 zI*qhI>==w(_6e%Uj_*sw{Kvi)QR!-U-`9*<@y|THDuy;6S%3-%u_I7IiJk;iXhCEE zTAOOfJG0$0Z1C@Dsiu? z2Y=t)d^Fm>Qd4>g@b3n;hyB)DaZBc~Y4rPnnRKMviRO~1bA1xAnZXnZOJDce6!KI;4iItp@opm=E ziI%r)0MB0LAj1#}g+THY5ndC0*uvb^6`TqlskV(=iL54OTz(1tnfbtx@cyc9jeO%X z64URrJ0~A?@*V&%4&fG8uN3m#G#c%QakKBA=KSCd_!Iz27D4O8B#f{y@7;bymZYt$ z)L&-3O@aPw8>mRlimbFj{UKrl>jCo@b>I2_X>xQ}!1CfzrTu)Wf^8}BmmL~U0d7UG z$C`qF%g+$0UqUH8(u?RWH)ogbn{T<%sZN2e_vWF0Y3BhK@6RHv)@e;cKjims0rWqu zg$_G(kADi}y*$otxti)I&Q=aM-GHVIg(%EQM>o9Y?w1P>%Lf@}L(PyWf z=->Lel!1_q3W^sm4+7;hjgJw|Xe6SyM4NK-4mop$TFDhy3g9n0y73M>K=*IlXKM1f za&AoTQ^yyY+fsG;n1-+(fRJ$8f+W{bhb@60Xfr_xUz`qc4FC|gCh~>I*Dd3a%$l`` zOKuu$5{37~C!#hWM#)F?*$bUJo%*avYQK5i+WXo@0|3AT@@zqmJLSU5q;x`EYC;Rv z4;=FPYe?&_q3H(kaF;t5-3aR>tbaxCNiH($MPFy=^x`7zt z*9yXHK_B?us+w@RQ=TVZiWmUP64gMC;O(qCOED^H4C;< z!>0F(ul7~kZV;$`S|~8!j~LqHkK9eeGgI!^1+feph1C|xcQ3B{BZl@OUkJD6Lk}B_ z3-4w2tIUtND8C=zEr&YC40Q>A*1Ifgp6O#-Gh8-OWd9mMe!mVE@w}NgYYS0iN!y{f z(=t6C>>Pt)PvHDR2u9?PSHfFjKmRPjZNu`CB|@JPuj#>$esHA*K%}zTnH`Vm9 zc0l{$8RO%&@G6vZs+l?SK!eO3=cuS6r)SUz3PxluKfqv<>l#&m{6ycn(M@KOlZn#Sb3HNMK$=IXfguXGf;3R88f@h+lmr zznziTVJt=gFcq>f(3O4paOhxVpap>2CMybHWQK9*O7eTTr{5?^cYK+?ivN ze4YUQr^+%|W+VoauDE?-Q(jW5<7sPT$b|FdsarVPNy>&()o0#f zfFqt3UlXHu{6tG7F0GaG+<4Y`F7KDdLjHbeIww~7>A}4PMxGYG*RG!tpYXq`x7|g~ z!{c9KW8}TpqmtK{j~G1tISRNGfCkd%-s43Yt-e-l)Gk5O&%~qIH~BYE%y$C zr-5}Tmopm0Uc$x;IGU$#v-1U10Hdgo2`yq`ER0h*FsQ}@;sMT<;El|Ja-n&#UkfD= z-XlJT2c0Z?I?j4(3OXSVHNCpbEj5|#DPcHkZ}SU#$H!4wm=6cg|K4>S?t3?wMhq>x z!dV+v6!~Jv5D4!Am}`Gq(zoMq`Hj zKw5kE|D;anPh3qj`Bn|K^8WO`Be8xWT_|Y^QYY!?^j8|MMpS(hE)PWu_w7|0WOUhI zf@#puweYsP0aY2xL%k~!)8-)^DCl3K#efMpb6?43*bhdhkLs`ZeoF!zlK@WOUlh%K zhFWB`IHo^vXk+}Xs?ia0>EypD%!Q9xX7`?1DWX;!+sG3fUE5<08uWpjer-fuqe0$_ zf0{il_037(9zdoN%x$z0;`1tb8rE`uJr;d?O5nTN_4Kae-!1>nMtEx~X|F9CC&@0C zU%ydQ%KP8UBqaSVG3sP6oo6UzQ0ai9DKayGswC)`rn|r3z>);A$+cbfoPQ3T>TUcu zyu`JD+N`iT*4q)#Ht@rJ+B#3N8kv}2sw8Mw(dt|AjpIu;4u=(k*&eT20Zv;&PzWKI zY84VtjcCk_+-T;KupE|fru1NuZzo$G$frnGBb|epqO9K8teD_3=g`75jmK47NMBR~ z7NG+(W^CpB{F*wBvZ+6Jy7ajKDSL`E8y%A|R|P3QsDDwp&_AaA8bp%|6h5{<3s9kn z;6X|1@mhs!iAR?-3{JWLfS7Ut3~tslrtJC8HO~v8P3`8W);568mJm$#51}FfW<&^_ z7oek8JN&u7*oe{h&c~VWx7QZ&4exhHW>!6tQi&zL*K*L0YLvhko*&z_dl~h^$(a;( zR8`{eC1!Ef*@0(35ga!o5WRiOkT#@wWCVKD{&SbZW&2iFw^zO?5(PeqY)dHo5%s1G z4RC9?c25#NU*DF1H-P^Muq)8slu9m9zn+PoE`>2Vmezc4fTkl6{|0ee;l*)?^~%Fa z&aOI|PKjiok9I<>H&}+c-$q{cOaDl3{eIP&P4UqBU4Z&GPoqIcrB|699_kpa$E0+7 z3yE_!V~D|lWKu9ef`^WIFc8b!H2>3~v>5Nc?`ieLhkU#z7|_gEZC z-s5*ebQ^ybV5#12>nyG8{!trWliyiOkw8cpaO^-Zzaxd2OM9=+m#rb`@?EhXUWjwQ zN6HlH4dc2kPGF^amVdq~^|98Pus%|>Nyua)k~92^i4sWIsEKmD^H4(=VDWs?Bmzy6 zj=omt6rpVK8RZcFwyvl2_;htYgh4H#f_Hm8eBacEN=wm7f93IAV_RKYVI4m1`@Q+{ z-GY1fuVvzou&x7MI|7d4=~F64M-Nv}Ulsb_`1R5HdH0WL6Ganl&TMEMWifA^J5YGS zJ0v~h4EaZ$9v?FB@$-_3Ele6@JkKWqPqaKOC_xZC(pa0t_cSH*7A%Q0hB zRWQ2(<=Q+)U7ckG^c6~f!{GA?faH24=4wJ!4pje!&SQBHYUmDDFxu$Df8Met8XaG4 zb8{y2DgaJ9!VCgB>+MUZP`##uukY8^IbCV2sA-CQGKF47x%R@(KUjqA4Ouaml|R1x z>2K`2025z&7K zx={26F=9eb?iWbk<)m(CbkneAbs-@!s!0liT+^k((!6o>xj#@fb0Q>=X~pIe>rL5p9JP2_7&x% zC(Gn&uA_Z_DP|gW3u?Tw952rt?oJX38dn^-F#L~V9zsGQIw=<#4xvgHRx#`)uh~7@ zyN1@uLv2A6T}NxjqPd8c48Qg2ugBv_h@o`E(1t3};lnS&Q?)(p&W0=XP@ z!7FM2U^ob@z}mH$l)887#g_hIByyLoFyL1Md^SL7Q#5giKd34H-5odo3zumKuu!25 zsI_1Kz3zD-R&2~s)}!4B^Cpd`wB08z!oc>GtK+Ljt}c(rMPZ^1j{xf$%n+Zu_{iEt zYw_4v-;Scs9(EpXGRWk*v-N8?>W(yL1Dz4DR(Vi!`LI=08`~a$wgU_nr|gX9Mkcvz z*m?ocJET>>Yfr%;3a>Ksx2eO7t}j5v1utYU4@+Fu?&`m9lh<>XLm=fdm)oqS*vCcl z+er!@O`UApn;Y{mTlTI_k1fw5zP5_vT)Daj0rLqoO?>rhr;9a7x;RuLbKGN)Z`V!U zI$LP=?$2)2N{`Mv8=FJ7G$kHXJ-ImcmPPaLro|eQ#hVP>?$W(RMbGU{?XWO&13XA> zG-21oewLO)wT?~`>sDzAe;&w+O9vlUgJ`YsT3xd-a?&CqE9N}+CP!3Q+_s6@8XWqG z{nFscMjPi2sWaBD>SQ3K05Cf^kg?<111Nz1QGaN+*H~XKr_0gifwUbQTc~$SKWjrh z;&0q-Y-{Wl-d+12hA%PoTYui=cb8&PxLV`Syy`^)SmaD4*d?) zI{D?L@k_%{$~(I0g|b7;r?{p?3fSk&u#a^*ww3h-rs42u_ba@ii!&=*4^GINZ~}J3 zQW2Jp`TSslXR_rQ58=tCW|o*g1o- z)|`ctT1wgzdRk@go4^@>GU9-4VV6bA>jI@pCxEsTau{;Yv1_> z`=aS%n{u4{LL{zi1JW`~{vHoP&RudA7Ae*_b}j$lU#SukiiAK;LJBnfVD$R)+QY?^ z*kA$1Z0BqPqEa3xZCSIMJI@k<-e@aLD zF+;YBwbO~L&)m!tntAvBCpN>K8krZIa*Qpk`4?HG_slQ=ryMwrOzsJ!PIkITW);`9 z0AHTTJIT)mp7xP1)^;aAC3%(TVRv_~oK6KB{`#wVu296qECBzT^fK5EeK*(AD+?N1 zb&FHJag=2Xi3od3OQe@G#nMFT<3)Md&^6dE;eI8LPjL5+5^h2B_Xg1^ccOVpRc{dF z5+;TooZ^hk0Pw%X3_2wKSY)qj(WKelueS(n8#<-J9^&M7+$ z=SKf=;YhNCZa(^fo{!Eb-`T0S%(Z#5G2n!jq|^Uwu})&q3y&@1b6Sd5M5=mZnvf8) zx3niv9)kr@S`oY6gO4hOjWM2MT}#L|fl?YYpI3^`WlN6Q=x7QsjW{wPd0=*6TUeiG zHP_HyE{yPu3YI-7yh{~hfLs$<@Y46jK}%wzSFS*_rlG+_dA?8X^=hOm_G=CP+dh*q zxSn^mYM(4w%d=3u4UON9{YBUzfZ5o@%-u&)FViZ1{|4soW|1Z`YEO`%W8QrjhNQIQ zm0qMod|8q6j^&Pv?0(6-j)FFuQ?+!@+2x_tLY>1RYPQj41q%5Xbz!V&Uq?&8SV4Wr zizWQuNjn}9*4ul%420xD72IpS9ypeMhDWfv!xLeR`ZAsEOh=bGl$4rCGwY|`c>?Gk z6{5lKe$BnxH}M37HNnkGNlY1lEg7J~5gGjZ&sK~}CL!zO+FIkyo{vwN7mh%A@I30y z>Do;TcGt4GnDj1r`~jzJ&4#lcfNl8%@ zvSjAZDsV^%U^arFCk{w*!@tEl&zC95sDCc8z; z8zQy;r}(eH!a!1-dcI9!>A?%`H~uW^^+y9ieO49pI-;o;OhNu?cCjk@htbp2 zNuAhj8nl4M>s#-u>uCx!Irhl&*4EJs!0ObjiELS=2*f$Ge zfg;>z`^kl!J};t5f6a$ZpJhWT&v8VwlMhyZS(P#G;Z$42;(2mx6tH^sY05Ku4`ls553pOZ%%@COh{0l09tZx}icvadr>CE~-;hIe)Dg_qIVf1GoZXDPDW z2e5%SWNiAKjmC+`QPCUw+YI-`B^e97(U5{P&44g)Ofl!CYbC OT{m@gI+HDZ!T$k}efzWk literal 0 HcmV?d00001 diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt new file mode 100644 index 00000000000..abc4136c586 --- /dev/null +++ b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt @@ -0,0 +1,135 @@ +columns format version: 1 +133 columns: +`WatchID` UInt64 +`JavaEnable` UInt8 +`Title` String +`GoodEvent` Int16 +`EventTime` DateTime +`EventDate` Date +`CounterID` UInt32 +`ClientIP` UInt32 +`ClientIP6` FixedString(16) +`RegionID` UInt32 +`UserID` UInt64 +`CounterClass` Int8 +`OS` UInt8 +`UserAgent` UInt8 +`URL` String +`Referer` String +`URLDomain` String +`RefererDomain` String +`Refresh` UInt8 +`IsRobot` UInt8 +`RefererCategories` Array(UInt16) +`URLCategories` Array(UInt16) +`URLRegions` Array(UInt32) +`RefererRegions` Array(UInt32) +`ResolutionWidth` UInt16 +`ResolutionHeight` UInt16 +`ResolutionDepth` UInt8 +`FlashMajor` UInt8 +`FlashMinor` UInt8 +`FlashMinor2` String +`NetMajor` UInt8 +`NetMinor` UInt8 +`UserAgentMajor` UInt16 +`UserAgentMinor` FixedString(2) +`CookieEnable` UInt8 +`JavascriptEnable` UInt8 +`IsMobile` UInt8 +`MobilePhone` UInt8 +`MobilePhoneModel` String +`Params` String +`IPNetworkID` UInt32 +`TraficSourceID` Int8 +`SearchEngineID` UInt16 +`SearchPhrase` String +`AdvEngineID` UInt8 +`IsArtifical` UInt8 +`WindowClientWidth` UInt16 +`WindowClientHeight` UInt16 +`ClientTimeZone` Int16 +`ClientEventTime` DateTime +`SilverlightVersion1` UInt8 +`SilverlightVersion2` UInt8 +`SilverlightVersion3` UInt32 +`SilverlightVersion4` UInt16 +`PageCharset` String +`CodeVersion` UInt32 +`IsLink` UInt8 +`IsDownload` UInt8 +`IsNotBounce` UInt8 +`FUniqID` UInt64 +`HID` UInt32 +`IsOldCounter` UInt8 +`IsEvent` UInt8 +`IsParameter` UInt8 +`DontCountHits` UInt8 +`WithHash` UInt8 +`HitColor` FixedString(1) +`UTCEventTime` DateTime +`Age` UInt8 +`Sex` UInt8 +`Income` UInt8 +`Interests` UInt16 +`Robotness` UInt8 +`GeneralInterests` Array(UInt16) +`RemoteIP` UInt32 +`RemoteIP6` FixedString(16) +`WindowName` Int32 +`OpenerName` Int32 +`HistoryLength` Int16 +`BrowserLanguage` FixedString(2) +`BrowserCountry` FixedString(2) +`SocialNetwork` String +`SocialAction` String +`HTTPError` UInt16 +`SendTiming` Int32 +`DNSTiming` Int32 +`ConnectTiming` Int32 +`ResponseStartTiming` Int32 +`ResponseEndTiming` Int32 +`FetchTiming` Int32 +`RedirectTiming` Int32 +`DOMInteractiveTiming` Int32 +`DOMContentLoadedTiming` Int32 +`DOMCompleteTiming` Int32 +`LoadEventStartTiming` Int32 +`LoadEventEndTiming` Int32 +`NSToDOMContentLoadedTiming` Int32 +`FirstPaintTiming` Int32 +`RedirectCount` Int8 +`SocialSourceNetworkID` UInt8 +`SocialSourcePage` String +`ParamPrice` Int64 +`ParamOrderID` String +`ParamCurrency` FixedString(3) +`ParamCurrencyID` UInt16 +`GoalsReached` Array(UInt32) +`OpenstatServiceName` String +`OpenstatCampaignID` String +`OpenstatAdID` String +`OpenstatSourceID` String +`UTMSource` String +`UTMMedium` String +`UTMCampaign` String +`UTMContent` String +`UTMTerm` String +`FromTag` String +`HasGCLID` UInt8 +`RefererHash` UInt64 +`URLHash` UInt64 +`CLID` UInt32 +`YCLID` UInt64 +`ShareService` String +`ShareURL` String +`ShareTitle` String +`ParsedParams.Key1` Array(String) +`ParsedParams.Key2` Array(String) +`ParsedParams.Key3` Array(String) +`ParsedParams.Key4` Array(String) +`ParsedParams.Key5` Array(String) +`ParsedParams.ValueDouble` Array(Float64) +`IslandID` FixedString(16) +`RequestNum` UInt32 +`RequestTry` UInt8 diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt new file mode 100644 index 00000000000..9034d8fd0a3 --- /dev/null +++ b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt @@ -0,0 +1 @@ +8873898 \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx new file mode 100644 index 00000000000..0cb239d932a --- /dev/null +++ b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx @@ -0,0 +1 @@ +?? \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/partition.dat b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..9c176950d3f897f5cc9714d7011107a21e47a229 GIT binary patch literal 4 Lcmdlj#LNHy11td1 literal 0 HcmV?d00001 diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/primary.idx b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/primary.idx new file mode 100644 index 0000000000000000000000000000000000000000..9a604fd108fd6ba7f48a9c05368431b9ac5fc126 GIT binary patch literal 15330 zcmX||cU+C%AHW-gRMK5din{yWL|M@jA{xX|T1r%;G*qNQi%LclMN5%UiH1s%G)1Yb zgpgDcz9kKf-{*O*`}_0#`pk2lb;Oq^&B6N71LTr@Km*K93|N|JcPLgIIDY7Cz7mSoC8YfJH5ob+pVQYGl< zWj#)-Tw}++htkoHCJ_-5cGQEsE8UGr5CwBEci#}s;}Ar{G*fPL=w9L?`U^a>l~rRZ zCKuW0%xe)5dF-yuG}WUMJk<6Dmz^$RK6X^+AkA((!ucenwaF&*f8Kbl1l1zXbe z`TUN{Y=WrRvK5o(pXqW5qG6m3bH+Er4Bj2XXKB^^`)`%2*vLs-6t0oXLFFsAzc5jw ztSH3uVAU+7s4~z~HBr#iUNF%pww;Zt7mC6)y6xlk@@y*uy;+89*3rzc0b>R#U4_fe zl7IMQDKk-}9!}LWeYeRMGYFz$*Ya7S$!ED}_8MGvbkSLCnKKjlnv25Sv*en?&=xZe zdbt~qHz(C(^W3aWDSn&$3UxU?7e!?!`}L z%wyG24*DI3XQrLv(UP-?fr{dBFOI(^yXAhdkY5TOVO>%a_sSVI3QQM;t0EOT8W70m zBa;lAAvt&iSBtKLMLarMUyMia zKAKQ`xs8Pw^>}6pmvZ`T2NkV~yab*jMx0~x7{!H6ur96Bu0%|W|3xTeL()rAHxnP^B) z492%I-hR)c95ymB!0Fm_XS*NT9Q4^1x67Z^{AMqci>e&OV3r(BmR#-VCjM4aN??R>o#2#AF3FgxM=uA4!~$QbWyw%L$5iy#U% z-DA0Z)^s*ORP2B9^ermy%y+=pMzi~2`DD3Eqr6a}&t|@ZN%InfE zF7go(huJ-R`{HjqTNZK^69<`A()jG}jXab$8<&|e#*G{!S*V024qoi)^FDvcr=#%& zIFYxm?f9+1KsigqA?w)>0%qRc!bVG0;dUZyCtunH7D4iu>i5FYBVK%@yg?jBG^Xl2 zQf~!MoDoj=HJWylfjklJQ{+Zj(fm?6YOus}Nk_)}+Yj-Ot~)MUZ*y*3n$AGo{y2%| znsgOwGthzr+>7E}O5ORBOr%_i%kWwHx}1(y)r!Mi5c2_BP&3R!9xrj9W}8GG9IfJ^ zn_qFeibuNcJN7V;+z3vbA$5!5cm~o~ISn5}N`rJ%01FxVPXk%pdNC=lBnDd1G7Y@E z_YNO&sN|tdo2NrOm-sr(eMji%bMbW09Mjmf=HpHV%I}#D^O7`>{_;>68_n^Qfcukv zS$zHAN_f9SNkC@ztp|cb;JusmLjvr&Bczx4ZDgXfD>ER%^XtvIap!o*Ky)T}Ik{OS zKMLM%{&p1@9!HLE@L8_g|i?RPqEq5iH(eZ#NP6F)#t533bE_3 z(^@K|N0C~giQV*nn%)nw>ywc}>iQtwXs`NzQtunN!xs<4s}z z=JDtxEJk*pr@Nt$zEvg0w#^XIroS#FvGCp%XyW(M4_84hWGu~ES9+|4Maallgw^0u zmxn@sArcx)(V|VhY*+WKVw=CwplLiDuLH_|9x0>7U^JSRErESWk$IDoXbX zyucSWVavw$Iw4KS)q8&T7gD0ug~VXQB_dFSM#HDrRxv4&v2q(J2JG+DZrfJ zM8k&WX^kRHc-P&)HLqFbs!48O5=6ls{q3L+>%&_SWHce52sCNfmmt5#aSsJDd2ISmUeP-ZO>~q$8#22jYao~DO#eqlN~ag) zrU~gnFJ|iXMM!0A50@R5LaMNs2&pR(Qi!eU<&fDziuVXEinuMLA5q57m#-Gm!IBm` zA^F>O=bF?BNmb!%{cIJX-JRxwE?r{*k+J^7qoQiCM!{!-j5YuCQ;CN4r+|`|o9foM z3uX07EG$`eLVEm4&8PK>kZv8fiJ9dsB>X8Ry%2f{BI{N)6$)kaa{CW{93hcA4~`W# z2s9~}U5>vpJDE*Hr(hZ%Li8IJ3F(b@qeG3JgiLU}|4cyb0`E-0H4XwtBE^K^}e zoP;v@-E-V$ED%z>Txe(QF!G`e6Uv^ZvPa_o$>cHdC%fj+=kSSm z^4Q83tqRX}m>09r^Y{4uLSfmAg#BQmWeQU892vVA$ObX#|0uNN^^WE66&1*;I%Ffw zV7(+D>m_%p11$tZ!rXQB#haH3ypS-zo^?j+<_JkmV`<Dt1GYM!EoZ2(Aef_t{DHuaLr* zUcYG3C!~q`dr!O9a|ulvrV_xKlg{P+BR77gEp}NbvtD@q@Z}pq!o8eV7HYO{j(=UM zC?I*Pt7*I1*l9lDMIQ6D@Y{X7NJyfum>DyZgjA|$*IhHtN3zTDD;8sCZ}0GlP7nz@ z9L7_90&f&tM#3aMwtr1N!z74|$b>06U(`RK%I{HnYQ{owU2l!cnV<8$_G$1LxYUR)GAA_Z%y zFy>|V>#CaQ^;&7rv|ikV>Nm1bZ>BV4J!T?qx!;#gkUSRJpH`!Pg^%uaO2gGWHrsqF ze<24oV=|!G7`k|Q*9aY*J|hDW-e@Xm9%XS+!$TSH`ONq1K37;LGiznQ4u2DkC$iDy zFq#D9r~^SD+rmM;0J{p zcxbhlB4}Qe`=UsfXZ)kJGnXv?IYs|R=PX;$$rvFm^q1PZsai-0tx1Qhl7%!~MKe-Y zQt0JoEG2cyOehlxZOfI0%7P%C;_pRe*XIbS#8a-20_johoB>_2`T9%{#dpT0Jq1E)1P*%O}VAp7^fXJBZShk2`5{s}Ce6<5ggygp4 z>9W{mLfVJY&Ar8hbYVnqU-54tneq_CE!gk+BFx~H&ND8n;r*d(No zqZtWiH9`uIzLh1NCM5hm+jU;(C4P-av&}Xk)i~yMROurh`aW`io`{l zXzfh=-o0Csekbq)1LZ~G&);8Tm#;z7V9%U_%XZ$J@^n2#1c!!;jLOYMtyagg1Z63l!^?dFjHsOe;FR)#mj`Q2K{ z^Om!ai;FTuxOFJ1W-Nw55DnYh)cm|KpNDQgQU<$|6SmL2B-toETm|g>)i2QdbdYDGt6?0Vd2V&JHILb-=`|lR+Y<5NXyQ@$22d7(7n`SQ!TE-C zG;cr)B9uoOWJQNF(SIsSz~}v1&z9P`T=e0+Hsqqwc#F$a<)JeMOTo_EU$&=iA=K+4 zRsii-sc_Q#I34X)TnUuFNJ*!`3`R?k#o^NjrJ))lppA8p3Nb7{+T$>;EmhsdvgRb3Z^L4LeYku8?H&gzLu`o<0-@WN3nNr6)u_(%Jv(7m~j6nAS#xl z@!a6D2ba*KVj&K<&yOw>lDF51Ps^kPM8mc`h}Ag{t8F|24WrcrU%BvJK=PQ3-XaML zSRvsud2BHjZREaE^B?_IxswbZ8ml@< z7iXZamijQ-KQ8?aTWgr;%sZSUu1{V$qr^ggbk=}fi&6N(x_KPr+_DyC@aX6-+L;ap zL1Zj3uhnDkFDCk=Y5~z=CP5jy#ZYJZ?W%USBG$r#lU)q7kD@6UN_XeSG5>I zgo4EEih)5UdbV&Q*uCA7bb4|l8x6K^0+QK!>TwUBjeIT}gXSkIg|`S>AU-d&8u0y^n_!K{YZEuBm!T-6z{SsEUH%$604@K^A z2Tl9@pfeRK;2CT30G}b^Gb9SNIB49`3rJ&Q?o%UWHp<&|5WL{0LZ{xc(3F8U*e$Yk zOX+|$N`{IrM6Wcd?&0(@(6_EbV7F|^UlmdU8@)c~2bv$r4GI!ab;pT<{aKq7^TU;m zZY}T!yYw-}cj+J|`dJYGG!$TC)#%AUep1K4E<5izm8{P|llyTB9cfQImq$m`@gSIa z?#243*G z&4ULlbm$XKYG{0mDpdWfl}k9ujZsp2!l&x?QG(V*M<@k~7 zex{Ji8?|V-3mzLwyYXl|bnZ(uL|7l-y5v0Uh;X7}jtLsik}q+P``j2Ha^s#NZD$T@ zToVhLf4 zG96?U2}zLkvm*{0BUUg7B4MEl_uIb&G6|v(&jWrsfNRQQHqpzHqGdJFt!&)xWq?A$ zE;x6>iHt@6kyc{D+Xg52{;hj%S)~s5H4qi+Ej<{$396Gg(Xgx)wBCiV?paxe%QS5g z-nq%anMV0#u&d`(Z{Wb!LNzZLG;0HnlIFvDsdIA*Xx`LJRMDRWdy>Qyi0&)5x+4!( zGj-LeAUonZrRoH$yxnbSkmr+;8PQv%VDGy<9c0tv-9pl=IVjjS9qblIto82>p%X;H zbi9pAx-cd|6zqM=1_k~on;k204ad2B+OO)b&jBSY2|$g_%5__VH?jkXnK zz|2=m-Wl;e%tVpfGl8h%JPmnxW8y@^uD88fPzU>WmE$PhJNv?JLTDjMVT@ z>+p5(;?;iA^QI#g{ms7tnvSc#-I9fqpSHrAK>OSVt1b<(P^(lnM9BVT7nZ?>)5j&* zkY_sYdT;t}CP9!(jNi)mFfJ;)oDH*NtbR&X3+@S=s07u+NpI4RZ1CBfm7s2r!a{Fj za)8XDE#@V@W+4xkT$mU9?6WGEjUFw%4PLgE++$3@sstwz_IvFgZ3S4n;Y7w9BDc|w zcC!egV4df$s5q7g?5LP&@oI7(R8??I8n#k7WO0}-k097<*BmRlUj?V!mvNtsqw61S z|II+jDYqfdkLGSx-=QYAxHb=>_eaJ$h%baYgmnkTeyO-(Q1U9+?Ys-odwA<-#lbAC zm&*q)2{&aL`%e^8sf4T( z4nHLwy2C^!_bb8Yr4?<<0-+*-6BVn7wmrKz9@f{Hk3o~Cka*^6B@3N3tpZIEIdA_@ zKj{QPe+W=E)Ykm`t010cy|c~UXeKIsSPfoYY|oILZNovoK2(F3OC`Y;GoG>tqG5*T zzSYkv;3A19H89tjbHCd#y16Jzt`73V&rsMSEOg~}JzQytJbOLWJAAb3QUgS&2x_=| z0^Y5M^`Aj57shNq4Xe?S-LGfh#b@TRI&~HgF}Y3P#m9e*6&<=hzLhnBtYjnauN|9* zuAO-fv`6Xb#mx`>u_gL*1rZ#Kc%{vbx?ua^zAi7 zNa-;fJkI7KbJsQ)QKL^-w-cQF{5|v*ymXcfmCS}}jAqPR7~ixJ$uT_+bdmtU8q7Cp zk>?8z%DD3ue10A}G}a7V7*9X8LuUH#80#bo=*WKGdyp+;B~R{$-Srx^50DGKLfQbU zAX{z+Xx{(b)ZRXmK@ijc+cI8Vgsu;7^A3pTP;s@^9M-fsb3OviJH__ea~Sq`2R?$A z!!bwJ_dkHUp!Xwq!S`btin9nA1@qlOj<9+v$p1%i*S43KkMfa~_GgIa()L&1h(kw_>`oY; z-Y|X0H4@Hme{_N@s~|1bPlHYn1#69qc~RUakjZ1EYl5vT*YnZdBVRzXb+scmz!;v} z8C@{fT8$~{y?{tB8(lRMPZVV7ks)eRJQT<@zHoY9_~(*v3- z=C+KcmcZ^>q!;ETcXn*1(g)}u>ch!r>}Lgigo~P6zrlz)MV5_Ene!3pMIYELDVbG3 z+Q~wFv%bTeos|D{#VeFS5ZLWzd6XXKAaHd)@+XCXJ0U!XZRD(@ zYH>g}9gWT!1TXXdE)Mzy`{@j~LAWYb0eKWFR}PAr7z8`KE5ap{jS8v1LDPcyb-f|f z*>Qr?|2N;&9z(xIrpj;7OsJk?FI~e%Z{mML1h0PYG3o<4TJYdE_`J}WH=R3=i_V4( zftM!rLyz-CSg3Q|AJCMXIu?9;E(1xt`~zNUO~+`)Ob$}~FbtYACZrQjRWs3^CnG@3 zmdl!2nNW4890j}866!ld&(H}XW1EL~-J7A-h09>|v?2T6>y_{}9U23gsmBU6hIa^K z`Z##Os{p-+SxAFB4ziszr@;AYJm`#?fIM4Tj3s=bt7UokBwVq=+Hd(Tj(qgYa0+PO z+1ThbrY3qnJ_TOdm{{QO6cZImh>F4)JKABBew%^nZ-|Lfup^;%zYj>j&Zbx#=$Fq> zeh7t)ux4>lD)v)V{ZAp(m(Ijb1NyMx-fA9)i^MCZi^^m7yhT2LQR5>cof#m*&w74u z=c2TM84zKa<(0H@sAQ}=GZV;+@zFYTSU_-A{zQE{S(!t~sF)~oARgN)AR6}Be(x_y zSnu8`nkh=gbWS{T?OF+IjpkXRa#-fw`nn74bb`njb1X2)vLD``AcM-foJ5*4^g-NK zmIO`lpkK#s!kz>tc`VsrPqwEvAN97+hFthxkFE^;1?%1aq`=Pkj^Qkm%^Va^E+b0A zMxO*r9)K=3s^=W=Qup%xGM|sImjR*>=eIKAY^0Dj2QvFo^660OX%;f}kOiN|A8H?5 zxPpn+M$ZMGH|ldz+AX+fX}uh1zFz-VqalEWPK1#mf|Acw>%=@R>iS552p4@NlIGUa z38E19wGOOl#YCwfJK$O{Qge)sC@X0|J@z(le?OulF|IuL>~@!kVCX^JMpgl21FPpo z;Po?=9t9ZfHRkxVM-lLq=~M(6b5n0+4fN@7Ta-YS`BLxk#VI(c->3pvH{@q)ErUw% z{ZtjmdiwNro-U~z^!La-813KTxMvg7Ip}oXe6YK2x994Ua6W32Qw5SroY&Q`h>z?o z)F2m?(vb0IuV9z_YXQif2koc4O6H>+8+C|&|KqRRpOtjvbyOYV*$3*X9lyvx&t7Uk zJV(YF%ct!e)agZsc=f!v3J+-xN_A%dE&Ao0{KyD;HT#+1bJehE@K5L;)KFuC%vw$F zV|*&C{9D)%!GHX!RW%RRgZ3QA<%d@HC(dIyQF+XPxlUDjU$Ny5tZ6ws(4@FpH-3<$ zquFJA$i>0oai%d;2pp)3K+`j9Zs>Ry7rk7h1rh3Ah@LKa3jJ;qTHrI2GIC41n~7v< z7XxL#?x^#-$wGf^w86`b;ZLO*P^I!W(}4(;12roRTId9ku{x!u?QNim%cxkG5puJK z&H$X`v1rA&AEm$WQGOq8cOd+&!e!Vq<3z%^_K7w>;9kNB<|VyxMmb^=1S_kKL38nN zE$7&7roNOV6<*} zJ=5`8-ikV$q*iggr7D>O3A)1`!j33Wd<}Rhs20n5Ig5|Bty%+KW}a7g`V{U7@e6BV zcDtgnh`Ev+bZ?IVzf;NF>oBg%=?ou9-%rSzQ-@e@3 zAhVQ#)V7(x>}qLv(08OzDMP8`jJp= z+P7dEcu9@?NtV}RqC+MKsBQUO{`TceWcn3BF6&>AckVjFM~~KZX$s=Ym6?}X8|9Z()yTh2nK0_{NaeMg^j?mGtZ zklh92>-G2hay}UD7fA;oFPD%-hB|bDNSH19LsAN6{v(oaT}y@w^r?a- z^n;Cx``g1#11Aa=DWbP&Q;5I|6&tMco?8Q7AY4Yn@XEeDsRq z4Dm7(mL~j3p`$+gy)bsXFMTQ4Ex5W5znY!W=P$tPUU~qCffUhTCnpad@enfM}TGz==N-$Doh8@c=~U$t>No&X14Q z*SLe`#o-nAHT>cGSKiA^iOy>AA-yh#LiFJJcp`-#$nLR z(R$Y$v5t%OF7SgfT;T4pN`aHm>mQGR-D9nx=GrCDhimB%@gmC=zHfNXLz^oCK$f$^ zafj?E1C{R%1TSe_PCdby@N9k!gk1Dyj;-q%;iIA%$6*XX{T?x<|G_>$F$id6f9p?4 z2dGP_1%qb!{55sa%9`k@XE69|UsNXRVZ}tQDIq}k4rR$qCYnwUgXs8^lb+2*Q6*sz z4?n|P@`Hnvd6}9xpTpJ--x$(Yloemugph?04 z7oS+V^fGiX5R_*|zo5BLNHS8|!+wqeO)~Z|%g12p4CrnEO)@r=`zxWhS4gxP z^6i}xhmgTZ(^TY>X;6E@?cgas*>G=dGLIm64DYKHhrZwB>$t2@G{|r@^vp8qqaZrt z zY`?~XEbE21mlBzS%KZ~y=KZ*Rh8+m{=Y21LZ0nT=N9&+ZM`H@7_|0xf(Hl6ZwDclq zYRZNd?D@z+o?=Om=lu|R!@je8RCYQEB3N8vlxzrP5+si$+L(0yfqOph`z4U6))j6D z3x^X_AbIRytnS*z*?d&-@iNGEB=0+G^q7G*olXH#7d6g^>4!6-fmDdckz3Of3bmQT z!)cIVH$ zs;VYK*BmZ`9sBcbDjUaG1W~a)oWV_bhq(mNu(XEh6_Hn=%D6TYqIXlvM?Y=npn8w1 zK-#K9f2To({M6_*@PePx?16Ji+k06+?5W0!S=*pKF`5O@t<1N-dXr8^H`}j6W_Zt# z2y{j0th)hOfA;UR{ys|+g{a*GyC=>g*k$OV3Aut3{r3r!cHX7XC8O z$p+F`k$7bPO*WdbD+jz3Y4CJzTJTZF$y|sq9Ef4tp#o|7F&Fa0_fGA{V5NHVHprY! zSC$XM&PH}y9?0T+$J9u_IVk+Y9gq#Z+ntNA-PR@C1=+@nerg@sP!~?f2fACJJ0_;X zMQwWvfG*nA%-jY2y8?>(Yj=KXl0y*eLay!CMbWS;2TdB$@12vwLq{$aK%P|Td7Am~ z*GB*A0vN+^TAW<#H&`dD6~Zh{ZNB3j_KS`}*o9zcwRz%XPc;{v7%KvLQl56FZ3Pd# z2)GA?zXwxod1!`pG30VJT&Jlaii6_klz^9Om_xtkB|1Sg?BhZoXKn-!4cL}Kykya~ zixKCbYiIL)7;RiT=hvwO2J)hmf!(VYuk}piV66`X&wKWx#gAvR2!g*8CYKtmX=4)v zt5~HHBi(ckK{QNyQhiI02#+AB1bfnUwZa#@AhHZH^D;ZGoe~6fKEHDCdBNeklP~PK zR%$+g=zB*eZFAP%MpclRduT~loiP`gJg9=JvfBQhqFx3c9XeeNJ}*g!y_Q@Ot2<5l zV0Q*Y!w%Y}s(jweL-FP{;1geo$7#R`&f;3o?EJJOjv_|JN|~rv_Tzy7l?!{`NPZ={#$oVj6T1Sqc)4ivsOZ!M77B9z0J61Z1D_tO z4Im#yn#p^<7_p-d=R@It{&rgNa{Y=0zjE%iEshAZAz>*rjP_%gtT{_u!7N zFotfq9r-c-OjL5S2jbm}MwX7oJhXGN2V}Df-?=#XFbRSR0mb=9-FG%YG@{ecAD$dp z^ zUmR4vfS@l3H2YHp1gBS^xo#<(&GWy(OgxOKjEX(ZMjo|&5S=d>wBQx=N9I-ZgRE;4@HoD*c1Fi$!r?W4Lk1{^|1kLUzQZ7gDvygt_FNkNg`0I($TO368`42SxzOUp& zJ>jF!=>tFq=5jPwUWTus=>TLs+wSb!U2p;Z_rIj+wc?zvIc=}De7-EuV*9b!$Ht=;x?w+!XFl1D~7Wr3!=kL1I{h(!q>2&0Jn=W zNG=Fp%to;XM`omLoXcXf8cCvYLasvxp>l%fz<8`UwwQ!2~XAG$2Wnsa3 zX+Fx9`3vKlrtY6O*#duX>HY;f{11y;1x&Q6_b-e=Zjnv8E9}m4NB%-)7p+6OKAzy9 zXW`>O>yxsU|IXl`3z_4P3*NcX0Au(_nE)@wy~|p|*Rqi3-3f@WiL_5w@+7>bDU5*8PU4iqHu$jQ5C{Da>=Cba literal 0 HcmV?d00001 diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..a72e2bdb3706bcc08ec2f7d6ff976c10e56f75bd GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cMSjn%S#tikUGF#tP8}kE# z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOBB}>nT0?1>@KmfG%gdU6%j|WMOn+E@EkJxd)

ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcnq1^RmKF(8EcDh KC_ym>w$1=Amu)2g literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt new file mode 100644 index 00000000000..193d3123896 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt @@ -0,0 +1,12 @@ +columns format version: 1 +10 columns: +`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt new file mode 100644 index 00000000000..7813681f5b4 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx new file mode 100644 index 00000000000..4b6c49f8d50 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx @@ -0,0 +1 @@ +…H…H \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac GIT binary patch literal 4 Lcmb0VWo7^X0I>iw literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx new file mode 100644 index 00000000000..99ab3ce7835 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx @@ -0,0 +1 @@ +…H°-œ_…H!.œ_ \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..49d83aebd914bce7c8ebb35018eb910ea5337b95 GIT binary patch literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyUJ56aG_hdC|uFP3IrAaDDV{y1~G} za1Mw++Olyb=a=S{=#^BIFy_n&-TQLZLq37-DI9oG#gHUryHuyxGq-0z*~@5 zRFYW&bhjQ*5z9{=eeu7Cnoj8~>F6<*FiB_NE-1>(O)RPe>E>AzQ2A(+8nc=5L>~VI Iw&B?f0H_RU;s5{u literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt new file mode 100644 index 00000000000..193d3123896 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt @@ -0,0 +1,12 @@ +columns format version: 1 +10 columns: +`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx new file mode 100644 index 00000000000..d3715e50d46 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx @@ -0,0 +1 @@ +¿H¿H \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..7af4421fcb5ebc50aa06a5f4dee6e454020552a6 GIT binary patch literal 4 Lcmb0UWo7^X0J#7) literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx new file mode 100644 index 00000000000..0156d831c6a --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx @@ -0,0 +1 @@ +¿Hˆ©ç_¿Hˆ©ç_ \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..f45270c50bd34e0f4400456a9a0ac4304ebae307 GIT binary patch literal 261 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyXp3#UiEg{lBXdkG?GYv@f%}=_UgM z!+9Y7XvfBxoL`z(qE}K;!W{Rh;=-DOtwxG>m6dsRwjN>NNJ%V7)Jw|DJCdSa{OCi# zm(z|B#fI`%l-4CPUb)Y{Vnx12R!7vt_w^?Z&Do&ziBq59LtbuCw($j~;%~BCO{aYv z_O55y$Dp`Sl0h^#GcPx>B0jY&HLoNdXl|-rW=aK%{)#M*%!oA;KkjQ)eIr+F%D`KY zSX7c(0(7_@P!Y>d9)0n@hni05Ea~VmmM}?Y;4Uc2%uOt+1nK5c-7Q?0BT~9Ltm4Vj Jxg9xO3;;OHZSw#C literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt new file mode 100644 index 00000000000..193d3123896 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt @@ -0,0 +1,12 @@ +columns format version: 1 +10 columns: +`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt new file mode 100644 index 00000000000..a21cae3648e --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt @@ -0,0 +1 @@ +454 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx new file mode 100644 index 00000000000..4fedf1265b2 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx @@ -0,0 +1 @@ +¾H¿H \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..7af4421fcb5ebc50aa06a5f4dee6e454020552a6 GIT binary patch literal 4 Lcmb0UWo7^X0J#7) literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx new file mode 100644 index 00000000000..698b75ec43a --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx @@ -0,0 +1 @@ +¾H˜ç_¿HÆ¥ç_ \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..5d9d47463d2dc3c588c2a68beb24126184e5955d GIT binary patch literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cvWrlG#xSW(BO;NigHQ+OX z+5i9m&;S4c@jnI$V{dhCbS`vwbOb;rV4Zj{6Wu0P+wm(Y%$fiQWMOn+E@EkJsoeoZ z83XkA_OM)y9%i<(>8`Z_;I@6CZZ2eDbOfQp0V$_ea0t>|RE;3Dk?sHsaB^vFVRCscX=Hc|1aaTY#PfTb M%D$$}R*TMQ0I^wb#{d8T literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt new file mode 100644 index 00000000000..193d3123896 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt @@ -0,0 +1,12 @@ +columns format version: 1 +10 columns: +`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt new file mode 100644 index 00000000000..0ac440f96a1 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt @@ -0,0 +1 @@ +1378 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx new file mode 100644 index 00000000000..4c89b29a856 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx @@ -0,0 +1 @@ +ËHÙH \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..a713aecd075daae7071deceaa93147871ef28fbf GIT binary patch literal 4 LcmXRFWo7^X0v!O3 literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx new file mode 100644 index 00000000000..6180f063fac --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx @@ -0,0 +1 @@ +ËH×Õ÷_ÙHjì ` \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..b91b7faa4f06a1664ebb74109586cac5fd23a1bf GIT binary patch literal 261 zcmV+g0s8)9Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bE+#qTQ)Lu{YCP+gXf%IE~ z*#H0l&j0`b@jeC#V{dhCbS`vwbOU(kK>xQg&QB-oQa%ok*L?s8WMOn+E@EkJi8YXH zVhIrlaZ>%T4I&riEH(kO-vz`k)u>G8-l<@lxx5?vo?Zzr0Pt>Ya%(ft0?3Luk)Hm4 z)H$e;_u%sC>HrmOX>M&{cwc39Wo~p|WMOn=E@@2I=Kk=xD!%mjDZJa%pX0a(OOkWOxk4<8|vG32}9| Lb!HVccCGgS;Baoy literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt new file mode 100644 index 00000000000..193d3123896 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt @@ -0,0 +1,12 @@ +columns format version: 1 +10 columns: +`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt new file mode 100644 index 00000000000..0e92c3c0fc9 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt @@ -0,0 +1 @@ +306 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx new file mode 100644 index 00000000000..0e3b7fb75b8 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx @@ -0,0 +1 @@ +êHëH \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx new file mode 100644 index 00000000000..b8d049dbb19 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx @@ -0,0 +1 @@ +êH«C!`ëH©¢"` \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..f448a75b6063fba410c431980059309d4b9a6aa1 GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$a$Hb&?2dEW#ESpcY+$ktzi z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOCIfRlVt&lYFUDR9N&B1^)mDWMOn+E@EkJ_5<}3 z3}5ITzipTtZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnsN(E!5h!GNyDe K$;I8+;Fkb~3T$=& literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt new file mode 100644 index 00000000000..193d3123896 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt @@ -0,0 +1,12 @@ +columns format version: 1 +10 columns: +`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx new file mode 100644 index 00000000000..9f363c02b40 --- /dev/null +++ b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx @@ -0,0 +1 @@ +ëHG£"`ëHG£"` \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..d331cb1fcc1616a320c584208bbbe060c20c1947 GIT binary patch literal 261 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyV5)|94mWJa#3U_`UDuPj{Tybd!OB z;XDw3v}5B;&M(a?(JQGaVLEni-X{0LThekT`8{LRP8u?Bq$HLk>Lq37bqTze&kSbS z^7D+CCQnYxcHy^-OLj7TynlAv5BGeAS4nH{ichgE;?!sOke6GOZG3^LKys6eD>t{J z*eAbZOH(qk8ANk4^Kug_;#12~^Gf1@=BDaprc|&b9%Ngvb93e*TLDw!M-#K08F&j4 zi%K#}fDYFKDq?YzOsi}P`os3}X~c_+{jJj(xC@Fha}$dyLArVRV>iAT$d33qg;0jf$7<_1PzbQF>&8 z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOPvZe4NH(<2?+h%j6?;@(ut9WMOn+E@EkJgdK+V z5ZhW2+=>to#{T20Lq-9{RA`+*_6V{cl3!Qme$Qq=nF%fc@NR8#Ycuc(P`_DAmgVQ_ z!Oxz{GfIzl02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcns^T1OQK`i$9(N KipbM-MdtvM<#0p* literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt new file mode 100644 index 00000000000..532baa6cba6 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt @@ -0,0 +1,41 @@ +columns format version: 1 +39 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`memory_usage` Int64 +`peak_memory_usage` Int64 +`thread_name` String +`thread_id` UInt64 +`master_thread_id` UInt64 +`query` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt new file mode 100644 index 00000000000..3cacc0b93c9 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt @@ -0,0 +1 @@ +12 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx new file mode 100644 index 00000000000..4b6c49f8d50 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx @@ -0,0 +1 @@ +…H…H \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac GIT binary patch literal 4 Lcmb0VWo7^X0I>iw literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx new file mode 100644 index 00000000000..d599340cc82 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx @@ -0,0 +1 @@ +…H!.œ_…Hm.œ_ \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..4398dc6a7abbe10202dcc9ff464b250d9453cfcf GIT binary patch literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$aI-o22|e$OHz$TackS;rWH z+5i9m&;S4c@jnI$V{dhCbS`vwbOTsTaBK6JI5}pr1~GpWy)OU=WMOn+E@EkJsmK8g z4bfp!g8FEMWeVZj)V}Wlv(^fr`M!nXj-ryTzn4HBVbGchFaYpwZE|Zf@CkwPt9t01 zsfP|9Ri`pFb>9FLZE0?8VR&C@6CZZ2eDbOiYfFBkp7g32tUh>SBBG-m({aB^vFVRCscX=Hc|vV`+0{_>ZK M%eyNa13TFQ07qJGTL1t6 literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt new file mode 100644 index 00000000000..532baa6cba6 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt @@ -0,0 +1,41 @@ +columns format version: 1 +39 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`memory_usage` Int64 +`peak_memory_usage` Int64 +`thread_name` String +`thread_id` UInt64 +`master_thread_id` UInt64 +`query` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt new file mode 100644 index 00000000000..cbd6012bc6f --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt @@ -0,0 +1 @@ +204 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx new file mode 100644 index 00000000000..4fedf1265b2 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx @@ -0,0 +1 @@ +¾H¿H \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..7af4421fcb5ebc50aa06a5f4dee6e454020552a6 GIT binary patch literal 4 Lcmb0UWo7^X0J#7) literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx new file mode 100644 index 00000000000..7f33001d913 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx @@ -0,0 +1 @@ +¾Hb˜ç_¿H»©ç_ \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..cc8c216e1b7494e413c0da9a5b31486c5d90c403 GIT binary patch literal 262 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyWf3AW-@%fo-?_t?&Cpwzzz3y2Zf2 zZ~=%v+Ou&c=a=S{=#^BIFgFFyXHxB3fAoMC=byk-#h(luDTyVCdP$ji9rGB|rzA|U zJX$kL_GQ{`uZTS%j8ms`sB}#X6EH4R>Y^4rRJ5y0}W2q%S@?YIU{wz_k+a5P6o-x>)Eq|k1_BT zBo>uqmH=I@2UNte@Gzt1@~{GqOEDq63R@?>W8f|*%FInHss!ogDczYWTxYG5;(xo& KrZ}f&2Lk}qd}`$Y literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt new file mode 100644 index 00000000000..532baa6cba6 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt @@ -0,0 +1,41 @@ +columns format version: 1 +39 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`memory_usage` Int64 +`peak_memory_usage` Int64 +`thread_name` String +`thread_id` UInt64 +`master_thread_id` UInt64 +`query` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt new file mode 100644 index 00000000000..aa59885c836 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt @@ -0,0 +1 @@ +143 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx new file mode 100644 index 00000000000..4c89b29a856 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx @@ -0,0 +1 @@ +ËHÙH \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..a713aecd075daae7071deceaa93147871ef28fbf GIT binary patch literal 4 LcmXRFWo7^X0v!O3 literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx new file mode 100644 index 00000000000..f166214418a --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx @@ -0,0 +1 @@ +ËHéÕ÷_ÙHåë ` \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..0921ff99748c1a04a7fd95cb8943ecfec846c1a0 GIT binary patch literal 261 zcmV+g0s8)9Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$aAz*8X&k@J-6kO<{X=LkxI z*#H0l&j0`b@jeC#V{dhCbS`vwbOJ{BwXp-jZa9E;ck*GPUV8utWMOn+E@EkJ+%Aqj zHZie?gISC4ZtT~N;D7=A*a4?auCebz5f>&NuopMmmh}lQ0Pt>Ya%(g23Fi<6sEDe~ zu$pGM;@RW~KL8bNX>M&{cwc39Wo~p|WMOn=E@@2I=Kk=xD!%mjDZJa%pX0a(OOkWOxh%C+Pf(wa}}N LsJMgcdn4umo7ixJ literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt new file mode 100644 index 00000000000..532baa6cba6 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt @@ -0,0 +1,41 @@ +columns format version: 1 +39 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`memory_usage` Int64 +`peak_memory_usage` Int64 +`thread_name` String +`thread_id` UInt64 +`master_thread_id` UInt64 +`query` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt new file mode 100644 index 00000000000..978b4e8e518 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt @@ -0,0 +1 @@ +26 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx new file mode 100644 index 00000000000..0e3b7fb75b8 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx @@ -0,0 +1 @@ +êHëH \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx new file mode 100644 index 00000000000..67f45d29597 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx @@ -0,0 +1 @@ +êHÍC!`ëHG£"` \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..2c30b88770877a5ddaa75c2f3daf2dbc321be19f GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cbNRM_*!6JF+5+jxa^3W%O z*Z=?k&Hw-a@jV6!V{dhCbS`vwbON#VDF3f}{FN>b+}S`mO_BfzWMOn+E@EkJz#hw3 zOT%v^;N`ZYj_KCHG`s=1SwsOGv?YnRsHJA*@;B-H;|VSR@NR8#Ycuc(Kw~dMnFgB~ zmXR7wn|S>302OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnl8Xz`TJd7NP!7 K53t!rdd~o;%5H-I literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt new file mode 100644 index 00000000000..532baa6cba6 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt @@ -0,0 +1,41 @@ +columns format version: 1 +39 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`query_start_time` DateTime +`query_start_time_microseconds` DateTime64(6) +`query_duration_ms` UInt64 +`read_rows` UInt64 +`read_bytes` UInt64 +`written_rows` UInt64 +`written_bytes` UInt64 +`memory_usage` Int64 +`peak_memory_usage` Int64 +`thread_name` String +`thread_id` UInt64 +`master_thread_id` UInt64 +`query` String +`is_initial_query` UInt8 +`user` String +`query_id` String +`address` IPv6 +`port` UInt16 +`initial_user` String +`initial_query_id` String +`initial_address` IPv6 +`initial_port` UInt16 +`interface` UInt8 +`os_user` String +`client_hostname` String +`client_name` String +`client_revision` UInt32 +`client_version_major` UInt32 +`client_version_minor` UInt32 +`client_version_patch` UInt32 +`http_method` UInt8 +`http_user_agent` String +`quota_key` String +`revision` UInt32 +`ProfileEvents.Names` Array(String) +`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt new file mode 100644 index 00000000000..9a037142aa3 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx new file mode 100644 index 00000000000..8f3c7c8809e --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx @@ -0,0 +1 @@ +ëH®©"`ëH¯©"` \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..e9f3f5dbfb1363c5743ef536fa11535cbb312446 GIT binary patch literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$c>P9c|kkD-grFaYpwZE|Zfumc7>wrI~> z!G@D=hNwi5yodl5ZE0?8VR&C_3*EI_1(J_Im&g zaA9(EX>@6CZZ2eDbOiRJGt`jAL&ph4JQlLni7WsMaB^vFVRCscX=Hc|qQj~+&$gS+ M9BY!1)str&0ESs^mH+?% literal 0 HcmV?d00001 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt new file mode 100644 index 00000000000..37cb59062b4 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt @@ -0,0 +1,15 @@ +columns format version: 1 +13 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`microseconds` UInt32 +`thread_name` LowCardinality(String) +`thread_id` UInt64 +`level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8) +`query_id` String +`logger_name` LowCardinality(String) +`message` String +`revision` UInt32 +`source_file` LowCardinality(String) +`source_line` UInt64 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt new file mode 100644 index 00000000000..19afd4ebd92 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt @@ -0,0 +1 @@ +527 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx new file mode 100644 index 00000000000..4b6c49f8d50 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx @@ -0,0 +1 @@ +…H…H \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/partition.dat b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac GIT binary patch literal 4 Lcmb0VWo7^X0I>iw literal 0 HcmV?d00001 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx new file mode 100644 index 00000000000..714ba773856 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx @@ -0,0 +1 @@ +…H®-œ_…H€.œ_ \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..e0dd83d152b7ad470cb7e6dc01063da79250d71e GIT binary patch literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyYGj&^g1ob@95*pY+^0H=Su~y1~G} za1Mw++Olyb=a=S{=#^BIFijMHvvzKnnEIOFX9gz}^Zqk%q$HLk>Lq37{ZYO&OJI4M zjCAP3lOg(izXaSEH$*n5CK&i9%nMq!Y`X9HpGOaI>M?xC%Pq<_-oU(R)d$Ag!s`2H zEpFbZu;!i*0UP(OA*i^mDlnNGyw=9*XUwBkGJZ(#GEfAi{z*~@5 zRFYW&bhjQ*5z9{=eeu7Cnoj8~>F6<*FiB_NE-1>(O)RPe>E?-ZOk)-2I#_?b=Uju7Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$b_qY0(p<%vUw$Fz9=x*t@6 z+W-In(EtDd@jwO%V{dhCbS`vwbOp+-iJ0^6C!0o57CJpSfd~KyWMOn+E@EkJ)Phz1 z%k1aRq;w4zWD2{8chZf$aFGnfz4lqGhN z^F@@yAM#diPMbsk6>Vv5ZDDv{Wp-t5bYEm)bY(7SWOxKobNpT`{e-ssQD+6%72KTw z4RB#{bZK;HZ*DGRVRQuf3@;b`!h*^yq=<|&7&K=93vhC2ZDDeGE@@u7Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$aUySV8>?*YX%1ymm|vajoc z+W-In(EtDd@jwO%V{dhCbS`vwbOk!)l;t<$46HHJYX_ISYDxeIWMOn+E@EkJvza1} zQyBilEl5*9x)JNQ3ivDm(!!zvxK+z2v32q!r+DVSDoo(32{8chZf$aFGvEe@okB*~ zbTR%3Y-@*T4j5|y6>Vv5ZDDv{Wp-t5bYEm)bY(7SWOxM38o*8P7?Ftp8RM@9Yg5Jm z4RB#{bZK;HZ*DGRVRQtc!vQI$R&WT?TvUx9wUO=s3vhC2ZDDeGE@@ literal 0 HcmV?d00001 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..7c01d242d7ed0b91099869d3ec8d9393edff1e66 GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$b;Du41;5Zw_V+h#FX$e?_J z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOOj1`)FV*dCjdq%k=Y^lq&!TWMOn+E@EkJ%pskK z4+T*^fU(%s4!?yZDEI--N?tRe=#`t*XPo$DrKeG@MhPwe@NR8#YcsF|50od+F4&OI z%2q0?M&%lG02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=%oGj0*JiNNvOS&DmLLRW01a?q za&&2QX>V>WWMOm!p~C?wr&e$X(p*%HAhnV101I$(X>DP0c`j*WcntT(8?D{|1xXDg K$kqa`ii-e@x@&s? literal 0 HcmV?d00001 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt new file mode 100644 index 00000000000..37cb59062b4 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt @@ -0,0 +1,15 @@ +columns format version: 1 +13 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`microseconds` UInt32 +`thread_name` LowCardinality(String) +`thread_id` UInt64 +`level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8) +`query_id` String +`logger_name` LowCardinality(String) +`message` String +`revision` UInt32 +`source_file` LowCardinality(String) +`source_line` UInt64 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt new file mode 100644 index 00000000000..b2412e34dff --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt @@ -0,0 +1 @@ +62 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx new file mode 100644 index 00000000000..76b5229cbee --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx @@ -0,0 +1 @@ +ÙHÙH \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/partition.dat b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..a713aecd075daae7071deceaa93147871ef28fbf GIT binary patch literal 4 LcmXRFWo7^X0v!O3 literal 0 HcmV?d00001 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx new file mode 100644 index 00000000000..e19de0711bb --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx @@ -0,0 +1 @@ +ÙHÁì `ÙHÑì ` \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..6ff207deeb3ac1e0093ae21d37c1b81c3a9d2d5b GIT binary patch literal 263 zcmV+i0r>u7Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cnp1>605;0weWScx&VIRYS z+W-In(EtDd@jwO%V{dhCbS`vwbOlE0NoHh|IjQ9=nBu&yL8kx+WMOn+E@EkJ-sK!V z{>Uu-LH|6^^@SRDj@mx~`{{rIlayO075KqB0u)m?sefCr2{8chZf$aFGvEeBYTdjB z7Ei**<+~!{Vky7?6>Vv5ZDDv{Wp-t5bYEm)bY(7SWOxJvIhh$>@v-l*wq3;?bvV`l z4RB#{bZK;HZ*DGRVRQsR8D@EcQv3$#iBAM}yn N|2B1HzutRv6aZJdZV>iAT$b7Fo=lt*g5R6KtA;xU-PVj z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOH&t)oZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnlzQ8dORAdLOmq K{L?d@g`fZn8gaG& literal 0 HcmV?d00001 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt new file mode 100644 index 00000000000..37cb59062b4 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt @@ -0,0 +1,15 @@ +columns format version: 1 +13 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`microseconds` UInt32 +`thread_name` LowCardinality(String) +`thread_id` UInt64 +`level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8) +`query_id` String +`logger_name` LowCardinality(String) +`message` String +`revision` UInt32 +`source_file` LowCardinality(String) +`source_line` UInt64 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt new file mode 100644 index 00000000000..dce6588ca14 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/partition.dat b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx new file mode 100644 index 00000000000..3b0e472e2cc --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx @@ -0,0 +1 @@ +ëH²©"`ëH¹©"` \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql new file mode 100644 index 00000000000..43ff67fb63d --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql @@ -0,0 +1,12 @@ +ATTACH TABLE _ UUID 'e6e8fcae-2f46-4616-a056-74fe935c46b1' +( + `event_date` Date, + `event_time` DateTime, + `event_time_microseconds` DateTime64(6), + `name` String, + `value` Float64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql new file mode 100644 index 00000000000..6cb4dcdc376 --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql @@ -0,0 +1,17 @@ +ATTACH TABLE _ UUID '5ec3f88a-cd1b-467a-9769-7ff65a09c665' +( + `event_date` Date, + `event_time` DateTime, + `timestamp_ns` UInt64, + `signal` Int32, + `thread_id` UInt64, + `query_id` String, + `trace` Array(UInt64), + `trace_full` Array(String), + `version` String, + `revision` UInt32, + `build_id` String +) +ENGINE = MergeTree +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql new file mode 100644 index 00000000000..fbdbe3c0592 --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql @@ -0,0 +1,264 @@ +ATTACH TABLE _ UUID '057dfcea-531d-4525-a19a-6720aa9ea4b0' +( + `event_date` Date, + `event_time` DateTime, + `event_time_microseconds` DateTime64(6), + `milliseconds` UInt64, + `ProfileEvent_Query` UInt64, + `ProfileEvent_SelectQuery` UInt64, + `ProfileEvent_InsertQuery` UInt64, + `ProfileEvent_FailedQuery` UInt64, + `ProfileEvent_FailedSelectQuery` UInt64, + `ProfileEvent_FailedInsertQuery` UInt64, + `ProfileEvent_QueryTimeMicroseconds` UInt64, + `ProfileEvent_SelectQueryTimeMicroseconds` UInt64, + `ProfileEvent_InsertQueryTimeMicroseconds` UInt64, + `ProfileEvent_FileOpen` UInt64, + `ProfileEvent_Seek` UInt64, + `ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64, + `ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64, + `ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64, + `ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64, + `ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64, + `ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64, + `ProfileEvent_ReadBufferAIORead` UInt64, + `ProfileEvent_ReadBufferAIOReadBytes` UInt64, + `ProfileEvent_WriteBufferAIOWrite` UInt64, + `ProfileEvent_WriteBufferAIOWriteBytes` UInt64, + `ProfileEvent_ReadCompressedBytes` UInt64, + `ProfileEvent_CompressedReadBufferBlocks` UInt64, + `ProfileEvent_CompressedReadBufferBytes` UInt64, + `ProfileEvent_UncompressedCacheHits` UInt64, + `ProfileEvent_UncompressedCacheMisses` UInt64, + `ProfileEvent_UncompressedCacheWeightLost` UInt64, + `ProfileEvent_IOBufferAllocs` UInt64, + `ProfileEvent_IOBufferAllocBytes` UInt64, + `ProfileEvent_ArenaAllocChunks` UInt64, + `ProfileEvent_ArenaAllocBytes` UInt64, + `ProfileEvent_FunctionExecute` UInt64, + `ProfileEvent_TableFunctionExecute` UInt64, + `ProfileEvent_MarkCacheHits` UInt64, + `ProfileEvent_MarkCacheMisses` UInt64, + `ProfileEvent_CreatedReadBufferOrdinary` UInt64, + `ProfileEvent_CreatedReadBufferAIO` UInt64, + `ProfileEvent_CreatedReadBufferAIOFailed` UInt64, + `ProfileEvent_CreatedReadBufferMMap` UInt64, + `ProfileEvent_CreatedReadBufferMMapFailed` UInt64, + `ProfileEvent_CreatedWriteBufferOrdinary` UInt64, + `ProfileEvent_CreatedWriteBufferAIO` UInt64, + `ProfileEvent_CreatedWriteBufferAIOFailed` UInt64, + `ProfileEvent_DiskReadElapsedMicroseconds` UInt64, + `ProfileEvent_DiskWriteElapsedMicroseconds` UInt64, + `ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64, + `ProfileEvent_NetworkSendElapsedMicroseconds` UInt64, + `ProfileEvent_ThrottlerSleepMicroseconds` UInt64, + `ProfileEvent_QueryMaskingRulesMatch` UInt64, + `ProfileEvent_ReplicatedPartFetches` UInt64, + `ProfileEvent_ReplicatedPartFailedFetches` UInt64, + `ProfileEvent_ObsoleteReplicatedParts` UInt64, + `ProfileEvent_ReplicatedPartMerges` UInt64, + `ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64, + `ProfileEvent_ReplicatedPartMutations` UInt64, + `ProfileEvent_ReplicatedPartChecks` UInt64, + `ProfileEvent_ReplicatedPartChecksFailed` UInt64, + `ProfileEvent_ReplicatedDataLoss` UInt64, + `ProfileEvent_InsertedRows` UInt64, + `ProfileEvent_InsertedBytes` UInt64, + `ProfileEvent_DelayedInserts` UInt64, + `ProfileEvent_RejectedInserts` UInt64, + `ProfileEvent_DelayedInsertsMilliseconds` UInt64, + `ProfileEvent_DuplicatedInsertedBlocks` UInt64, + `ProfileEvent_ZooKeeperInit` UInt64, + `ProfileEvent_ZooKeeperTransactions` UInt64, + `ProfileEvent_ZooKeeperList` UInt64, + `ProfileEvent_ZooKeeperCreate` UInt64, + `ProfileEvent_ZooKeeperRemove` UInt64, + `ProfileEvent_ZooKeeperExists` UInt64, + `ProfileEvent_ZooKeeperGet` UInt64, + `ProfileEvent_ZooKeeperSet` UInt64, + `ProfileEvent_ZooKeeperMulti` UInt64, + `ProfileEvent_ZooKeeperCheck` UInt64, + `ProfileEvent_ZooKeeperClose` UInt64, + `ProfileEvent_ZooKeeperWatchResponse` UInt64, + `ProfileEvent_ZooKeeperUserExceptions` UInt64, + `ProfileEvent_ZooKeeperHardwareExceptions` UInt64, + `ProfileEvent_ZooKeeperOtherExceptions` UInt64, + `ProfileEvent_ZooKeeperWaitMicroseconds` UInt64, + `ProfileEvent_ZooKeeperBytesSent` UInt64, + `ProfileEvent_ZooKeeperBytesReceived` UInt64, + `ProfileEvent_DistributedConnectionFailTry` UInt64, + `ProfileEvent_DistributedConnectionMissingTable` UInt64, + `ProfileEvent_DistributedConnectionStaleReplica` UInt64, + `ProfileEvent_DistributedConnectionFailAtAll` UInt64, + `ProfileEvent_CompileAttempt` UInt64, + `ProfileEvent_CompileSuccess` UInt64, + `ProfileEvent_CompileFunction` UInt64, + `ProfileEvent_CompiledFunctionExecute` UInt64, + `ProfileEvent_CompileExpressionsMicroseconds` UInt64, + `ProfileEvent_CompileExpressionsBytes` UInt64, + `ProfileEvent_ExternalSortWritePart` UInt64, + `ProfileEvent_ExternalSortMerge` UInt64, + `ProfileEvent_ExternalAggregationWritePart` UInt64, + `ProfileEvent_ExternalAggregationMerge` UInt64, + `ProfileEvent_ExternalAggregationCompressedBytes` UInt64, + `ProfileEvent_ExternalAggregationUncompressedBytes` UInt64, + `ProfileEvent_SlowRead` UInt64, + `ProfileEvent_ReadBackoff` UInt64, + `ProfileEvent_ReplicaPartialShutdown` UInt64, + `ProfileEvent_SelectedParts` UInt64, + `ProfileEvent_SelectedRanges` UInt64, + `ProfileEvent_SelectedMarks` UInt64, + `ProfileEvent_SelectedRows` UInt64, + `ProfileEvent_SelectedBytes` UInt64, + `ProfileEvent_Merge` UInt64, + `ProfileEvent_MergedRows` UInt64, + `ProfileEvent_MergedUncompressedBytes` UInt64, + `ProfileEvent_MergesTimeMilliseconds` UInt64, + `ProfileEvent_MergeTreeDataWriterRows` UInt64, + `ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64, + `ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64, + `ProfileEvent_MergeTreeDataWriterBlocks` UInt64, + `ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64, + `ProfileEvent_CannotRemoveEphemeralNode` UInt64, + `ProfileEvent_RegexpCreated` UInt64, + `ProfileEvent_ContextLock` UInt64, + `ProfileEvent_StorageBufferFlush` UInt64, + `ProfileEvent_StorageBufferErrorOnFlush` UInt64, + `ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64, + `ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64, + `ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64, + `ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64, + `ProfileEvent_DictCacheKeysRequested` UInt64, + `ProfileEvent_DictCacheKeysRequestedMiss` UInt64, + `ProfileEvent_DictCacheKeysRequestedFound` UInt64, + `ProfileEvent_DictCacheKeysExpired` UInt64, + `ProfileEvent_DictCacheKeysNotFound` UInt64, + `ProfileEvent_DictCacheKeysHit` UInt64, + `ProfileEvent_DictCacheRequestTimeNs` UInt64, + `ProfileEvent_DictCacheRequests` UInt64, + `ProfileEvent_DictCacheLockWriteNs` UInt64, + `ProfileEvent_DictCacheLockReadNs` UInt64, + `ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64, + `ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64, + `ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64, + `ProfileEvent_PolygonsAddedToPool` UInt64, + `ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64, + `ProfileEvent_RWLockAcquiredReadLocks` UInt64, + `ProfileEvent_RWLockAcquiredWriteLocks` UInt64, + `ProfileEvent_RWLockReadersWaitMilliseconds` UInt64, + `ProfileEvent_RWLockWritersWaitMilliseconds` UInt64, + `ProfileEvent_DNSError` UInt64, + `ProfileEvent_RealTimeMicroseconds` UInt64, + `ProfileEvent_UserTimeMicroseconds` UInt64, + `ProfileEvent_SystemTimeMicroseconds` UInt64, + `ProfileEvent_SoftPageFaults` UInt64, + `ProfileEvent_HardPageFaults` UInt64, + `ProfileEvent_VoluntaryContextSwitches` UInt64, + `ProfileEvent_InvoluntaryContextSwitches` UInt64, + `ProfileEvent_OSIOWaitMicroseconds` UInt64, + `ProfileEvent_OSCPUWaitMicroseconds` UInt64, + `ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64, + `ProfileEvent_OSReadBytes` UInt64, + `ProfileEvent_OSWriteBytes` UInt64, + `ProfileEvent_OSReadChars` UInt64, + `ProfileEvent_OSWriteChars` UInt64, + `ProfileEvent_PerfCpuCycles` UInt64, + `ProfileEvent_PerfInstructions` UInt64, + `ProfileEvent_PerfCacheReferences` UInt64, + `ProfileEvent_PerfCacheMisses` UInt64, + `ProfileEvent_PerfBranchInstructions` UInt64, + `ProfileEvent_PerfBranchMisses` UInt64, + `ProfileEvent_PerfBusCycles` UInt64, + `ProfileEvent_PerfStalledCyclesFrontend` UInt64, + `ProfileEvent_PerfStalledCyclesBackend` UInt64, + `ProfileEvent_PerfRefCpuCycles` UInt64, + `ProfileEvent_PerfCpuClock` UInt64, + `ProfileEvent_PerfTaskClock` UInt64, + `ProfileEvent_PerfContextSwitches` UInt64, + `ProfileEvent_PerfCpuMigrations` UInt64, + `ProfileEvent_PerfAlignmentFaults` UInt64, + `ProfileEvent_PerfEmulationFaults` UInt64, + `ProfileEvent_PerfMinEnabledTime` UInt64, + `ProfileEvent_PerfMinEnabledRunningTime` UInt64, + `ProfileEvent_PerfDataTLBReferences` UInt64, + `ProfileEvent_PerfDataTLBMisses` UInt64, + `ProfileEvent_PerfInstructionTLBReferences` UInt64, + `ProfileEvent_PerfInstructionTLBMisses` UInt64, + `ProfileEvent_PerfLocalMemoryReferences` UInt64, + `ProfileEvent_PerfLocalMemoryMisses` UInt64, + `ProfileEvent_CreatedHTTPConnections` UInt64, + `ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64, + `ProfileEvent_QueryProfilerSignalOverruns` UInt64, + `ProfileEvent_CreatedLogEntryForMerge` UInt64, + `ProfileEvent_NotCreatedLogEntryForMerge` UInt64, + `ProfileEvent_CreatedLogEntryForMutation` UInt64, + `ProfileEvent_NotCreatedLogEntryForMutation` UInt64, + `ProfileEvent_S3ReadMicroseconds` UInt64, + `ProfileEvent_S3ReadBytes` UInt64, + `ProfileEvent_S3ReadRequestsCount` UInt64, + `ProfileEvent_S3ReadRequestsErrors` UInt64, + `ProfileEvent_S3ReadRequestsThrottling` UInt64, + `ProfileEvent_S3ReadRequestsRedirects` UInt64, + `ProfileEvent_S3WriteMicroseconds` UInt64, + `ProfileEvent_S3WriteBytes` UInt64, + `ProfileEvent_S3WriteRequestsCount` UInt64, + `ProfileEvent_S3WriteRequestsErrors` UInt64, + `ProfileEvent_S3WriteRequestsThrottling` UInt64, + `ProfileEvent_S3WriteRequestsRedirects` UInt64, + `ProfileEvent_QueryMemoryLimitExceeded` UInt64, + `CurrentMetric_Query` Int64, + `CurrentMetric_Merge` Int64, + `CurrentMetric_PartMutation` Int64, + `CurrentMetric_ReplicatedFetch` Int64, + `CurrentMetric_ReplicatedSend` Int64, + `CurrentMetric_ReplicatedChecks` Int64, + `CurrentMetric_BackgroundPoolTask` Int64, + `CurrentMetric_BackgroundMovePoolTask` Int64, + `CurrentMetric_BackgroundSchedulePoolTask` Int64, + `CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64, + `CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64, + `CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64, + `CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64, + `CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64, + `CurrentMetric_DiskSpaceReservedForMerge` Int64, + `CurrentMetric_DistributedSend` Int64, + `CurrentMetric_QueryPreempted` Int64, + `CurrentMetric_TCPConnection` Int64, + `CurrentMetric_MySQLConnection` Int64, + `CurrentMetric_HTTPConnection` Int64, + `CurrentMetric_InterserverConnection` Int64, + `CurrentMetric_PostgreSQLConnection` Int64, + `CurrentMetric_OpenFileForRead` Int64, + `CurrentMetric_OpenFileForWrite` Int64, + `CurrentMetric_Read` Int64, + `CurrentMetric_Write` Int64, + `CurrentMetric_SendScalars` Int64, + `CurrentMetric_SendExternalTables` Int64, + `CurrentMetric_QueryThread` Int64, + `CurrentMetric_ReadonlyReplica` Int64, + `CurrentMetric_MemoryTracking` Int64, + `CurrentMetric_EphemeralNode` Int64, + `CurrentMetric_ZooKeeperSession` Int64, + `CurrentMetric_ZooKeeperWatch` Int64, + `CurrentMetric_ZooKeeperRequest` Int64, + `CurrentMetric_DelayedInserts` Int64, + `CurrentMetric_ContextLockWait` Int64, + `CurrentMetric_StorageBufferRows` Int64, + `CurrentMetric_StorageBufferBytes` Int64, + `CurrentMetric_DictCacheRequests` Int64, + `CurrentMetric_Revision` Int64, + `CurrentMetric_VersionInteger` Int64, + `CurrentMetric_RWLockWaitingReaders` Int64, + `CurrentMetric_RWLockWaitingWriters` Int64, + `CurrentMetric_RWLockActiveReaders` Int64, + `CurrentMetric_RWLockActiveWriters` Int64, + `CurrentMetric_GlobalThread` Int64, + `CurrentMetric_GlobalThreadActive` Int64, + `CurrentMetric_LocalThread` Int64, + `CurrentMetric_LocalThreadActive` Int64, + `CurrentMetric_DistributedFilesToInsert` Int64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql new file mode 100644 index 00000000000..ea3aabd51e4 --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql @@ -0,0 +1,25 @@ +ATTACH TABLE _ UUID '721d6adf-4b1c-49bc-8fbe-1599ac4e048d' +( + `event_type` Enum8('NewPart' = 1, 'MergeParts' = 2, 'DownloadPart' = 3, 'RemovePart' = 4, 'MutatePart' = 5, 'MovePart' = 6), + `event_date` Date, + `event_time` DateTime, + `duration_ms` UInt64, + `database` String, + `table` String, + `part_name` String, + `partition_id` String, + `path_on_disk` String, + `rows` UInt64, + `size_in_bytes` UInt64, + `merged_from` Array(String), + `bytes_uncompressed` UInt64, + `read_rows` UInt64, + `read_bytes` UInt64, + `peak_memory_usage` UInt64, + `error` UInt16, + `exception` String +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql new file mode 100644 index 00000000000..bcafc4b0530 --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql @@ -0,0 +1,52 @@ +ATTACH TABLE _ UUID '3f558312-ca2a-49dc-a6bf-a5daea1869df' +( + `type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4), + `event_date` Date, + `event_time` DateTime, + `event_time_microseconds` DateTime64(6), + `query_start_time` DateTime, + `query_start_time_microseconds` DateTime64(6), + `query_duration_ms` UInt64, + `read_rows` UInt64, + `read_bytes` UInt64, + `written_rows` UInt64, + `written_bytes` UInt64, + `result_rows` UInt64, + `result_bytes` UInt64, + `memory_usage` UInt64, + `current_database` String, + `query` String, + `exception_code` Int32, + `exception` String, + `stack_trace` String, + `is_initial_query` UInt8, + `user` String, + `query_id` String, + `address` IPv6, + `port` UInt16, + `initial_user` String, + `initial_query_id` String, + `initial_address` IPv6, + `initial_port` UInt16, + `interface` UInt8, + `os_user` String, + `client_hostname` String, + `client_name` String, + `client_revision` UInt32, + `client_version_major` UInt32, + `client_version_minor` UInt32, + `client_version_patch` UInt32, + `http_method` UInt8, + `http_user_agent` String, + `quota_key` String, + `revision` UInt32, + `thread_ids` Array(UInt64), + `ProfileEvents.Names` Array(String), + `ProfileEvents.Values` Array(UInt64), + `Settings.Names` Array(String), + `Settings.Values` Array(String) +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql new file mode 100644 index 00000000000..56ff3366702 --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql @@ -0,0 +1,46 @@ +ATTACH TABLE _ UUID '8b2738c3-4bea-49f7-b547-0a92673830cc' +( + `event_date` Date, + `event_time` DateTime, + `event_time_microseconds` DateTime64(6), + `query_start_time` DateTime, + `query_start_time_microseconds` DateTime64(6), + `query_duration_ms` UInt64, + `read_rows` UInt64, + `read_bytes` UInt64, + `written_rows` UInt64, + `written_bytes` UInt64, + `memory_usage` Int64, + `peak_memory_usage` Int64, + `thread_name` String, + `thread_id` UInt64, + `master_thread_id` UInt64, + `query` String, + `is_initial_query` UInt8, + `user` String, + `query_id` String, + `address` IPv6, + `port` UInt16, + `initial_user` String, + `initial_query_id` String, + `initial_address` IPv6, + `initial_port` UInt16, + `interface` UInt8, + `os_user` String, + `client_hostname` String, + `client_name` String, + `client_revision` UInt32, + `client_version_major` UInt32, + `client_version_minor` UInt32, + `client_version_patch` UInt32, + `http_method` UInt8, + `http_user_agent` String, + `quota_key` String, + `revision` UInt32, + `ProfileEvents.Names` Array(String), + `ProfileEvents.Values` Array(UInt64) +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql new file mode 100644 index 00000000000..874466c610f --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql @@ -0,0 +1,20 @@ +ATTACH TABLE _ UUID 'b619f0a8-9836-4824-9594-a722b0a7d500' +( + `event_date` Date, + `event_time` DateTime, + `event_time_microseconds` DateTime64(6), + `microseconds` UInt32, + `thread_name` LowCardinality(String), + `thread_id` UInt64, + `level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8), + `query_id` String, + `logger_name` LowCardinality(String), + `message` String, + `revision` UInt32, + `source_file` LowCardinality(String), + `source_line` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(event_date) +ORDER BY (event_date, event_time) +SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql new file mode 100644 index 00000000000..04517208a2e --- /dev/null +++ b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql @@ -0,0 +1,17 @@ +ATTACH TABLE _ UUID '72dc35e3-6e64-44bf-af3f-4d66fb70834a' +( + `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 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..15f590ec5a9f148c51c40e5f8643b93ed6312e48 GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bUKSS==f{FYCNyEFtWbCto z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOT5}}ui?1Ks zvg>S#d$7rg02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcnm83Pgz_C>>6$e Kar3Qw+64edhHlyb literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt new file mode 100644 index 00000000000..5bfe4600a4e --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`name` String +`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt new file mode 100644 index 00000000000..afbe847262c --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt @@ -0,0 +1 @@ +126 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx new file mode 100644 index 00000000000..4b6c49f8d50 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx @@ -0,0 +1 @@ +…H…H \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac GIT binary patch literal 4 Lcmb0VWo7^X0I>iw literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx new file mode 100644 index 00000000000..38e2c642aeb --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx @@ -0,0 +1 @@ +…HÖ-œ_…HN.œ_ \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..288379f59b4d0506587d2b915da29b5429f3c32c GIT binary patch literal 262 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyVUf?3K)m;9pkOXWjEYr54|4y2Zf2 zZ~=%v+Ou&c=a=S{=#^BIupV9eg5{F*3qhl-g^$@cNz7y5NJ%V7)Jw|DJHCbKd3ghe zFgEolo@6;#*(z-(X1n4kgRx0RBc}nwhrHaPY~v25bs?^=WxQQ@ znPqo7Se%>sj6pOvGcPx>B0jY&HLoNdXmF}tW=aK1VDXg_AS<=yCEMbz)z+F(3nVVQt3DPa}a>cj&7s0hFg}*8) KC3BoV%>V$TlWN`o literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt new file mode 100644 index 00000000000..5bfe4600a4e --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`name` String +`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt new file mode 100644 index 00000000000..23644ad9685 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt @@ -0,0 +1 @@ +12390 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx new file mode 100644 index 00000000000..07abc658bdb --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx @@ -0,0 +1 @@ +¾HÁH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..7af4421fcb5ebc50aa06a5f4dee6e454020552a6 GIT binary patch literal 4 Lcmb0UWo7^X0J#7) literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx new file mode 100644 index 00000000000..d57044faf98 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx @@ -0,0 +1 @@ +¾H>˜ç_¿HÇç_ÁH’}ê_ \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..d702cbf79d2a5a91c07163c9680526ae0c503ceb GIT binary patch literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bUW>V4jhooyz_~f{Za(pN8Q&6FaYpwZE|ZfumSZKokr7~ z|403vu$n{)-iQDdZE0?8VR&C@6CZZ2eDbOfQp0V$_ea0t>|RE;3Dk?sHsaB^vFVRCscX=Hc|m=Z;zemg0s MudjjE3#Lvc0M|-wEC2ui literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt new file mode 100644 index 00000000000..5bfe4600a4e --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`name` String +`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt new file mode 100644 index 00000000000..721ca6f552a --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt @@ -0,0 +1 @@ +6048 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx new file mode 100644 index 00000000000..4c89b29a856 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx @@ -0,0 +1 @@ +ËHÙH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..a713aecd075daae7071deceaa93147871ef28fbf GIT binary patch literal 4 LcmXRFWo7^X0v!O3 literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx new file mode 100644 index 00000000000..8e5b4736172 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx @@ -0,0 +1 @@ +ËHÖ÷_ÙHšì ` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..318b8a518b960630dc1e834f27ce63e27b6593ab GIT binary patch literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$c7oqGLyZE0?8VR&C@6CZZ2eDbOb>eW_f~A{08aeSmV>iAT$cC2%NmSkKqG{?=Jq@?bnHd z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOJne3tc?*x$p?c>F_{T(_H`vWMOn+E@EkJfCsQb zD@h%ngTaZ|pyvL@;Y0zh4$4=AdlcKPVg5TpMxB@3V+k$*@NR8#YcsF`21}@_f-!EI z40m1$`G2OI02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnrlp=b;Ym@z8Dp Kp(2L^K0p9kBW<++ literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt new file mode 100644 index 00000000000..5bfe4600a4e --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`name` String +`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt new file mode 100644 index 00000000000..f70d7bba4ae --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt @@ -0,0 +1 @@ +42 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx new file mode 100644 index 00000000000..47460b95be9 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx @@ -0,0 +1 @@ +ëHª¥"`ëHª¥"` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..705c91fa699e37af9a5bc17585f859123272e27b GIT binary patch literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyZ=8e&PS-lZH>%6zMI~Ocl~>y1~G} za1Mw++Olyb=a=S{=#^BIFxixH$J%_|`GMoa%MT9Wmtz??QW8rN^^!94dfBVyP4w8E z^d;5x=sNDHm&BhluH*BS+J9T=m`+Tsp_!zbhWTwyJ%$f?xkcH=8yJhzv;sK_cCKB% z@k(c3X6Zc!(cH|u+{B9b)UwpPl6au8sd||y6)cGd*%s{FoVmzWz|{EB#4Kk9-h#xU zlFSmIyY+yISR5tODw~4-u)Ta5@gie?>vRV0f}+ga#G*=&Zk}DNanDuX@+;a)C2mzy I?byly0Nr6}0ssI2 literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt new file mode 100644 index 00000000000..5bfe4600a4e --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`name` String +`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt new file mode 100644 index 00000000000..f70d7bba4ae --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt @@ -0,0 +1 @@ +42 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx new file mode 100644 index 00000000000..dc2b3df8c14 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx @@ -0,0 +1 @@ +ëHæ¥"`ëHæ¥"` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/checksums.txt new file mode 100644 index 0000000000000000000000000000000000000000..a79136dea2ae4694404058e307f30c58f2c95216 GIT binary patch literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$b+KNoPO=+Yv2+Llc&Eg)Ee z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOJne3tc?*x$p?c>F_{T(_H`vWMOn+E@EkJk_VW( zJ28oFC&TW$V{^28W&8oI4&bJ;l7ZNHLvKrW15Q;okqIsU@NR8#YcsF`wn)byrXbCF zZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnoZhRO7$)udRQM Kfs2=!gM9!_8*V`W literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt new file mode 100644 index 00000000000..5bfe4600a4e --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`event_date` Date +`event_time` DateTime +`event_time_microseconds` DateTime64(6) +`name` String +`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt new file mode 100644 index 00000000000..f70d7bba4ae --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt @@ -0,0 +1 @@ +42 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt new file mode 100644 index 00000000000..061d1280b89 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx new file mode 100644 index 00000000000..16cc6680505 --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx @@ -0,0 +1 @@ +ëHëH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/partition.dat new file mode 100644 index 0000000000000000000000000000000000000000..da540c2d1a825be3ef80d13baa834834b2248491 GIT binary patch literal 4 LcmXRDWo7^X0wDm8 literal 0 HcmV?d00001 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx new file mode 100644 index 00000000000..bd60e38174b --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx @@ -0,0 +1 @@ +ëH"¦"`ëH"¦"` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt new file mode 100644 index 00000000000..56a6051ca2b --- /dev/null +++ b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c8f48f2ed1f..8d824d85d37 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -49,6 +49,14 @@ #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include #include #include #include @@ -956,7 +964,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

QueryProcessingStage::WithMergeableState && - !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube; + !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube && + !query.group_by_with_grouping_sets; if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) { @@ -1238,7 +1247,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aggregationKeys()) + keys.push_back(header_before_transform.getPositionByName(key.name)); + + const Settings & settings = context->getSettingsRef(); + + Aggregator::Params params(header_before_transform, keys, query_analyzer->aggregates(), + false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0, + settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, + context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression); + + auto transform_params = std::make_shared(params, true); + + QueryPlanStepPtr step; + step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); + + query_plan.addStep(std::move(step)); +} void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description) { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 4298cbbb794..8ea04cf7b27 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -160,6 +160,7 @@ private: }; void executeRollupOrCube(QueryPlan & query_plan, Modificator modificator); + void executeGroupingSets(QueryPlan & query_plan); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. * diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 1c5a4310f1b..f07a4a328e2 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -125,6 +125,9 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (group_by_with_cube) s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH CUBE" << (s.hilite ? hilite_none : ""); + if (group_by_with_grouping_sets) + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH GROUPING SETS" << (s.hilite ? hilite_none : ""); + if (group_by_with_totals) s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ASTSelectQuery.h b/src/Parsers/ASTSelectQuery.h index 2b004e9e040..a6169c37013 100644 --- a/src/Parsers/ASTSelectQuery.h +++ b/src/Parsers/ASTSelectQuery.h @@ -83,6 +83,7 @@ public: bool group_by_with_rollup = false; bool group_by_with_cube = false; bool group_by_with_constant_keys = false; + bool group_by_with_grouping_sets = false; bool limit_with_ties = false; ASTPtr & refSelect() { return getExpression(Expression::SELECT); } diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 90ab5911d6b..f77fff658d6 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -50,6 +50,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_by("BY"); ParserKeyword s_rollup("ROLLUP"); ParserKeyword s_cube("CUBE"); + ParserKeyword s_grouping_sets("GROUPING SETS"); ParserKeyword s_top("TOP"); ParserKeyword s_with_ties("WITH TIES"); ParserKeyword s_offset("OFFSET"); @@ -184,18 +185,22 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->group_by_with_rollup = true; else if (s_cube.ignore(pos, expected)) select_query->group_by_with_cube = true; + else if (s_grouping_sets.ignore(pos, expected)) + select_query->group_by_with_grouping_sets = true; - if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !open_bracket.ignore(pos, expected)) + if ((select_query->group_by_with_rollup || select_query->group_by_with_cube || select_query->group_by_with_grouping_sets) && + !open_bracket.ignore(pos, expected)) return false; if (!exp_list.parse(pos, group_expression_list, expected)) return false; - if ((select_query->group_by_with_rollup || select_query->group_by_with_cube) && !close_bracket.ignore(pos, expected)) + if ((select_query->group_by_with_rollup || select_query->group_by_with_cube || select_query->group_by_with_grouping_sets) && + !close_bracket.ignore(pos, expected)) return false; } - /// WITH ROLLUP, CUBE or TOTALS + /// WITH ROLLUP, CUBE, GROU PING SETS or TOTALS if (s_with.ignore(pos, expected)) { if (s_rollup.ignore(pos, expected)) diff --git a/src/Processors/QueryPlan/GroupingSetsStep.cpp b/src/Processors/QueryPlan/GroupingSetsStep.cpp new file mode 100644 index 00000000000..d409e0e061a --- /dev/null +++ b/src/Processors/QueryPlan/GroupingSetsStep.cpp @@ -0,0 +1,46 @@ +#include +#include +#include + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = false, + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = false, + } + }; +} + +GroupingSetsStep::GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_) + : ITransformingStep(input_stream_, params_->getHeader(), getTraits()) + , params(std::move(params_)) +{ + /// Aggregation keys are distinct + for (auto key : params->params.keys) + output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name); +} + +void GroupingSetsStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*settings*/) +{ + pipeline.resize(1); + + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type == QueryPipelineBuilder::StreamType::Totals) + return nullptr; + + return std::make_shared(header, std::move(params)); + }); +} + +} diff --git a/src/Processors/QueryPlan/GroupingSetsStep.h b/src/Processors/QueryPlan/GroupingSetsStep.h new file mode 100644 index 00000000000..f27d7f1280e --- /dev/null +++ b/src/Processors/QueryPlan/GroupingSetsStep.h @@ -0,0 +1,25 @@ +#pragma once +#include +#include + +namespace DB +{ + +struct AggregatingTransformParams; +using AggregatingTransformParamsPtr = std::shared_ptr; + +/// WITH CUBE. See CubeTransform. +class GroupingSetsStep : public ITransformingStep +{ +public: + GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_); + + String getName() const override { return "Grouping Sets"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + +private: + AggregatingTransformParamsPtr params; +}; + +} diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 2a515fdf3be..c9d516f8a11 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -15,6 +15,7 @@ public: AggregatedArenasChunkInfo(Arenas arenas_) : arenas(std::move(arenas_)) {} + }; class AggregatedChunkInfo : public ChunkInfo diff --git a/src/Processors/Transforms/CubeTransform.cpp b/src/Processors/Transforms/CubeTransform.cpp index c64f39af5de..13e4963573c 100644 --- a/src/Processors/Transforms/CubeTransform.cpp +++ b/src/Processors/Transforms/CubeTransform.cpp @@ -45,7 +45,7 @@ Chunk CubeTransform::generate() consumed_chunks.clear(); auto num_rows = cube_chunk.getNumRows(); - mask = (UInt64(1) << keys.size()) - 1; + mask = (UInt64(1) << keys.size()); current_columns = cube_chunk.getColumns(); current_zero_columns.clear(); @@ -55,11 +55,11 @@ Chunk CubeTransform::generate() current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows)); } - auto gen_chunk = std::move(cube_chunk); + // auto gen_chunk = std::move(cube_chunk); - if (mask) + if (mask > 1) { - --mask; + mask = mask >> 1; auto columns = current_columns; auto size = keys.size(); @@ -72,6 +72,7 @@ Chunk CubeTransform::generate() chunks.emplace_back(std::move(columns), current_columns.front()->size()); cube_chunk = merge(std::move(chunks), false); } + auto gen_chunk = std::move(cube_chunk); finalizeChunk(gen_chunk); return gen_chunk; diff --git a/src/Processors/Transforms/GroupingSetsTransform.cpp b/src/Processors/Transforms/GroupingSetsTransform.cpp new file mode 100644 index 00000000000..637f244ff96 --- /dev/null +++ b/src/Processors/Transforms/GroupingSetsTransform.cpp @@ -0,0 +1,81 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +GroupingSetsTransform::GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params_) + : IAccumulatingTransform(std::move(header), params_->getHeader()) + , params(std::move(params_)) + , keys(params->params.keys) +{ +// if (keys.size() >= 8 * sizeof(mask)) +// throw Exception("Too many keys are used for CubeTransform.", ErrorCodes::LOGICAL_ERROR); +} + +Chunk GroupingSetsTransform::merge(Chunks && chunks, bool final) +{ + BlocksList rollup_blocks; + for (auto & chunk : chunks) + rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); + + auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final); + auto num_rows = rollup_block.rows(); + return Chunk(rollup_block.getColumns(), num_rows); +} + +void GroupingSetsTransform::consume(Chunk chunk) +{ + consumed_chunks.emplace_back(std::move(chunk)); +} + +Chunk GroupingSetsTransform::generate() +{ + if (!consumed_chunks.empty()) + { + if (consumed_chunks.size() > 1) + grouping_sets_chunk = merge(std::move(consumed_chunks), false); + else + grouping_sets_chunk = std::move(consumed_chunks.front()); + + consumed_chunks.clear(); + + auto num_rows = grouping_sets_chunk.getNumRows(); + mask = (UInt64(1) << keys.size()); + + current_columns = grouping_sets_chunk.getColumns(); + current_zero_columns.clear(); + current_zero_columns.reserve(keys.size()); + + for (auto key : keys) + current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows)); + } + + // auto gen_chunk = std::move(cube_chunk); + + if (mask > 1) + { + mask = mask >> 1; + + auto columns = current_columns; + auto size = keys.size(); + for (size_t i = 0; i < size; ++i) + /// Reverse bit order to support previous behaviour. + if ((mask & (UInt64(1) << (size - i - 1))) == 0) + columns[keys[i]] = current_zero_columns[i]; + + Chunks chunks; + chunks.emplace_back(std::move(columns), current_columns.front()->size()); + grouping_sets_chunk = merge(std::move(chunks), false); + } + auto gen_chunk = std::move(grouping_sets_chunk); + + finalizeChunk(gen_chunk); + return gen_chunk; +} + +} diff --git a/src/Processors/Transforms/GroupingSetsTransform.h b/src/Processors/Transforms/GroupingSetsTransform.h new file mode 100644 index 00000000000..0e300453215 --- /dev/null +++ b/src/Processors/Transforms/GroupingSetsTransform.h @@ -0,0 +1,35 @@ +#pragma once +#include +#include + + +namespace DB +{ + +/// Takes blocks after grouping, with non-finalized aggregate functions. +/// Calculates all subsets of columns and aggregates over them. +class GroupingSetsTransform : public IAccumulatingTransform +{ +public: + GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params); + String getName() const override { return "GroupingSetsTransform"; } + +protected: + void consume(Chunk chunk) override; + Chunk generate() override; + +private: + AggregatingTransformParamsPtr params; + ColumnNumbers keys; + + Chunks consumed_chunks; + Chunk grouping_sets_chunk; + Columns current_columns; + Columns current_zero_columns; + + UInt64 mask = 0; + + Chunk merge(Chunks && chunks, bool final); +}; + +} diff --git a/src/Processors/ya.make b/src/Processors/ya.make new file mode 100644 index 00000000000..a1386acb6d0 --- /dev/null +++ b/src/Processors/ya.make @@ -0,0 +1,160 @@ +# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. +LIBRARY() + +PEERDIR( + clickhouse/src/Common + contrib/libs/msgpack + contrib/libs/protobuf +) + +CFLAGS(-g0) + +SRCS( + Chunk.cpp + ConcatProcessor.cpp + DelayedPortsProcessor.cpp + Executors/ExecutingGraph.cpp + Executors/PipelineExecutingBlockInputStream.cpp + Executors/PipelineExecutor.cpp + Executors/PullingAsyncPipelineExecutor.cpp + Executors/PullingPipelineExecutor.cpp + ForkProcessor.cpp + Formats/IInputFormat.cpp + Formats/Impl/BinaryRowInputFormat.cpp + Formats/Impl/BinaryRowOutputFormat.cpp + Formats/Impl/ConstantExpressionTemplate.cpp + Formats/Impl/CSVRowInputFormat.cpp + Formats/Impl/CSVRowOutputFormat.cpp + Formats/Impl/JSONAsStringRowInputFormat.cpp + Formats/Impl/JSONCompactEachRowRowInputFormat.cpp + Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp + Formats/Impl/JSONCompactRowOutputFormat.cpp + Formats/Impl/JSONEachRowRowInputFormat.cpp + Formats/Impl/JSONEachRowRowOutputFormat.cpp + Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp + Formats/Impl/JSONRowOutputFormat.cpp + Formats/Impl/LineAsStringRowInputFormat.cpp + Formats/Impl/MarkdownRowOutputFormat.cpp + Formats/Impl/MsgPackRowInputFormat.cpp + Formats/Impl/MsgPackRowOutputFormat.cpp + Formats/Impl/MySQLOutputFormat.cpp + Formats/Impl/NativeFormat.cpp + Formats/Impl/NullFormat.cpp + Formats/Impl/ODBCDriver2BlockOutputFormat.cpp + Formats/Impl/PostgreSQLOutputFormat.cpp + Formats/Impl/PrettyBlockOutputFormat.cpp + Formats/Impl/PrettyCompactBlockOutputFormat.cpp + Formats/Impl/PrettySpaceBlockOutputFormat.cpp + Formats/Impl/ProtobufRowInputFormat.cpp + Formats/Impl/ProtobufRowOutputFormat.cpp + Formats/Impl/RawBLOBRowInputFormat.cpp + Formats/Impl/RawBLOBRowOutputFormat.cpp + Formats/Impl/RegexpRowInputFormat.cpp + Formats/Impl/TabSeparatedRowInputFormat.cpp + Formats/Impl/TabSeparatedRowOutputFormat.cpp + Formats/Impl/TemplateBlockOutputFormat.cpp + Formats/Impl/TemplateRowInputFormat.cpp + Formats/Impl/TSKVRowInputFormat.cpp + Formats/Impl/TSKVRowOutputFormat.cpp + Formats/Impl/ValuesBlockInputFormat.cpp + Formats/Impl/ValuesRowOutputFormat.cpp + Formats/Impl/VerticalRowOutputFormat.cpp + Formats/Impl/XMLRowOutputFormat.cpp + Formats/IOutputFormat.cpp + Formats/IRowInputFormat.cpp + Formats/IRowOutputFormat.cpp + Formats/LazyOutputFormat.cpp + Formats/OutputStreamToOutputFormat.cpp + Formats/PullingOutputFormat.cpp + Formats/RowInputFormatWithDiagnosticInfo.cpp + IAccumulatingTransform.cpp + IInflatingTransform.cpp + IProcessor.cpp + ISimpleTransform.cpp + ISink.cpp + ISource.cpp + LimitTransform.cpp + Merges/Algorithms/AggregatingSortedAlgorithm.cpp + Merges/Algorithms/CollapsingSortedAlgorithm.cpp + Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp + Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp + Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp + Merges/Algorithms/MergingSortedAlgorithm.cpp + Merges/Algorithms/ReplacingSortedAlgorithm.cpp + Merges/Algorithms/SummingSortedAlgorithm.cpp + Merges/Algorithms/VersionedCollapsingAlgorithm.cpp + Merges/IMergingTransform.cpp + Merges/MergingSortedTransform.cpp + OffsetTransform.cpp + Pipe.cpp + Port.cpp + printPipeline.cpp + QueryPipeline.cpp + QueryPlan/AddingDelayedSourceStep.cpp + QueryPlan/AggregatingStep.cpp + QueryPlan/ArrayJoinStep.cpp + QueryPlan/ConvertingStep.cpp + QueryPlan/CreatingSetsStep.cpp + QueryPlan/CubeStep.cpp + QueryPlan/DistinctStep.cpp + QueryPlan/ExpressionStep.cpp + QueryPlan/ExtremesStep.cpp + QueryPlan/FillingStep.cpp + QueryPlan/FilterStep.cpp + QueryPlan/FinishSortingStep.cpp + QueryPlan/GroupingSetsStep.cpp + QueryPlan/IQueryPlanStep.cpp + QueryPlan/ISourceStep.cpp + QueryPlan/ITransformingStep.cpp + QueryPlan/LimitByStep.cpp + QueryPlan/LimitStep.cpp + QueryPlan/MergeSortingStep.cpp + QueryPlan/MergingAggregatedStep.cpp + QueryPlan/MergingSortedStep.cpp + QueryPlan/OffsetStep.cpp + QueryPlan/PartialSortingStep.cpp + QueryPlan/QueryPlan.cpp + QueryPlan/ReadFromPreparedSource.cpp + QueryPlan/ReadFromStorageStep.cpp + QueryPlan/ReadNothingStep.cpp + QueryPlan/RollupStep.cpp + QueryPlan/TotalsHavingStep.cpp + QueryPlan/UnionStep.cpp + ResizeProcessor.cpp + Sources/DelayedSource.cpp + Sources/RemoteSource.cpp + Sources/SinkToOutputStream.cpp + Sources/SourceFromInputStream.cpp + Sources/SourceWithProgress.cpp + Transforms/AddingMissedTransform.cpp + Transforms/AddingSelectorTransform.cpp + Transforms/AggregatingInOrderTransform.cpp + Transforms/AggregatingTransform.cpp + Transforms/ArrayJoinTransform.cpp + Transforms/ConvertingTransform.cpp + Transforms/CopyTransform.cpp + Transforms/CreatingSetsTransform.cpp + Transforms/CubeTransform.cpp + Transforms/DistinctTransform.cpp + Transforms/ExpressionTransform.cpp + Transforms/ExtremesTransform.cpp + Transforms/FillingTransform.cpp + Transforms/FilterTransform.cpp + Transforms/FinishSortingTransform.cpp + Transforms/GroupingSetsTransform.cpp + Transforms/JoiningTransform.cpp + Transforms/LimitByTransform.cpp + Transforms/LimitsCheckingTransform.cpp + Transforms/MaterializingTransform.cpp + Transforms/MergeSortingTransform.cpp + Transforms/MergingAggregatedMemoryEfficientTransform.cpp + Transforms/MergingAggregatedTransform.cpp + Transforms/PartialSortingTransform.cpp + Transforms/ReverseTransform.cpp + Transforms/RollupTransform.cpp + Transforms/SortingTransform.cpp + Transforms/TotalsHavingTransform.cpp + +) + +END() From 2403b03597d524bae4e9e5a6d03834b30dad566c Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 21:42:20 +0300 Subject: [PATCH 073/358] rm unnessesary data committed by mistake --- .../201403_1_6_1/checksums.txt | Bin 14305 -> 0 bytes .../201403_1_6_1/columns.txt | 183 ------------ .../201403_1_6_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../201403_1_6_1/minmax_StartDate.idx | 1 - .../201403_1_6_1/partition.dat | Bin 4 -> 0 bytes .../201403_1_6_1/primary.idx | Bin 3240 -> 0 bytes .../201403_7_9_1/checksums.txt | Bin 14273 -> 0 bytes .../201403_7_9_1/columns.txt | 183 ------------ .../201403_7_9_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../201403_7_9_1/minmax_StartDate.idx | 1 - .../201403_7_9_1/partition.dat | Bin 4 -> 0 bytes .../201403_7_9_1/primary.idx | Bin 1314 -> 0 bytes .../format_version.txt | 1 - .../202010_1_28_6/checksums.txt | Bin 262 -> 0 bytes .../202010_1_28_6/columns.txt | 259 ----------------- .../202010_1_28_6/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202010_1_28_6/minmax_event_date.idx | 1 - .../202010_1_28_6/partition.dat | Bin 4 -> 0 bytes .../202010_1_28_6/primary.idx | 1 - .../202012_29_2358_466/checksums.txt | Bin 11371 -> 0 bytes .../202012_29_2358_466/columns.txt | 259 ----------------- .../202012_29_2358_466/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_29_2358_466/minmax_event_date.idx | 1 - .../202012_29_2358_466/partition.dat | Bin 4 -> 0 bytes .../202012_29_2358_466/primary.idx | 1 - .../202101_2359_3529_235/checksums.txt | Bin 11145 -> 0 bytes .../202101_2359_3529_235/columns.txt | 259 ----------------- .../202101_2359_3529_235/count.txt | 1 - .../default_compression_codec.txt | 1 - .../minmax_event_date.idx | 1 - .../202101_2359_3529_235/partition.dat | Bin 4 -> 0 bytes .../202101_2359_3529_235/primary.idx | Bin 18 -> 0 bytes .../202102_3530_4606_216/checksums.txt | Bin 10969 -> 0 bytes .../202102_3530_4606_216/columns.txt | 259 ----------------- .../202102_3530_4606_216/count.txt | 1 - .../default_compression_codec.txt | 1 - .../minmax_event_date.idx | 1 - .../202102_3530_4606_216/partition.dat | Bin 4 -> 0 bytes .../202102_3530_4606_216/primary.idx | 1 - .../202102_4607_4607_0/checksums.txt | Bin 260 -> 0 bytes .../202102_4607_4607_0/columns.txt | 259 ----------------- .../202102_4607_4607_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_4607_4607_0/minmax_event_date.idx | 1 - .../202102_4607_4607_0/partition.dat | Bin 4 -> 0 bytes .../202102_4607_4607_0/primary.idx | 1 - .../202102_4608_4608_0/checksums.txt | Bin 260 -> 0 bytes .../202102_4608_4608_0/columns.txt | 259 ----------------- .../202102_4608_4608_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_4608_4608_0/minmax_event_date.idx | 1 - .../202102_4608_4608_0/partition.dat | Bin 4 -> 0 bytes .../202102_4608_4608_0/primary.idx | 1 - .../format_version.txt | 1 - .../hits.sql | 141 ---------- .../visits.sql | 189 ------------- .../202010_1_5_1/checksums.txt | Bin 260 -> 0 bytes .../202010_1_5_1/columns.txt | 47 ---- .../202010_1_5_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202010_1_5_1/minmax_event_date.idx | 1 - .../202010_1_5_1/partition.dat | Bin 4 -> 0 bytes .../202010_1_5_1/primary.idx | 1 - .../202012_6_47_9/checksums.txt | Bin 262 -> 0 bytes .../202012_6_47_9/columns.txt | 47 ---- .../202012_6_47_9/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_6_47_9/minmax_event_date.idx | 1 - .../202012_6_47_9/partition.dat | Bin 4 -> 0 bytes .../202012_6_47_9/primary.idx | 1 - .../202101_48_74_6/checksums.txt | Bin 261 -> 0 bytes .../202101_48_74_6/columns.txt | 47 ---- .../202101_48_74_6/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202101_48_74_6/minmax_event_date.idx | 1 - .../202101_48_74_6/partition.dat | Bin 4 -> 0 bytes .../202101_48_74_6/primary.idx | 1 - .../202102_75_79_1/checksums.txt | Bin 260 -> 0 bytes .../202102_75_79_1/columns.txt | 47 ---- .../202102_75_79_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_75_79_1/minmax_event_date.idx | 1 - .../202102_75_79_1/partition.dat | Bin 4 -> 0 bytes .../202102_75_79_1/primary.idx | 1 - .../202102_80_80_0/checksums.txt | Bin 260 -> 0 bytes .../202102_80_80_0/columns.txt | 47 ---- .../202102_80_80_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_80_80_0/minmax_event_date.idx | 1 - .../202102_80_80_0/partition.dat | Bin 4 -> 0 bytes .../202102_80_80_0/primary.idx | 1 - .../format_version.txt | 1 - .../all_1_1_0/checksums.txt | Bin 188 -> 0 bytes .../all_1_1_0/columns.txt | 13 - .../all_1_1_0/count.txt | 1 - .../all_1_1_0/default_compression_codec.txt | 1 - .../all_1_1_0/primary.idx | 1 - .../all_2_2_0/checksums.txt | Bin 188 -> 0 bytes .../all_2_2_0/columns.txt | 13 - .../all_2_2_0/count.txt | 1 - .../all_2_2_0/default_compression_codec.txt | 1 - .../all_2_2_0/primary.idx | 1 - .../format_version.txt | 1 - .../202012_1_18_4/checksums.txt | Bin 260 -> 0 bytes .../202012_1_18_4/columns.txt | 20 -- .../202012_1_18_4/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_1_18_4/minmax_event_date.idx | 1 - .../202012_1_18_4/partition.dat | Bin 4 -> 0 bytes .../202012_1_18_4/primary.idx | 1 - .../202101_19_22_1/checksums.txt | Bin 260 -> 0 bytes .../202101_19_22_1/columns.txt | 20 -- .../202101_19_22_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202101_19_22_1/minmax_event_date.idx | 1 - .../202101_19_22_1/partition.dat | Bin 4 -> 0 bytes .../202101_19_22_1/primary.idx | 1 - .../format_version.txt | 1 - .../201403_1_29_2/checksums.txt | Bin 10792 -> 0 bytes .../201403_1_29_2/columns.txt | 135 --------- .../201403_1_29_2/count.txt | 1 - .../default_compression_codec.txt | 1 - .../201403_1_29_2/minmax_EventDate.idx | 1 - .../201403_1_29_2/partition.dat | Bin 4 -> 0 bytes .../201403_1_29_2/primary.idx | Bin 15330 -> 0 bytes .../format_version.txt | 1 - .../202010_1_4_1/checksums.txt | Bin 260 -> 0 bytes .../202010_1_4_1/columns.txt | 12 - .../202010_1_4_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202010_1_4_1/minmax_event_date.idx | 1 - .../202010_1_4_1/partition.dat | Bin 4 -> 0 bytes .../202010_1_4_1/primary.idx | 1 - .../202012_35_35_0/checksums.txt | Bin 260 -> 0 bytes .../202012_35_35_0/columns.txt | 12 - .../202012_35_35_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_35_35_0/minmax_event_date.idx | 1 - .../202012_35_35_0/partition.dat | Bin 4 -> 0 bytes .../202012_35_35_0/primary.idx | 1 - .../202012_5_34_6/checksums.txt | Bin 261 -> 0 bytes .../202012_5_34_6/columns.txt | 12 - .../202012_5_34_6/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_5_34_6/minmax_event_date.idx | 1 - .../202012_5_34_6/partition.dat | Bin 4 -> 0 bytes .../202012_5_34_6/primary.idx | 1 - .../202101_36_115_17/checksums.txt | Bin 262 -> 0 bytes .../202101_36_115_17/columns.txt | 12 - .../202101_36_115_17/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202101_36_115_17/minmax_event_date.idx | 1 - .../202101_36_115_17/partition.dat | Bin 4 -> 0 bytes .../202101_36_115_17/primary.idx | 1 - .../202102_116_127_3/checksums.txt | Bin 261 -> 0 bytes .../202102_116_127_3/columns.txt | 12 - .../202102_116_127_3/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_116_127_3/minmax_event_date.idx | 1 - .../202102_116_127_3/partition.dat | Bin 4 -> 0 bytes .../202102_116_127_3/primary.idx | 1 - .../202102_128_128_0/checksums.txt | Bin 260 -> 0 bytes .../202102_128_128_0/columns.txt | 12 - .../202102_128_128_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_128_128_0/minmax_event_date.idx | 1 - .../202102_128_128_0/partition.dat | Bin 4 -> 0 bytes .../202102_128_128_0/primary.idx | 1 - .../202102_129_129_0/checksums.txt | Bin 261 -> 0 bytes .../202102_129_129_0/columns.txt | 12 - .../202102_129_129_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_129_129_0/minmax_event_date.idx | 1 - .../202102_129_129_0/partition.dat | Bin 4 -> 0 bytes .../202102_129_129_0/primary.idx | 1 - .../format_version.txt | 1 - .../202010_1_5_1/checksums.txt | Bin 260 -> 0 bytes .../202010_1_5_1/columns.txt | 41 --- .../202010_1_5_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202010_1_5_1/minmax_event_date.idx | 1 - .../202010_1_5_1/partition.dat | Bin 4 -> 0 bytes .../202010_1_5_1/primary.idx | 1 - .../202012_6_45_8/checksums.txt | Bin 262 -> 0 bytes .../202012_6_45_8/columns.txt | 41 --- .../202012_6_45_8/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_6_45_8/minmax_event_date.idx | 1 - .../202012_6_45_8/partition.dat | Bin 4 -> 0 bytes .../202012_6_45_8/primary.idx | 1 - .../202101_46_72_6/checksums.txt | Bin 262 -> 0 bytes .../202101_46_72_6/columns.txt | 41 --- .../202101_46_72_6/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202101_46_72_6/minmax_event_date.idx | 1 - .../202101_46_72_6/partition.dat | Bin 4 -> 0 bytes .../202101_46_72_6/primary.idx | 1 - .../202102_73_77_1/checksums.txt | Bin 261 -> 0 bytes .../202102_73_77_1/columns.txt | 41 --- .../202102_73_77_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_73_77_1/minmax_event_date.idx | 1 - .../202102_73_77_1/partition.dat | Bin 4 -> 0 bytes .../202102_73_77_1/primary.idx | 1 - .../202102_78_78_0/checksums.txt | Bin 260 -> 0 bytes .../202102_78_78_0/columns.txt | 41 --- .../202102_78_78_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_78_78_0/minmax_event_date.idx | 1 - .../202102_78_78_0/partition.dat | Bin 4 -> 0 bytes .../202102_78_78_0/primary.idx | 1 - .../format_version.txt | 1 - .../202010_1_29_6/checksums.txt | Bin 262 -> 0 bytes .../202010_1_29_6/columns.txt | 15 - .../202010_1_29_6/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202010_1_29_6/minmax_event_date.idx | 1 - .../202010_1_29_6/partition.dat | Bin 4 -> 0 bytes .../202010_1_29_6/primary.idx | 1 - .../202012_2358_2361_1/checksums.txt | Bin 260 -> 0 bytes .../202012_2358_2361_1/columns.txt | 15 - .../202012_2358_2361_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_2358_2361_1/minmax_event_date.idx | 1 - .../202012_2358_2361_1/partition.dat | Bin 4 -> 0 bytes .../202012_2358_2361_1/primary.idx | 1 - .../202012_30_2357_1895/checksums.txt | Bin 263 -> 0 bytes .../202012_30_2357_1895/columns.txt | 15 - .../202012_30_2357_1895/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_30_2357_1895/minmax_event_date.idx | 1 - .../202012_30_2357_1895/partition.dat | Bin 4 -> 0 bytes .../202012_30_2357_1895/primary.idx | 1 - .../202101_2362_3536_470/checksums.txt | Bin 263 -> 0 bytes .../202101_2362_3536_470/columns.txt | 15 - .../202101_2362_3536_470/count.txt | 1 - .../default_compression_codec.txt | 1 - .../minmax_event_date.idx | 1 - .../202101_2362_3536_470/partition.dat | Bin 4 -> 0 bytes .../202101_2362_3536_470/primary.idx | Bin 24 -> 0 bytes .../202101_3537_3539_1/checksums.txt | Bin 260 -> 0 bytes .../202101_3537_3539_1/columns.txt | 15 - .../202101_3537_3539_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202101_3537_3539_1/minmax_event_date.idx | 1 - .../202101_3537_3539_1/partition.dat | Bin 4 -> 0 bytes .../202101_3537_3539_1/primary.idx | 1 - .../202102_3540_4637_415/checksums.txt | Bin 263 -> 0 bytes .../202102_3540_4637_415/columns.txt | 15 - .../202102_3540_4637_415/count.txt | 1 - .../default_compression_codec.txt | 1 - .../minmax_event_date.idx | 1 - .../202102_3540_4637_415/partition.dat | Bin 4 -> 0 bytes .../202102_3540_4637_415/primary.idx | 1 - .../202102_4638_4638_0/checksums.txt | Bin 260 -> 0 bytes .../202102_4638_4638_0/columns.txt | 15 - .../202102_4638_4638_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_4638_4638_0/minmax_event_date.idx | 1 - .../202102_4638_4638_0/partition.dat | Bin 4 -> 0 bytes .../202102_4638_4638_0/primary.idx | 1 - .../format_version.txt | 1 - .../asynchronous_metric_log.sql | 12 - .../crash_log.sql | 17 -- .../metric_log.sql | 264 ------------------ .../part_log.sql | 25 -- .../query_log.sql | 52 ---- .../query_thread_log.sql | 46 --- .../text_log.sql | 20 -- .../trace_log.sql | 17 -- .../202010_1_3_1/checksums.txt | Bin 260 -> 0 bytes .../202010_1_3_1/columns.txt | 7 - .../202010_1_3_1/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202010_1_3_1/minmax_event_date.idx | 1 - .../202010_1_3_1/partition.dat | Bin 4 -> 0 bytes .../202010_1_3_1/primary.idx | 1 - .../202012_4_295_59/checksums.txt | Bin 262 -> 0 bytes .../202012_4_295_59/columns.txt | 7 - .../202012_4_295_59/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202012_4_295_59/minmax_event_date.idx | 1 - .../202012_4_295_59/partition.dat | Bin 4 -> 0 bytes .../202012_4_295_59/primary.idx | 1 - .../202101_296_439_30/checksums.txt | Bin 262 -> 0 bytes .../202101_296_439_30/columns.txt | 7 - .../202101_296_439_30/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202101_296_439_30/minmax_event_date.idx | 1 - .../202101_296_439_30/partition.dat | Bin 4 -> 0 bytes .../202101_296_439_30/primary.idx | 1 - .../202102_440_574_27/checksums.txt | Bin 262 -> 0 bytes .../202102_440_574_27/columns.txt | 7 - .../202102_440_574_27/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_440_574_27/minmax_event_date.idx | 1 - .../202102_440_574_27/partition.dat | Bin 4 -> 0 bytes .../202102_440_574_27/primary.idx | 1 - .../202102_575_575_0/checksums.txt | Bin 260 -> 0 bytes .../202102_575_575_0/columns.txt | 7 - .../202102_575_575_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_575_575_0/minmax_event_date.idx | 1 - .../202102_575_575_0/partition.dat | Bin 4 -> 0 bytes .../202102_575_575_0/primary.idx | 1 - .../202102_576_576_0/checksums.txt | Bin 260 -> 0 bytes .../202102_576_576_0/columns.txt | 7 - .../202102_576_576_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_576_576_0/minmax_event_date.idx | 1 - .../202102_576_576_0/partition.dat | Bin 4 -> 0 bytes .../202102_576_576_0/primary.idx | 1 - .../202102_577_577_0/checksums.txt | Bin 260 -> 0 bytes .../202102_577_577_0/columns.txt | 7 - .../202102_577_577_0/count.txt | 1 - .../default_compression_codec.txt | 1 - .../202102_577_577_0/minmax_event_date.idx | 1 - .../202102_577_577_0/partition.dat | Bin 4 -> 0 bytes .../202102_577_577_0/primary.idx | 1 - .../format_version.txt | 1 - 324 files changed, 3761 deletions(-) delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/checksums.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/partition.dat delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/primary.idx delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/checksums.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/partition.dat delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/primary.idx delete mode 100644 programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/checksums.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/partition.dat delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/checksums.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/columns.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/count.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/default_compression_codec.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/minmax_event_date.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/partition.dat delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/primary.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/checksums.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/columns.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/count.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/default_compression_codec.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/minmax_event_date.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/partition.dat delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202101_2359_3529_235/primary.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/checksums.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/columns.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/count.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/default_compression_codec.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/minmax_event_date.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/partition.dat delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_3530_4606_216/primary.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/checksums.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/partition.dat delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/checksums.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/partition.dat delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx delete mode 100644 programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt delete mode 100644 programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql delete mode 100644 programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/checksums.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/partition.dat delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/checksums.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/partition.dat delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/checksums.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/columns.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/count.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/default_compression_codec.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/minmax_event_date.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/partition.dat delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/primary.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/checksums.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/partition.dat delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/checksums.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/columns.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/count.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/default_compression_codec.txt delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/minmax_event_date.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/partition.dat delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/primary.idx delete mode 100644 programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/format_version.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/checksums.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/checksums.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx delete mode 100644 programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/checksums.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/partition.dat delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/checksums.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/partition.dat delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx delete mode 100644 programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/checksums.txt delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/partition.dat delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/primary.idx delete mode 100644 programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/checksums.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/partition.dat delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/checksums.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/partition.dat delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/checksums.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/partition.dat delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/checksums.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/partition.dat delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/checksums.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/partition.dat delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/checksums.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/partition.dat delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/checksums.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/columns.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/count.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/default_compression_codec.txt delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/minmax_event_date.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/partition.dat delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/primary.idx delete mode 100644 programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/format_version.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/checksums.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/partition.dat delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/checksums.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/partition.dat delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/checksums.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/partition.dat delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/checksums.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/partition.dat delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/checksums.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/partition.dat delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx delete mode 100644 programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/checksums.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/partition.dat delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/checksums.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/columns.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/count.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/default_compression_codec.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/minmax_event_date.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/partition.dat delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/primary.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/checksums.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/columns.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/count.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/default_compression_codec.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/minmax_event_date.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/partition.dat delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_30_2357_1895/primary.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/checksums.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/columns.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/count.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/default_compression_codec.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/minmax_event_date.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/partition.dat delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_2362_3536_470/primary.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/checksums.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/partition.dat delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/checksums.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/columns.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/count.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/default_compression_codec.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/minmax_event_date.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/partition.dat delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/primary.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/checksums.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/partition.dat delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx delete mode 100644 programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql delete mode 100644 programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/checksums.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/partition.dat delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/checksums.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/partition.dat delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/checksums.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/partition.dat delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/checksums.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/columns.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/count.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/default_compression_codec.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/minmax_event_date.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/partition.dat delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/primary.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/checksums.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/partition.dat delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/checksums.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/partition.dat delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/checksums.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/partition.dat delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx delete mode 100644 programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/checksums.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/checksums.txt deleted file mode 100644 index da44b12b708c635f7165a2c8b8437d5fd40cbb12..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 14305 zcmX|IcOaGj_kTaneJ-xOcVuLhl@W=oGD2h*nHM+X%9e~oMo5Svqe3!DMOId1WklJO zQpzaz+GPBm^7;P$z6V{1KAsMK2Ltdvz67Gz@q_Zz z?VqOPubs%3R#X^Fo*H1>j;uTez!g0JHV#K&7&M#%PILz-SGmo243KC&ecX>u2_S-^7rXgRoMniX%9$dE9AAjJQ^BMy58h(CwFJ}iYr*mMP=l9({B-tiDE1Y5JYY;_7 zk4cc?B$St(gyTDrKvAQ3?5TC9HG9&IxhK7vpiEqa0V& zehL99ErTYA%-t4EU-;_2!DnhiYwsSj%fq=EjZjhhBUnt3QiFrSas7IPPGO|8F%?c0 zz(#<~dA#s!_;tI3=I*%fo8r{Tt$E6aix6o zF9EFIU#-6SHR#^{{#Q4&2XnBHZ*YXNODHGgHxAc$BRm=afxW>JCVbrY;)5_(PTx?a zouvF09AK0N_UOHMdizD9){3NGm7V37OZa5mJS#VDpQ~-_0yg2=2d~R3wgJpEZF-7L_(4?;P_v#CGL8|8^GG> z*FCs!T>#pFySQfi;=5ZjSc7f!9cCgp4)oCI-S`xmKlqGZ5x`dHR;9%` zowhcRpTFFu--iX$)E$5vwTYe%1g}ukR(~_HPDAa$q4Axfwp6|I*%R}J=y5|a`ja2Z z`^S09;zc)|al-2dp#L4*m64nW)ZvDu0?I0NL0bYe%J9Kc0aqcY5g{Y%o!ASI!Df)k z?Cp=0k}8@XY&7jqV|{{=QjzgLIM$DJrqqzFrrBXc8Ne6OTkX3YP1w_Ooc1Z`@Vb2D zY9ojO&>E`*kAKoV+E{a%_Dra9w)gI|Bhb(x5}gkNpoZET>A<8uJyQLCERz+wCd5{5 zwA!z<}9rwX#<7DHU>jI8o-y9k9@tO^lQ;@B6@rhy8Q~fe(fISYF>-Z4; zy}MvCC7SUz9zy3PxW2&y6Ka2W{unW1EpNMkJvh%`9HY0>W7L18ui*ZV(|nzQsSuvOpv}&8D#x5CA=V)PU&f2*}(M$vBHQH<+*IyAF9EWg640)3>1aN$rSM z%1PNSZpT_jE#*gtwGQDK9081m#Cbr*oK-eixS4>ze@n2#v+#^b6W|g8Ukx36-0^-| z4t{u7qE9d$Fth_UxF4a@1ZYgsbpf~mp_T#+0L#PA`vL$6eeMJ3AS`?fMt@*3K7}xh zp>O2F`}a*Z0AL(~hyW-aFmz+tVNDo^`lkz^YXeM-Yry-gw;zfXzTNN=XU{WC@7=rN zk(aWJU%ybap}d`lsZ=7HygI- zwM$$01bm!#Rdz~lNl>J0>|O>KbXhj&vK}JGKmSg%l%q?2;bx3y@x@|+FhGK#CC-9} zn7PWkW1+EZ>`I?fC|G(vm_T3p&kiMih)y46_Feh1$CUJ``qu=6MVIA@TY+Szq`;Z$ zf<+5rsp2#f-^c8LmYa`*@81klnT1V0EiegdRZnYw)*{?<)(WUcKb%iL4r0^%LViyZ zuzDguMcE!8|7ENy@uWeK*ItS}YD(Jefm)A(AuVGM2PZq!)@&Q5=H3x5zWcqh#%s4# z9-mIC`Wtt3Yr!5&Gi2?w$cf1Kw>vw*(EsA@AcKi1L)DexlSjF}JrT}-3fhryV;=(1 zhu}Aa;e~l9DgU7i-7bt`jq^FYFVWxUEnu+6u%W-Be`9b4)aIsm54=+$B?-Cx2%i0| zX|^)d`0GNiTy>ZA1;IkfCFt~9vyJ6Q1sUtG#vr#4I~ylJiN>iD8VBiC`Qx1vadlDe z0sp^MxeeN9|r5&Dm zb#1aEKj_$(3hE`Ogk|IpgGJzUYNb)DQh7kZ`@j1_cef0g%zos!q@?;jjw!e*dMc>R z4+Aa&xaY51zQfet*0!In$awXr?a`RNo?fmMyae@TkDnY&a;Hv-*Qpav60PyW?7I>f zW0QRNbLmSpOwWfu*L?UUya4S2SmHN7XYiLVHC=-$ISxpQ_?i}O(|F0ZQ%6QYwLN;D zxELIFKhHYPQN(^btXpG<#Q}K$n*%bGf@C(i9J%*v^T@EO>0&?u8zhSYRtY7-L`BA4 z>wE+g=YDw8?T!*Ml#pzRTjO-! zhT6w>&gam5lL}I*1=usxWbCi}J?K&1J*BJKUAf0=f8g{RZ;4+-4bAC14asPGlcnq- zX!oP>;{0m>qX1)x8lnwNj9=p_d>G;b>)yYcEUOa|Niz|)bZ|H;xGyNIVzj_A(*zHgsJnX0KDFC z7enmv$-vgBH;lZ?R8t|Hfg{d!yg3;%3JRZwD9?=;b4> z^AlL14VX;2t>x{N5WRLZXvik+;}qk+m4|SC{lgd7O*6jEDw6#5^e>sqh zv39yH`7U|Td59{ju5EHw4$zs=3*vn~qj;4#$NrU&ZNX0XshpTF+yCKhkyI55^A~9O zo}LQ1ID74QxoqatQQm!+Cs7t2i+&mBNqzI8^#p~rjrSU$)%N#sz*7Qx^d7qJ--N;* z543KVy}FB(HeUL%8&P5fR$<-+qlKCKdxu@F4f^lMQmoMZyHXOxJ%?b6&p#&nH%25! zh~2jKfclh|^Dey1hx+ZO#9hWmGj`7lMN;p+HG5mb^7$1vM1IgE5@ySNjT0%0iYx)l zl~}36G^pR2qV)W|k)Fysoh`kR$N2nGy5ND$2mi5iy3D3#^Ybb#LU;B!5^J5d!%=e{ z4!&-N4wr~Nlqk=Tr(KZ zty+T>h~e0oxck;uT=ebwZ_S^QF}DVS1TWMRk@hW)S{CM4JsS5j1GWoV4lK%bTBGTq z-&w)%ahgG&sA9MiN1yJTHGuwEB4hf&*;4B=l3ebuV`73SZmodnD!_EYWSrdJa9__6 z#k)=Gu1cPtuIx{@3Uj>II*lW&W4$x1hR+q`uG0;&1iy zzAN<3|2DP^>T*x<7n2RW%3#ks%$rQm$Bi&Cg>`3yLnLx7Tn&813>PtSCD9a z&hsl=DPOFL3uRVUwwh)DtU7p<&O3PMg<@p!zJ3YRJ3LP&5-^Q?yZp3;aXMkEHZ|(W zWIZT3^A~ngErzRaojQEXK+TwARPG&{8Va~z2U8O5c4x|O;*LVDrLF2WH)_Bw4vc*X zSMV~O=yt0ghtzaknn~#r#wiE7dKA6f(BnfZ7~FG<5}mY6I{~lHx8u7CCU~^a3dRSJ z8Tq~)nBa`Aew5W+IhhGpdje)1qJzgdyn~Y)-dDq`l$neyK4F~V)_vRlhY;2qv^#j4 zbKLD5<`1l1{_-pNX^;NLdm`&tvPst>b6iDU=;-#+IVlj z1(b__$)fm_buk9cau_UsACphWUQKicJ1|CFqT{RN1^>aU8;z|j2D)r${zgkBDAQf1 z#Uf<~|CRjk%$+=z(;WcggDhc+mzX$C-o#aG-jtDHe`PLo;VU0@0}76A2{(4N_Me$L z+XPSd-Rl5gKSN?>-Bb?=HHyoP$7=-p{iYrO>^H$n-$O@FY5Dv3;JuvAQ}1!zxm?XF zt)L-x{kBT6&)VhPCQGPe{WYX%c|7jr#Ob>z(IT;LQ3R z|K9xVWt<9F0VvzWv7NT5VSai*xQ;JtV+=}O+=SO-ZEFIy zp31|`^GDB%nD;Hg0Oj)kqJ0Ts_%239*xTLTF9#2vscfh)hJgsMH^uu996Ta1WaNFL zu<{G{FO@qxdd^vrl9AGdmk&a#Q0m=b>dqg`OfRZWw9Uuas_X#Fzgt~`&0hbKp=V4? z&z+MLeljFu0(2mXFQ}nf5~ucwyfF2q0KXw8j}M+UVV}{*Fli%?aPq9|*zt`o>Ssv$ z_ZheTqr)qd!?>QoQ}tt-kMvn25~!mu09rJA=rpmBkdjh}Z6n%b)ggv2K)u$g5 zL^OLyxKU;uJ}Yqk=-k0CTF-wTru#=RNHXfoE;qf6pH@@CECTZ_^4BSQ_3)8yUv1&bxxTW{HI->1LHAsyn< z5SY}U0}Oh`MtHwKqK|tROvVmx)IpWiw*^hIwt*q9$RKl-%$x<-fMrSAUcq%XhS2tb z7K3wDsf;K^N9zE}=G~mGr67sWyNq4T$B|%nK&|KHM6_ceArtq7`@k1ewyLr8QhkO? zx>+l*i)u1+>=J!BTH$h;507TJa%F1J90jlm*u;crPiS4YIv>%Tqmf(wOYbY7+e9g2 zNv?ike%|3D?hG#J_j46g4Sph6$~D-x>|WLqUYTF^tsxSwa!~NUTAk{g%A?DVRqLg= zs8c58p5x;It)8zY(LailgnWE*KuP7o8H(S`9H~czsh+E(NtU1p9++|Qp^D?9DBI3a z&u`uSxA#w*MAwxtfV;Tu_8~A-`r6B@d+C5)&sUom=;c9laIS_)$cHVsfNDj~{l)LE zm%MhcrlBp7mT2&hz3C}veH4mrVLj!(^NPpqqa+DB%cE!f>LfM8dj_jWA_slMQ=n-` zbR-N@laQ|;ndj4w56WJP0;3KgtO-YvJkF(*uyXW(395OfYL7dAy=*?bgB z)MB;gF681kcd1e7>MY7gH4v?VaY7xV^mszq^5#9Yd|+?w;N^@*Us~lHHVb5X;ZRZZ z`-`)=i$KkJ`t`HA-GO z#)TY{M-MC!%P)wY!5pC1cL;D0r6nOZ+jx(Lv(=sMf*zC~n6!G*67snWJ#o@PiF>Kl zoWJb6oa7C4!t8%1j-yMn}Jr05AJLpF{Fbax6VCLJ>15~}l>a3U;R?=8 zoEV52jk|2sL;7-MlNRL@=BPqw2z$}7y?1%t^k~5Eljm#_;F)8)iz1IU!3Xc;_nK>u zqX9BAdU2M(aZgo!sqp!qrNwM0?}*TkWU&}Atok9V-;N8`XB>Go02aOs{$Qke8t>-Y zF<=^O4woD{HNXS7WdL%7b(;%a+$)IgcSZ8Bx^Dck@6&$VrP*!NQbzQ39^|tT&k3hD z!e`5KI$$6IRz#5H-&?mJiq_X0mG*wu@XbNg$b!2c`j#AD$4%_Jv^RWu(?LbP z!35%eZy{_fp8!apfH^SR93vDyp9Nlhkat~hkm-AHB^+r$-#ld2=)Wn~eEByKs^}cl zWMNeX3=V*ig&2g{VBe!}2c~wz1LX;N%HKOYhnF^?FY;^zKKs)TRA{$_J{>M^)0J}o z5SqA9n8V~5sD>>i-;(6n4yL?DIk^ki;(##GIi6#ebLg;q{+q~^N85CX#c5iL@|}y& z{Yt2~@1gIJ^YIGrFYjbl@cyi4|0m>Z!9l4b;Bq`c?Q3`z;VyaD!SjI=X zwXiL2d-=?J%@=g-FUFuIt93=U%cpKz#i}*5STPxBU*K>7FlxsYO@QxT9#dTHw4LnE zpX>e3_I4d`9R?95{tjM#1ixT_U1QI{^ti-I|NiB02=S@d271!Kz{8WX4HVx2xzT`Y@KeOKmq+@{VvBb6#hTWJDp)3eX*b*qDR?Fk=Pr z&=$uOBy->UwQERq%{ZpL?p=wbFBD5dk<60((=ajKU!9mz+!QawjFJ^p8-wT9suc%5 zOg$wo+7;*tdKrUj00UG16GzA$usc`1{x@Gx@A-Ip4EO4%<8kU`=q1NT@}BFpfWxlO zHiN0tRMP(^j$!n278l05?n~gSVw*W%=HLLsS4v@Hw<|0pObpA388-g7S*z;3cQ_N} zPnc3AiVfKBk-(*qk=b(x1Y>e%JRZBEEgQ5ivW}ro{yZ?@!^#tPXSBe4nHmFFg@K0Q z-+G#6hx>k-Kx)kxvwLfo2xl8h2}#%fDfH{}EvWG{Rq>TK(!X7zw+ELi+Ku<=>nerq z^!N1UFoPx@c8%yS-^8i_EsEl;Oe7>@g0fzpU7XvK?sea@Lo9LYSqu#*o}4KcGvn{k zhdiV;gN;8dkpH6=EzU~cnB>c8Vwgx zoII`$RLxvn{XLFEQJyu;WL}}Q+q07T$l;pxG+SL~Cj9YDz~;HMEzvB{YW zUv=A_S3VJcfl9i`>)|AyJ9Mo@UybR(`L^R0ruThz(k;@oF8`H83c&{ERh!R?citHU zW)Cp19;C_$SPuat8m*7Ml98=4@#7cep3^e0zEBbXdgaV$1EqK>>vufW8;!d~hZjT= zj%@5Z*b1i9cIO8!xBHUh?xCl9%(kBayEs5D3&Bq)_qe7upaD^lx`(qr)tWi`7Kn#m z4?x|;`i7w=Jk(xnoUIztZSQw~JNjQivy18e@lc=7l1;2qJ*v5fAU+7V4+CTnEh)!8 zAYB~SK(k0r0ayBo#e_p z)AJX7>3Bdk@Iz^S0Kp00Ou51}Rt>r4DKHnwILTOY)k%r>QJse76AYepq+WW$z^C}Y zajc>E3C$v>7ob8@q&pfGKTQPGU*A8vUQt>b6W`GE5M%;MRQddbuC09)!`2XS94E&4 zVdI#uj5}Qe;S;(x>@2B?aZx{|eljyru&U_{uQ%EUK;6L)tU*`mzu<@q$7BAOON=Kq zNGTnJIZW+%S2*bD6rL@Bq$-(U{2UMCGA$h?|a(4S@r@Ol>G4D z_9Py4xr(=M9!*_Lyy6|xtXID6nGCQH1`nVV@Gph^E+i3p`(IJ!wVe^_l1Yh(Twqv(JHqcOG+UG^&EhgfW^F|+_G$JCRlDu) z?cf~64G!ufb>rtAHlvLX2OkH|R2Ea68MtqSg2ue4B}CP zK&@45?GWZ!pXEJ{m3^sZ6T_e#bI*gpcOV*wNdE%W^DEQ8K+fk0Pinf7$s8@{0S`{$sF%u%XLwT=G4 z15qr-Q_2s@#dhsXaL&NWK z!LuJ&3NtodpiKvL4$Jeex7yUBc=de3dIfo&{jceO!*tFtaA^U@Yj~yc19e*hfCD{r zU%az3*gJ%hzBBk>*}pnj_d8xGEA~^pDgGbYU$6VuQ|5a>Z*DHb-9JsU>}US8E?GaB zTgSNIIn@D#O82Sqh=I)(ACc!0yosZ6(~N7ATYN|QJ|Kf z3a&lQU*)tG0lD-Rv8_d-LDVMvwbnY&EwTwKn{Qtr&=VedlkL+}aje%J0HlU%58`Ai zAHf|~a?Sew@nLrjUeg^l(}mB`d zcoSmG8EukOjGm>EO>jEs@vLP4#-Qoz5~`%!egE)0AM2Vlw^xX6s?&xzpt;KMFGaj* z#~d%PvW*G7DRg6M>Ntplh_#k|zm^h%SfQ0Z&CS|yR)H!mIeFA&e3dU`{BJbP;S z`ufPHHf@~J^AvOaxnJQdV?6z~IhUJ%Nl!HRpI04&1fXNjV zAmHC1yWG8%_C}Fu!3r0QQ;Gh9M#7_Cu~5^>N6I3e_JRiU!YZFn9ieJ1MO*Bsy6FaX zF;juZDm*?ER$miz*v@Y~?E-jEsy>Ax)e&}q$^Gm(cj#sUpBz_5s?^sxx2TU>mPLu%0pe^o(mXJ74 z!V_y^siO9zP3CQ!nkQ8MrU9#R=9!F-S8hbd={rwk%07GiJ*?(Ks}Km3TYhpBcKcp+ zvufQTrv3zk4p<1U2{KBz6K(Xjz@HEwO_rqVaLf{sVKZkdHD7wr&nW5+#oYxNDuRyF^_Q`6=d;E8>o4^u0ZvhX zy^kK(e7aqo&%3pv88`q-4>tLb@atmo~Yc9x&XpdW55mx0wu?Gd8C5)@a zy49fZ9C7&5soKdy_5FZ03BW%o$;iv%urn96b(mLt(j<|R;fXrcp>WJ9d?(g3N&t6f zYV$|gpVz9+X8-Bu&^ZGTDls#>w*H~3#9O#R#sX-168-Sql1oTs(jv4eRA9EsaV4Kc zvb#4a?a2^o2`*)4(K(p&O6YJ!-3EU_#(sZh6;4|n8Ckfy!^9+Z3^e6GwM%PheRq4$oZRCrS@wi(^~-+MFN(DY0a(3 z$tzZH>ORxcKxz8PAFyPL9Q5#K6vVzbKtg(>1Py<4tfZCTY_g@|m2+}(6HV#1;rVga zlOjyKh;qW^;tOJaJbw}FHOfvW8{xoMCH}Kf2_I#$f9q`mF<_#LC%C$$qdH${mELWr zF+q4jam0}B-sz|KLaTIS4ZfwedD{LYzWK8cyQt8+CrfO2fcd9zZ$Q@$P_EX}rbYO^ z?-8OceH`2gj#?6&L-@!@Tc!S-o<*sXkiZC$zTtk&#up7d8liKJ;f^w`D`ECupNSfX zew2FnuQ+uEWSZyt$#bsK(;OlV6Mb{fv^4UKg+^>9dHGNR=u@?M+6KEIwWAdcGHt+}I#00ii_8S6A zn9D!jZG8MHuF)1E{vtH2{{&zZ1@ML_h(b1b))zNm-Pqx~QX>qRZf%{#*~PcsLt9NN z*e5zO&chR(_1`qgdtc~XDq=!I(GF$KxhcmwSiA;|+-=;bRHwE4{xRq4=+XB!6Nv(( zlbdC4R2COWGASZd6qT4A_;D5?ZhrUX}d5fbK$=1#?!zQ3s9d849*5Q}pHP3RCG+^kJJAc?I4 zRR%%}ch3Ig#G%7|27p9uoP8IOmKxuB=zKzAYnJjA;4me4pfiddV=yLH2x>tU$h2vG z9zC7Ql}OI%Gsy)x%?*EoZP0kW{H}CnH(<)wmpk)l zr~F@pZl>OYf*qx9Pn~wL6DHh0u+(_-&|jc$rbg$WI#f@!o*MdfuU(I4wqz`h!&zP& zy9fQs2(LGmWIGf-Hs>W6Hc9=LcgRH{d(dQ@sm7I0<5mcQ*@KdGYB_%;@^4z_eAN`P^e#hJ!-6hp2`4lWKpn7redaFf z$w^UqaXjEMB|4!E1X%{$9X=A$e;teexl<=zME~F>InSV2Sb9%;dymd zu`|@lVl*L8>PKTcGdAA1m~$E}o-3wLyf07eYma1%$N9>Ez;9cJ6wlk&gdl3QfKG}< zT^k)yqD2|Q${(gzBFbMDT#0x$FUT=@&bmht4Lz1`x}q1U(|;5ll^{;wJBI!@%qBY* zrx#}w%vY*EZrwW@FnlFd>4 zK(Rz2v10(*ne97t)V(p_38Ph@zthKlN25<{KLGgGEnmSL9E?9<&h)cE+=6Mn)>x(s zFk1q+n+lz)`oq2W({VSWuplDaN_U`{jQdG;rGi#Jr{hd+)SNT1r3>q#!MR}f7<%>- z!WMe;?UbRLwEjW%%y)O|YEA=oQGm>$F&wjU)`Gz+O1{=Cu>6gbX+q)EzByzMp82Ix zVVD2@Y}sbcz2U^PjsJ$t)@Y~=Kcp05l_C>x91S5o*S`a1GorTvJ^=60jn;-8A9??* zX4Iy9!Y$sln{tKB4Z2*=G=5w}PjHy4)}9qdBNc-L(U_{@A9TnJ7s7&o*IK&Of)DfYP6A5b*5dSF&# zwA#F#9b$0v8dA`cp;;(Jwph+aAPCSz95MV(VI{_|gL2X zHBra+MlD%?mEz*P5uIQ z;d9M-T&=`jf6xq%CqPi;#|K^2>U|Llx7C?M zVf#O6Nau<3KtLMM=cC18{Fdse8)39R6n|CE9{SBZMe-yDquDQdLC`IRU+9N}kIe3F z!-j>RKA?dkz#fKOMJBO_Yg>Aps!PmCsOKBtN8Om>4b>jcaO(u-jgdj9*>CcVWub1r znl-mT=u!0swQ|B4>3emek7RZ}j!P^)pggDtaBwKt#(bUMc> zRXMTwZ!*-4)XSZ)sx&Dvb_PPKfC@23J)t>?{x@%jcVWrRhc?4DY&uUz&OqN@$H3)v zs?EK%IfgatMs#BL5d~obKnF*mGsJq{T+Ek9qTP9}0yRS=n^K?z#tbw5t17bEU9W$4 zetKf2htsrV{MP1{x$qri1Gdus-qm|VEiNiMX6Ne#<_V5KuwS5TU?ka|1#~?o$CeHa zQIr(^+6K&r0c?yaWDNxZ?bLA(^J~rBRch;QHF96T-lzNnFBN~j@=QjL>)^cMb%n}F zI~Ek19l#dF&N*Wlwodt9PV$rnLpK!c024X}c8PI|yLCl+)$ncKO7b>O4_jAt-Odsh zl%KFbshF|tWc@lzTGlv4Kni3M{u>OzV520Wn?A;V7*#LZk-Qv-0URO#+eS}DqN>F2 z?da8*eLYpSIQ}@n+45}A5%k1G?}e7)S6{!J+rV3?c1zs(uWnwVAWL@EN2)k$XJvmF z4~Wi_0rcpKQt6>o+I8-qEdtq9*^~rn9{wb%odblfc?3$Ujm4-TOun@rlLXvq`h1z_ zx#9iZ`a0&bRpzrUzXN1F1)}spxq*MpDr^gyXKJTK<8GN=x-K;Zr9+JjKI zs~O9gfCyVs8AOV;zpbJ?S^SLl=+#@xqhk)=PyA0xA5H1rH0o={E(*Mzyi%Y^7E>GIdIK=T9$&@-D)WG04p>0^C$zF|g-{_l-C{x74%ZhfasLtk$j^wDg zy0tWTrPYd+GIQN05k>$#O1b}zoqHuPkU8tx5SEWWc;Bb{P!X7Dy`BBB!T9T~a|2Snme6pM^@EEEqp=aV~s}@HZMu?(fg#`+ExklO=#c_XWjwU9^nn0=`<1b*t(JHBZ-Y z^q#9mJtIEi;A6SNpF`xIgg&*pmO8ckcfuF?H*-OTgnopkaZQ~k-TU-eg{e$KwFQ~-&_U)?r}1^8ZjRJ z=AaxT*!oExs=RCH91&;ET&8PM#O_ofxY1!l`)|&Q3=JIHJer?%JH5(Ie*J?(tG)^b zm;&TEdZ#t(*xwn!+Y^`qL$mS%@A6ye3D41m^;BdWH7$(boCPh*)~<@eCotH5!;j0j z%}u{M6aMz#_`&*P7ri%x1rOaeJMK|0{_#*p*5u zO>Ai6?T-{n!WY*S%l!&!6|*ybQR3AE{8~(z%JaPXgal?xRNnH>Ri zTJT?{|Ck|iN`owa#ySmG6G@Y){%dr}5v>sW%<%&(#hVG)^K31H;dP_v3;W)RIr(T| z!(Pr}o>ijEGU2-bEufAAuq^cwvYZ`@=@-t`_qT5ycqjHG(;8HkPL zjpoP}v_mU!vR`a^D2-})`X5h-0A2mDs%Ym0D6`XkgamR7Z zC9-~3;ITw@hdB$HaZr_-vGV2b8snjU7p}FH3Yizz)B&p{-Vr5&L>~<&KSDsdD1-P3 zwgdYKolZ=Hl?83O`YSOypG6ofPO)I~{tdJ{0=n3fAJ{DkUe3h8{V(45vHiQZ%ETa_ zp1ig5iBIGdJNKT>>%7rThWB9Z#E|zV95>B=GE6VnMNc;VM{_qY&mwhWHq0vSXkIRO z#kO~w53r&59LxsZ(~>d0u6=L(dDJq4AK51@EqoR+#S+lDIoVjLDay?I9tVI7jCAIwswG*MrH&g_4 z-e`Z15Wqnak#ctML$;5-)l!U#J;l+&h>2}@E(U14eF&ZoKEYDx+eKWnZGXh}%008% Na{YAY`X7sc{{tR9x848% diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt deleted file mode 100644 index 0d87e8d0e8d..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/columns.txt +++ /dev/null @@ -1,183 +0,0 @@ -columns format version: 1 -181 columns: -`CounterID` UInt32 -`StartDate` Date -`Sign` Int8 -`IsNew` UInt8 -`VisitID` UInt64 -`UserID` UInt64 -`StartTime` DateTime -`Duration` UInt32 -`UTCStartTime` DateTime -`PageViews` Int32 -`Hits` Int32 -`IsBounce` UInt8 -`Referer` String -`StartURL` String -`RefererDomain` String -`StartURLDomain` String -`EndURL` String -`LinkURL` String -`IsDownload` UInt8 -`TraficSourceID` Int8 -`SearchEngineID` UInt16 -`SearchPhrase` String -`AdvEngineID` UInt8 -`PlaceID` Int32 -`RefererCategories` Array(UInt16) -`URLCategories` Array(UInt16) -`URLRegions` Array(UInt32) -`RefererRegions` Array(UInt32) -`IsYandex` UInt8 -`GoalReachesDepth` Int32 -`GoalReachesURL` Int32 -`GoalReachesAny` Int32 -`SocialSourceNetworkID` UInt8 -`SocialSourcePage` String -`MobilePhoneModel` String -`ClientEventTime` DateTime -`RegionID` UInt32 -`ClientIP` UInt32 -`ClientIP6` FixedString(16) -`RemoteIP` UInt32 -`RemoteIP6` FixedString(16) -`IPNetworkID` UInt32 -`SilverlightVersion3` UInt32 -`CodeVersion` UInt32 -`ResolutionWidth` UInt16 -`ResolutionHeight` UInt16 -`UserAgentMajor` UInt16 -`UserAgentMinor` UInt16 -`WindowClientWidth` UInt16 -`WindowClientHeight` UInt16 -`SilverlightVersion2` UInt8 -`SilverlightVersion4` UInt16 -`FlashVersion3` UInt16 -`FlashVersion4` UInt16 -`ClientTimeZone` Int16 -`OS` UInt8 -`UserAgent` UInt8 -`ResolutionDepth` UInt8 -`FlashMajor` UInt8 -`FlashMinor` UInt8 -`NetMajor` UInt8 -`NetMinor` UInt8 -`MobilePhone` UInt8 -`SilverlightVersion1` UInt8 -`Age` UInt8 -`Sex` UInt8 -`Income` UInt8 -`JavaEnable` UInt8 -`CookieEnable` UInt8 -`JavascriptEnable` UInt8 -`IsMobile` UInt8 -`BrowserLanguage` UInt16 -`BrowserCountry` UInt16 -`Interests` UInt16 -`Robotness` UInt8 -`GeneralInterests` Array(UInt16) -`Params` Array(String) -`Goals.ID` Array(UInt32) -`Goals.Serial` Array(UInt32) -`Goals.EventTime` Array(DateTime) -`Goals.Price` Array(Int64) -`Goals.OrderID` Array(String) -`Goals.CurrencyID` Array(UInt32) -`WatchIDs` Array(UInt64) -`ParamSumPrice` Int64 -`ParamCurrency` FixedString(3) -`ParamCurrencyID` UInt16 -`ClickLogID` UInt64 -`ClickEventID` Int32 -`ClickGoodEvent` Int32 -`ClickEventTime` DateTime -`ClickPriorityID` Int32 -`ClickPhraseID` Int32 -`ClickPageID` Int32 -`ClickPlaceID` Int32 -`ClickTypeID` Int32 -`ClickResourceID` Int32 -`ClickCost` UInt32 -`ClickClientIP` UInt32 -`ClickDomainID` UInt32 -`ClickURL` String -`ClickAttempt` UInt8 -`ClickOrderID` UInt32 -`ClickBannerID` UInt32 -`ClickMarketCategoryID` UInt32 -`ClickMarketPP` UInt32 -`ClickMarketCategoryName` String -`ClickMarketPPName` String -`ClickAWAPSCampaignName` String -`ClickPageName` String -`ClickTargetType` UInt16 -`ClickTargetPhraseID` UInt64 -`ClickContextType` UInt8 -`ClickSelectType` Int8 -`ClickOptions` String -`ClickGroupBannerID` Int32 -`OpenstatServiceName` String -`OpenstatCampaignID` String -`OpenstatAdID` String -`OpenstatSourceID` String -`UTMSource` String -`UTMMedium` String -`UTMCampaign` String -`UTMContent` String -`UTMTerm` String -`FromTag` String -`HasGCLID` UInt8 -`FirstVisit` DateTime -`PredLastVisit` Date -`LastVisit` Date -`TotalVisits` UInt32 -`TraficSource.ID` Array(Int8) -`TraficSource.SearchEngineID` Array(UInt16) -`TraficSource.AdvEngineID` Array(UInt8) -`TraficSource.PlaceID` Array(UInt16) -`TraficSource.SocialSourceNetworkID` Array(UInt8) -`TraficSource.Domain` Array(String) -`TraficSource.SearchPhrase` Array(String) -`TraficSource.SocialSourcePage` Array(String) -`Attendance` FixedString(16) -`CLID` UInt32 -`YCLID` UInt64 -`NormalizedRefererHash` UInt64 -`SearchPhraseHash` UInt64 -`RefererDomainHash` UInt64 -`NormalizedStartURLHash` UInt64 -`StartURLDomainHash` UInt64 -`NormalizedEndURLHash` UInt64 -`TopLevelDomain` UInt64 -`URLScheme` UInt64 -`OpenstatServiceNameHash` UInt64 -`OpenstatCampaignIDHash` UInt64 -`OpenstatAdIDHash` UInt64 -`OpenstatSourceIDHash` UInt64 -`UTMSourceHash` UInt64 -`UTMMediumHash` UInt64 -`UTMCampaignHash` UInt64 -`UTMContentHash` UInt64 -`UTMTermHash` UInt64 -`FromHash` UInt64 -`WebVisorEnabled` UInt8 -`WebVisorActivity` UInt32 -`ParsedParams.Key1` Array(String) -`ParsedParams.Key2` Array(String) -`ParsedParams.Key3` Array(String) -`ParsedParams.Key4` Array(String) -`ParsedParams.Key5` Array(String) -`ParsedParams.ValueDouble` Array(Float64) -`Market.Type` Array(UInt8) -`Market.GoalID` Array(UInt32) -`Market.OrderID` Array(String) -`Market.OrderPrice` Array(Int64) -`Market.PP` Array(UInt32) -`Market.DirectPlaceID` Array(UInt32) -`Market.DirectOrderID` Array(UInt32) -`Market.DirectBannerID` Array(UInt32) -`Market.GoodID` Array(String) -`Market.GoodName` Array(String) -`Market.GoodQuantity` Array(Int32) -`Market.GoodPrice` Array(Int64) -`IslandID` FixedString(16) diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt deleted file mode 100644 index 7821086ced1..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -1194218 \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx deleted file mode 100644 index 0cb239d932a..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/minmax_StartDate.idx +++ /dev/null @@ -1 +0,0 @@ -?? \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/partition.dat b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/partition.dat deleted file mode 100644 index 9c176950d3f897f5cc9714d7011107a21e47a229..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmdlj#LNHy11td1 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/primary.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_1_6_1/primary.idx deleted file mode 100644 index 4731cba6d463bb6755b51e8dc1e552175657502e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3240 zcmXApc|4SR8^#TXtXXG}B_i1)yOZ-+24R?CG`6uPOGWl!?5||YQXwKsgpjgK2qC-3 z64{AJmV}cm#ru1Xf4`sWzOL)Of6wQUp`xPVfI_j^Ob7Yz8obxM9TZ|mW;$%TuYSIHA}GJR&WA=e{u6-x?-dP}2dM7}E>c4a3290!)z5`(S0Gda0ixqV+<0 zFhM~DtxX}yI=4DnrNeH)ga;zIK43b49!HIsrDJ3=;jBBkwo~{!AI~3Y$1;NY2!t z&(kvwQ*BS$lA5k&hqISb10P~t;a9Sy@BK%@rI>?%sew-hk-w?w^GFQseW|cWM;buB z_hzXw#gGaG8~5!Gb6}Eku-xNuQA92Fm8V7A4h?YLp2qbI9N(c?{3ph_bN?@}DL6sn*VmH|5JYWJ|Cs%|w>5C%1|E9d%3He{+>1f&k-qbw_qSFaM z^@OaAEslNNV4^aC2^aLhtBw%d6Wy09izlBDm6vku%OeZxQBi{=m>{9%M|KKh9oiOd z#pm2l2s3@^v-urvj(2qYelS5nX+LU^-U^ks`0=mIPKW`aCe{LjMWH8e2A+`8=JsFV z)yrM3lXI3Z;fDTM?+rbzF&!@cR_f{rS$7FL+!%^c_Vp_Wh6xW8Z%q|(v!5qeIW5EW zgb0f(9Zjrv_{$c&CBCs|0$W^OJdm$lWny8Apf~HDWCqu`@F)0ECO3x9$o3yJOpwrP z6W!vvdV|KZ?L#D(@Ibi=T6x~Qp4&_d!8XieEMRJAy1ybIXe7DCTH-ectPnLBnw zzPF%s`v2Ns0a;J^>~l!3MRlWZ_}R)H!&79;<=0W@B+W3K%Ur@?zkRQC|)dhX>4 z??&02kSv4l_NeA9+xwGujbOqFnXymFX1VK1R|keUoe=gE>nw|AUy&wne>_aMAl=z3 zh%dN`CyUv}rYA&nNzyCo;r`Mk5hJS;@_udN)0tC?rAY`UO_(5|FTs*%IeJwm&y-*G zCuHT=q*{q;PpMMff(#QB6fA@kdQ9ntWb(}Zn?gZ4#3_gZB z-+`}jErt1d?;8c_9bYy>-6eQ0MssOvhtKDSuSgAEq4cr>&C6mX#OL$-u1gmldt7_L z4zgCJjB)a@wfkDS7cF-5IKb3xmz7y}2CREjT@wZ-JkaO2^hGJudigujN``T?C@|%( zSg(jo&aSjCkqB+0;{h7`IVoY>B7d}E^h&6f3_r+^sp+x=ai-aFz({LO<+Td+GWqn0tpzGXgS zf?~~0xDGV>XJ+5bRho>2hQe?TaO5WwHw~+PjB$iMT5>CRydq&@xSjD~U%C85;Oz0HG{N8UYbG1{nxPUh z6=?SP7Z4LRrud+&4Y#SiG|+jW%=Och9~oonkR2w8N(UO#&^({RY*~h6QiT6i*DUZ} zNOXwHs&WptgMQP_At)OVPxQd@G-&0+6QWLaN8!0k17KJgmA-zXH4e&cMHLH9tqXmcHeR(xL#;EEn!gNpnj? zl9mi}0~-Kw-tSQf_ggg6++FfE5o!Wi%c)i8gxhbsvr+V}XlA_wq&)S~az;yToyxZ7 z?cVd9;9~Wsgmc!=1{p%3xp+m6KCqV;eyXZ_RV|#(n9VM(Y5=_8eqG&O%SIQ4#$Vh? zh;0~TDUu0U>@Rz=$X7^S{hc2{XS`q6ajV8|8w14(Dzc-FcFnJCl{a&Kl++4tQ8hQj5#JZY5 z(bpF((bgKkTSqYO`f_C%ln2L_X&RglsKUsM7vSeP`ZlQU8E zt#8)#+pOY?lmL+mUz^gd__>A0?PCQy)Iev$mQntIyvx;Q#=Zf&NG;%8Ozov>*;h0^ z^XP$rFXt7|kPzprZXQ7*&+Cp^^Vwm6ro@Tw>%9^tvL8O#!{iT4y4#8vrUaYn94;Jc ztH$EN?hzqpnSRK~?74Lf>Bx;*Avhs=_UTEalho_@k|KYzEj!S8WyA>`qlR2)pe?v5 zsNw*89Ql5isS!ELgIU|Hq&HkaL*{kcnvWcY73SqpL9yYUz&U|)T%;l|B5QIS5!P^I@$UsAUP3L>a-xB?^#1uU)5DBtesz*y@A8ejkcImwOJ2DoK zSaCzgI?L3=>V_)a^MOfV$^}K)B05@cUrSNNw-GXc^S?&*e@ymNvQ%+DEQ0Pm0aJR~ zXMPd*coR6X#jmzo#!EQBcWH zZ(8$`C=>MaX$5$T!g-9lnDqD?;cnt>lf_rSc};WXB2xMoQ!)^UN4vfT7t1TPOivDt zk~21T44FS{1e(O0#2u}MWZ#GSyFmuaZ6G~~Z^9UDy}r0{Dj@W?O9vnn#=Y?|TSm1< zGnd_0^}0d2BC|cMmmMj(UprDqL2Cf`AjB}M;*2Xx?$S9%Aw45NgJ*bzx>}%qIh_lM zA#_gw&5ProqD@_m{k)ebK}T$#L7xjaAL?U|2$i4jZ)1(tR=|5v+~VH4B9j-ex{Nvp zKHtG!qMJK~60EZF`|B?LqaE8o!$2vf>XS@Aqa2c@EiJtd2yco;qFOdDZd@}SV?%ok j(w$po#558Imm2ua8h8CU23d;MR=5E3ApT7w51;=7A{bPx diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/checksums.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/checksums.txt deleted file mode 100644 index ab58ad06ceeb0d11e4723395c0f1b415399d055b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 14273 zcmX|IcRbbq_kX?L?|bdNHz6e>Nk;Z6$|l-%ac!5b9hH%pB1(2e6j_m3vLcm~vOXj- zl9cYXvi)At_xFzn*L|ORUgMnS^E}VF4lYCocOO4bA2BB{vM0e;%%4d1arGh{6jNfr z$A+4pV9tNsH|Rtd_#W6Cep>|qR$~B`WUgYYx{m&aBxhF=(bz!V-j%eHfLa;ptFG`r zRWu$*OH3UcN=6di33DebTc)d3(ma_{vRqbR_{F9IC`_JYcg22YR&_RRnv+UjldwiP z&%DM_K&$Iav<4L1LN2?fgqp|6+X;^y*OB_{=7x3nGTNeFI2E<{+VRtF7G@p%ZaccY z06hw*OnaG1Cz#je+L+nPzdy9#V?s4(z8K*^?f0lT{UuL z{9~L40ZIV#sK=G5x8Pv!e*7NZ%b7{H*}yJfZ$cnB`w=)13T~~OVPr|A__YqUKbr3Y z5zlh$ls%Y34I1rvFO3GjcOOaX&};ntn#&&G)dA`NX7N97qeGlH@vs+OWhT-HTR))F zH+g|j`CmjJZF5seRusW9+LzZgS+v=uk>^kS&r;gouP-IJdtbd+!c6#GNUs1;Q$XdD zYqm$V@1W#{*t24=r;2DTeuo!lqzSWjU&|R>+ zFN#~6>zVayWD_-NsGQci`e}&p=Hk=Sj50R_bpa0b2D8}5?s99|hWqSCSAL2Ph*`7bvt>-`}TsM*A zYi#ixUcc3w*bVESPc-Q0R@8Bi)l~T>Kb!-zNEgTHsN`wOPxf>(sq4(|wMb+M(} ziVnq8(Nnz7a5(!R^87yQRc~$$fD*wRM(VIkx@%-Tk?%fgZ|wg5XjsR*+`0Rc?Cw%vjI$SM71CPsjx5 zU6#{9os=SV$Kj;5{C$r;hY-NFIVzS}m(s^k;lWWF{5+bk+W!vVi2=tBlfC@BU!oKm znDFBCOoWZhnG-{H{Hk&t-Fy7^q9fQut(wz>O}mqm7^!dVdH1HZ^J)Tsy1>;AZL*#B z=f7(CX=MoCmzJFh=ukxyFXu6YDv%q6zg9}+V16T}_J-p(b8?E$7_#*JCo(oSNq20e ztKne+J?<4YNyBOnP#6*?0Ts)ga(9f|DNO&#*>i->F251*OMp>R0@faEc1(01PDo=5~Nq;221NbpahnW%%+jv83SFetX)0aLt|P z0PbJD%Have-sg8J8b@D#bG_;-L?16dax-8(j&Z`j;s4+rWH@F`^dLGU)6VjJC_;EC7O#cHOVZ;m zn3stAT`<^6Yb-J}Uxw~pA1$&@sy7SfWp)6x+rWhuQU8#=4)z{;TmMpAH1ZV<2*nd^ z2xMoXZy7?Rg;7=X5P6nrsgRrZ8z1tp`(b46-{EpTz9C-rZ4F=R=cithjG(drXHPfFTS&gleCA8?J>;EWu-Ho`p`9jM z7#)E3dP#I(B5#g69XG9gQQEn+EV#HiB&i~zLHjfH65W$R+X})g)g1$;0os74V$;jW ziO3R{;-|l!-U8?42Sepn zdIN)(XXwCwfZ6~m(z%?R%$$4q>n$-S%@w9f5q4xBgsN0p+&V1A74In+K9xk9?2}KMg2|SB6r6iHhmWuIQ*qHV%5r8h|PR^ceK& z(&K|vMuzL>4$64ES86xVCMoAyQ&VAJ>Va(5*fk_h%5b&!~vI}Wmi zJe_K-o2F&CK90F{6=2Kgl@eA~J5fr+jjlK*u7kbJ+hm4{&C~IW?Ugo6nAUAvuS)As z0L=SXa9MDje>z!Al7nx@vU@d!e7UKrt1)%45&}CO*FJ`P7(JL5XVFnnM0Vc>b_(s~ zq(9t`0-7A3 zCB7#r`gi8?pDN)Vht@m@J}#yNH!m`V{NXQ%X4&JY zlsn9=Nmo>BZ}l?gX604}dL|oM>9aVnBUp;Bs(Qvnf7{$VGOz1qdqlR6PI_T6&105^ z<^<CBe_ZP31G(zy|JP{`hoxkzlLEyBtSPz?2}#_2WU=;oBW%7gc*+pRW==0g%6r zQL*P59+o@x!qrRcI(hpGnu-B?2*8?Aw*8gR+GHDxn3h>jU*N!~bs#p1TA%jVfvupo zOv-%B4=Pgg66?RO-l|^U_?zHYkb<@(b5~zqgMM3|ugg`o+UEe9(m80RqmAbHhQI#d z>9XRueQw_#--y;a2bPg?Q3}3HC#X)&)#lM9KOeDkfOG5ZjCU4_g;G9QgwMGP5UR*( zZvoRGvX`e3+%8BNa3gQfb}hMcvPm?RN7k~eK|deGfo1`{9+I6cVN9z#qIZZf=L}mQ zGbE0-fPyhl7COIeFfj=%9U2;_P89)+JAi@>!5IcWUv3_9j_4RM2g;7!Mt?icyw)v3 zUbl(fFR#r9zPuKRoK~vzT%lgEU@M2g(v30EoqRv6J+iC8c;T9vjc1t-eN5FxtX{?W^U3(6-o3|WfMP$cwi1zN8Dl~QQ_7C zs54l{E^|klC%LWWI)xKE6IuCZfPgfx@NvCJRCo=STUMx)ejuz~_mTkThx*FNGaZ)d zBCuk~YMb-s#4QDWKFINL)JkwtP#;z-WI$!J3N2l@siAt9rGAKRPrn-r;5h6>@US8h z99)P#x}LwF~GP5WRh;VT8^OVqug&)lm=b@wLj7&J4J%U2_2I z=#SaqOPg79ankj|t#)rFFK!nE%Scg=GuiTWw5}$v1$TUcIH?U_9*}c)=g~ck9?`q> z@^iI`(|mPrg9@DkSVEd^Hv9~Fe#3r0>&DFQGV)9t0Nc3{YyNTP^rqC|oQ|Fei=^^T zjesW(RQT*vG}QMalZhk;U4~7*CK>BJ!jBYMrRa1t6PNV$lY5cI_l9f^eY@1Xj(*2a z>7Op{vJU)5Y^i8}aQhqEV>&ZOtR99u6MG9;H2^Mv9tR27V)K5r&TgmOx#GR-e#80l zM_zHdfCc1m)JwauA$l*);a9%?N$kd-S%V^-WB9s8^rf!G!#Bw{6Ro$yoQn25!Dy#` zFx7rKHkJ4zNn9D2BcKt`YA{oI=Y`K?h@(tNFDYer@wbhie4|0G_OnYjT2MP!P{Tu7ENY$RU5p}=jYZtWqreeet zHIJY`3Y@nll3fWN7cmO%>2)^4)%>QxWE+K=AU&@c5L_yPEFcBD%~DoumfJ#>qQ%~i z&3LW??C+7zqCLk?3);JdC`g6U`1Ez8<^V&4rXQo?B4U~Lmv72T3Mj~&pR(c0q~+A_ z$Cgl$$U#raiI;O6uKKrY6BE?fAdfx`=4mfyPq!_;2qgpWZB9a%9jq_@16Nx86EmDu01udkNGcMnZ@fqXJ1({Hx(Z; za+^Dl21k1xmRTg(hc*^Z2AT3;L-n@|i z>ah)2LcPyaXaYO44++7`4gK{-tp6aKHer>=BumZ9F*{#BA&fu70x%ep9J~}bAyTfh z^mPqs?kxT`n_6^M1JQonxeB!x+zyUe&HU~%`;eRP#OE+GsMXMDoEYp zUrkOoIZ$iPksYjis^(e>$=W9xFd6&kdHI>}K}$7E=5rs(E4-Xcua4evW8YXQEQG*) zk|Z45|NbFkk1xUUzNP2bKj3EW99ZorXYsmJ$a#$S!H+%@z+&uU;1xjf@FF;t!)lHJu|LjVDo~f6;YZD}h-(u;tT5PlPzL~xdK#$_9Dtd|Pb$k*Kr`v7VVDX2P^yRT$`ho%+(nLHzr#xoDV`4a?^BQekl zg49EayHfG;ha5t+g)~1mzxQ`lwS-O$3bI~d{aDtgOtN8%=+Cp7+-pA@^sT9) z`e8m7SN}&X{Wm{eHUFjgcB0MUUG3WfSHNiEN^-wQJ1-3R;PYhT)VCWF-cDBP-WM6r zwNHVH$Z$snLgzKbea^bvz3-3sVdj^o028ExHc$z?GvH}SKM*CU6rykm?C%6XP?swe z0*-+zndsnqm4A~**Jb?F4{7P?H+HeitLHU>QjmG>=BnkW-G`KAZ2IOsViY+Rg>(QO z3X7}Qs`~Po47qUY6Zc-8cQ7bV0RjqucAaL8TYL9$u8`=Lx3yeQ8zcRD2*=I&8R%`10%x$syMfU>NNbW8&GoeWz!w%Zqk<| z_roL6{8~~z>0D}nl>pc{Ayr-CiJ)svJtKYZmxEaxk8+XRK2rvyRDiK|l#8A2l;WEw z|M4dGTE8Ix1mXk%{K2YsvUiVy(yE7}28V@jDyRbPSRm=;c#?O6w_bR3b~nu`ioTtZ zebCw)rW%1fbUEzC6U65nbNcj{&<&}P26-NMCx~DLZL~=VWced@pry=6BzeDWIM@Rv zWV&SmxcVu&xUOe0GLN;Q@5vThC0|Yf$ak4tS?}Me%qzTOYYABIeZKgE*9o9d_Hl;E z@4u8I-_pBINPQ`|6+NIxP5}He;DV(ef#mDzTgAA@la8LerZg8;E$Z<&GoQ*o5)%aX zWUh|s)K15Bb<&BUBef;pb`_woLcwbT)DtlGp)>>pmxM0zLvO}~QrCYdGh8|=uh8`E z_FcJAO{ChJqkN^s-XKuU#55}Wu(rS+9|Q+s&jHp&4di}N*(WsjOqZk@@X15GU=<+J z09^wMTT7e6oA7+4cD?;3RVvghd|UEt5?mU3%)~t72UTE?#1ouhV~9E*GeQfAhbv%K zL%3OG0ZQegAo2tI6kDJ7`5kYNs216MTSgvMhQ2i)%ZfH*jGb;UNLNbEGqSyy1!T^mN0~J>gg{(TKC4T2o3Hof>>_&eh;;0m+kStz`Sx4}f)mR>H#NB!sqSR65VM zDsi$R>2%`~2Tt4yHfi5SvxFuJXUo$EFedlf7*FkdT^PHKk$Mph@Iw|;Epc^8YW^B)hY!`mQZOB!+yX9a#HY!WJl?+hI$ z(`;$?K zY!A`M_07eVRqui=Rif?((;u~o`OovW2539ROBNKl}{IqirwUaXFp?Q+dO|4EE%r$1@{i7F44cP zH2-bfzmt_83a6+S%;`@~4aa(a^SV8PS4hRzq}>N{z}JfCL?jc*-hhg)DT_riFSNw)ug8H*DVk^a8Ya$V+umpQqWHO}ERB zcG(aE^1#m8mq7L%h4%i7NJj!p!|gd(;?CZvA`yzrSpvz>-hT13^rmm=ShAHK*nDCb zJ|##7^acR+0Tg`ZP317*$DB{syOx$z&i2v)8gpw;NHSl%;+Qv#G&~eE;H4drsFTmY z=#zuY;U1+jIp(rFTOv(N4wEaxb!pA5BRJ-m!gdCIx~!a+cOh3TO|w$e8L*jq6G=Y4 z1Yg|{nq~f%caT4Y<)51)kKYyamh`&Bz|4pIekO9*{Mj9*Dsc<5*XyK$7riv>kZ9(E zCG2CE{qy+Y_Jc%qyBXs8zy-jy6W}5dDsH|2lU&Rz_baOP*a{>LzQ~t>xi!APe|HZtw5vKzuh6x^w=@zj^d;j zlDa9vUKS;qvdL33ub|yT)Ojz&=LzUwjq3&jqhQp8tJ&*=yp}IK) zU8@*FCizkmXEt^&w7iXRm}R_lVhFm1DXzwbo2;%J#MWJd(^RkgmHOdz@CA#=?fgqo z!Vhii_xn6@-&LVpCW`V}?-yF&o%AmE8o=AW?#~NM$2QfLl3kIkLeZ16*REa-tsJ=J z@?YrjYs$&O=vMpD5#BhZV~$N0YV^DctmEn9cfzXy4#6A&aufWOaMLys@61GsOwGFe zfdi?nxm7~8J#Z3~^ZMAz1p7_ftTkw3guT#mP-TIP`x>lr`)HNf6Ay4BNUQ4jun)7s(C=*3jCFwIN1 zx5>*qU4d-<-FmCGwVTy8Yc;Qr|ALJT0-4|$!U+Lvh3~U6@ibG+5%pAF_M+R}XbWpH zQapKFgX13i;A1llIo0`7+CB|#9Pp~1h3`*QQZ9_8#BvVs9kO!ZN%{>qr2wjlQLz4< zj$FQzQ&Qh}d3Mzh9^18x4KuBwPIr$Nn#> z=rIMRcxZ`jKl}$DB@E$tAf9QhH+lETK&)NqOE*eee+AJ<~>@NLj`Ud3w6gE#D z+x7c-ju369&<}wa6eP~-)tRJB+?$)Ml-X={gJur6K{IQ#=~@LFJN$!r zCWSsfCjMy&Mc#eGj=#y&;9qEd_&DpU?UibGHE4$OfHty6o)S^IrG-=(or+xd?dn18hIz--y2c4C7!iN&VK0-U*~o z`Ua9!wPw*}q)t;aZrx^nkz*x&vL@=qQ?9kJSm>=SG~_wu}0m2dGg+zP5u(#R=} z&ksZ_F58!!po$6 zEoLk@YPa1F6Z;gyaeWO^ew7QSCXj)bF<9`^xmvp0G4uWGb9=w} zo}S==Dc%KQPe7fK<;7Lai?ywbvPs5{89;Ckp!tBKVuK3hR*?+R0T!t2KIcn$8%p6{ zAR7^gsVFZa7-U{V=s1izeeVbvz{2ixJG*m1@*|l=D`6HJu;La32%kekQEFfp7g~jH= zIbUq!uNBXYmh);LF#X^pJW~8JHvELtpPYje z*U}KpF&hnp_V4=UQK!DFFDLi=*dcZ*6nv12Ft`DftccEpU32{5VFF({6U48l@^|8^ zN7$Z_tppIt$1CaDoL7{c9&bZUO^r=ES7+XLo%yNbi z9p)7}?!gCo2)3L5gbt)wuV#~zcURg|YQXkCJ zgPG#PaibdEYOTj05?&C%>N`1^LKzpNj7i7xdbVvXph3DJzcY8-z135@>-ifN85ytx zASM!2{JrlHh?YkU&vce1d(T9E<1lUKxe%UtRB>nE!K*E#_f<63pLqhoXY3% z3(W0{pp)XG^y%=ub*2s3U!H(-tC8#Bx7DzZ5nkiJnZv^Q+v}nP@wez>PnJIuIhCMTczw^Tal4>~J0_>s?3CAsINE`tovWjT2;8qV(TvdI1H5_Mc-fbk>ZwP%?m$@sF^=x?Xlq<} zQ7grBbXa2X!99RN&l~`iQN-rCbp{iz#gzp%g+$H@0A?#MdoN!S(I*YMmQl~>zH+zA ze0sSebjIuyNpKhQPGk`sE;y=JD3ffgUc_D;=1kDsW)4gAMhn7xo`L~fA%E&dauO_0 z0S;>-f$ZQis|Y7$zOw0S3puqn+>m>t_E~@+eZHa=tfFT`JRAkj>YuzE*~c8~dX8mV z`k%!V_9*s>v&uf^JDy1&5tg3~*meO+i>(2G!<@^44_aO$?;qLTbftRi@dz-}+8vEB zAe(8i*ie_=Re=u*7i)(5X}=2=|8`_$#DR)WC> zxOU0m{SK~x*7^-U)b_5je!jfxzhE{WhP6F48<#QyrUMWP+Hm^$V<&xEZbP<**h5U( zaeLm5B0L@a?k2li^X0c((t2_{zSDrt8qSGrWts6^4g$Y(-nKJYHI25FdD13*hVyx- zR>z)=w9U_2otMW|b?9>J>EYa-37{}EB=#Q{i2m`EzI&F!bW_q2@LIci!0|(mJ`~$0 zg*=6-<3b+uLCb?}Z_3#C&Iv>t`+Q+fKMkXrFRP2exJ%G~pU8Kc!kDkQ|4$pb%3r1L z2|G2Zw|mif9yg*{&%D3_sdgY9@`D5)*^H)3J z`}GUj`(!UlH>dw+zEauq`0y5Kc*?Megzg-z%xY9w3KYVgW_F!a{pSxBDSlwD^yo}R zHNOClll{-!qE*)xViNJyv&W`my=JbcJ_c zm;k@Emjmn<2(Z$7!?1bPhMI>M4l8Ta20T`|bOq@XP}hS<@yfzLxVBx){8d$uK-8rT z83A252~-N9{-#sT&HED5yBzP~=3=1cri9vP(L2cxXoUb~Zq9uM1mAvG zWHw|ld|;LhxmB6q{eg7$zCHPC#sL_9Y1Qk$cSrDaE)v*FV|Ec1L#PC<40Kp&o z>!rieEZZ7J>;TRcM&6xye-$gkgZ*Z|eRcROs`@2`lougI&P?FJ7sfo;#VDhCeZ%5+ zyB$FOPP|dEc%WEQwBOsgOYCXuYl}e(z!U_qUcp&`rN#=HFKu}x@kdy1-07;4#MjKi zq0NdE*4y27H!Q=ev+ev-6fMgLQsV%?Ca)djF`@RIp9klfE%JAlPd)f7;tV+K0NO|g zT#!|PwNe${f!vFaMqb=IX+*i_Atk>9N%{5`3+GCq8DSiZ_`Wab2$vE)4JXW1_Fx60 zZ*EVyZ|1)9f&V+#bfx~6fM*xL$+E*v(8@uaVB}>PpOtGBY={9IHeTK)M1P`32P_Nw+8E^{ zSLG9u);kU<%8kf<_4wrkGgYUULRE8=t!bie6_+OOZq;qCsC2UxF==WEE=%)~&UbFt zM0o&K8!umihm++v2S)4|mZCC|N3D$VrsWG<_+1Rk9;g9}HOM@WbMN1N{48 zw-DJBSi7sR?mo5Ym?oUZGp9pZvMRShW~X1E0>k7#3~uW_*=vdfi^E@11@z#p-%^ef zZ?YyG#x_MAR7zx7N_%Pnw>&^~VUoG_Lp1!mZF7{<(B2w}5mQ#vK>rc2f$rQl`pn!7 zD~`Px6hv3=vc{tc*66k;mz96!w>xr;k_`vn9DiSXx(VC{N^l}ePhnNyQ4iYmS&`)Y zK=NU|#%pV7X2D=&aV-ylvnDEy{AV|m(QL4;eaUSQf?m?B2=yzwy$~ z_gC+!1v~ZvmJC=OmJEr~dG<57e(mTNtLZyEe>MGi5X^psjyp?4ySVQ3JG<9^Ff*o0FNP6z%6T{+GEwjYSO)Tj= zeaOT5QaoqC>?}dI_vy#-}3At6SzupJ2b-;Wqj3B-pJ4 zSQThs!)q}Kucq_j)vxmC4?Y1LtO%{ zyMm|hk`$Q-yix!!#!bb2=;M&n z`q}XTRc4MD(&4SnN)wQU8-6El!o9F}kLewnMib)hb9ec4VEya?W@(1Cd#>*+aLN8? z`9(f!eRB}7K##7nAdUDx0=Yj}<{!yt53@b;_N#nV(Xy*YADMp|i3AFz-DvtM-j>UH z=AfRQiU%vys!zgBL?Z`vtVw%WH0zzi?t)9HK3gp+d<#r9na?hSlB+$=@R)W)iAU{$BV5zvexq-`(O~0aD1$DW?-q8Tw=m$175H zQE{uO=sgDKV`yG>Y@A2YRBqC@BXFbzG5SoebP!1%kUt_7$+nU5U=d(2H6I65nwO1- zhzlc52E9bi>Ce-3vw-cGjlL@_5<&O3I9#0bk7v)co=RCfU2{(ToV1=F?*Pp*`naIZ z|A6Owzgw~GKB7w;vL#BM0rNCE#4jm2ye>^2DxdePGJF{TtjBCjgSCJ%?Gk@vD^1oL z|A4O&0hM+I^uB=i3Wj7g@6pTUrZ_o{2s-`*gVQcjp8wO)vwV6EocT7KKa+VekwN`# z2jGM=q$goG*W9LS4({1&tl=20)y4eoh(>=H53+{Nb;K26&R=+%j(k*$9(8)k1!=M) z6vf<$uaqLc$`{^4)~@)g^i=^CNV~U&%1M`4seDxww3l2HIXocs%OO_H-LTil8JQUs z%ZPXSdE^ep+FH=XEqYhFW#R9BodAniwYJ?pmWUu(Z_cO<+T}Bifa3(fJ%^Q0<`dk+ zYhKUP>$X~g4-`EMr^)^tR3y7xLM@~6)b|fqt#ar!C5xpkwFmhNWfZ2LI+2S}SBV2x zB(wf_{2CB}Cuw79N_2Gf+xo5mUqwHz==4c`ZL5iY5@;9SCw@xE&%M?D zL&>(=uW-p1p&+FV?(Y~rVf#KB6OQg5GIXBXnqokU2y{Ps^Gv^u8lkbtmw49SlCK}t)2nDUMIjZE;=UStdZK@qCJyQV= zp}-p8p1`M|^nXV3$-l>N_oe^b&zO4x-7b8=#0SQBpQA$}EwgFC^lsksJS7)P6D2s? zu#!K5Ha>T5u|0UjMmB}z#5Rjtz*=H6vPbuch0v{8oD#{A=>Z(#z~b+;UNfW!%}<|jQiymmkMH1jH&MDh%qWbOIS^To7mzXbQI?Yz2#7~2HW)cE!>IK+~} z&CI&zV0Tnp{BL-dyk|`x{3`Wijm}_j@ZmMrcQ^sqG(`g53B7f%pHqFNMtOTOgwY#5 z@K-pXV41RwDvCwzQn7zT@TWvw&Ta{(s5KuIU!7Ql(6Wl$KHR<@*deFVdF*g|$^~Q& zJxonM_;Z3F{yMb4mObga_P_B)Ea_Lc{rJb($t2VP-aMD_ki`mVyxB|df5Jnh)2B<=M&%%% zCvB=0^IxXAfW2Je4@*u*_ElO{lqT;#PzE@50DL*`)^kz=wlusOnIN3@5tn_gsH z2VYlz(k|9_I-J7W0|(4+b+|ZT&-H~2n_~>hN-^NXXa7}d;QaVrZ8DFYE6LF-fJY4@ zaQ{8D8=v;q{-j}=?tJ#4YGk-9Utpj<9^FKj^}^dJc(KNDvqQFhpU?CC>&h%)5((gw zOBctC+Ruph88YwE0XKwar)^uMGqPqMC4WYhkG{|s$&Svx5I5xlMR$hi%%UebPwfzsYx=++e73Tv<|vT?P5@Z|s#LS~Tj_u|_rzcCi{jY3ds7wA zJ8XR|Ox`!pm*%iyd_N!$e;14XTr~Zxa}#iRx{^Ezf#-bbkO+1PldEGOPCrjAKd^+- zHhJls6x{<;Wx(tW`+KhEc_~uzjs#!ano6s_^3@na-WE12=EalUfYF=m>Pa96$-}2} X7;-Zk*B`ohbhoWo-401f;RXK(?Rkf` diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt deleted file mode 100644 index 0d87e8d0e8d..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/columns.txt +++ /dev/null @@ -1,183 +0,0 @@ -columns format version: 1 -181 columns: -`CounterID` UInt32 -`StartDate` Date -`Sign` Int8 -`IsNew` UInt8 -`VisitID` UInt64 -`UserID` UInt64 -`StartTime` DateTime -`Duration` UInt32 -`UTCStartTime` DateTime -`PageViews` Int32 -`Hits` Int32 -`IsBounce` UInt8 -`Referer` String -`StartURL` String -`RefererDomain` String -`StartURLDomain` String -`EndURL` String -`LinkURL` String -`IsDownload` UInt8 -`TraficSourceID` Int8 -`SearchEngineID` UInt16 -`SearchPhrase` String -`AdvEngineID` UInt8 -`PlaceID` Int32 -`RefererCategories` Array(UInt16) -`URLCategories` Array(UInt16) -`URLRegions` Array(UInt32) -`RefererRegions` Array(UInt32) -`IsYandex` UInt8 -`GoalReachesDepth` Int32 -`GoalReachesURL` Int32 -`GoalReachesAny` Int32 -`SocialSourceNetworkID` UInt8 -`SocialSourcePage` String -`MobilePhoneModel` String -`ClientEventTime` DateTime -`RegionID` UInt32 -`ClientIP` UInt32 -`ClientIP6` FixedString(16) -`RemoteIP` UInt32 -`RemoteIP6` FixedString(16) -`IPNetworkID` UInt32 -`SilverlightVersion3` UInt32 -`CodeVersion` UInt32 -`ResolutionWidth` UInt16 -`ResolutionHeight` UInt16 -`UserAgentMajor` UInt16 -`UserAgentMinor` UInt16 -`WindowClientWidth` UInt16 -`WindowClientHeight` UInt16 -`SilverlightVersion2` UInt8 -`SilverlightVersion4` UInt16 -`FlashVersion3` UInt16 -`FlashVersion4` UInt16 -`ClientTimeZone` Int16 -`OS` UInt8 -`UserAgent` UInt8 -`ResolutionDepth` UInt8 -`FlashMajor` UInt8 -`FlashMinor` UInt8 -`NetMajor` UInt8 -`NetMinor` UInt8 -`MobilePhone` UInt8 -`SilverlightVersion1` UInt8 -`Age` UInt8 -`Sex` UInt8 -`Income` UInt8 -`JavaEnable` UInt8 -`CookieEnable` UInt8 -`JavascriptEnable` UInt8 -`IsMobile` UInt8 -`BrowserLanguage` UInt16 -`BrowserCountry` UInt16 -`Interests` UInt16 -`Robotness` UInt8 -`GeneralInterests` Array(UInt16) -`Params` Array(String) -`Goals.ID` Array(UInt32) -`Goals.Serial` Array(UInt32) -`Goals.EventTime` Array(DateTime) -`Goals.Price` Array(Int64) -`Goals.OrderID` Array(String) -`Goals.CurrencyID` Array(UInt32) -`WatchIDs` Array(UInt64) -`ParamSumPrice` Int64 -`ParamCurrency` FixedString(3) -`ParamCurrencyID` UInt16 -`ClickLogID` UInt64 -`ClickEventID` Int32 -`ClickGoodEvent` Int32 -`ClickEventTime` DateTime -`ClickPriorityID` Int32 -`ClickPhraseID` Int32 -`ClickPageID` Int32 -`ClickPlaceID` Int32 -`ClickTypeID` Int32 -`ClickResourceID` Int32 -`ClickCost` UInt32 -`ClickClientIP` UInt32 -`ClickDomainID` UInt32 -`ClickURL` String -`ClickAttempt` UInt8 -`ClickOrderID` UInt32 -`ClickBannerID` UInt32 -`ClickMarketCategoryID` UInt32 -`ClickMarketPP` UInt32 -`ClickMarketCategoryName` String -`ClickMarketPPName` String -`ClickAWAPSCampaignName` String -`ClickPageName` String -`ClickTargetType` UInt16 -`ClickTargetPhraseID` UInt64 -`ClickContextType` UInt8 -`ClickSelectType` Int8 -`ClickOptions` String -`ClickGroupBannerID` Int32 -`OpenstatServiceName` String -`OpenstatCampaignID` String -`OpenstatAdID` String -`OpenstatSourceID` String -`UTMSource` String -`UTMMedium` String -`UTMCampaign` String -`UTMContent` String -`UTMTerm` String -`FromTag` String -`HasGCLID` UInt8 -`FirstVisit` DateTime -`PredLastVisit` Date -`LastVisit` Date -`TotalVisits` UInt32 -`TraficSource.ID` Array(Int8) -`TraficSource.SearchEngineID` Array(UInt16) -`TraficSource.AdvEngineID` Array(UInt8) -`TraficSource.PlaceID` Array(UInt16) -`TraficSource.SocialSourceNetworkID` Array(UInt8) -`TraficSource.Domain` Array(String) -`TraficSource.SearchPhrase` Array(String) -`TraficSource.SocialSourcePage` Array(String) -`Attendance` FixedString(16) -`CLID` UInt32 -`YCLID` UInt64 -`NormalizedRefererHash` UInt64 -`SearchPhraseHash` UInt64 -`RefererDomainHash` UInt64 -`NormalizedStartURLHash` UInt64 -`StartURLDomainHash` UInt64 -`NormalizedEndURLHash` UInt64 -`TopLevelDomain` UInt64 -`URLScheme` UInt64 -`OpenstatServiceNameHash` UInt64 -`OpenstatCampaignIDHash` UInt64 -`OpenstatAdIDHash` UInt64 -`OpenstatSourceIDHash` UInt64 -`UTMSourceHash` UInt64 -`UTMMediumHash` UInt64 -`UTMCampaignHash` UInt64 -`UTMContentHash` UInt64 -`UTMTermHash` UInt64 -`FromHash` UInt64 -`WebVisorEnabled` UInt8 -`WebVisorActivity` UInt32 -`ParsedParams.Key1` Array(String) -`ParsedParams.Key2` Array(String) -`ParsedParams.Key3` Array(String) -`ParsedParams.Key4` Array(String) -`ParsedParams.Key5` Array(String) -`ParsedParams.ValueDouble` Array(Float64) -`Market.Type` Array(UInt8) -`Market.GoalID` Array(UInt32) -`Market.OrderID` Array(String) -`Market.OrderPrice` Array(Int64) -`Market.PP` Array(UInt32) -`Market.DirectPlaceID` Array(UInt32) -`Market.DirectOrderID` Array(UInt32) -`Market.DirectBannerID` Array(UInt32) -`Market.GoodID` Array(String) -`Market.GoodName` Array(String) -`Market.GoodQuantity` Array(Int32) -`Market.GoodPrice` Array(Int64) -`IslandID` FixedString(16) diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt deleted file mode 100644 index 91d679bb3ab..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -485007 \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx deleted file mode 100644 index 0cb239d932a..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/minmax_StartDate.idx +++ /dev/null @@ -1 +0,0 @@ -?? \ No newline at end of file diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/partition.dat b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/partition.dat deleted file mode 100644 index 9c176950d3f897f5cc9714d7011107a21e47a229..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmdlj#LNHy11td1 diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/primary.idx b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/201403_7_9_1/primary.idx deleted file mode 100644 index 6765990e38d151d57b7f18ee97cef776d10afa02..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1314 zcmWlWeLT|%9LIkfbtfq=jO5`;VIDKrgED^QVO-ecan`Y3+vPFyba~p2$@RqD@@i2{ zj>>tskZy@YSmSc#p(3L^OkGSfgc6qf_V4HQ{k%V)@AqqiAc!t{Fjmofqgz$l&^?9GGcP7X*~wPp&bP(`jJIVv)0UyaSh*M zkuB(XdTF)(i?wCF@H^}&##iOQF2_o~-?g#M!Im&&1`Dn|yKj^p)NMXm>R#*m@vsVz zZPu-jeCFLQ^*t=4wO&zy^zU)wnbq%T1!(hoy2-Dqh&DPQy4R{(Jf|lS`+48Rtq1Z% z+p6j7Aj-yjDKgiGgKD6p7;jXsEbNaG5W+dm#7&^^dvi-JGbrjCHOfQztD2DYU%(d! z_km{*?D<8$u8rD=7HVi!`$;+)YUcXREOR_e7t*5_Sy7x$jSSCB9R@B@AJIWIMzef< z3Uw7OG_zz{t^v53DhRdu7Ts9UVjjbIt1;~5oEx$&ZnkY`XEKg6Q1=~l9;uuQSab-U z#&&&*lslThj3&V`5YsPdF+2MuVBabpIL9b?mi=3wdnT~g2Ro7Np>s~>|6Bs(AUElxC6TdRbjifCdaVjLpWd!AZw5AV&PY5$@T}XA z^(4s1ON}ZHvdGCGNg9RVBH+)R8dqiZA=7xW-8B*1Fucm!@4hc{>u-malLxhWAZu@w z>3}+RKS%!N^eBM4i}qK|Q#IO!>bB`~e)>_iKo*9*A?|!p zXVNE9x)4w7!RVl$OP|C#G)NWF;ge|*8blx;7~?4tsMQkBBDQZviaU_!V*cXFjvM8x z)Y4c6VHDsXJ%ht38|aCSp+a{%E)`~wq?dU2+D(Mbt!XU3xiIML(%*IN=>qKrH$H-# z+{DD-(9xe%Vkx|(>6@RWjC?c}vW707wIzx0JeK%rQn5n{C_33M>P`}RWBddyB&^9P zAYW=9S!vp$h>sXJkV(z`9n#0rXCkeFy>DF)%Am-9xB^-A7uuAI`$;vt9$XA9rx^CC z^v>v3dOAO+mFLBZLrTEKWf^;w`U`FDdE5%?W0!)WUE{P=rKmK}F40f7XsrU5@xd(f zyy@1>lcGKTq?J3M9P+zky~BBxX~_A*+b+EUvTDWzf`uoxMQ-jhNn-v(n8^5p;em`cqBLGPl2ap)i&2)(v|-Qn7p;U%sQbWzw7yEO`m8-r)J^xyXs) zEPHDF#dT6}c_cgvT|2Jy4E0w{MMkC|i|L@K-t{5Ec11Xp*e;)ipUxMUZo|{*nv>t~ zJ*=%i!xs75-@o78c!U$0_#yQ&bs0F+Y{rsMi*sH!>>AObT!YI?zVnalJ{)GSWU3eb EKZSBM)c^nh diff --git a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt b/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/checksums.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/checksums.txt deleted file mode 100644 index 1200b25bd52ed0332f4e976e5e109661007e72bb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$axV}lqE!_QlD*Y>FAiIEP1 z+5i9m&;S4c@jnI$V{dhCbS`vwbOV+!t186UB-eP+=laeWci#XAWMOn+E@EkJtA+vp zC@FDdb-WItZml__r5>08+N>Jd_3*EI_1(J_Im&g zaA9(EX>@6CZZ2eDbOiRJGt`jAL&ph4JQlLni7WsMaB^vFVRCscX=Hc|qQj~+&$gS+ M9BY!1)str&05BDACIA2c diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt deleted file mode 100644 index 4835061786d..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/columns.txt +++ /dev/null @@ -1,259 +0,0 @@ -columns format version: 1 -257 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`milliseconds` UInt64 -`ProfileEvent_Query` UInt64 -`ProfileEvent_SelectQuery` UInt64 -`ProfileEvent_InsertQuery` UInt64 -`ProfileEvent_FailedQuery` UInt64 -`ProfileEvent_FailedSelectQuery` UInt64 -`ProfileEvent_FailedInsertQuery` UInt64 -`ProfileEvent_QueryTimeMicroseconds` UInt64 -`ProfileEvent_SelectQueryTimeMicroseconds` UInt64 -`ProfileEvent_InsertQueryTimeMicroseconds` UInt64 -`ProfileEvent_FileOpen` UInt64 -`ProfileEvent_Seek` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64 -`ProfileEvent_ReadBufferAIORead` UInt64 -`ProfileEvent_ReadBufferAIOReadBytes` UInt64 -`ProfileEvent_WriteBufferAIOWrite` UInt64 -`ProfileEvent_WriteBufferAIOWriteBytes` UInt64 -`ProfileEvent_ReadCompressedBytes` UInt64 -`ProfileEvent_CompressedReadBufferBlocks` UInt64 -`ProfileEvent_CompressedReadBufferBytes` UInt64 -`ProfileEvent_UncompressedCacheHits` UInt64 -`ProfileEvent_UncompressedCacheMisses` UInt64 -`ProfileEvent_UncompressedCacheWeightLost` UInt64 -`ProfileEvent_IOBufferAllocs` UInt64 -`ProfileEvent_IOBufferAllocBytes` UInt64 -`ProfileEvent_ArenaAllocChunks` UInt64 -`ProfileEvent_ArenaAllocBytes` UInt64 -`ProfileEvent_FunctionExecute` UInt64 -`ProfileEvent_TableFunctionExecute` UInt64 -`ProfileEvent_MarkCacheHits` UInt64 -`ProfileEvent_MarkCacheMisses` UInt64 -`ProfileEvent_CreatedReadBufferOrdinary` UInt64 -`ProfileEvent_CreatedReadBufferAIO` UInt64 -`ProfileEvent_CreatedReadBufferAIOFailed` UInt64 -`ProfileEvent_CreatedReadBufferMMap` UInt64 -`ProfileEvent_CreatedReadBufferMMapFailed` UInt64 -`ProfileEvent_CreatedWriteBufferOrdinary` UInt64 -`ProfileEvent_CreatedWriteBufferAIO` UInt64 -`ProfileEvent_CreatedWriteBufferAIOFailed` UInt64 -`ProfileEvent_DiskReadElapsedMicroseconds` UInt64 -`ProfileEvent_DiskWriteElapsedMicroseconds` UInt64 -`ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64 -`ProfileEvent_NetworkSendElapsedMicroseconds` UInt64 -`ProfileEvent_ThrottlerSleepMicroseconds` UInt64 -`ProfileEvent_QueryMaskingRulesMatch` UInt64 -`ProfileEvent_ReplicatedPartFetches` UInt64 -`ProfileEvent_ReplicatedPartFailedFetches` UInt64 -`ProfileEvent_ObsoleteReplicatedParts` UInt64 -`ProfileEvent_ReplicatedPartMerges` UInt64 -`ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64 -`ProfileEvent_ReplicatedPartMutations` UInt64 -`ProfileEvent_ReplicatedPartChecks` UInt64 -`ProfileEvent_ReplicatedPartChecksFailed` UInt64 -`ProfileEvent_ReplicatedDataLoss` UInt64 -`ProfileEvent_InsertedRows` UInt64 -`ProfileEvent_InsertedBytes` UInt64 -`ProfileEvent_DelayedInserts` UInt64 -`ProfileEvent_RejectedInserts` UInt64 -`ProfileEvent_DelayedInsertsMilliseconds` UInt64 -`ProfileEvent_DuplicatedInsertedBlocks` UInt64 -`ProfileEvent_ZooKeeperInit` UInt64 -`ProfileEvent_ZooKeeperTransactions` UInt64 -`ProfileEvent_ZooKeeperList` UInt64 -`ProfileEvent_ZooKeeperCreate` UInt64 -`ProfileEvent_ZooKeeperRemove` UInt64 -`ProfileEvent_ZooKeeperExists` UInt64 -`ProfileEvent_ZooKeeperGet` UInt64 -`ProfileEvent_ZooKeeperSet` UInt64 -`ProfileEvent_ZooKeeperMulti` UInt64 -`ProfileEvent_ZooKeeperCheck` UInt64 -`ProfileEvent_ZooKeeperClose` UInt64 -`ProfileEvent_ZooKeeperWatchResponse` UInt64 -`ProfileEvent_ZooKeeperUserExceptions` UInt64 -`ProfileEvent_ZooKeeperHardwareExceptions` UInt64 -`ProfileEvent_ZooKeeperOtherExceptions` UInt64 -`ProfileEvent_ZooKeeperWaitMicroseconds` UInt64 -`ProfileEvent_ZooKeeperBytesSent` UInt64 -`ProfileEvent_ZooKeeperBytesReceived` UInt64 -`ProfileEvent_DistributedConnectionFailTry` UInt64 -`ProfileEvent_DistributedConnectionMissingTable` UInt64 -`ProfileEvent_DistributedConnectionStaleReplica` UInt64 -`ProfileEvent_DistributedConnectionFailAtAll` UInt64 -`ProfileEvent_CompileAttempt` UInt64 -`ProfileEvent_CompileSuccess` UInt64 -`ProfileEvent_CompileFunction` UInt64 -`ProfileEvent_CompiledFunctionExecute` UInt64 -`ProfileEvent_CompileExpressionsMicroseconds` UInt64 -`ProfileEvent_CompileExpressionsBytes` UInt64 -`ProfileEvent_ExternalSortWritePart` UInt64 -`ProfileEvent_ExternalSortMerge` UInt64 -`ProfileEvent_ExternalAggregationWritePart` UInt64 -`ProfileEvent_ExternalAggregationMerge` UInt64 -`ProfileEvent_ExternalAggregationCompressedBytes` UInt64 -`ProfileEvent_ExternalAggregationUncompressedBytes` UInt64 -`ProfileEvent_SlowRead` UInt64 -`ProfileEvent_ReadBackoff` UInt64 -`ProfileEvent_ReplicaPartialShutdown` UInt64 -`ProfileEvent_SelectedParts` UInt64 -`ProfileEvent_SelectedRanges` UInt64 -`ProfileEvent_SelectedMarks` UInt64 -`ProfileEvent_SelectedRows` UInt64 -`ProfileEvent_SelectedBytes` UInt64 -`ProfileEvent_Merge` UInt64 -`ProfileEvent_MergedRows` UInt64 -`ProfileEvent_MergedUncompressedBytes` UInt64 -`ProfileEvent_MergesTimeMilliseconds` UInt64 -`ProfileEvent_MergeTreeDataWriterRows` UInt64 -`ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64 -`ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64 -`ProfileEvent_MergeTreeDataWriterBlocks` UInt64 -`ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64 -`ProfileEvent_CannotRemoveEphemeralNode` UInt64 -`ProfileEvent_RegexpCreated` UInt64 -`ProfileEvent_ContextLock` UInt64 -`ProfileEvent_StorageBufferFlush` UInt64 -`ProfileEvent_StorageBufferErrorOnFlush` UInt64 -`ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64 -`ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64 -`ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64 -`ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64 -`ProfileEvent_DictCacheKeysRequested` UInt64 -`ProfileEvent_DictCacheKeysRequestedMiss` UInt64 -`ProfileEvent_DictCacheKeysRequestedFound` UInt64 -`ProfileEvent_DictCacheKeysExpired` UInt64 -`ProfileEvent_DictCacheKeysNotFound` UInt64 -`ProfileEvent_DictCacheKeysHit` UInt64 -`ProfileEvent_DictCacheRequestTimeNs` UInt64 -`ProfileEvent_DictCacheRequests` UInt64 -`ProfileEvent_DictCacheLockWriteNs` UInt64 -`ProfileEvent_DictCacheLockReadNs` UInt64 -`ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64 -`ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64 -`ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64 -`ProfileEvent_PolygonsAddedToPool` UInt64 -`ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64 -`ProfileEvent_RWLockAcquiredReadLocks` UInt64 -`ProfileEvent_RWLockAcquiredWriteLocks` UInt64 -`ProfileEvent_RWLockReadersWaitMilliseconds` UInt64 -`ProfileEvent_RWLockWritersWaitMilliseconds` UInt64 -`ProfileEvent_DNSError` UInt64 -`ProfileEvent_RealTimeMicroseconds` UInt64 -`ProfileEvent_UserTimeMicroseconds` UInt64 -`ProfileEvent_SystemTimeMicroseconds` UInt64 -`ProfileEvent_SoftPageFaults` UInt64 -`ProfileEvent_HardPageFaults` UInt64 -`ProfileEvent_VoluntaryContextSwitches` UInt64 -`ProfileEvent_InvoluntaryContextSwitches` UInt64 -`ProfileEvent_OSIOWaitMicroseconds` UInt64 -`ProfileEvent_OSCPUWaitMicroseconds` UInt64 -`ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64 -`ProfileEvent_OSReadBytes` UInt64 -`ProfileEvent_OSWriteBytes` UInt64 -`ProfileEvent_OSReadChars` UInt64 -`ProfileEvent_OSWriteChars` UInt64 -`ProfileEvent_PerfCpuCycles` UInt64 -`ProfileEvent_PerfInstructions` UInt64 -`ProfileEvent_PerfCacheReferences` UInt64 -`ProfileEvent_PerfCacheMisses` UInt64 -`ProfileEvent_PerfBranchInstructions` UInt64 -`ProfileEvent_PerfBranchMisses` UInt64 -`ProfileEvent_PerfBusCycles` UInt64 -`ProfileEvent_PerfStalledCyclesFrontend` UInt64 -`ProfileEvent_PerfStalledCyclesBackend` UInt64 -`ProfileEvent_PerfRefCpuCycles` UInt64 -`ProfileEvent_PerfCpuClock` UInt64 -`ProfileEvent_PerfTaskClock` UInt64 -`ProfileEvent_PerfContextSwitches` UInt64 -`ProfileEvent_PerfCpuMigrations` UInt64 -`ProfileEvent_PerfAlignmentFaults` UInt64 -`ProfileEvent_PerfEmulationFaults` UInt64 -`ProfileEvent_PerfMinEnabledTime` UInt64 -`ProfileEvent_PerfMinEnabledRunningTime` UInt64 -`ProfileEvent_PerfDataTLBReferences` UInt64 -`ProfileEvent_PerfDataTLBMisses` UInt64 -`ProfileEvent_PerfInstructionTLBReferences` UInt64 -`ProfileEvent_PerfInstructionTLBMisses` UInt64 -`ProfileEvent_PerfLocalMemoryReferences` UInt64 -`ProfileEvent_PerfLocalMemoryMisses` UInt64 -`ProfileEvent_CreatedHTTPConnections` UInt64 -`ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64 -`ProfileEvent_QueryProfilerSignalOverruns` UInt64 -`ProfileEvent_CreatedLogEntryForMerge` UInt64 -`ProfileEvent_NotCreatedLogEntryForMerge` UInt64 -`ProfileEvent_CreatedLogEntryForMutation` UInt64 -`ProfileEvent_NotCreatedLogEntryForMutation` UInt64 -`ProfileEvent_S3ReadMicroseconds` UInt64 -`ProfileEvent_S3ReadBytes` UInt64 -`ProfileEvent_S3ReadRequestsCount` UInt64 -`ProfileEvent_S3ReadRequestsErrors` UInt64 -`ProfileEvent_S3ReadRequestsThrottling` UInt64 -`ProfileEvent_S3ReadRequestsRedirects` UInt64 -`ProfileEvent_S3WriteMicroseconds` UInt64 -`ProfileEvent_S3WriteBytes` UInt64 -`ProfileEvent_S3WriteRequestsCount` UInt64 -`ProfileEvent_S3WriteRequestsErrors` UInt64 -`ProfileEvent_S3WriteRequestsThrottling` UInt64 -`ProfileEvent_S3WriteRequestsRedirects` UInt64 -`ProfileEvent_QueryMemoryLimitExceeded` UInt64 -`CurrentMetric_Query` Int64 -`CurrentMetric_Merge` Int64 -`CurrentMetric_PartMutation` Int64 -`CurrentMetric_ReplicatedFetch` Int64 -`CurrentMetric_ReplicatedSend` Int64 -`CurrentMetric_ReplicatedChecks` Int64 -`CurrentMetric_BackgroundPoolTask` Int64 -`CurrentMetric_BackgroundMovePoolTask` Int64 -`CurrentMetric_BackgroundSchedulePoolTask` Int64 -`CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64 -`CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64 -`CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64 -`CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64 -`CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64 -`CurrentMetric_DiskSpaceReservedForMerge` Int64 -`CurrentMetric_DistributedSend` Int64 -`CurrentMetric_QueryPreempted` Int64 -`CurrentMetric_TCPConnection` Int64 -`CurrentMetric_MySQLConnection` Int64 -`CurrentMetric_HTTPConnection` Int64 -`CurrentMetric_InterserverConnection` Int64 -`CurrentMetric_PostgreSQLConnection` Int64 -`CurrentMetric_OpenFileForRead` Int64 -`CurrentMetric_OpenFileForWrite` Int64 -`CurrentMetric_Read` Int64 -`CurrentMetric_Write` Int64 -`CurrentMetric_SendScalars` Int64 -`CurrentMetric_SendExternalTables` Int64 -`CurrentMetric_QueryThread` Int64 -`CurrentMetric_ReadonlyReplica` Int64 -`CurrentMetric_MemoryTracking` Int64 -`CurrentMetric_EphemeralNode` Int64 -`CurrentMetric_ZooKeeperSession` Int64 -`CurrentMetric_ZooKeeperWatch` Int64 -`CurrentMetric_ZooKeeperRequest` Int64 -`CurrentMetric_DelayedInserts` Int64 -`CurrentMetric_ContextLockWait` Int64 -`CurrentMetric_StorageBufferRows` Int64 -`CurrentMetric_StorageBufferBytes` Int64 -`CurrentMetric_DictCacheRequests` Int64 -`CurrentMetric_Revision` Int64 -`CurrentMetric_VersionInteger` Int64 -`CurrentMetric_RWLockWaitingReaders` Int64 -`CurrentMetric_RWLockWaitingWriters` Int64 -`CurrentMetric_RWLockActiveReaders` Int64 -`CurrentMetric_RWLockActiveWriters` Int64 -`CurrentMetric_GlobalThread` Int64 -`CurrentMetric_GlobalThreadActive` Int64 -`CurrentMetric_LocalThread` Int64 -`CurrentMetric_LocalThreadActive` Int64 -`CurrentMetric_DistributedFilesToInsert` Int64 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt deleted file mode 100644 index 1e3852840b5..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/count.txt +++ /dev/null @@ -1 +0,0 @@ -211 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx deleted file mode 100644 index 4b6c49f8d50..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -…H…H \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/partition.dat b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/partition.dat deleted file mode 100644 index b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0VWo7^X0I>iw diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx deleted file mode 100644 index 714ba773856..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202010_1_28_6/primary.idx +++ /dev/null @@ -1 +0,0 @@ -…H®-œ_…H€.œ_ \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/checksums.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202012_29_2358_466/checksums.txt deleted file mode 100644 index 8031891da7f5bd9e7a744c4e46bca92f86a61006..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 11371 zcmYLPWk8f!7ryt6iFe{6BOODTASt1MgoHtZf{DX0z`!trGlPJLEP`Mwirs;Yxng&& z3brDyg|)^?n1I4}2Xw!GZ{9lRp65BwxzaS5G((e}sj*5?sWT;7t0^+IMy^s0wi?K6 zW%(_Ax9Uw%Cs*1leY>)FhMxlff0qK#8BQ+=2hf}xqKRzfe1@63$QiXG9$%}iEg9`GF^A|RoKKy+X18a0Xvu(BUq@n9ZMMb6< zY+Oa67{G?*;3s+CVc^~M{y{EFG(kPZ$}tU~e>tGsSn3eDLZ*pRt$<(=Ko40WVFN(X zfP@QyU?9oA2U$ZvWUNe?TndbMD+$hS78nM)W+>*vv%y(uvP_v;qKHr>mqP@hi1(CW zLIBFcOJJD?_J=A|NfJd|np!6L1e%Gt^z_R;y?6X=vSH1r4j1u?0bHUTR!QDpda}aK zn%|2%vtaC`gQoyD7{DCRPALCY@2;|c(d*dzr76Sh_8$Nj2#za=4#1Y1@&z=JGw3m6 zU!*u09qYa77Wdc$w<4EP+}`Z>*TvjP)umhdmafS#a9x$aLL&jdp4CopE?68{BAWf; zyUW7QuV?991_NOEu(-IWp(>?Pc9}+v(1_LPnWR`f16cE% zI4A;;Xf~dn#cbe(P5%?3Y(m5fBsf6;WdoAO&}^3N`4L%E(WvbM9 zacaDu<;v7H*hF1=4?}(P=&k)KTYAnM&yKvgzhNq#LELSv&GElAZXGKew^}56ZEc6V zVE|6U4sEb#jEgfm{49UNi zC_e7300^EFyaM1KnI%)+rH5kYiy1>Tkw-bDDTiDy-wQ7-SbyI9ai8O=CeRM8pG-gR z^z}*4_4V;SlY21I0?^$?C$Vrq=dp2_Yl2(-gQ`?6vtI&tzz&F4%e7;1V{M!XLW|2m zwGTcgZfqVgd0Obo>A0~ef_6|)1%95ry{t!JN!9or1baFlh67lLO}I3(S6d6~a^B3Q zquO~z-q(O0`A3vQtqsrCN-P*Ko8eq^0PR6hlv*as%+kt^vydD+|IYt>2l6~Vuw3eyQFiqqF#^R!=X4cnrz+ms=g+AM5>8ZHyFA@6`(@pIQOY_uIJlo!Eav)F` zHs4Ip2^}SFuU*O)E|;9LU48Q1(tlty(&AnjV5qYIRrD~#YsjU#z7hzbso{4Lj}By!O=KxeN9}o2=^HxT z;^4~{wGZj79oVdBYNuP=@KsCmFHc)|YxVWCJtXu6FctiOOFunL3Gp0napdWfo#lL| z4UsI+AM8^p6?rkTEQMSuQ3Lc9FYj9VmuJ2N0(Stj!9omDLv=8lilggoS}+`-LTpz% zisbMhrmC3GGwQ@{eS$#U?brY8A92?2p9< zG7>`Xe66{kNOs4{A8?7+t+tmt>OLJF)N|3LYsZ)I`0x!4)Oceprqjcgxo=O7Ngp3N za9*z6is^o=UACW0kQd=s9#=SK+uIiFMf>y zhASgp1MJ3L>I)}ItxCbD7d!=Xti%A!v?Kuvg-UuKPy0K)<<$K3n`ck(sXfwq{^*0- zr{BX)VuZPVc0z;OI#1L2gEkWVp>-%=FhX<^;TM7jd~QC&u_(KA_{hD+bzmT%1Ba$% zD_z-5WD9+ca$g%P-FEDS`rFOMgA3);@GqCG(eGE@wSze zz5C&2RBcAA|f&T8`>QISNsnW(^?3f~3qL0K}L$Q4{=&1wfB0}su1VfuM zX&t9x17ft93bQx7UA+1KXDX)RAh1Z4nT2N@pw(XHqcwz)*eV3mIt*)~ME@snPCkHtI$SPQt28pHN|`*HWz72miLsFV9B@Yiv=T#MJR~ARvX#>7 z$c&jwl5+svy{(WjL8xN0rBa!Og2w19AgFF$pnK6Xc@KyQ&Xq~Cx3chkUqK|$F-G1O z(pr_dmavi~59AXENw+|Ss*~45md9R{WqldCp{gTntH&a6zV==X^^=f!zmm3|&Gfx6 zY1W~fy|s-F=>%vbFyR51@ce)W91a@!ta`f1!)?-!8Nyu?9#E(pKx!|yi8@>XE4L0w zpJsjZ#WRcf6K*w?O)JNd?q&Va%Tvr=M{{x)7V1w=B5@!ik4Q2`IWcfoYV2Ls2Qq&<~VpH4Zx~W%> z>N{Q--*SC4)d}VI={-r4MY*3|<>kyi8J+h7^66-Ry|@#G^Y;%cT<17u*0Le5seiwv z?SLeECl{R|@N6N`jYEPYN_WHHG3A3)?jI#Ug6$9JF|dh+6QIp`&&-;xDHD#) zo8jj>bd%-ZHalS_+{c`7PJDGZIO}oawVL{ve_jCrjVD|VTFEAdV7uU4O?yaR`u`1kO_-hxcNSnp03Dg-MH8H0 zNP7e94d%=;Hd+8k;+Kg%NW9X7Q@C7DXo6mW9(YT1d6=D?Ai->NFn}YD2&Rlt5lOMT z0SkEn4rH@Y2{1t^QFX0y_&n&}f6ux7+`)AgQ=p=%~{V#;ZV%vI1 z(%nIvA4&Aku!*Q(g(OQej9{vJG?722*k^6tDA-$0Z0#_3@os%%+(OO|IIm~PoPVc1 zTVGkyZXO-Y;|?UTldy3X*g;ITT(RiR)xB?D4)w`9mJ{X<`~kcMQ3b%8cg0~yOKC{i!1scaCkCuoSaQ(XH6tFYj0#cyw&SC3eE$T1QKWM&Vg4!_YM+s_t-1UCjsI zDt)!g)XAqopRM1QRc;GLFo^+L3?)-&Bc?$gjRt(U5(r|r0w+iva$;5j{kR`I6u}h< z93hGWY_t@w{6a1Y01#ACE|{VjnIjiyWbSqa3ReN{7=YG;-ifhVi9%M!SmGPK?q{Uf z>lFr?*gU25or5#ZVk@cjs%&j=u2d#VmakJGkLuU;KZ zNV?bv+F(S7+>|&Y`E>1oQIr`$agQoYt%02sO*_)f+Es?J& z9^bt@X-?wsF^&c0O9I8GdMAE2z&HTe+sRLy6>%TjV~YW=VQ71=X=_N#xt&qAkk!R@Pyqb!Ut^ZmUW;ynO?`@r@;wD3? zSDorT^}x9;TTFHYK0LYRr5=tmnDVs|F}4S?QH|5gEBp#W4y*7WSPpH5N1m}|8pTc2 z$+eLA2kvq5z5ZbCsEU*CH*Om{9k+Ji`dnVp+tJti1A)iQ^dmdd1YJaS2A*l}e~wjO zbUr@*pJT?GQ%?3NYzJIV0D1f-@{eho23)gn@o#zMI;eN6)x?mu1-L0^^6rj>yX^yM zd*>PDRUS{nW~V;&5pGLKw9og6uYGJA4#)b@=3}Y1*sbJ#1f+&)f6)Y-bRL zLyasS537^vv(zZI<(Omh*7Mfd>_a?mlb*+c9=FiTVxG}>k z`WD3(8{3Y~+haVId9&vT`P90&-Ps56!h zvKc1-ikWUFJ$SWICF^v+;K-8c&CTmpZ)3wYINP?`-79}#ab4@N&GDW#Sp^W!fxub} zICFKux{A98wkz&U@3kQz`7K4e0uaI{4)@XNFg`QOqHDgTZCG3zYS&WCS%_T@a*8kf zyp=oH`em5(>+{-UkKR(<{w*lCH!)NZv=66={1o+9{gpmG17v;C%7n1_KEBqIF?GtB zv)TU2S?f!lGs{LUXrHyl2*(+mD7M2*ZkZK(SLXSd{|w9v7==teTjIw>aX1MN!ocjp zF;rJ#x;QJ>_;5zQhY8Due*6#yg_#rY=Hua1hMn`-eyHWVW`~6H_nBn2H-U97HYOk) zWVGezF)Js3Fx?&{<=?7&@)~#^#YhC8Re<8GhhE-jF7X(<(3~8v@Dhuo{oO(j-Cp

tE*-EM!D=JNt|@+#bc$NKO04cs5&wmf+Gu>W_lI>|ngkKO=) zBKe?+y1fey%C5RKs^DYPnIRirzL<6X|;J~dwz#j+T zCcJ@voL_Y@sQ~1DuGzQiqT4wc$kj)i@UmbK(@~#_w9_8K+X{~d4dU_TIhn?VWy|oK zp9MelYm8emy$2Z>e`KRC2%%%cW_&I(}!ANkVFv8=^n!(`991K?XEigeByruJ zy}wz*<3L}N!DddGG?C9Hid;KwlsO@zr(EWU!)}Ku=iv6@kB48n?02b5Qa;#K8`V&s zPXJ*Az-+u7ZVaol+{)Wld!((^Xx06my)5*fM#!`|Ds@JTOe&L4c`d?oKEYJ%bLi3E z*>eArdHOC9$rp1f@SGPly?lQDX6tE}#FMDJ+v5}o$KGW6|FtJay5jkT z#Iy0`??hxsx9GYCTbFEPnLoZ%ddYn-J8?pw9{X;O{%df18oyQN{2sJ(BS`j0n>%-} zz7zc-9PKcnPA}bj&Ex1n6YjS@|M+qE0k#|FgU4+-qim=SU*xYRVh*r%JX04M4zQWw z2>Ah$^szu|%#9ei0cJWpv|NY1%(=+ep;4no%hlR!iK3BU?in_bE7|E29`1OrPJDaQ zwDE$dwJ#X+6{}T+=Fmq0LICBW; z7y%Zf+F=3lETUj-+@;lxa~7AR3l|lbL`KFAk1Pcku{Y2czD)Cm&` zD@Jcg&sed%dURyVR?j&aK-dmVGjPShgKdz#Yv$69aO(|M%s)ClN%C&Qu2!%hxyQ+U z1-ILh#!d;fx$;f}AkoE9eh`B#LI+U_ip(5Z6pn^-k=~7fyj%modIs5A% z5ff+TcsS#+j!q=f%43^`>|js2le4kxx~*$Z9=8iGI^pK5CEl~1Y-Ak>lPdgj*jZSm z#)PmdY;>}o_fC@y`;pMP>hQsph0EXCWrJ3t;zPlqpcnD)nsV&@1gEvemc7|zHXwe% z&+=KXtr_l!L!-*8{Eq1`E+A zU?KE|oNjeOm``AJc1Da0*VAN5X}A!np$I?ZcV#e!#F|i4gT#1HfY+VVFkdNH1f8?8 zhrSwsNRE-vhoHL!f8>IhY9`?brU0V5PoW7pd1zL4xI9&zX^LKv#um2K6w^g84GMKY z6&Duxk+2XJg6@YT)5!+!Ac@Iu3t=sEr-j|wnlOh92+qt_%z&tccIqA)$?Rw0ZN>gT zV>WEX&=&vE@eW4l3(z`f&2%uoskShRX4^p2k2DslNpAO233m`MC4~48j2uV{eDk}r z`;d*3>t*==G7(q?bwwbIfd*0Ga%He`tM4DlG1*Eb&QY8MsG2qsF6=53;V(#G35AP9 zXcjON_Mmv(+N5wZVI3no!6FD2I>|D`x4(N2q|~~EMj=af<$@t`ATJDOhK51~fhR)f z@(P?(@U24BTnBiN5Y-%nRe{*lL%6dW|N%fYos2bO4u`gQMUMOx?PIAbYwll}H znl;RJOYwZKL9=T92sl?yN+uS{(dvQK|FjKqirW~y7NcbH^qq**@DZ$yN{|gzz36QmOecn zpg&0w<3=#^06Pl6`9N@nV46w1idau@#bxn!5p%)?YsKuAhzTL<3ktW?a7(7A31LN(zT-#1II_ZKS~~>cWA}Krfr?uG7l|y7HwF zGn9*mPoA|Qe`SUmQh+yTBFmhS)Bi;yGY=e%fSw1avMUXeX{2g-mbNIWiM%l3H3E#H7ru-7lg~yKf z_uHM9I*P=&S}%gzF|0ZE^jr2pRB2ySJceI=E}c6;j79(fI^}J_;LC|vKHdQYa*|2p zf{}p#$+C%!!EVk1n0>?S!^|0nr3)>V9w@E%j`hRKZ`B%peQcJ~+T9+9JO8)||C+?@ zsza3kL7SJr3m|J#Oc~d*Gi)^)N8(YnYecHY_@r&Z5?p57GNn3wVA7)WMZUzn=X;l! z^-dv#U6rYJxFctIcHE*B_ZM~S7mS;AYUNW#8Y4@Uf%Z=UcSaOr$w*(J`pn?BN~Om~Xa4*${%bqpNYROePv*%9NWo%qcL zSJu;Uejhr{n)eMb-6h?2$X8RaFap>a?~_Xuv1!@bWL1u`9iY#&2*1L1C3GmmccGy- z(1BULzEhZLnY#(UmJoD@HhvL7c>*AS$T-ph-{8gQ61SLj0$gl^s1Idf=3qbwv7X(n zlwoqkCGeZfNEB^k!n|Ub#enG+*U$vW#WBwkVMHeTnBiiYPV?ZobGc(6IT%<+rchRB z9WusGupcvBetntX!UX`<(C7v!Lg|o8{u0+X6hV%L)X8`pit$+k9@h}Ex^;CXn6Zj= zTg;+G9K^=38O;24{A(294EV(YG!-N9H3~aP*YN{{lX=Qm5;+u~Ix$KzW+Dk<24MbE z_qgz`KkCG|Tt+Y@;8*{;n?Yb5_U#INj+=8pY#{g7Xtzy6gWQt{| zGGTW#(8YZIEdnkVAOq5zvFBpiqAR5Hml8PJMMk78e**EV6o!>Wf~h6YeFHk`2H zn)@KYkT(ca3JjUkEM6%fyH}E5PH@ivWIBVI=#wmxo7qCo;SxWF@Qb* z%J7D>WPDk6smHQe;i0OQTW7zpdU4bPyC_yA3;Y=FpRmU-kryz_F4DaCoC1H(Hs7h) z_17T|wQTn|^z7!gDRa zu2bSgwLm9f3^Qt?o>V~p_`L?RdPtvLdnZu0md^TD$CSTA!%^WCPwmH)9I-y`CgtWORAgr?oAH< z!MZ7s82ElpVW7o__;m-%pJFnG`4K+X(FsG>a97n?^&MRyA929M%fT}Tw@wEZQuqfG=PD+e^|vgNWOcu0LPD7E-=y0=q64$C;a zI;kBxapdAlZi-P2b|);j{qA^Xu;3el#ABX<&B9r3;bXgb?Hh;Gp6k;YZSY!8|5$}8 z=PMUnAkhT2$kY^2RF@7Y2t>REaIh*xYvb8Wd}M833XW#4o4M*FH*D60ygtUG+hO## z`Z3`et1U8NHN2lBo$oG$WZ&g;+lb=JgmU^V1;f=)V0qS45bu_BE*obC*cCQYUI z8E3*oyKplg8i%(7E=!nW>604yB+5&)ZJkTk*Xc!cNzg+PXTLhp` zFb?3K?BFdH3WYZK^4AlRw~8B`r_stXy#a4}d^7Qx^>NO%>ZTV%w~aEs$|@%3TnTQ6 zn{#tF9(#1J>hp~vCcW6I{3;(uTwS579ZoErSo5rZ#w*{qAFL~P*A;wW^y1=dC~Z6{ z9Bs**V18w60t<>6N0CL`LYX>Nfs4Wv-#+2;gf#UTQB6y5-}g(e@b#4 z6kvkeqn8D9kM}z+G0{-q0~W^1N3l>rS$qFvBQY=uZIj8 zds__+uoRD82uq9e`;p9MFvz zJsr>LR^C%Q*uFrx!yH-wupDoiD`|-@NkXI&+WjoT)~5X<^BKH1hrgcRSuOr4}2xgMOZ_}lmFVzAW=>->oJ#gyaTRd{F; zF|umc#hUGtoyJADKfkvP$A_NNK-m&?&z{?sFPgJvh$6^;7@7nO;BHL*{Jz}}o~C-A z;aZpFxa!J+pCAlapabR>CWAL}(QbgqNdQ&>0Yfrcnl6z-e@LG3Lb*o9%&hZlal8iB7qv5aevKziE9)#09 zBPOLk>k}ZlGCtGD;9R+8q@f8umPL`>Ngxx3=)J%xR3;^inUUxh?l6WN+6DZf6Wnto zYFTrbURLc$coU&?3|fYq-_C(gkWhZZm2xgFZo88d>`F&Wox|FD0X7o7nVL8oG*HbOb%@4@OW#j!c0 zGw6m#W)DLTfGT{NOsSC^_Od|sY>L~BddP@PfjK-x64oe|pm^OR0QnIV;n|M~8G_1i zZoWiFOQE0}`A{6B^x}B~FlEwYsgxAk1f{{>wK;39^p-_4N1Lmq%a)vAfi-&buF0x{ zga3?N0fe%C8Njay*pc6ikHZ%I58~OiLe1;%+x%*01RY~yDrFM@ta|7dnNW0kelkv<-s@P}yqg?lCRrP$Aw2wFT z`~&MYQ3qGJfBzo7E60sEzj;Mf%8~>CJpfDqE#$`x!PQC67W|uK8D+h}bC@qMn+kvp zuushF)1#Sudzy9UPTIgyX6}!_FZjpXEswD`>?CYzx{XgCuFFRHM^fJ}<_*f8Ub4<@8sZ0Kzn>fVeNiQTHmCb7Fv;i zUG8LXEw4Jb<500;eH4F9kBU?mgsx0Ns5yQT(=)hGr&DSS(Mp3(m6;x-$jr;pX$v)3 zL511bN?n+`P@kKSnXAkyR4Ze(T6LmApVvJ@rP)VaUpe&4oV2iCldZ+KJ{gu~;wkHy z5j%rCC0kCbHQGo#ek$$Yn}!^ldHK3L&xjFk<1UY%l)Ld^^UKpi#=0k?TEsmPqSC{( z8HEOARxJu3X-Gh75iRc(k2#N`m3qA*M;WBk<|%cXQ73ut;iv=I3V(NwMqD>-Q5|-W zcZoq}WIEh)@XS{?ad{W%3?FksVnD>v0c9=KceerOT0}+#?xT zlO>3@aSTBpwHkx6#1N&;%u80N3|+BI-aY~e$rwaiJ9wiKy6kvCKyYFk0>?_ zky1DU`%<(sT&>Mes1tK_N<}@YqfgN{t)@I4-cG!3ynjW|o}*-TiS;-Rouv4lC@v5TSM&wO+ep*VG`Gu@ zIpY1%HIruYhIAi;ns9R2+r7`9f0@)d{oI@>uP3?;=8^#jfr~Uz!-}@qwO{18GNsqJ zmj^A+6(d(%9g&zA8?4o6lxH~PBbEt!QeY)qhQxW4b*Dlc7YSC@Rnl@@1#%R+@yS3$ zlc}&jyQ#<*Z2uFJLP{zOqWB?*6e3#4a(Ety+l0Z;QehY+X|<@Z5kHPr=4*AOi8=^O zl_sYV%(%MXT^C;;wTv1W>3Ku{bYGgL6b|HGQskm}Ha`YzF7+oQ2#-PH z$CPjk3WzCCYX0T~gY(4?X6opDTtl@evCLZh?_};O&-L`^cXe>&=F`9N`kbxv_uewe zQfkqDQA1*`t$-M|+n-W$Gt?n`waz8DdD zT&zN8h%PiJY*{pnB6q*bM;od6dWO(Ab!#H ze0+J>%JodC$Dd_x8_yi(U>7t#R;N_v7Z{WWxI_iPu<3uUgoWN5+yikvd3E$F%0)qj zZ}a=U_gL75Uk`?5oOlDf*6+qOLtd=5J5cEUF>3hP1s#nJg$rTJfnPTpmT;e{cYe-y z^qWz*1pUi(h)?zuQtlj-NINQZN7y=g6TeOr8M(=QJO1_Ki)h01vJQ?QNvdH4-O6K&+Wg{XNKi$>t#AL)W#}JuoY5{f-FfC7sCD_{H8x5;N z1#}&~l0Lco?&Z_rj#c|TwysrLu5tba`$=l{SU3kw!Z@=_^s2{P7#(fda0ed)~ zB|cf>e%8cds;1&)A8rg6bwxY08g*&BvOulMROk@-4@|O*)8^gJk=WaKzOw+Zf{l=y z0~z`{M;wVrIrxSX!}USwVM>65AE+~P22rhfPq>B8^+sgA$cCImT-=(^ZLNnJ{Ti0YT$`?cMQ*Lm|UG{&-1W6q@mXbnFr4UKL3a5BMpjLxZ#g0cxj4A#QBvllX2 zi$5C9$h)oTs~wX*MJhry=t^h_cjwLOafc)dN#Tm3Exdf(e%Sv^8m&D-d z;yQX6eP_>))f4~vF~ncW56t;@oMQoGqCp|wmml7%mOfo^YL2^4T|+k-vO6w?b-%xO zrjtp(Za3F`TFsmvefdN<-+ho$_l!cpV)q=S?mn3CF!f~Ig&$ni@ke(Kz3dwP$4lE$ zV8Y$Q{mFMlKM#)uCG>!=_Q~mVYjXR6+5QFoHbazqw*E1;zQf>}P@xFF&hSBol)W<_#K?fMVbTj)(w@TlJZ6c z&Z2Lc3g-^hY6mEl1xnp0Gh{&IFOF2$i2`FoQV}Q7!owVn4{}dX>eU#`@5Z!wfjPw~ zM<&Tj%mMOb!l%YO3~8PD!jIKyvjI6ni-2#X2LkI-1gh2A%v-SSdis2aB?qc4qsp|E z6_<7?*(q;vGj(l&)AtQqUr#7{l=^Xm@19!6aKr-0G*iXm5zE!u{Qu5p)E%mwB^rc! zAWl$lZlT7QDxT9Teq9P)n{s3Rjfz)q@7vc0j)qqXv|HbM42^4da_^_f<4m?BwjYfc zx+Q0Sr&oq%=477<^*cXp^=SVNx4NRP-spU=LZi_d;sJh&lyl68Two!0i9_v-AU~dx zyY~Zhbf+B?wLC=SakRkXAYA&0{-5nw%R|s?ZGHjlIM84?BO*&FncN`^C>{X4yt5a= zL7^oDy7{!Zeho#e8R`F%IX6EfIufnQ)M@p~OsytsD%Vo@1ye&Y?+FqNLZk-JG#OJd zVTGE^i^PVVPSev6r!}mIB}Hiy3Ntg6dInpPV~Ao}xrOnMc3Im{d}xU>vv55Zt~-lk zkdZwKztE1_{AHAbGOLG(+C}#WQfr%qb(GDX3g^eH_ml=hN@(uKQ|!h|%unj{(y(nq zH&uTf@cfu-$xfFg-9}L8BL$oZ0cZMw%_7D+ueXAr<^9u|$lz^Tp2oktSU4N<-0fF41}!aGGpdR-52NPB3!fqrAT4mrP4pNilYnr^ zj8MraHPIGpeUa^D#EGy$y#NNE6_t6&Cp2Db+Oe#3PNl&x7LNxj`yawX*Jt%EFfSBI zj#LDXwhM-QqeqRj--x?;cS?S~j{nx+H|8(LEvS<_8UYLo64_@9vVyV{swpE&!Bdg= zZw~2$s3}p}oKTHHS9;ExY@pgx)6nnJI)x>Ei2Rh=3pW_|&4Ufna_4mzbG>Qzz{nU! zidbQ(ykit1fC)6Qz?7g8#e^wT>P=EB<0ZU>jc<|jNE8ztttgOT8R63ucLu`I$cVG# zG!miS2nMo~yROATUIL_ku+%mIrKPx+fMV#fRJRv|NWyXD)Hd}YeP4m&tW6W zdJSTlan+?o@r7p^#fi~Yi;sCNPW*vIoR)KK#x@$!`0|J?cDGz)#JqWnV)oqAHyM1gexSRo&Yt1+t;R)i=Fiok4xa+kxd5S6ha(}(G_`HoVwje@h4 zT6+Ta4n|_K3{21?6hit3hpa?h670b@+JklEar`?Vp>sNfrb@SfYsGG30R4?~(X53?&euM={jcR0gFWt9cMQuz0SI5fWFCUhy_8}hv${OGsqW91 zx3(*%Htf1S4;b5pujfq{@7TO~^S*TtB?)c?-R&?*giXx{)VgaIp$WF5t+k1N)n0j8 zoAStvC+WVP8I^gsikNFhhYJ~AVNkOyG#M;A!N$WjY?)w5TCZSZ5 zInc!&O5nB33T>ewv?Noh%*x)((2$|xViDFu3Q zgi2|qxmzYwI_eJwxE`Opif_(u#I7Ao?u7ik_e-q9tJ}eU`Q`h{_`wJga5MgxvO!vP z)9)OLwT<0$bhpE21TdvkD>DtH$d-IYnBTUdjizXt3%b}91_?G-RT?L;A}G9dx3iX&2}TY)aBkZz&QJ^6poGFcQ$O!)2744g0wsbB5J8(oA6kf zVD2p>(!+$I&R*V0NBAXXo*Pp} z!H_m%f)=mwSBkY$FFv9)m;IlaK+*y7*GLuL>;J@UTfF-}<6LUDNLI{13vv6%m=)IS zJ9!;FJGt6n`ID$M-ob~KT^}*6e4Nb~7|i`}gMR*QRu(sX#fFr5-j3N~(jN>w9yXdi z;nA!fogRsPoZI>N;@7N4jY!ZFVS%WQu8n@vv|*3c>upO?Q|`>(l>XOg<4`BRjSP;n zvw3(pdrT#L@Qy_zYF$?oo+G}x?fW06)(uOW%DLU%YI>zL^%swGgCaAyQZ|}ywV&9~ zn=KXb+!4CN7nO>D9!ptQE=0S)hzeHrd$?2p;xz%4U+Z`ESa#LQ)k{~0jz9e2%l7m65&iwsYfN@UYKpY#LX81Ls|s8Hgkn{LudV1BhMdm@OcI+=>~RzrtHeE~MQiB?@Hn%jM}ZlXzMu%)>6Vh|7Z5V9?;tSKP5@6Qn_1&O`l2adt5duYPC zTjrarUVXW0`$ziYgw;&+D~98VP$*ETPR1gkfwFMm+6MOSWjo}3QH^C+@CK~>^de{d zp#Gm<@JH8GEY1#i#k9Jfcy~uD3oXUz!>^;q+h~#&oqsa1{SxyBg)93X^4K*JC=TN- z*E)>cy8lbFKxrFN8X|*)0M%S0)hy+Um+bf=og5K5ZLG@%8acCYKOhmx>oOU3=UeF3 z3hU^{JulhRE!tnZt@fgiZJIQj#FoKwqS;lQo(ul8dB+)d!7ss(-iHD~iw7j^7yWH% zyXWiTf84I>mV0&9Zl43F=YD_;gdGCahkQ&*lLXWQ;I>FAH;SD``H~$dA}Vv={k8<6_&D%eGM_{xz& z<<8;LsomSdJ+c+3(9D<%aWmcG5J!6Jcnh6-=+3!)JFzG6>JvlQ0$6@($B6ikeX=H< z=_c4R{LQT%%S0pwaqx5qlrugu)5-g{ol`hI-oL){oU2hH;s=Y15`k8_-|z2~$F&Vx zi{@mVd9?Z7*1`jeV6bA%tbxqikcUo3UoIQf>yIjU12QaL2WPQ+eC+DIFPAhP(@)+r zKG7Ky% zi3&m(?1*o9-n_Bi?@5xY?)ikhlQU9RVJy!QU|7}!*S_VKE~S*!NOGp_nK0c{M2&^# zCz^#>%=++(%*r1!;-_`${Z$yjCQ-{KS1V$734V63ksW{Q;(7hq^?FBq0Ga7kJobQ9 z9bF;+cXjCX!;|NBSmbieYeo41i)k?4wfpx^1-{%B@M`=Mb?ON7@=+9$3`Epa^qsoq zGJIi#>3>-^r#f7{v7LuJEAgX&N<*<$mlvCTMKT z@Y_xk09CvKOrct;nD@z^8o50ni#jmVa?MLA9fr_P zSMVO@6LpCR>-@w1%g!HXbJpj~5j5g|ZeObn5cF+Wx_*|lz8k_XrEM7Bg!6io z?WJs|e+`PbZ!UF3mDi_x%g3g!q{4{J9tJ`XN>S2JT{CFe*ia+p7z#m_7wRS zs-h%B7J{=f?D3d{;Mk-=DxINFq5ep**DO9!rsYJK9?*?%1PD1LwUMyD-eM;wUIvL2!;h=F-b z^-s5F%0~)skDr~byP%=WYpMx_AL(9io@{Tu#4e+8M2_wPZG>M?ewuK3+)qPPv5Uv+ zQ{fK1lxyafTE)a9M8-@)0EnCD;{%u9*Sgj&Tzkgj;buSehe=?t6MLhBCkDKw1KB-2 zVm6##+;I~&vR0!MHB!Kaa2x>nEh&diEg|%Y9&_*Psq$+thol@?y7QHS7M;t6sE!{Pa? zNPV#XQo!AvaZ`sjphuW$p#e=8M}6vlwuh?s!RStQ zELN&=u;t0e`Y&b9qodGDc?4R*SfO5cjT#fI7%Fn2B3eFgBFJ-YC6L zA6%OG%iEg0C#qj)h6Xc0-u!PhjJ4qkRZ>{7dWfZ8N74b@_dX$>X0TQCaQn)E< zVf+?UbT@K=$YD`P*jfXy;~6|2yB8D&KXN4t`7((g#c9>oi3d1;KGhu+A*yvwi4{FE zxS%jvm7^PBOTX~vt}OJjc8ztpD9O)H%HQ1 z-bjk+z|j0cbp>C%c7tP>?Qw_ba7n0XBAB0E^6Hqg9$v&C0&x zvf>Lh8esKqVzP%Llgw)A%fJ(~WFiAy88QXgNK6@FtB@yILpjDP4{<4qOa9_oz~%4R zgBYE$1W+rpJO!v{BGN@cDPRDU1>rHOdyX@dAhTMGldgu?Ww# zMZ3L8yu?;`fdi)`Ffw>6i?2LRds`4b=r60v8-!WU9#7uAg_ZBzeLRFT;?8-4Q}6AM z6z!><_=9cy1MRY;T14I^96r(P-m0|A)q6I)IR57Lf%VT!a989Wt_ z3jDGVU+B2}^2-0Np;<*u2OOtCDr^3F+W!~9^PWywEqXts>G)S;{lvjDfIP`B`J~1! zZMVp4*NLqgyX0X?ht4#7FPLP3Dr`w8XguC9}jZD_$e%_V3F@i>kyc zz7Lf{jUy)e``EGYGYtgSS|rY;*)#!yEH^+7c)G0Hon!y5n45U!XSCX9-JHpjH}wM> z2;`sk9UoDibSAZ;?&0CWlWE4#!qCC|NgZe(&}l{Ns^xvDC*hXz9ycI^6S*}ott-sM zkrx_zt^9S2_vy``N&oykdLpbg!YBH~fWP_AXUq!h-fUkd3Mx`v`WYaT=^>f1wedju54{f0RTw)5I*f<`PVV6auL_fnj2*j{{Vjw6TM|&y5mv^K7?~COjAiE_wpE-WsaOCh z(4K%?MLXL-Bg&{52)zW<&NxsD9b>ZZIFhBr67GO!vqq9d&QJne1?_1exk5|H2rQu6 z#3?>Q(YBb`oeY`TQg~b_u)y3_O_~j69TL3ea5<)Y!UGr$W`Bpm1qS$sXuOz=0%X3x z7)xSf`=3Dl%M|&>1jE-Gk_<~WhN3DE+lq{!jtTJP0H6$(>|~0ah}z~iUi3n`cx9GK zr_8+Al~ln@lPM!1FzVt^DG2n?h}RknYNu2(1Lmugv}On+2#Bsx_~Ed z%UHDlf90=Vue~HNVh2cvO}I#Wq+6+zc5$PVe5YuSwRD~g?4SmcCYi-be{b~MP1D*=s@ou z4BQOpQp-@Kue;@W&wR5p<)rtrjIkBYFq9{>&?kf9v5x+dwK*;2)VWvl=jAfZ zz}zeOM~3V#>AXpL!a3vW=$+!$v5a(&cp};>UZIh&qy#+bQU@j*Lm^~r3z1b6$go~5 zCivR9+;U;G?;Ebw?weh_SNu8c(MM282Pe-}mvwcWW%GJ_?~`=JL;IOD`llz|#(Mko zJhNEbZcstGTjYwjYPXmDA_EfC+TsQQIVF)3_KE0=B)^FMKi7O6fHWw>B}G@UgLxN0V+0ncOw0 zPp9LY?k7S$1w)78Zyd)M&E*2}6ll53`3>_#CsB9{7OmS9jz!?hsa@2BF)UnT7)Di z$S8P{2rNM-J>gCVg0Mt948coI_zIVV<`k{NE{+J&*WQ4}7l+ z^URk_7fac9_xrSd@pq?&5E^Q|3|P6QZY&J zg;Nj>5q6LX^6*p27jq7Uz1jG#!r3V<2)=77c|S?NHqpQK<;?B>)Snz2{K;4n8b4zY zm-k>3v3$_HKHb?jA|tUg(wzvJwc+6mG1b?`12~2R5qGYPO~t_WewP;}7%~~cxm-xr zG8V3WtI`=hyuL^hZHw&?S_l9=!FOrWvytIlwkLU(zvP{FuGY91t65pSBoE%k=ATa?OFU5-g6}<6N18!MA`C1f!BEC zM1PePSvL#IW_3wDTT?%1N7*m;JNWoid^d4see-QgH)j{cw0keMcOX3w)}aPGa-Zq> z6z%CzlXgf`g4dC#K}c$t;aP9FaUM11+{4AzmD4v$MsFPF`gwy2Pn=6-2`G9X zVprs68QsrmFv30v4@RHqwdE53u01!)GcM$IJ$_`k9NCOQs6BE?&v!PhV}7ln-@MOS z7W^D_%b58pQ+!tGB@x0`nL~o#6~}xzzGqY4j#Spd%eKMbEbx^E&7UrNq2-JvQuX{i;_@X^&08 zH;19m>_6SNh^BpslDz!<_18*?<0NFBuhQfzO46%QovHET&F-qK66T`y(datM>Z!=; z*>+xD!f7NffG@*TLA*NB9Ufv>1{fJ;aAp9pP!tuhyA|WvEjG5# zXFt2U8;qX{DBm6i-}m1=H&5)d*4k^|%v?ogo~|%o=a8+@=F9aCMGCD>sZn=x=*rgq zqnTf@_WN~A*;aRiE=@wHDSi^iFQibbRjBn*3cXgDnciETnU|y06sog& z7iMQGwBf2kU2Z~Vt|F^YrHIvNREct3Ugr#@`Y$5ZtclyBaTr}!Id|1h7yskfo~( zg?dHSMifNTkdSUfjD0{n=01v2=ydWNMQ^PpPodq6Tye^ueTI^uo zJ_ec6N;ucH2jU8fNiDNW7@l&+qW`$RPN2N!rrnx64;GUqGXFHsV#?8H;iK;F)AkQZ zRJ!y-Qak}QF{6$==SYe^zFSq)^TKKOTXm?TKjPAD7}qk}$g?$eGcOG}bEsX})Uihz|r-v7X-U#^%v$$Jp;4nj?) zA##|!P-Uhb7DKKrNh-*a>lJZ@ib6$ixgIX*mN6FgT?ZpOT!kcy7<-QhWER*@QL+MA z+IuD=Yq}IMHjbg_u|};|jL`SjWacHym3nvR-zM^P-;>3!KOX24gcJ6X_Jif8a1)76 zKA1K5N838zv;X`GZ!@RHsSgt2G0?xDuF`Jd$p6X|)q=6RvKreX2Osn(RH2fWD6%5e zI)zp*z?Py@NECtaC?uJSZJb9TPw2Tn(Znl;7b@MZ5xDQKZNU*Jz^@+8o6M=;0aBQ6+Eq`Md2R^ea902V|LCoq!u~_Jhcd*=m3=VHm>2sDWfJ%yeLZ8}v8tzLO!^dbiU_<=TrjM8 z(KmSCUJ`S|@#31%Hi2O*)GUX9#*Fqc@3DKb~XY93Q z$WhcqK%)^uD`0qb704fA<`<@gL?-G@1fhr)B1Xh=`JRZ^faxeC6NMA0aY03{%;P9U zzD8S;s0AQUs&ndL6(7nkU%55lXVcmqAv9vP{iu2MB`_cDhq7}PS2r8Kj}c*Wzm z9FE_S4@Z99DNad`nLY*?-_b;7j6Swi=yt4LwM28G`he~v@Vxq zMCfE#sK+E0&_Y}C9Rg`5nNn4XF6`gE@UidxgKbj_``5yDGC%jZdec8Xn>wBRw@O@O z?f;k%m_e9899kiqKYdY5*7d_FkEYoMUp~Uc9nkn#twNDspjRB^Q6+4JP5(s_wzkW| zxGUm$@oSmQB$H43Yc1^n*5uRV{)mB+13SvN9oH4cyBiB_;C|`Le5?-k_A7wIqx(b+jbj+dm@|z z6Wi)W(Thv(E&G=4{Z*SDe#k$DhuqN)japR_uP9I{Gv!)D3t(=Sb6cYHGnV)m=4mDn zU5J6Ia{y|uawU<7mck;pQ#moGrw|WEAlR{t zt#u0VE>i0L<{1{MF>3DzkCEe|A6N89t(ju}S32DpJG(W0YZlnMME?V|m-a(+0t~{E5&EORcujEzV@WTwT$XKSvgIWB{6R3C!}RA?sx`ZuB^d zVr^qL9jkVT5O@qwXrB=jBJs#kXz#(oZ6)qY@^Ut`J`{4#zx}s?cb4863GuXc!p9F8 z!#ZrKII||bW8d>>>5LJY8i+L>8+fKFWkOzzV@*Wc2kX%oE`G&DTG(8hNJ;J!y4Z)V zi0J$E%-r{j!?dWLv}zrHx@K35h2r_?RL^g723rncjJr$n%~TY~ojX{g>8DT>D6}I@ zksi?+u1w@gU|9hNBuNY`F4rSLp;KWv>cVMRZ8O5HMkdKi&48gSL{AO435&}4CWzH) zvVrKria?L02ZJh;2dh+?%$qRKD{h zeC|=p^X85bhz0I$A_py-Mqd~@|I53_W75w}nI}M95w~|pZlT(+wf`ko7dYiPpX%wK zld&nCDLQg$v_aPdzBO^G4;iO_%A^L zkoahnGE=M3DKa(ctO}l`=o==3G5-k?4nTA<5MVMUG2w;k%nQ_ptz?*~h-=(F)RGWQ zLSbg6LdU_D^f)3MqZ%+=Xq~kU#fObhWETF#gSML_1{nmV=o{mx$zM(!6j@!xWH;m8 zTcv3d!DL#l9@<`>%6g71O>6MBwOMuTBIg%*R{zCHS|GQU`8pJSsM@dTIEjoxcLriE=zfR%XWGiPE&b*X~YkrX;4Kv~s0&AKV@3IbKo(~fk92v6)TiCbjk0?MnL(MS} zmZO+(xl*-BW@Tu?BiQf=J%_|GQBm>&3v59J4B<@!R5SoPL;pr%)D2N9Y-itTEf(<; z0O(=4wh4)b@GgQh&}J!bI-oWnWY*bCeY2}yxyLtK{cu``4tBP?#c4uy|5)sPQvC2M zx475h@-z1uv4q<^vrYJ?#C}JgO4Kv0a#rQZg)^EqVurs#NRJU~x)Bp#7CaeAGB}0? z3tiZVy=Vf^Hhi8A#DpJ&xPx#V7m}r4eHCMlN59#fc57N<{qxglKRjs@Zc|Y3KSe7S z`7!;FwTW*&6}&(aE(~`z`pF!)b7P6i%(yFy#=0-qQ}OR8WHpiu4~E8@dKDYyvCV?L?S+||S;mk&8vMuT)u#XPoSkmzfhi_yl>W>7N&!-cS6=fI> znutI^GzVWb;V^C_Fj1L58L}>DvBpl&dWOTm_h({Dd*{2D;Tlm~979^^_#`$=B`?tR zA?%)3wafz774bJK+u8l>srFCweD3qMZvfkf^Y;vP-_*ma{LMUDTCR9CvMU2o!iSXc zH#reHqO$no&{-!W*S02-6OAYc;q#czL!e?yONO-}w>^T?Dhb>q!&Mv-iK zupX$SkZZv{GNl8M7%xVWIH9C7=5%V$E=Hz_4I(;&Cxa94Ha?vXb3a2!gAqn+Ii8@X zTH8`jY+>vD50)-L!a<0xMD5ZO^m3JA659qECdPA&z5fd!nuHQ{W^@M+$avRsR%#0M zVIwjXimVL2m3=@ua-y3dN_4H-h8BwL$zQ0QBI~}4j)YkGTS{XQk4}fnV=Dk7jC5>Gtt@O=D={66OlTF{1f7jQh#TofJ7dTH9-SkmJrVLrQfJzkQ4efu z<(o<95O4|B!6OejG!iiQ3>OV@rJ)=P!-+)18V1bn=5vA}i_~cK#~{>p^QFdxX1@vq zzKBY&11k`MEN2X;!}2it#p8TuFyEEV!WPi}bGyF-o;iOSn?D%p80J3xw^;0-?@7^!g?H>cY5V6sWRPZ z)`dOqXN5X%XvKvn$%pk`H6?9c(01+EuQ^FOCFUW%-`db_)}UnO_g(%^gIGIiN61WX zp~EYS>h^k6kCWZE?T{9CU9jH|-+Q-PGs=OAn-l#8ti8FX^Bnwj$moO8jZasf%X(&V zqK{mg6$|D|xV%u6?p;UzAdD77(Ur0#lM`1?cRb!%boV7%h@B#1R$8;~?Q5CxRGDqY*^$z%^9Rh# z+wM5m(d0XH&+~r+MxS~+b4ibm58kDjoypFY(7v!z<6)eo^_l@0Sv}Wx8(Y?Q$!Ff@ zdL;BiSSYR|=i+Y4mshTGzwF_X_GtBxa%)n37W{f4kItFElq!TZ|waIy+rI_)D2@tMOKbWGeKPFpH{TX_4TpckK?u_e5%~9 z`ih%yszn;tP4M-9pZyRX4+$HCjYDX_b1ROB_WM^J#J77bcMfI9v47#7p z{n6uw3%)xPTun)JHJ@X(4Vh~|7W>sQd#o>*oEMx*nmOU^zTcOt`<3d3g8=d^TrpY~ zaV{j&HG8Vw!q3tI_Bf(7D3NC$HZfyTrA7Wx$KzMi;N^rr;^@K~w!^?nGwJ zn}7RF=)=`zXDh?!O+kJKfB_J82v!~TH7Q9FlKW!14D_hM;51m4Y_yoZLa0qh8~EJs zL3Ia~w>vN-_f}E*qy1NFA_oM+#5R3g;$uGOQevl?7xQ)xAKM5|0Fr_2S5Gb-+-|Mh zvu5ql>qkCZ+2MQ=6`C5L4L32hJdh7lqFR^6uN#~{(0T5P*E^-EHNO@>_uXE%UK-G- zS*^O$Nw?PJ%iQH+8iTmF5+-Hr#W9<*^P38Gr6%XgR($Q)Urc)-MEi@uAH5rG5;Eo2 z%Fll_9+}d2TG`%)VT<5p?ss|HKR#cW9j^~Ke!FhVb$9|W6R(50*j4vQzH8TezwIyD z_Bxd1+YQ;+5^grwITIY)=~`7DC|RsKwDRQE1<3=FD&b9yxy1o@Z}mKLd~tT?dE1Ya z=vn9z>5VWfBi?Z4-y5lHC9K*@zC8e|D_Mwq=Cr-5@6)^2(v)n6-8Yg}9(vkM09$Hv$IB-3+uJ?3cDn3>iDW>n@6DZo4ebn0YQM5u|tkLGhD>465fjTHI^O!wXVADn@rjB4PSg- z$(V42eszN20gk^;Jj+Fc`@c{PjhAE`yVdlmcp3UhdUXnyR+-6G9oAZAO!jh%SV9<*+AW5<+5=1}L@qLb>V_VXv4Q#<#t*Y5tt}{3oG#FRIrc z3x9nSBG;sBcg6~F(4`ea=M6{?e)#?!GLR-{4#@z*?xHvtq2*o0Gk|(7kh-=H#vDue z@cM-Fjx6WJoP{%Bez-rX=Dr^n(4%sqHB%cz@s3Z8?fcq@5^&OSN#fm3ng<|N3SPDSOb+Lm`46;SpbJ>BKD z_!`7DghV;K%>00fMZvj}dDYBK^EcZb ztVlE;4L#$B4!N`F*0B#eZC-zvv@C6&r^v_yO?cUj_&YD26nl-BK2_OqOp!>3h;v9T z^{OK`Nljgw@UG4Q%UsS65bzoc-uN})8AIXtcZ+QoANY9R|31dR-Ng8GDrUtzeN2W8%D4XN-xxflw|() zv1TtwfIVn2aHm5KA9*1hBBfuDowPd^7;|yyC<2yaS-b*@O$v2pKPlB>8JrY1>veN7 zKZ2g8_RuXPGAej>JO_`*9tDLVk2=wX0t;ym!Zn)O)DxsuKIx2#5HWsKYQ+o>DJYCm z=4gjm(-({-ThyBI!Te_RWrnWMz{LK&KN34>DPmg`SrZ>sBL)=QXW}UJ;Yy7einN#s z4$Ci8m0|js+r`*7hTX}7UBG|O1qJB961)E-=xEWbPIPJ*TLF3QIZel*9lMCSaav55H4V4ZcAG6pfr8-Q#s%LCge4$#s z*s+T;KT2vTo!MLpfg2d+66$-kjv*ut6sWlPKK#&_Zf9e4rxg^m9 zfvvPJ%QMclC<2T}jjAL^qt*pyWht@}HBd2p=3*)8FEm(-%`V}2F5{0JBGqFNo@ez~@>eDJSq=rhT|R{zd`MJab!X?2bI5h6 zOXH=Kx?VygH}F3Qq;I-Q_XOAt7X7{Q_{+5g6-ocebD#X94 zq@9*uk$#w}t*txVe=QnsOT)mI@c}w|+Ohm?FaHIZS+dXVb5v^&OzX57$hrsmToW_y zS^7(_;JEQ$=MMGz#-XW5hSvihz4Z17>4bL=sg}v&`Zr#z7-SDQjfC#+ZpXr58hE7} zktCO4Q-uhyt*<>0=*o6|^fIAv!|e39`<~BBj|FV(1v7u>;*CURTDi08?2Zg(mt#|^ zVLxH075*ZBUEZ&JyD_xyl+CjTnVopH3y_<(u?C%Yi!4m?zvGtuTqp0B`1^mS7A>6! zgg11Bm2Xy>K)9(Pta}xi7{%cmjGJcmT<&mh+=K_~Z;l%OZeeUKZ}NRMGhUIS7*TN3 z1}rH<^@*h1ZZ%=}n@LMbMnxC6YaPeJ3)k;68E13g(%%#B4mV5Fei(9}HKq$vU7Scu zLWe=ld$2w2F0iz3F{dT!Ll|4k*^^ACAWP8+k=zpOOD%={qUgexhr@{BirA8Pq zK)v`1!Sf~5jIm`q2tkh+YQ*CC1k?z%V4TINq@~CMUnlG(LI;|g8!k^oY|H2a5DO9u z%5LWDQJ!cLVvM{go=t?uz&2(a;0TbF)50q(A8xk9=4>CKXa?dNImOB>QskMSP;gYW z%o>}HTn8^-ZR7ZB)ot&5-Q>{ax2PU}vi>4oz1Fn;(Syq$BQoryv&?W0gcYC)UzW8W zca<}s?$o~3SLH>q%wir}-IX3SlZpOe_{P-!v>AI*$bZz8I<?bd89{I6*Qm(eN-2cMXV-+BUnsA2> zGj6D^4J*8UUxFq)>|NZY@sn~lJZO6L_n@&J$;F6+GQ>LaX zsNmELVRx@#U6V6CO49W%|A0?fMsG?IL`5mM znbtcP5T)B`w{v?$w#b1^awb2re?GWA?XlgFbHWM zEJY(3x)%O4LRczELGVype3?fhuoE1%oMA+ZgmEQ0C|Y$#B3*JVbB@fLKG#qGzn*CYSyb;}cM=4`V4M-13zsGKz5%o8D#Km{MV>LYtt1LL(77v*!eVfQ>x|xn5U%r#$%pnq!1pn~# zW`OK&6OHwtIl}V;RwWIJkA0bP=0tC%0rx)ieZOmUv3TU+m!kY8-qSh^{WKKnC2n49 zBDNnew|i%HAKxIM4C0~>!`kq1hJ>4F3)4zHcou9b22u4#$(vxn6NGblu$C4qwEoz? za$kN=Bwc5V?GRcBtav53(C4*s#KOTA5zZG&x;g!ndH|1emyWr-)6F9?vrWnT>oM(j z@%Zp=VvoUA;WpbUYjt(LF5EHmm>|uBtihp^(Ha}DRVcS73~*Dn`DE*?ii^EpPA;vU zQj#gZ{ui*>2+uA*PE~B~et!HDmxwc0NBH%j>BtOk1_x?;NBj7|UZ%e`cm?+)`;Xs8 z{gE9>3q|q3He8uiw=f~y0zY~TMR+Tc2*Kgza=kNfAZ8Bw(?SaX>;IQ4-&wYM^Z!bE z2MECX%pcrm$BDTB{X4AA6q#2 zCl=se15TGn^}Qr3>cenwv%a_P(qGw1Xa(TXa>9Tyob;$dm0tOdSWqv9Lt08lulU^c z*ds=tyMfT}*q(jW91_jge++TR&uq$vI$kjpR*aoA9NH%oGakrDJ?vJN+X-vnCnU8_ zzSG~1cHnb_Mn<6)d=9D*QHD6u1i}#+DV7Ej4)JQw2KAyd1TuDxl$(hqMpmJTn1g=~ zgJ0Nm1sXMnZ$)$QpW2S2F!PdKoj^ZA&v~{e**Z_TebKai9hUijt*k>*g_Bg!$;hX*2Cf^eubFp+-yZu)(?TWA#eZ{3~O_y|V zuSk1%^-lcnB5FGT$@G<8Uo0<_k)ug^+sRX^!=)ArtEVk__C1wWu62&84{1O5W$=pydeKkva!Tswv%GI9yN4WS(7%J0w}Pll zL7&+#GuJA6EwQR-bIPXQY_A@#ky*Y{oi87e4%O>2Tq{;)jo>f=i>=b@c7EEqH}$k# zq5n-JDS%&}l)d@2E}h}uM^3O+c1c-$Bk#?NfvV>iAT$cR><@3KuhWpPvmL#83`$Lc z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOB(Jyhd}l+lM_DLQ|R1S9JggWMOn+E@EkJ+)pEt zS0B@B+5na71A~^(h?oKJe_7^kUF)~R705&JjF@Ba(+MsB@NR8#YcsGwevk!s&HdgP zC{Zt|h7q9Z02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnnTemQio;UGet* Kde0cxq6Ywpes2{3 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt deleted file mode 100644 index 4835061786d..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/columns.txt +++ /dev/null @@ -1,259 +0,0 @@ -columns format version: 1 -257 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`milliseconds` UInt64 -`ProfileEvent_Query` UInt64 -`ProfileEvent_SelectQuery` UInt64 -`ProfileEvent_InsertQuery` UInt64 -`ProfileEvent_FailedQuery` UInt64 -`ProfileEvent_FailedSelectQuery` UInt64 -`ProfileEvent_FailedInsertQuery` UInt64 -`ProfileEvent_QueryTimeMicroseconds` UInt64 -`ProfileEvent_SelectQueryTimeMicroseconds` UInt64 -`ProfileEvent_InsertQueryTimeMicroseconds` UInt64 -`ProfileEvent_FileOpen` UInt64 -`ProfileEvent_Seek` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64 -`ProfileEvent_ReadBufferAIORead` UInt64 -`ProfileEvent_ReadBufferAIOReadBytes` UInt64 -`ProfileEvent_WriteBufferAIOWrite` UInt64 -`ProfileEvent_WriteBufferAIOWriteBytes` UInt64 -`ProfileEvent_ReadCompressedBytes` UInt64 -`ProfileEvent_CompressedReadBufferBlocks` UInt64 -`ProfileEvent_CompressedReadBufferBytes` UInt64 -`ProfileEvent_UncompressedCacheHits` UInt64 -`ProfileEvent_UncompressedCacheMisses` UInt64 -`ProfileEvent_UncompressedCacheWeightLost` UInt64 -`ProfileEvent_IOBufferAllocs` UInt64 -`ProfileEvent_IOBufferAllocBytes` UInt64 -`ProfileEvent_ArenaAllocChunks` UInt64 -`ProfileEvent_ArenaAllocBytes` UInt64 -`ProfileEvent_FunctionExecute` UInt64 -`ProfileEvent_TableFunctionExecute` UInt64 -`ProfileEvent_MarkCacheHits` UInt64 -`ProfileEvent_MarkCacheMisses` UInt64 -`ProfileEvent_CreatedReadBufferOrdinary` UInt64 -`ProfileEvent_CreatedReadBufferAIO` UInt64 -`ProfileEvent_CreatedReadBufferAIOFailed` UInt64 -`ProfileEvent_CreatedReadBufferMMap` UInt64 -`ProfileEvent_CreatedReadBufferMMapFailed` UInt64 -`ProfileEvent_CreatedWriteBufferOrdinary` UInt64 -`ProfileEvent_CreatedWriteBufferAIO` UInt64 -`ProfileEvent_CreatedWriteBufferAIOFailed` UInt64 -`ProfileEvent_DiskReadElapsedMicroseconds` UInt64 -`ProfileEvent_DiskWriteElapsedMicroseconds` UInt64 -`ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64 -`ProfileEvent_NetworkSendElapsedMicroseconds` UInt64 -`ProfileEvent_ThrottlerSleepMicroseconds` UInt64 -`ProfileEvent_QueryMaskingRulesMatch` UInt64 -`ProfileEvent_ReplicatedPartFetches` UInt64 -`ProfileEvent_ReplicatedPartFailedFetches` UInt64 -`ProfileEvent_ObsoleteReplicatedParts` UInt64 -`ProfileEvent_ReplicatedPartMerges` UInt64 -`ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64 -`ProfileEvent_ReplicatedPartMutations` UInt64 -`ProfileEvent_ReplicatedPartChecks` UInt64 -`ProfileEvent_ReplicatedPartChecksFailed` UInt64 -`ProfileEvent_ReplicatedDataLoss` UInt64 -`ProfileEvent_InsertedRows` UInt64 -`ProfileEvent_InsertedBytes` UInt64 -`ProfileEvent_DelayedInserts` UInt64 -`ProfileEvent_RejectedInserts` UInt64 -`ProfileEvent_DelayedInsertsMilliseconds` UInt64 -`ProfileEvent_DuplicatedInsertedBlocks` UInt64 -`ProfileEvent_ZooKeeperInit` UInt64 -`ProfileEvent_ZooKeeperTransactions` UInt64 -`ProfileEvent_ZooKeeperList` UInt64 -`ProfileEvent_ZooKeeperCreate` UInt64 -`ProfileEvent_ZooKeeperRemove` UInt64 -`ProfileEvent_ZooKeeperExists` UInt64 -`ProfileEvent_ZooKeeperGet` UInt64 -`ProfileEvent_ZooKeeperSet` UInt64 -`ProfileEvent_ZooKeeperMulti` UInt64 -`ProfileEvent_ZooKeeperCheck` UInt64 -`ProfileEvent_ZooKeeperClose` UInt64 -`ProfileEvent_ZooKeeperWatchResponse` UInt64 -`ProfileEvent_ZooKeeperUserExceptions` UInt64 -`ProfileEvent_ZooKeeperHardwareExceptions` UInt64 -`ProfileEvent_ZooKeeperOtherExceptions` UInt64 -`ProfileEvent_ZooKeeperWaitMicroseconds` UInt64 -`ProfileEvent_ZooKeeperBytesSent` UInt64 -`ProfileEvent_ZooKeeperBytesReceived` UInt64 -`ProfileEvent_DistributedConnectionFailTry` UInt64 -`ProfileEvent_DistributedConnectionMissingTable` UInt64 -`ProfileEvent_DistributedConnectionStaleReplica` UInt64 -`ProfileEvent_DistributedConnectionFailAtAll` UInt64 -`ProfileEvent_CompileAttempt` UInt64 -`ProfileEvent_CompileSuccess` UInt64 -`ProfileEvent_CompileFunction` UInt64 -`ProfileEvent_CompiledFunctionExecute` UInt64 -`ProfileEvent_CompileExpressionsMicroseconds` UInt64 -`ProfileEvent_CompileExpressionsBytes` UInt64 -`ProfileEvent_ExternalSortWritePart` UInt64 -`ProfileEvent_ExternalSortMerge` UInt64 -`ProfileEvent_ExternalAggregationWritePart` UInt64 -`ProfileEvent_ExternalAggregationMerge` UInt64 -`ProfileEvent_ExternalAggregationCompressedBytes` UInt64 -`ProfileEvent_ExternalAggregationUncompressedBytes` UInt64 -`ProfileEvent_SlowRead` UInt64 -`ProfileEvent_ReadBackoff` UInt64 -`ProfileEvent_ReplicaPartialShutdown` UInt64 -`ProfileEvent_SelectedParts` UInt64 -`ProfileEvent_SelectedRanges` UInt64 -`ProfileEvent_SelectedMarks` UInt64 -`ProfileEvent_SelectedRows` UInt64 -`ProfileEvent_SelectedBytes` UInt64 -`ProfileEvent_Merge` UInt64 -`ProfileEvent_MergedRows` UInt64 -`ProfileEvent_MergedUncompressedBytes` UInt64 -`ProfileEvent_MergesTimeMilliseconds` UInt64 -`ProfileEvent_MergeTreeDataWriterRows` UInt64 -`ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64 -`ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64 -`ProfileEvent_MergeTreeDataWriterBlocks` UInt64 -`ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64 -`ProfileEvent_CannotRemoveEphemeralNode` UInt64 -`ProfileEvent_RegexpCreated` UInt64 -`ProfileEvent_ContextLock` UInt64 -`ProfileEvent_StorageBufferFlush` UInt64 -`ProfileEvent_StorageBufferErrorOnFlush` UInt64 -`ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64 -`ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64 -`ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64 -`ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64 -`ProfileEvent_DictCacheKeysRequested` UInt64 -`ProfileEvent_DictCacheKeysRequestedMiss` UInt64 -`ProfileEvent_DictCacheKeysRequestedFound` UInt64 -`ProfileEvent_DictCacheKeysExpired` UInt64 -`ProfileEvent_DictCacheKeysNotFound` UInt64 -`ProfileEvent_DictCacheKeysHit` UInt64 -`ProfileEvent_DictCacheRequestTimeNs` UInt64 -`ProfileEvent_DictCacheRequests` UInt64 -`ProfileEvent_DictCacheLockWriteNs` UInt64 -`ProfileEvent_DictCacheLockReadNs` UInt64 -`ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64 -`ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64 -`ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64 -`ProfileEvent_PolygonsAddedToPool` UInt64 -`ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64 -`ProfileEvent_RWLockAcquiredReadLocks` UInt64 -`ProfileEvent_RWLockAcquiredWriteLocks` UInt64 -`ProfileEvent_RWLockReadersWaitMilliseconds` UInt64 -`ProfileEvent_RWLockWritersWaitMilliseconds` UInt64 -`ProfileEvent_DNSError` UInt64 -`ProfileEvent_RealTimeMicroseconds` UInt64 -`ProfileEvent_UserTimeMicroseconds` UInt64 -`ProfileEvent_SystemTimeMicroseconds` UInt64 -`ProfileEvent_SoftPageFaults` UInt64 -`ProfileEvent_HardPageFaults` UInt64 -`ProfileEvent_VoluntaryContextSwitches` UInt64 -`ProfileEvent_InvoluntaryContextSwitches` UInt64 -`ProfileEvent_OSIOWaitMicroseconds` UInt64 -`ProfileEvent_OSCPUWaitMicroseconds` UInt64 -`ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64 -`ProfileEvent_OSReadBytes` UInt64 -`ProfileEvent_OSWriteBytes` UInt64 -`ProfileEvent_OSReadChars` UInt64 -`ProfileEvent_OSWriteChars` UInt64 -`ProfileEvent_PerfCpuCycles` UInt64 -`ProfileEvent_PerfInstructions` UInt64 -`ProfileEvent_PerfCacheReferences` UInt64 -`ProfileEvent_PerfCacheMisses` UInt64 -`ProfileEvent_PerfBranchInstructions` UInt64 -`ProfileEvent_PerfBranchMisses` UInt64 -`ProfileEvent_PerfBusCycles` UInt64 -`ProfileEvent_PerfStalledCyclesFrontend` UInt64 -`ProfileEvent_PerfStalledCyclesBackend` UInt64 -`ProfileEvent_PerfRefCpuCycles` UInt64 -`ProfileEvent_PerfCpuClock` UInt64 -`ProfileEvent_PerfTaskClock` UInt64 -`ProfileEvent_PerfContextSwitches` UInt64 -`ProfileEvent_PerfCpuMigrations` UInt64 -`ProfileEvent_PerfAlignmentFaults` UInt64 -`ProfileEvent_PerfEmulationFaults` UInt64 -`ProfileEvent_PerfMinEnabledTime` UInt64 -`ProfileEvent_PerfMinEnabledRunningTime` UInt64 -`ProfileEvent_PerfDataTLBReferences` UInt64 -`ProfileEvent_PerfDataTLBMisses` UInt64 -`ProfileEvent_PerfInstructionTLBReferences` UInt64 -`ProfileEvent_PerfInstructionTLBMisses` UInt64 -`ProfileEvent_PerfLocalMemoryReferences` UInt64 -`ProfileEvent_PerfLocalMemoryMisses` UInt64 -`ProfileEvent_CreatedHTTPConnections` UInt64 -`ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64 -`ProfileEvent_QueryProfilerSignalOverruns` UInt64 -`ProfileEvent_CreatedLogEntryForMerge` UInt64 -`ProfileEvent_NotCreatedLogEntryForMerge` UInt64 -`ProfileEvent_CreatedLogEntryForMutation` UInt64 -`ProfileEvent_NotCreatedLogEntryForMutation` UInt64 -`ProfileEvent_S3ReadMicroseconds` UInt64 -`ProfileEvent_S3ReadBytes` UInt64 -`ProfileEvent_S3ReadRequestsCount` UInt64 -`ProfileEvent_S3ReadRequestsErrors` UInt64 -`ProfileEvent_S3ReadRequestsThrottling` UInt64 -`ProfileEvent_S3ReadRequestsRedirects` UInt64 -`ProfileEvent_S3WriteMicroseconds` UInt64 -`ProfileEvent_S3WriteBytes` UInt64 -`ProfileEvent_S3WriteRequestsCount` UInt64 -`ProfileEvent_S3WriteRequestsErrors` UInt64 -`ProfileEvent_S3WriteRequestsThrottling` UInt64 -`ProfileEvent_S3WriteRequestsRedirects` UInt64 -`ProfileEvent_QueryMemoryLimitExceeded` UInt64 -`CurrentMetric_Query` Int64 -`CurrentMetric_Merge` Int64 -`CurrentMetric_PartMutation` Int64 -`CurrentMetric_ReplicatedFetch` Int64 -`CurrentMetric_ReplicatedSend` Int64 -`CurrentMetric_ReplicatedChecks` Int64 -`CurrentMetric_BackgroundPoolTask` Int64 -`CurrentMetric_BackgroundMovePoolTask` Int64 -`CurrentMetric_BackgroundSchedulePoolTask` Int64 -`CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64 -`CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64 -`CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64 -`CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64 -`CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64 -`CurrentMetric_DiskSpaceReservedForMerge` Int64 -`CurrentMetric_DistributedSend` Int64 -`CurrentMetric_QueryPreempted` Int64 -`CurrentMetric_TCPConnection` Int64 -`CurrentMetric_MySQLConnection` Int64 -`CurrentMetric_HTTPConnection` Int64 -`CurrentMetric_InterserverConnection` Int64 -`CurrentMetric_PostgreSQLConnection` Int64 -`CurrentMetric_OpenFileForRead` Int64 -`CurrentMetric_OpenFileForWrite` Int64 -`CurrentMetric_Read` Int64 -`CurrentMetric_Write` Int64 -`CurrentMetric_SendScalars` Int64 -`CurrentMetric_SendExternalTables` Int64 -`CurrentMetric_QueryThread` Int64 -`CurrentMetric_ReadonlyReplica` Int64 -`CurrentMetric_MemoryTracking` Int64 -`CurrentMetric_EphemeralNode` Int64 -`CurrentMetric_ZooKeeperSession` Int64 -`CurrentMetric_ZooKeeperWatch` Int64 -`CurrentMetric_ZooKeeperRequest` Int64 -`CurrentMetric_DelayedInserts` Int64 -`CurrentMetric_ContextLockWait` Int64 -`CurrentMetric_StorageBufferRows` Int64 -`CurrentMetric_StorageBufferBytes` Int64 -`CurrentMetric_DictCacheRequests` Int64 -`CurrentMetric_Revision` Int64 -`CurrentMetric_VersionInteger` Int64 -`CurrentMetric_RWLockWaitingReaders` Int64 -`CurrentMetric_RWLockWaitingWriters` Int64 -`CurrentMetric_RWLockActiveReaders` Int64 -`CurrentMetric_RWLockActiveWriters` Int64 -`CurrentMetric_GlobalThread` Int64 -`CurrentMetric_GlobalThreadActive` Int64 -`CurrentMetric_LocalThread` Int64 -`CurrentMetric_LocalThreadActive` Int64 -`CurrentMetric_DistributedFilesToInsert` Int64 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt deleted file mode 100644 index 301160a9306..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -8 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/partition.dat b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx deleted file mode 100644 index 3709b3e4f82..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4607_4607_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëHª©"`ëH±©"` \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/checksums.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/checksums.txt deleted file mode 100644 index df2f868f3e238b0545f87bcd10382a47e43e9b21..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$c3z@`!4L&OV%^7r2tD!?&< z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOEL%%`Kt3xZ#Bpu>(3pY7Tm^@j&dUd`w1=p@NR8#YcsGwZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnsy^i!!b(-JKN- K=7_>$hb#bNOKuqe diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt deleted file mode 100644 index 4835061786d..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/columns.txt +++ /dev/null @@ -1,259 +0,0 @@ -columns format version: 1 -257 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`milliseconds` UInt64 -`ProfileEvent_Query` UInt64 -`ProfileEvent_SelectQuery` UInt64 -`ProfileEvent_InsertQuery` UInt64 -`ProfileEvent_FailedQuery` UInt64 -`ProfileEvent_FailedSelectQuery` UInt64 -`ProfileEvent_FailedInsertQuery` UInt64 -`ProfileEvent_QueryTimeMicroseconds` UInt64 -`ProfileEvent_SelectQueryTimeMicroseconds` UInt64 -`ProfileEvent_InsertQueryTimeMicroseconds` UInt64 -`ProfileEvent_FileOpen` UInt64 -`ProfileEvent_Seek` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64 -`ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64 -`ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64 -`ProfileEvent_ReadBufferAIORead` UInt64 -`ProfileEvent_ReadBufferAIOReadBytes` UInt64 -`ProfileEvent_WriteBufferAIOWrite` UInt64 -`ProfileEvent_WriteBufferAIOWriteBytes` UInt64 -`ProfileEvent_ReadCompressedBytes` UInt64 -`ProfileEvent_CompressedReadBufferBlocks` UInt64 -`ProfileEvent_CompressedReadBufferBytes` UInt64 -`ProfileEvent_UncompressedCacheHits` UInt64 -`ProfileEvent_UncompressedCacheMisses` UInt64 -`ProfileEvent_UncompressedCacheWeightLost` UInt64 -`ProfileEvent_IOBufferAllocs` UInt64 -`ProfileEvent_IOBufferAllocBytes` UInt64 -`ProfileEvent_ArenaAllocChunks` UInt64 -`ProfileEvent_ArenaAllocBytes` UInt64 -`ProfileEvent_FunctionExecute` UInt64 -`ProfileEvent_TableFunctionExecute` UInt64 -`ProfileEvent_MarkCacheHits` UInt64 -`ProfileEvent_MarkCacheMisses` UInt64 -`ProfileEvent_CreatedReadBufferOrdinary` UInt64 -`ProfileEvent_CreatedReadBufferAIO` UInt64 -`ProfileEvent_CreatedReadBufferAIOFailed` UInt64 -`ProfileEvent_CreatedReadBufferMMap` UInt64 -`ProfileEvent_CreatedReadBufferMMapFailed` UInt64 -`ProfileEvent_CreatedWriteBufferOrdinary` UInt64 -`ProfileEvent_CreatedWriteBufferAIO` UInt64 -`ProfileEvent_CreatedWriteBufferAIOFailed` UInt64 -`ProfileEvent_DiskReadElapsedMicroseconds` UInt64 -`ProfileEvent_DiskWriteElapsedMicroseconds` UInt64 -`ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64 -`ProfileEvent_NetworkSendElapsedMicroseconds` UInt64 -`ProfileEvent_ThrottlerSleepMicroseconds` UInt64 -`ProfileEvent_QueryMaskingRulesMatch` UInt64 -`ProfileEvent_ReplicatedPartFetches` UInt64 -`ProfileEvent_ReplicatedPartFailedFetches` UInt64 -`ProfileEvent_ObsoleteReplicatedParts` UInt64 -`ProfileEvent_ReplicatedPartMerges` UInt64 -`ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64 -`ProfileEvent_ReplicatedPartMutations` UInt64 -`ProfileEvent_ReplicatedPartChecks` UInt64 -`ProfileEvent_ReplicatedPartChecksFailed` UInt64 -`ProfileEvent_ReplicatedDataLoss` UInt64 -`ProfileEvent_InsertedRows` UInt64 -`ProfileEvent_InsertedBytes` UInt64 -`ProfileEvent_DelayedInserts` UInt64 -`ProfileEvent_RejectedInserts` UInt64 -`ProfileEvent_DelayedInsertsMilliseconds` UInt64 -`ProfileEvent_DuplicatedInsertedBlocks` UInt64 -`ProfileEvent_ZooKeeperInit` UInt64 -`ProfileEvent_ZooKeeperTransactions` UInt64 -`ProfileEvent_ZooKeeperList` UInt64 -`ProfileEvent_ZooKeeperCreate` UInt64 -`ProfileEvent_ZooKeeperRemove` UInt64 -`ProfileEvent_ZooKeeperExists` UInt64 -`ProfileEvent_ZooKeeperGet` UInt64 -`ProfileEvent_ZooKeeperSet` UInt64 -`ProfileEvent_ZooKeeperMulti` UInt64 -`ProfileEvent_ZooKeeperCheck` UInt64 -`ProfileEvent_ZooKeeperClose` UInt64 -`ProfileEvent_ZooKeeperWatchResponse` UInt64 -`ProfileEvent_ZooKeeperUserExceptions` UInt64 -`ProfileEvent_ZooKeeperHardwareExceptions` UInt64 -`ProfileEvent_ZooKeeperOtherExceptions` UInt64 -`ProfileEvent_ZooKeeperWaitMicroseconds` UInt64 -`ProfileEvent_ZooKeeperBytesSent` UInt64 -`ProfileEvent_ZooKeeperBytesReceived` UInt64 -`ProfileEvent_DistributedConnectionFailTry` UInt64 -`ProfileEvent_DistributedConnectionMissingTable` UInt64 -`ProfileEvent_DistributedConnectionStaleReplica` UInt64 -`ProfileEvent_DistributedConnectionFailAtAll` UInt64 -`ProfileEvent_CompileAttempt` UInt64 -`ProfileEvent_CompileSuccess` UInt64 -`ProfileEvent_CompileFunction` UInt64 -`ProfileEvent_CompiledFunctionExecute` UInt64 -`ProfileEvent_CompileExpressionsMicroseconds` UInt64 -`ProfileEvent_CompileExpressionsBytes` UInt64 -`ProfileEvent_ExternalSortWritePart` UInt64 -`ProfileEvent_ExternalSortMerge` UInt64 -`ProfileEvent_ExternalAggregationWritePart` UInt64 -`ProfileEvent_ExternalAggregationMerge` UInt64 -`ProfileEvent_ExternalAggregationCompressedBytes` UInt64 -`ProfileEvent_ExternalAggregationUncompressedBytes` UInt64 -`ProfileEvent_SlowRead` UInt64 -`ProfileEvent_ReadBackoff` UInt64 -`ProfileEvent_ReplicaPartialShutdown` UInt64 -`ProfileEvent_SelectedParts` UInt64 -`ProfileEvent_SelectedRanges` UInt64 -`ProfileEvent_SelectedMarks` UInt64 -`ProfileEvent_SelectedRows` UInt64 -`ProfileEvent_SelectedBytes` UInt64 -`ProfileEvent_Merge` UInt64 -`ProfileEvent_MergedRows` UInt64 -`ProfileEvent_MergedUncompressedBytes` UInt64 -`ProfileEvent_MergesTimeMilliseconds` UInt64 -`ProfileEvent_MergeTreeDataWriterRows` UInt64 -`ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64 -`ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64 -`ProfileEvent_MergeTreeDataWriterBlocks` UInt64 -`ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64 -`ProfileEvent_CannotRemoveEphemeralNode` UInt64 -`ProfileEvent_RegexpCreated` UInt64 -`ProfileEvent_ContextLock` UInt64 -`ProfileEvent_StorageBufferFlush` UInt64 -`ProfileEvent_StorageBufferErrorOnFlush` UInt64 -`ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64 -`ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64 -`ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64 -`ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64 -`ProfileEvent_DictCacheKeysRequested` UInt64 -`ProfileEvent_DictCacheKeysRequestedMiss` UInt64 -`ProfileEvent_DictCacheKeysRequestedFound` UInt64 -`ProfileEvent_DictCacheKeysExpired` UInt64 -`ProfileEvent_DictCacheKeysNotFound` UInt64 -`ProfileEvent_DictCacheKeysHit` UInt64 -`ProfileEvent_DictCacheRequestTimeNs` UInt64 -`ProfileEvent_DictCacheRequests` UInt64 -`ProfileEvent_DictCacheLockWriteNs` UInt64 -`ProfileEvent_DictCacheLockReadNs` UInt64 -`ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64 -`ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64 -`ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64 -`ProfileEvent_PolygonsAddedToPool` UInt64 -`ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64 -`ProfileEvent_RWLockAcquiredReadLocks` UInt64 -`ProfileEvent_RWLockAcquiredWriteLocks` UInt64 -`ProfileEvent_RWLockReadersWaitMilliseconds` UInt64 -`ProfileEvent_RWLockWritersWaitMilliseconds` UInt64 -`ProfileEvent_DNSError` UInt64 -`ProfileEvent_RealTimeMicroseconds` UInt64 -`ProfileEvent_UserTimeMicroseconds` UInt64 -`ProfileEvent_SystemTimeMicroseconds` UInt64 -`ProfileEvent_SoftPageFaults` UInt64 -`ProfileEvent_HardPageFaults` UInt64 -`ProfileEvent_VoluntaryContextSwitches` UInt64 -`ProfileEvent_InvoluntaryContextSwitches` UInt64 -`ProfileEvent_OSIOWaitMicroseconds` UInt64 -`ProfileEvent_OSCPUWaitMicroseconds` UInt64 -`ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64 -`ProfileEvent_OSReadBytes` UInt64 -`ProfileEvent_OSWriteBytes` UInt64 -`ProfileEvent_OSReadChars` UInt64 -`ProfileEvent_OSWriteChars` UInt64 -`ProfileEvent_PerfCpuCycles` UInt64 -`ProfileEvent_PerfInstructions` UInt64 -`ProfileEvent_PerfCacheReferences` UInt64 -`ProfileEvent_PerfCacheMisses` UInt64 -`ProfileEvent_PerfBranchInstructions` UInt64 -`ProfileEvent_PerfBranchMisses` UInt64 -`ProfileEvent_PerfBusCycles` UInt64 -`ProfileEvent_PerfStalledCyclesFrontend` UInt64 -`ProfileEvent_PerfStalledCyclesBackend` UInt64 -`ProfileEvent_PerfRefCpuCycles` UInt64 -`ProfileEvent_PerfCpuClock` UInt64 -`ProfileEvent_PerfTaskClock` UInt64 -`ProfileEvent_PerfContextSwitches` UInt64 -`ProfileEvent_PerfCpuMigrations` UInt64 -`ProfileEvent_PerfAlignmentFaults` UInt64 -`ProfileEvent_PerfEmulationFaults` UInt64 -`ProfileEvent_PerfMinEnabledTime` UInt64 -`ProfileEvent_PerfMinEnabledRunningTime` UInt64 -`ProfileEvent_PerfDataTLBReferences` UInt64 -`ProfileEvent_PerfDataTLBMisses` UInt64 -`ProfileEvent_PerfInstructionTLBReferences` UInt64 -`ProfileEvent_PerfInstructionTLBMisses` UInt64 -`ProfileEvent_PerfLocalMemoryReferences` UInt64 -`ProfileEvent_PerfLocalMemoryMisses` UInt64 -`ProfileEvent_CreatedHTTPConnections` UInt64 -`ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64 -`ProfileEvent_QueryProfilerSignalOverruns` UInt64 -`ProfileEvent_CreatedLogEntryForMerge` UInt64 -`ProfileEvent_NotCreatedLogEntryForMerge` UInt64 -`ProfileEvent_CreatedLogEntryForMutation` UInt64 -`ProfileEvent_NotCreatedLogEntryForMutation` UInt64 -`ProfileEvent_S3ReadMicroseconds` UInt64 -`ProfileEvent_S3ReadBytes` UInt64 -`ProfileEvent_S3ReadRequestsCount` UInt64 -`ProfileEvent_S3ReadRequestsErrors` UInt64 -`ProfileEvent_S3ReadRequestsThrottling` UInt64 -`ProfileEvent_S3ReadRequestsRedirects` UInt64 -`ProfileEvent_S3WriteMicroseconds` UInt64 -`ProfileEvent_S3WriteBytes` UInt64 -`ProfileEvent_S3WriteRequestsCount` UInt64 -`ProfileEvent_S3WriteRequestsErrors` UInt64 -`ProfileEvent_S3WriteRequestsThrottling` UInt64 -`ProfileEvent_S3WriteRequestsRedirects` UInt64 -`ProfileEvent_QueryMemoryLimitExceeded` UInt64 -`CurrentMetric_Query` Int64 -`CurrentMetric_Merge` Int64 -`CurrentMetric_PartMutation` Int64 -`CurrentMetric_ReplicatedFetch` Int64 -`CurrentMetric_ReplicatedSend` Int64 -`CurrentMetric_ReplicatedChecks` Int64 -`CurrentMetric_BackgroundPoolTask` Int64 -`CurrentMetric_BackgroundMovePoolTask` Int64 -`CurrentMetric_BackgroundSchedulePoolTask` Int64 -`CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64 -`CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64 -`CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64 -`CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64 -`CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64 -`CurrentMetric_DiskSpaceReservedForMerge` Int64 -`CurrentMetric_DistributedSend` Int64 -`CurrentMetric_QueryPreempted` Int64 -`CurrentMetric_TCPConnection` Int64 -`CurrentMetric_MySQLConnection` Int64 -`CurrentMetric_HTTPConnection` Int64 -`CurrentMetric_InterserverConnection` Int64 -`CurrentMetric_PostgreSQLConnection` Int64 -`CurrentMetric_OpenFileForRead` Int64 -`CurrentMetric_OpenFileForWrite` Int64 -`CurrentMetric_Read` Int64 -`CurrentMetric_Write` Int64 -`CurrentMetric_SendScalars` Int64 -`CurrentMetric_SendExternalTables` Int64 -`CurrentMetric_QueryThread` Int64 -`CurrentMetric_ReadonlyReplica` Int64 -`CurrentMetric_MemoryTracking` Int64 -`CurrentMetric_EphemeralNode` Int64 -`CurrentMetric_ZooKeeperSession` Int64 -`CurrentMetric_ZooKeeperWatch` Int64 -`CurrentMetric_ZooKeeperRequest` Int64 -`CurrentMetric_DelayedInserts` Int64 -`CurrentMetric_ContextLockWait` Int64 -`CurrentMetric_StorageBufferRows` Int64 -`CurrentMetric_StorageBufferBytes` Int64 -`CurrentMetric_DictCacheRequests` Int64 -`CurrentMetric_Revision` Int64 -`CurrentMetric_VersionInteger` Int64 -`CurrentMetric_RWLockWaitingReaders` Int64 -`CurrentMetric_RWLockWaitingWriters` Int64 -`CurrentMetric_RWLockActiveReaders` Int64 -`CurrentMetric_RWLockActiveWriters` Int64 -`CurrentMetric_GlobalThread` Int64 -`CurrentMetric_GlobalThreadActive` Int64 -`CurrentMetric_LocalThread` Int64 -`CurrentMetric_LocalThreadActive` Int64 -`CurrentMetric_DistributedFilesToInsert` Int64 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt deleted file mode 100644 index c7930257dfe..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -7 \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/partition.dat b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx deleted file mode 100644 index dc6e4026178..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/202102_4608_4608_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëH²©"`ëH¸©"` \ No newline at end of file diff --git a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt b/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql b/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql deleted file mode 100644 index 81ffc1acf48..00000000000 --- a/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/hits.sql +++ /dev/null @@ -1,141 +0,0 @@ -ATTACH TABLE _ UUID '72667623-6371-4258-be57-799522ee4e64' -( - `WatchID` UInt64, - `JavaEnable` UInt8, - `Title` String, - `GoodEvent` Int16, - `EventTime` DateTime, - `EventDate` Date, - `CounterID` UInt32, - `ClientIP` UInt32, - `ClientIP6` FixedString(16), - `RegionID` UInt32, - `UserID` UInt64, - `CounterClass` Int8, - `OS` UInt8, - `UserAgent` UInt8, - `URL` String, - `Referer` String, - `URLDomain` String, - `RefererDomain` String, - `Refresh` UInt8, - `IsRobot` UInt8, - `RefererCategories` Array(UInt16), - `URLCategories` Array(UInt16), - `URLRegions` Array(UInt32), - `RefererRegions` Array(UInt32), - `ResolutionWidth` UInt16, - `ResolutionHeight` UInt16, - `ResolutionDepth` UInt8, - `FlashMajor` UInt8, - `FlashMinor` UInt8, - `FlashMinor2` String, - `NetMajor` UInt8, - `NetMinor` UInt8, - `UserAgentMajor` UInt16, - `UserAgentMinor` FixedString(2), - `CookieEnable` UInt8, - `JavascriptEnable` UInt8, - `IsMobile` UInt8, - `MobilePhone` UInt8, - `MobilePhoneModel` String, - `Params` String, - `IPNetworkID` UInt32, - `TraficSourceID` Int8, - `SearchEngineID` UInt16, - `SearchPhrase` String, - `AdvEngineID` UInt8, - `IsArtifical` UInt8, - `WindowClientWidth` UInt16, - `WindowClientHeight` UInt16, - `ClientTimeZone` Int16, - `ClientEventTime` DateTime, - `SilverlightVersion1` UInt8, - `SilverlightVersion2` UInt8, - `SilverlightVersion3` UInt32, - `SilverlightVersion4` UInt16, - `PageCharset` String, - `CodeVersion` UInt32, - `IsLink` UInt8, - `IsDownload` UInt8, - `IsNotBounce` UInt8, - `FUniqID` UInt64, - `HID` UInt32, - `IsOldCounter` UInt8, - `IsEvent` UInt8, - `IsParameter` UInt8, - `DontCountHits` UInt8, - `WithHash` UInt8, - `HitColor` FixedString(1), - `UTCEventTime` DateTime, - `Age` UInt8, - `Sex` UInt8, - `Income` UInt8, - `Interests` UInt16, - `Robotness` UInt8, - `GeneralInterests` Array(UInt16), - `RemoteIP` UInt32, - `RemoteIP6` FixedString(16), - `WindowName` Int32, - `OpenerName` Int32, - `HistoryLength` Int16, - `BrowserLanguage` FixedString(2), - `BrowserCountry` FixedString(2), - `SocialNetwork` String, - `SocialAction` String, - `HTTPError` UInt16, - `SendTiming` Int32, - `DNSTiming` Int32, - `ConnectTiming` Int32, - `ResponseStartTiming` Int32, - `ResponseEndTiming` Int32, - `FetchTiming` Int32, - `RedirectTiming` Int32, - `DOMInteractiveTiming` Int32, - `DOMContentLoadedTiming` Int32, - `DOMCompleteTiming` Int32, - `LoadEventStartTiming` Int32, - `LoadEventEndTiming` Int32, - `NSToDOMContentLoadedTiming` Int32, - `FirstPaintTiming` Int32, - `RedirectCount` Int8, - `SocialSourceNetworkID` UInt8, - `SocialSourcePage` String, - `ParamPrice` Int64, - `ParamOrderID` String, - `ParamCurrency` FixedString(3), - `ParamCurrencyID` UInt16, - `GoalsReached` Array(UInt32), - `OpenstatServiceName` String, - `OpenstatCampaignID` String, - `OpenstatAdID` String, - `OpenstatSourceID` String, - `UTMSource` String, - `UTMMedium` String, - `UTMCampaign` String, - `UTMContent` String, - `UTMTerm` String, - `FromTag` String, - `HasGCLID` UInt8, - `RefererHash` UInt64, - `URLHash` UInt64, - `CLID` UInt32, - `YCLID` UInt64, - `ShareService` String, - `ShareURL` String, - `ShareTitle` String, - `ParsedParams.Key1` Array(String), - `ParsedParams.Key2` Array(String), - `ParsedParams.Key3` Array(String), - `ParsedParams.Key4` Array(String), - `ParsedParams.Key5` Array(String), - `ParsedParams.ValueDouble` Array(Float64), - `IslandID` FixedString(16), - `RequestNum` UInt32, - `RequestTry` UInt8 -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(EventDate) -ORDER BY (CounterID, EventDate, intHash32(UserID), EventTime) -SAMPLE BY intHash32(UserID) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql b/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql deleted file mode 100644 index cd2d3509c4e..00000000000 --- a/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080/visits.sql +++ /dev/null @@ -1,189 +0,0 @@ -ATTACH TABLE _ UUID '044d7000-c91c-482d-9f0f-719f226eff0f' -( - `CounterID` UInt32, - `StartDate` Date, - `Sign` Int8, - `IsNew` UInt8, - `VisitID` UInt64, - `UserID` UInt64, - `StartTime` DateTime, - `Duration` UInt32, - `UTCStartTime` DateTime, - `PageViews` Int32, - `Hits` Int32, - `IsBounce` UInt8, - `Referer` String, - `StartURL` String, - `RefererDomain` String, - `StartURLDomain` String, - `EndURL` String, - `LinkURL` String, - `IsDownload` UInt8, - `TraficSourceID` Int8, - `SearchEngineID` UInt16, - `SearchPhrase` String, - `AdvEngineID` UInt8, - `PlaceID` Int32, - `RefererCategories` Array(UInt16), - `URLCategories` Array(UInt16), - `URLRegions` Array(UInt32), - `RefererRegions` Array(UInt32), - `IsYandex` UInt8, - `GoalReachesDepth` Int32, - `GoalReachesURL` Int32, - `GoalReachesAny` Int32, - `SocialSourceNetworkID` UInt8, - `SocialSourcePage` String, - `MobilePhoneModel` String, - `ClientEventTime` DateTime, - `RegionID` UInt32, - `ClientIP` UInt32, - `ClientIP6` FixedString(16), - `RemoteIP` UInt32, - `RemoteIP6` FixedString(16), - `IPNetworkID` UInt32, - `SilverlightVersion3` UInt32, - `CodeVersion` UInt32, - `ResolutionWidth` UInt16, - `ResolutionHeight` UInt16, - `UserAgentMajor` UInt16, - `UserAgentMinor` UInt16, - `WindowClientWidth` UInt16, - `WindowClientHeight` UInt16, - `SilverlightVersion2` UInt8, - `SilverlightVersion4` UInt16, - `FlashVersion3` UInt16, - `FlashVersion4` UInt16, - `ClientTimeZone` Int16, - `OS` UInt8, - `UserAgent` UInt8, - `ResolutionDepth` UInt8, - `FlashMajor` UInt8, - `FlashMinor` UInt8, - `NetMajor` UInt8, - `NetMinor` UInt8, - `MobilePhone` UInt8, - `SilverlightVersion1` UInt8, - `Age` UInt8, - `Sex` UInt8, - `Income` UInt8, - `JavaEnable` UInt8, - `CookieEnable` UInt8, - `JavascriptEnable` UInt8, - `IsMobile` UInt8, - `BrowserLanguage` UInt16, - `BrowserCountry` UInt16, - `Interests` UInt16, - `Robotness` UInt8, - `GeneralInterests` Array(UInt16), - `Params` Array(String), - `Goals.ID` Array(UInt32), - `Goals.Serial` Array(UInt32), - `Goals.EventTime` Array(DateTime), - `Goals.Price` Array(Int64), - `Goals.OrderID` Array(String), - `Goals.CurrencyID` Array(UInt32), - `WatchIDs` Array(UInt64), - `ParamSumPrice` Int64, - `ParamCurrency` FixedString(3), - `ParamCurrencyID` UInt16, - `ClickLogID` UInt64, - `ClickEventID` Int32, - `ClickGoodEvent` Int32, - `ClickEventTime` DateTime, - `ClickPriorityID` Int32, - `ClickPhraseID` Int32, - `ClickPageID` Int32, - `ClickPlaceID` Int32, - `ClickTypeID` Int32, - `ClickResourceID` Int32, - `ClickCost` UInt32, - `ClickClientIP` UInt32, - `ClickDomainID` UInt32, - `ClickURL` String, - `ClickAttempt` UInt8, - `ClickOrderID` UInt32, - `ClickBannerID` UInt32, - `ClickMarketCategoryID` UInt32, - `ClickMarketPP` UInt32, - `ClickMarketCategoryName` String, - `ClickMarketPPName` String, - `ClickAWAPSCampaignName` String, - `ClickPageName` String, - `ClickTargetType` UInt16, - `ClickTargetPhraseID` UInt64, - `ClickContextType` UInt8, - `ClickSelectType` Int8, - `ClickOptions` String, - `ClickGroupBannerID` Int32, - `OpenstatServiceName` String, - `OpenstatCampaignID` String, - `OpenstatAdID` String, - `OpenstatSourceID` String, - `UTMSource` String, - `UTMMedium` String, - `UTMCampaign` String, - `UTMContent` String, - `UTMTerm` String, - `FromTag` String, - `HasGCLID` UInt8, - `FirstVisit` DateTime, - `PredLastVisit` Date, - `LastVisit` Date, - `TotalVisits` UInt32, - `TraficSource.ID` Array(Int8), - `TraficSource.SearchEngineID` Array(UInt16), - `TraficSource.AdvEngineID` Array(UInt8), - `TraficSource.PlaceID` Array(UInt16), - `TraficSource.SocialSourceNetworkID` Array(UInt8), - `TraficSource.Domain` Array(String), - `TraficSource.SearchPhrase` Array(String), - `TraficSource.SocialSourcePage` Array(String), - `Attendance` FixedString(16), - `CLID` UInt32, - `YCLID` UInt64, - `NormalizedRefererHash` UInt64, - `SearchPhraseHash` UInt64, - `RefererDomainHash` UInt64, - `NormalizedStartURLHash` UInt64, - `StartURLDomainHash` UInt64, - `NormalizedEndURLHash` UInt64, - `TopLevelDomain` UInt64, - `URLScheme` UInt64, - `OpenstatServiceNameHash` UInt64, - `OpenstatCampaignIDHash` UInt64, - `OpenstatAdIDHash` UInt64, - `OpenstatSourceIDHash` UInt64, - `UTMSourceHash` UInt64, - `UTMMediumHash` UInt64, - `UTMCampaignHash` UInt64, - `UTMContentHash` UInt64, - `UTMTermHash` UInt64, - `FromHash` UInt64, - `WebVisorEnabled` UInt8, - `WebVisorActivity` UInt32, - `ParsedParams.Key1` Array(String), - `ParsedParams.Key2` Array(String), - `ParsedParams.Key3` Array(String), - `ParsedParams.Key4` Array(String), - `ParsedParams.Key5` Array(String), - `ParsedParams.ValueDouble` Array(Float64), - `Market.Type` Array(UInt8), - `Market.GoalID` Array(UInt32), - `Market.OrderID` Array(String), - `Market.OrderPrice` Array(Int64), - `Market.PP` Array(UInt32), - `Market.DirectPlaceID` Array(UInt32), - `Market.DirectOrderID` Array(UInt32), - `Market.DirectBannerID` Array(UInt32), - `Market.GoodID` Array(String), - `Market.GoodName` Array(String), - `Market.GoodQuantity` Array(Int32), - `Market.GoodPrice` Array(Int64), - `IslandID` FixedString(16) -) -ENGINE = CollapsingMergeTree(Sign) -PARTITION BY toYYYYMM(StartDate) -ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) -SAMPLE BY intHash32(UserID) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/checksums.txt deleted file mode 100644 index 939ea6ccf187b34a8db6e4def524f9dd13691092..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bhh(-qXp!7uuz9v-dm@^cD z*Z=?k&Hw-a@jV6!V{dhCbS`vwbON#VDF3f}{FN>b+}S`mO_BfzWMOn+E@EkJ`W*io z<|Oi02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcns^T1OQK`i$9(N KipbM-Mdtv4FmBEO diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt deleted file mode 100644 index 5ea415161bf..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/columns.txt +++ /dev/null @@ -1,47 +0,0 @@ -columns format version: 1 -45 columns: -`type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4) -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`result_rows` UInt64 -`result_bytes` UInt64 -`memory_usage` UInt64 -`current_database` String -`query` String -`exception_code` Int32 -`exception` String -`stack_trace` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`thread_ids` Array(UInt64) -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) -`Settings.Names` Array(String) -`Settings.Values` Array(String) diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt deleted file mode 100644 index 9a037142aa3..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx deleted file mode 100644 index 4b6c49f8d50..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -…H…H \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/partition.dat b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/partition.dat deleted file mode 100644 index b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0VWo7^X0I>iw diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx deleted file mode 100644 index d599340cc82..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202010_1_5_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -…H!.œ_…Hm.œ_ \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/checksums.txt deleted file mode 100644 index 747078ab876a6913b8e0a9a92d3d0a3e3a1deb0f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyWHkv{Wolw154dF1>;$$|v}nZZR-0 zTma&a_H3NV`K5U!dL%eM=a3btktj z6fTJR@=T@IyyV0s#)et!Y;O}d4yN1KT?$>ea}nQfR!#$k4|%yo*~S~V4GOsV4O}$! zZrdrRHA`0hWDw2G%*#!zh)*p`%`1rq8l0+^nNq=`zaq;cGh)rekNa9x-^dl4GVm58 z7L{a{09~#JRK)U=M_>H!p{7$hOFDXtB}~#8xC@Fha}$dyLArT1wS3n4_i1|9>D}5g J%+@!V7yzW3XQcoD diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt deleted file mode 100644 index 5ea415161bf..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/columns.txt +++ /dev/null @@ -1,47 +0,0 @@ -columns format version: 1 -45 columns: -`type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4) -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`result_rows` UInt64 -`result_bytes` UInt64 -`memory_usage` UInt64 -`current_database` String -`query` String -`exception_code` Int32 -`exception` String -`stack_trace` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`thread_ids` Array(UInt64) -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) -`Settings.Names` Array(String) -`Settings.Values` Array(String) diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt deleted file mode 100644 index 780fea92d29..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/count.txt +++ /dev/null @@ -1 +0,0 @@ -77 \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx deleted file mode 100644 index 4fedf1265b2..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -¾H¿H \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/partition.dat b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/partition.dat deleted file mode 100644 index 7af4421fcb5ebc50aa06a5f4dee6e454020552a6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0UWo7^X0J#7) diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx deleted file mode 100644 index 7f33001d913..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202012_6_47_9/primary.idx +++ /dev/null @@ -1 +0,0 @@ -¾Hb˜ç_¿H»©ç_ \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202101_48_74_6/checksums.txt deleted file mode 100644 index eec9f2828ad60100f834ee2ec7c2ecdd572e4d2f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 261 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyUI``tQA+<)x6f|4kELHWvPFy2-%6 za2|+1+Ocsa=a=S{=#^BIFika>Ad;2)__e!oc(_Su$x;T6l*E!my`;>%-kfLCXPdd* ze|I8LNK`6r-p_c(P3^3@vsGTkiX2IichpIIeqjR>r#{1nyxgK};|<&n9adSpqodSI zI&*}wm1=%7h~{SI(O)RPe>EV>iAT$c8rTFS^0FlRNAYdvChb+}S`mO_BfzWMOn+E@EkJqa$s) z<(i%4_5Su;$|AM?s{R3`Wi?hP8GT=Tu0P7G#$xv>0SPVu@NR8#YcsG56t3`xr|pUp z;{9}LW*LD002OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=138%)U-7Z;v9?{s9d$U?01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnn{AlpFuovTY~I Kn?YGm%iaJK*Kw-= diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt deleted file mode 100644 index 5ea415161bf..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/columns.txt +++ /dev/null @@ -1,47 +0,0 @@ -columns format version: 1 -45 columns: -`type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4) -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`result_rows` UInt64 -`result_bytes` UInt64 -`memory_usage` UInt64 -`current_database` String -`query` String -`exception_code` Int32 -`exception` String -`stack_trace` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`thread_ids` Array(UInt64) -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) -`Settings.Names` Array(String) -`Settings.Values` Array(String) diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt deleted file mode 100644 index 9a037142aa3..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx deleted file mode 100644 index 0e3b7fb75b8..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -êHëH \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/partition.dat b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx deleted file mode 100644 index 0b5fcd2fdea..00000000000 --- a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_75_79_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -êHÌC!`ëHG£"` \ No newline at end of file diff --git a/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/checksums.txt b/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df/202102_80_80_0/checksums.txt deleted file mode 100644 index a466972c338851bb9e376b5fec0b27c54f763a01..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyUI6hQ;Hcxb2IaD6D#6V%Tn`7;(^Ad>Sd->up}O2Td;F;<|11GQ{zVyvz!@t3lfV; zGE0E&)&nYHagV>iAT$cnNbtE_VA_ox+gYO}M76zw zkN^Mxh5!Hn@jV0yV{dhCbS`vwbOCIfRlVt&lYFUDR9N&B1^)mDWMOn+E@EkJ&k!F~ zY}HzkJ8Te4L?36TX=VY?FDFqniJ!N25;D3H*NTaDs0l6r@Fs0?Ycucyl#~%aH>Tie q3B6siBLUYk01I$(X>DP0c`j*WcnlPm#22y_7RAN-7E)Vm7x4gDE=jup diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt deleted file mode 100644 index 725a526b957..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/columns.txt +++ /dev/null @@ -1,13 +0,0 @@ -columns format version: 1 -11 columns: -`event_date` Date -`event_time` DateTime -`timestamp_ns` UInt64 -`signal` Int32 -`thread_id` UInt64 -`query_id` String -`trace` Array(UInt64) -`trace_full` Array(String) -`version` String -`revision` UInt32 -`build_id` String diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx deleted file mode 100644 index b3b53069cfb..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_1_1_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ØH9Ö `ØH9Ö ` \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/checksums.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/checksums.txt deleted file mode 100644 index 41a34e68660fcf30f2ebf3886af70965d1c50d97..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 188 zcmV;t07L&{Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cC)=R{W#rbM&U?0f^HAjYm zkN^Mxh5!Hn@jV0yV{dhCbS`vwbOCIfRlVt&lYFUDR9N&B1^)mDWMOn+E@EkJ&=4}7 zAlkfU=Me)DyD->zPV@oLFCA^8bN;KGdEy@&Pv!bL6$vf?@Fs0?Ycucy8(uAC5Vito q|AbpmqG_6t01I$(X>DP0c`j*WcnnVC4s8oxdoa!Z4je{MT>k*Ano9!! diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt deleted file mode 100644 index 725a526b957..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/columns.txt +++ /dev/null @@ -1,13 +0,0 @@ -columns format version: 1 -11 columns: -`event_date` Date -`event_time` DateTime -`timestamp_ns` UInt64 -`signal` Int32 -`thread_id` UInt64 -`query_id` String -`trace` Array(UInt64) -`trace_full` Array(String) -`version` String -`revision` UInt32 -`build_id` String diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx deleted file mode 100644 index 24c83f6002c..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/all_2_2_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ØHÏØ `ØHÏØ ` \ No newline at end of file diff --git a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt b/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/checksums.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/checksums.txt deleted file mode 100644 index 94103f5a76ccb7f25b7a74030c04dbc2413b0242..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyY$U>CqQtR5+I_knLZ-OHz2#4F(2= zb3pvjmW?wxzcjBzucV@cX`=X>wR6M7)Yk+*GdQ7`_n(0yC9xz?FDWyxQ#5noG1-=x zMe7?wHfev7dG?oaNwm=1jm!7WVfXwmTE9_VNPjb@9>a&c+@fsb3oO0Y#oSLXwp>@( z{bpjsq?khtqPdxQxrr6=sb#5oCGkLGQ}r@aDp&%G|HSG1ZQ1rGFrD>==$*L?yakCx zC7C5ack2NavHaxG7yo;x>6FfrjviwPlXM2|f}+ga#G*=&Zk~`GM+A);B?QWr7Vt)1 I5q`=5051<|M*si- diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt deleted file mode 100644 index 7775d6f8eb4..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/columns.txt +++ /dev/null @@ -1,20 +0,0 @@ -columns format version: 1 -18 columns: -`event_type` Enum8('NewPart' = 1, 'MergeParts' = 2, 'DownloadPart' = 3, 'RemovePart' = 4, 'MutatePart' = 5, 'MovePart' = 6) -`event_date` Date -`event_time` DateTime -`duration_ms` UInt64 -`database` String -`table` String -`part_name` String -`partition_id` String -`path_on_disk` String -`rows` UInt64 -`size_in_bytes` UInt64 -`merged_from` Array(String) -`bytes_uncompressed` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`peak_memory_usage` UInt64 -`error` UInt16 -`exception` String diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt deleted file mode 100644 index eb13855b7d7..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/count.txt +++ /dev/null @@ -1 +0,0 @@ -79 \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx deleted file mode 100644 index 07abc658bdb..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -¾HÁH \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/partition.dat b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/partition.dat deleted file mode 100644 index 7af4421fcb5ebc50aa06a5f4dee6e454020552a6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0UWo7^X0J#7) diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx deleted file mode 100644 index 16d6a0eea67..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202012_1_18_4/primary.idx +++ /dev/null @@ -1 +0,0 @@ -¾He¢ç_ÁH(|ê_ \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/checksums.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/checksums.txt deleted file mode 100644 index 2b7fff5ccb9418c7ab7aa25bd1fb7d93eceec517..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyZ>tGIK5Cl6r%;##xVQ61m%&ZZI$~ zoCD&Iwrrfq`K5U!dLfa6k3NUy@yiSHP=3yLyx6N@TAx_KfGyWeaxDL=3MNbdhr I+Xd$s04~;Rw*UYD diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt deleted file mode 100644 index 7775d6f8eb4..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/columns.txt +++ /dev/null @@ -1,20 +0,0 @@ -columns format version: 1 -18 columns: -`event_type` Enum8('NewPart' = 1, 'MergeParts' = 2, 'DownloadPart' = 3, 'RemovePart' = 4, 'MutatePart' = 5, 'MovePart' = 6) -`event_date` Date -`event_time` DateTime -`duration_ms` UInt64 -`database` String -`table` String -`part_name` String -`partition_id` String -`path_on_disk` String -`rows` UInt64 -`size_in_bytes` UInt64 -`merged_from` Array(String) -`bytes_uncompressed` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`peak_memory_usage` UInt64 -`error` UInt16 -`exception` String diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt deleted file mode 100644 index 9a037142aa3..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx deleted file mode 100644 index 7d66cd0aae2..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ËHËH \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/partition.dat b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/partition.dat deleted file mode 100644 index a713aecd075daae7071deceaa93147871ef28fbf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRFWo7^X0v!O3 diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx deleted file mode 100644 index 3e5042c4bf2..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/202101_19_22_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ËHÝÕ÷_ËHEÙ÷_ \ No newline at end of file diff --git a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt b/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/checksums.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/checksums.txt deleted file mode 100644 index 322c6666b3efd30007e1786d71ebf7dbb6bc46f1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10792 zcmX|HcRbbq_kX?b`(FFno3hEsY|xNB%F0MWT<*m+?&Z2hG;B&%s8l3WDkCF18Ym-F zMwCdUVcmP}mEXI3zQ2F`aqoQ`uh)5<^E}V3N(6qi0e)R^Q=BoWMwR6U5K%rs`c zqebZ+C)gsghP0ebQHp1vhrPLI*c<=ZJ<`Lhf@PpljHCTEw>F?*11Nr)#cv=Ph4Sk{ zQLLo`hh*=scf4W)%=+F02SCT9rCf)2dF=i;=;5C>o-L#63GI$Q0ca@JhfltIz~o-- zVUp^>;#xklXaLv>(0EdQs&h*!P=04R%#`ObPt5>c0}44Tm_RWkhmfe0GfrS#Y_lu| zx|E`M@cx-EORtQYclOXES)l&BMwX0*vC@&#Tjz%O_zQrt*9PAq0OP>|%EJ6XqEKe|$sBK0qv> zd*fHs$uDQ1;-^d8BM$~`#{cxhaCw|pof<5$WkoJ)4d^_lS6&$lo*2oEywRSzc&(=y z@EQ6O2_&jfV}_%mZaSgmq`#_nod)CXsPIh1ssY%(%dE-G4-{53w~=WN`J~iO0qLO zTj>9Ihk_eeM7^mtzDyk%|8P&J&}n|p%YE2=06vL)t_WF_v&c$`FU?zELT>H33Zwz| zJ}0msRuQ!h*9loRNR=OjpIhTU2Gxu!L-~uthHtNoO-Y_qdX{85ruZvQPleM7@b5c? zoT@RrLus9trQ*%V{WXS!*a+a20e}zEI7KE2Atz%LK7ddQrDqnX52l{lGtE%9xZrMR z9xB*XMq81Zw$y2IKR4X|Nyi@g`viDEYVL9h_4M z;)hwLE=NifG>O4)0goY>L?U<&Lh3}4w>x67G^PnXG@XOz=$xF&3dtdE9P)b{uhSIro;(5EhGeoIkzh>1dH731 zbaZ73T8oE#RZblwzm_49+*bYxBs|NQn%t^Ej(yc<|WRl5jM z#-GooSiP z8d=$eAg`MK@;#jOM~4xwCm1NPN9p^qpgWnTz!J)~t$xqtfD!Fl`^RB}AwkKU`vHu6 z&e_Jh@cd2Ny4V5b;<`yAZv`M=WNm3k4hZxoPzi`fF;7>cVMl+zh2V!t2Xud&@MBNC zT<8J*pjxlKC1^{RKTLeyeQrNVieK;ufPDcSX6^UrU@OR-h+x}nm=XUu2}G4a4dOBq z)q;$}6Yylj=CAA(G}gaQ|A^G$fru9xc6G`>9LZn{H5}fKHPzkR*>Lz&-OHaNLt;h% z4O0P)c}>l%zT~(AbYs9E-HB@eXQ=n z*P3_L6vX6a5c+Gf!_F#Z`^LrVFLRPa3;+ra1axL?Z&3?1p3~L)U7q*cPz-AYJVs;^ zwHMMbBT{iFI%Y2Z38cVoi~W>_3s}V-AMUteAmM>pLyaVDU3$t=YZ*@5`Nz-xoaKL! z2Q$iy8su7D3N73@Wa#DARSvL@7!4yFbtm$;Mc4LoYVELtIHZ(oQ8|HqgI|Y_pqEe= zv%G7#&1Y-{&7Z!k=%^jve%yIza>8T2FN2A*t=`#2*Y$tHti!|2lHF+Mk1uABukB zn!Oxc2-r;=NW>t1ctLFR_6sz4dvB~&tng@kiXu91QI)~2O~x5O!qaTB3xx+pm3N!$wHG9geseT)!qb*6WqhacSj>6-94Yne;1 z$>-py+)A*5lFIA!d9(98d&TH-qu!#B*>>jB*)^AvV)y0NYW^@e^P%N*av9wI}np#%G1i#?H+fJ6J_wGv>DMX{2Qmq12TQjL3&mM+g0d-0QHuewu-)K^%JkTm|TCw+YqL^;O*`Hde34 z1@7NG1^7(~BmxEJ&xw{M1n0t=Vk2`gOdL|tV7QrBP@*}Ywz~%N6H2XaReBO7sRqZD zP*i@Yx#6N?uW!G?CM>fNmRssD9?uuGS?T?BEEb3=0h?fA1VLRM!KU&D7g3ClCIw7d zPL+Ae_O+|dVVxVOE!2Itf~au$^CC*m7yi2WJCr zQ!>s!*p`6v^daDV!K!H9oJzFeNbr6Ez1N8ey>InPx@Dn_8y#pone#{0Z(fz1^${`= zji-#a@pl!rcO5ZUVosefC?p*c8N9_01myvy7kSsx+6!KRrv_LmY5r~cyVZrT31@o{ z_()9g@07lGXD&}A>G{eBAS!eO0N4+K4?cEPde%_+)Qfqxl){o{VE}7NCgXM3z$?TU z<2U-kMbqz{`DOWVqkzSdrsb{ANWSNvW&W7Sr+G6V6AIzX*EzeL?+mo-FAtAW#)?#| z;-_*hyPR3TY!-#2z+7xUc4Pi>YqT%^T(s@=1F0clN-tp=%KnMxgsjEo9+|1IlfAFr z*KN}{2HW+#yP>?(VskE>&y9{xYnuS9nZ3P@F@y7y~kO8T5@KRYjd=9g7Z_4er)9Dlr2@-Ak7@xOa4-_9n+ zGYD{*+gK5(VPuM53v#lh%4O8Vk|!#)_$)qQ%!gC!=CE)JLTScwBMkIe><)7%svcBO zzU;9LYz*diyn6OTTl|YQE4BUm-D%T+&79;(rZLgc?_Qp}LeB3fo={v)6&mP9<>vd(<0X?c0jGJe5jl+HPsZt^>FB6xEQoX|&R;9>7Wgq3 zMjW)`a6v#Y(B$xH*4=%+zu0$pnjS4XybZI97@1`HSDKdAw_g%+tHCpbw}9O|_#G0h zKdVu092dXw?XiAJZF}VlP+sp&B0Yt4(^TQ9JG@Ogb-^r(hrav#MKTA3M-~+5=zuQ@ z;TQahm~P(c%tD+VY(XUXu_ED6)2=f-zW3YXyXVKNWi#Wwonq(g5w|zHPlbFIs_?#X zC~wTAE-ZZ8?F`n_yXE@XBRw7&a35QGszxXQ9Ol86WDjB(2OV9WyXTp?2$NrIcrpX8 z;iWy=tmS}2*2&k+lh^4<)26%5>ucM{wEh?OkiSp*o%UNxwT>g7ZpV1dTGT<&R%EIH z(o=a>vC=Uw<}#q%>c_J`Bfq|>3AHpbmhF1tjClT5gvnyP`A)~f*)Jf%JuQ|0JkPSf z^XMz>>BwhKyq9q;cMQ)1Zu4Mke>{bahhV@$N5?!t#hGj_|7Kfu%T_XS_E2hOMk!BQjMM(1Ok<{3a7k1WMb}5Zp zdGMVqg0itCdyuJZi2J`Z@uoko>8a2V`HuOO+}ilbDgxFtn#KaPY~LqaR%nVgiaMR>g*hxq32_xGt1u7OXv z7Ey*Pav?!F=Y#cx7mr>X^K{?#{t~lWf7}hV3;8{IF0ASEw!__k^C&J9ca@WluFG|J zS5l#L`)ZlP)R!;GfguVx&|+uiea0mGes((M;$j1e+eO`RWmfcP zWpaH-0qgEtwwNu{H`2V1sP}lN)&BeB`>%_42NJv#1x@MbL=`LgbwY9r8=G@Lh z|B+$_?9pz1@f)0v303inEI4@V7~rahWo&#Lc^GhRtS0StQa{%nnxUo<(}#>QfoP@` zRLL-LfA!%N;EF|&6jd^;qTVT02tWevTp7KOquZj}jymjZq{pl^wEi{~+$qCF`# z1N(Yb7c!G?a{5fIzPZ8o4(eLG%tE#-@y4XM=mq%=Tm7P6#j1x^~L8{TzMh|8{^5AP#vtW@atKR*{>hRf(%W0wfcrmk|fCIt%I3&LZCwSni2Ni@ZrWJTM zk1su4%aV#jEuz{R$4!|t%9$M4yEOdv$5a2S98e~QW=x1!)puc2VXl?(Y9~j4jzX+# z6?4aF{0c?yc-9;x-m$o2pz&@G3mqCUTbw4$K3g6fS8J$kiG1h?pzZpIi8UOrHLP8K zgcL9^Nofd-iKa!&$bYGy7DCy#7AJ(-mKM#sMvIPrhnY33qqcP;^myYK~ z6%T}>n62$B01abK;_OX-{Z=)F*s^pm+d~_0S_cYqNL%5U5p+oGW33)>$ZnxFUA18u z2;x~?t0h3oDEa*Q!`Wluwn7meRKp{pG5?GC`A?dkKU+n1ucXLa$5O9!0B%`e7fi)b z^;6NSqD^DWLG`^~(@N@!(G{&Xa*NSfP(@-9x+JAbU}ba90f|>H_#aT|@HW;gz@tWq zyIXzc4=)Nl6<2I1l=w@!VhnKsfjFW!i2$}lD}FF__0--B`1SH?U&l=EmokkT&{kq0 zI*qhI>==w(_6e%Uj_*sw{Kvi)QR!-U-`9*<@y|THDuy;6S%3-%u_I7IiJk;iXhCEE zTAOOfJG0$0Z1C@Dsiu? z2Y=t)d^Fm>Qd4>g@b3n;hyB)DaZBc~Y4rPnnRKMviRO~1bA1xAnZXnZOJDce6!KI;4iItp@opm=E ziI%r)0MB0LAj1#}g+THY5ndC0*uvb^6`TqlskV(=iL54OTz(1tnfbtx@cyc9jeO%X z64URrJ0~A?@*V&%4&fG8uN3m#G#c%QakKBA=KSCd_!Iz27D4O8B#f{y@7;bymZYt$ z)L&-3O@aPw8>mRlimbFj{UKrl>jCo@b>I2_X>xQ}!1CfzrTu)Wf^8}BmmL~U0d7UG z$C`qF%g+$0UqUH8(u?RWH)ogbn{T<%sZN2e_vWF0Y3BhK@6RHv)@e;cKjims0rWqu zg$_G(kADi}y*$otxti)I&Q=aM-GHVIg(%EQM>o9Y?w1P>%Lf@}L(PyWf z=->Lel!1_q3W^sm4+7;hjgJw|Xe6SyM4NK-4mop$TFDhy3g9n0y73M>K=*IlXKM1f za&AoTQ^yyY+fsG;n1-+(fRJ$8f+W{bhb@60Xfr_xUz`qc4FC|gCh~>I*Dd3a%$l`` zOKuu$5{37~C!#hWM#)F?*$bUJo%*avYQK5i+WXo@0|3AT@@zqmJLSU5q;x`EYC;Rv z4;=FPYe?&_q3H(kaF;t5-3aR>tbaxCNiH($MPFy=^x`7zt z*9yXHK_B?us+w@RQ=TVZiWmUP64gMC;O(qCOED^H4C;< z!>0F(ul7~kZV;$`S|~8!j~LqHkK9eeGgI!^1+feph1C|xcQ3B{BZl@OUkJD6Lk}B_ z3-4w2tIUtND8C=zEr&YC40Q>A*1Ifgp6O#-Gh8-OWd9mMe!mVE@w}NgYYS0iN!y{f z(=t6C>>Pt)PvHDR2u9?PSHfFjKmRPjZNu`CB|@JPuj#>$esHA*K%}zTnH`Vm9 zc0l{$8RO%&@G6vZs+l?SK!eO3=cuS6r)SUz3PxluKfqv<>l#&m{6ycn(M@KOlZn#Sb3HNMK$=IXfguXGf;3R88f@h+lmr zznziTVJt=gFcq>f(3O4paOhxVpap>2CMybHWQK9*O7eTTr{5?^cYK+?ivN ze4YUQr^+%|W+VoauDE?-Q(jW5<7sPT$b|FdsarVPNy>&()o0#f zfFqt3UlXHu{6tG7F0GaG+<4Y`F7KDdLjHbeIww~7>A}4PMxGYG*RG!tpYXq`x7|g~ z!{c9KW8}TpqmtK{j~G1tISRNGfCkd%-s43Yt-e-l)Gk5O&%~qIH~BYE%y$C zr-5}Tmopm0Uc$x;IGU$#v-1U10Hdgo2`yq`ER0h*FsQ}@;sMT<;El|Ja-n&#UkfD= z-XlJT2c0Z?I?j4(3OXSVHNCpbEj5|#DPcHkZ}SU#$H!4wm=6cg|K4>S?t3?wMhq>x z!dV+v6!~Jv5D4!Am}`Gq(zoMq`Hj zKw5kE|D;anPh3qj`Bn|K^8WO`Be8xWT_|Y^QYY!?^j8|MMpS(hE)PWu_w7|0WOUhI zf@#puweYsP0aY2xL%k~!)8-)^DCl3K#efMpb6?43*bhdhkLs`ZeoF!zlK@WOUlh%K zhFWB`IHo^vXk+}Xs?ia0>EypD%!Q9xX7`?1DWX;!+sG3fUE5<08uWpjer-fuqe0$_ zf0{il_037(9zdoN%x$z0;`1tb8rE`uJr;d?O5nTN_4Kae-!1>nMtEx~X|F9CC&@0C zU%ydQ%KP8UBqaSVG3sP6oo6UzQ0ai9DKayGswC)`rn|r3z>);A$+cbfoPQ3T>TUcu zyu`JD+N`iT*4q)#Ht@rJ+B#3N8kv}2sw8Mw(dt|AjpIu;4u=(k*&eT20Zv;&PzWKI zY84VtjcCk_+-T;KupE|fru1NuZzo$G$frnGBb|epqO9K8teD_3=g`75jmK47NMBR~ z7NG+(W^CpB{F*wBvZ+6Jy7ajKDSL`E8y%A|R|P3QsDDwp&_AaA8bp%|6h5{<3s9kn z;6X|1@mhs!iAR?-3{JWLfS7Ut3~tslrtJC8HO~v8P3`8W);568mJm$#51}FfW<&^_ z7oek8JN&u7*oe{h&c~VWx7QZ&4exhHW>!6tQi&zL*K*L0YLvhko*&z_dl~h^$(a;( zR8`{eC1!Ef*@0(35ga!o5WRiOkT#@wWCVKD{&SbZW&2iFw^zO?5(PeqY)dHo5%s1G z4RC9?c25#NU*DF1H-P^Muq)8slu9m9zn+PoE`>2Vmezc4fTkl6{|0ee;l*)?^~%Fa z&aOI|PKjiok9I<>H&}+c-$q{cOaDl3{eIP&P4UqBU4Z&GPoqIcrB|699_kpa$E0+7 z3yE_!V~D|lWKu9ef`^WIFc8b!H2>3~v>5Nc?`ieLhkU#z7|_gEZC z-s5*ebQ^ybV5#12>nyG8{!trWliyiOkw8cpaO^-Zzaxd2OM9=+m#rb`@?EhXUWjwQ zN6HlH4dc2kPGF^amVdq~^|98Pus%|>Nyua)k~92^i4sWIsEKmD^H4(=VDWs?Bmzy6 zj=omt6rpVK8RZcFwyvl2_;htYgh4H#f_Hm8eBacEN=wm7f93IAV_RKYVI4m1`@Q+{ z-GY1fuVvzou&x7MI|7d4=~F64M-Nv}Ulsb_`1R5HdH0WL6Ganl&TMEMWifA^J5YGS zJ0v~h4EaZ$9v?FB@$-_3Ele6@JkKWqPqaKOC_xZC(pa0t_cSH*7A%Q0hB zRWQ2(<=Q+)U7ckG^c6~f!{GA?faH24=4wJ!4pje!&SQBHYUmDDFxu$Df8Met8XaG4 zb8{y2DgaJ9!VCgB>+MUZP`##uukY8^IbCV2sA-CQGKF47x%R@(KUjqA4Ouaml|R1x z>2K`2025z&7K zx={26F=9eb?iWbk<)m(CbkneAbs-@!s!0liT+^k((!6o>xj#@fb0Q>=X~pIe>rL5p9JP2_7&x% zC(Gn&uA_Z_DP|gW3u?Tw952rt?oJX38dn^-F#L~V9zsGQIw=<#4xvgHRx#`)uh~7@ zyN1@uLv2A6T}NxjqPd8c48Qg2ugBv_h@o`E(1t3};lnS&Q?)(p&W0=XP@ z!7FM2U^ob@z}mH$l)887#g_hIByyLoFyL1Md^SL7Q#5giKd34H-5odo3zumKuu!25 zsI_1Kz3zD-R&2~s)}!4B^Cpd`wB08z!oc>GtK+Ljt}c(rMPZ^1j{xf$%n+Zu_{iEt zYw_4v-;Scs9(EpXGRWk*v-N8?>W(yL1Dz4DR(Vi!`LI=08`~a$wgU_nr|gX9Mkcvz z*m?ocJET>>Yfr%;3a>Ksx2eO7t}j5v1utYU4@+Fu?&`m9lh<>XLm=fdm)oqS*vCcl z+er!@O`UApn;Y{mTlTI_k1fw5zP5_vT)Daj0rLqoO?>rhr;9a7x;RuLbKGN)Z`V!U zI$LP=?$2)2N{`Mv8=FJ7G$kHXJ-ImcmPPaLro|eQ#hVP>?$W(RMbGU{?XWO&13XA> zG-21oewLO)wT?~`>sDzAe;&w+O9vlUgJ`YsT3xd-a?&CqE9N}+CP!3Q+_s6@8XWqG z{nFscMjPi2sWaBD>SQ3K05Cf^kg?<111Nz1QGaN+*H~XKr_0gifwUbQTc~$SKWjrh z;&0q-Y-{Wl-d+12hA%PoTYui=cb8&PxLV`Syy`^)SmaD4*d?) zI{D?L@k_%{$~(I0g|b7;r?{p?3fSk&u#a^*ww3h-rs42u_ba@ii!&=*4^GINZ~}J3 zQW2Jp`TSslXR_rQ58=tCW|o*g1o- z)|`ctT1wgzdRk@go4^@>GU9-4VV6bA>jI@pCxEsTau{;Yv1_> z`=aS%n{u4{LL{zi1JW`~{vHoP&RudA7Ae*_b}j$lU#SukiiAK;LJBnfVD$R)+QY?^ z*kA$1Z0BqPqEa3xZCSIMJI@k<-e@aLD zF+;YBwbO~L&)m!tntAvBCpN>K8krZIa*Qpk`4?HG_slQ=ryMwrOzsJ!PIkITW);`9 z0AHTTJIT)mp7xP1)^;aAC3%(TVRv_~oK6KB{`#wVu296qECBzT^fK5EeK*(AD+?N1 zb&FHJag=2Xi3od3OQe@G#nMFT<3)Md&^6dE;eI8LPjL5+5^h2B_Xg1^ccOVpRc{dF z5+;TooZ^hk0Pw%X3_2wKSY)qj(WKelueS(n8#<-J9^&M7+$ z=SKf=;YhNCZa(^fo{!Eb-`T0S%(Z#5G2n!jq|^Uwu})&q3y&@1b6Sd5M5=mZnvf8) zx3niv9)kr@S`oY6gO4hOjWM2MT}#L|fl?YYpI3^`WlN6Q=x7QsjW{wPd0=*6TUeiG zHP_HyE{yPu3YI-7yh{~hfLs$<@Y46jK}%wzSFS*_rlG+_dA?8X^=hOm_G=CP+dh*q zxSn^mYM(4w%d=3u4UON9{YBUzfZ5o@%-u&)FViZ1{|4soW|1Z`YEO`%W8QrjhNQIQ zm0qMod|8q6j^&Pv?0(6-j)FFuQ?+!@+2x_tLY>1RYPQj41q%5Xbz!V&Uq?&8SV4Wr zizWQuNjn}9*4ul%420xD72IpS9ypeMhDWfv!xLeR`ZAsEOh=bGl$4rCGwY|`c>?Gk z6{5lKe$BnxH}M37HNnkGNlY1lEg7J~5gGjZ&sK~}CL!zO+FIkyo{vwN7mh%A@I30y z>Do;TcGt4GnDj1r`~jzJ&4#lcfNl8%@ zvSjAZDsV^%U^arFCk{w*!@tEl&zC95sDCc8z; z8zQy;r}(eH!a!1-dcI9!>A?%`H~uW^^+y9ieO49pI-;o;OhNu?cCjk@htbp2 zNuAhj8nl4M>s#-u>uCx!Irhl&*4EJs!0ObjiELS=2*f$Ge zfg;>z`^kl!J};t5f6a$ZpJhWT&v8VwlMhyZS(P#G;Z$42;(2mx6tH^sY05Ku4`ls553pOZ%%@COh{0l09tZx}icvadr>CE~-;hIe)Dg_qIVf1GoZXDPDW z2e5%SWNiAKjmC+`QPCUw+YI-`B^e97(U5{P&44g)Ofl!CYbC OT{m@gI+HDZ!T$k}efzWk diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt deleted file mode 100644 index abc4136c586..00000000000 --- a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/columns.txt +++ /dev/null @@ -1,135 +0,0 @@ -columns format version: 1 -133 columns: -`WatchID` UInt64 -`JavaEnable` UInt8 -`Title` String -`GoodEvent` Int16 -`EventTime` DateTime -`EventDate` Date -`CounterID` UInt32 -`ClientIP` UInt32 -`ClientIP6` FixedString(16) -`RegionID` UInt32 -`UserID` UInt64 -`CounterClass` Int8 -`OS` UInt8 -`UserAgent` UInt8 -`URL` String -`Referer` String -`URLDomain` String -`RefererDomain` String -`Refresh` UInt8 -`IsRobot` UInt8 -`RefererCategories` Array(UInt16) -`URLCategories` Array(UInt16) -`URLRegions` Array(UInt32) -`RefererRegions` Array(UInt32) -`ResolutionWidth` UInt16 -`ResolutionHeight` UInt16 -`ResolutionDepth` UInt8 -`FlashMajor` UInt8 -`FlashMinor` UInt8 -`FlashMinor2` String -`NetMajor` UInt8 -`NetMinor` UInt8 -`UserAgentMajor` UInt16 -`UserAgentMinor` FixedString(2) -`CookieEnable` UInt8 -`JavascriptEnable` UInt8 -`IsMobile` UInt8 -`MobilePhone` UInt8 -`MobilePhoneModel` String -`Params` String -`IPNetworkID` UInt32 -`TraficSourceID` Int8 -`SearchEngineID` UInt16 -`SearchPhrase` String -`AdvEngineID` UInt8 -`IsArtifical` UInt8 -`WindowClientWidth` UInt16 -`WindowClientHeight` UInt16 -`ClientTimeZone` Int16 -`ClientEventTime` DateTime -`SilverlightVersion1` UInt8 -`SilverlightVersion2` UInt8 -`SilverlightVersion3` UInt32 -`SilverlightVersion4` UInt16 -`PageCharset` String -`CodeVersion` UInt32 -`IsLink` UInt8 -`IsDownload` UInt8 -`IsNotBounce` UInt8 -`FUniqID` UInt64 -`HID` UInt32 -`IsOldCounter` UInt8 -`IsEvent` UInt8 -`IsParameter` UInt8 -`DontCountHits` UInt8 -`WithHash` UInt8 -`HitColor` FixedString(1) -`UTCEventTime` DateTime -`Age` UInt8 -`Sex` UInt8 -`Income` UInt8 -`Interests` UInt16 -`Robotness` UInt8 -`GeneralInterests` Array(UInt16) -`RemoteIP` UInt32 -`RemoteIP6` FixedString(16) -`WindowName` Int32 -`OpenerName` Int32 -`HistoryLength` Int16 -`BrowserLanguage` FixedString(2) -`BrowserCountry` FixedString(2) -`SocialNetwork` String -`SocialAction` String -`HTTPError` UInt16 -`SendTiming` Int32 -`DNSTiming` Int32 -`ConnectTiming` Int32 -`ResponseStartTiming` Int32 -`ResponseEndTiming` Int32 -`FetchTiming` Int32 -`RedirectTiming` Int32 -`DOMInteractiveTiming` Int32 -`DOMContentLoadedTiming` Int32 -`DOMCompleteTiming` Int32 -`LoadEventStartTiming` Int32 -`LoadEventEndTiming` Int32 -`NSToDOMContentLoadedTiming` Int32 -`FirstPaintTiming` Int32 -`RedirectCount` Int8 -`SocialSourceNetworkID` UInt8 -`SocialSourcePage` String -`ParamPrice` Int64 -`ParamOrderID` String -`ParamCurrency` FixedString(3) -`ParamCurrencyID` UInt16 -`GoalsReached` Array(UInt32) -`OpenstatServiceName` String -`OpenstatCampaignID` String -`OpenstatAdID` String -`OpenstatSourceID` String -`UTMSource` String -`UTMMedium` String -`UTMCampaign` String -`UTMContent` String -`UTMTerm` String -`FromTag` String -`HasGCLID` UInt8 -`RefererHash` UInt64 -`URLHash` UInt64 -`CLID` UInt32 -`YCLID` UInt64 -`ShareService` String -`ShareURL` String -`ShareTitle` String -`ParsedParams.Key1` Array(String) -`ParsedParams.Key2` Array(String) -`ParsedParams.Key3` Array(String) -`ParsedParams.Key4` Array(String) -`ParsedParams.Key5` Array(String) -`ParsedParams.ValueDouble` Array(Float64) -`IslandID` FixedString(16) -`RequestNum` UInt32 -`RequestTry` UInt8 diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt deleted file mode 100644 index 9034d8fd0a3..00000000000 --- a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/count.txt +++ /dev/null @@ -1 +0,0 @@ -8873898 \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx deleted file mode 100644 index 0cb239d932a..00000000000 --- a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/minmax_EventDate.idx +++ /dev/null @@ -1 +0,0 @@ -?? \ No newline at end of file diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/partition.dat b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/partition.dat deleted file mode 100644 index 9c176950d3f897f5cc9714d7011107a21e47a229..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmdlj#LNHy11td1 diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/primary.idx b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/201403_1_29_2/primary.idx deleted file mode 100644 index 9a604fd108fd6ba7f48a9c05368431b9ac5fc126..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 15330 zcmX||cU+C%AHW-gRMK5din{yWL|M@jA{xX|T1r%;G*qNQi%LclMN5%UiH1s%G)1Yb zgpgDcz9kKf-{*O*`}_0#`pk2lb;Oq^&B6N71LTr@Km*K93|N|JcPLgIIDY7Cz7mSoC8YfJH5ob+pVQYGl< zWj#)-Tw}++htkoHCJ_-5cGQEsE8UGr5CwBEci#}s;}Ar{G*fPL=w9L?`U^a>l~rRZ zCKuW0%xe)5dF-yuG}WUMJk<6Dmz^$RK6X^+AkA((!ucenwaF&*f8Kbl1l1zXbe z`TUN{Y=WrRvK5o(pXqW5qG6m3bH+Er4Bj2XXKB^^`)`%2*vLs-6t0oXLFFsAzc5jw ztSH3uVAU+7s4~z~HBr#iUNF%pww;Zt7mC6)y6xlk@@y*uy;+89*3rzc0b>R#U4_fe zl7IMQDKk-}9!}LWeYeRMGYFz$*Ya7S$!ED}_8MGvbkSLCnKKjlnv25Sv*en?&=xZe zdbt~qHz(C(^W3aWDSn&$3UxU?7e!?!`}L z%wyG24*DI3XQrLv(UP-?fr{dBFOI(^yXAhdkY5TOVO>%a_sSVI3QQM;t0EOT8W70m zBa;lAAvt&iSBtKLMLarMUyMia zKAKQ`xs8Pw^>}6pmvZ`T2NkV~yab*jMx0~x7{!H6ur96Bu0%|W|3xTeL()rAHxnP^B) z492%I-hR)c95ymB!0Fm_XS*NT9Q4^1x67Z^{AMqci>e&OV3r(BmR#-VCjM4aN??R>o#2#AF3FgxM=uA4!~$QbWyw%L$5iy#U% z-DA0Z)^s*ORP2B9^ermy%y+=pMzi~2`DD3Eqr6a}&t|@ZN%InfE zF7go(huJ-R`{HjqTNZK^69<`A()jG}jXab$8<&|e#*G{!S*V024qoi)^FDvcr=#%& zIFYxm?f9+1KsigqA?w)>0%qRc!bVG0;dUZyCtunH7D4iu>i5FYBVK%@yg?jBG^Xl2 zQf~!MoDoj=HJWylfjklJQ{+Zj(fm?6YOus}Nk_)}+Yj-Ot~)MUZ*y*3n$AGo{y2%| znsgOwGthzr+>7E}O5ORBOr%_i%kWwHx}1(y)r!Mi5c2_BP&3R!9xrj9W}8GG9IfJ^ zn_qFeibuNcJN7V;+z3vbA$5!5cm~o~ISn5}N`rJ%01FxVPXk%pdNC=lBnDd1G7Y@E z_YNO&sN|tdo2NrOm-sr(eMji%bMbW09Mjmf=HpHV%I}#D^O7`>{_;>68_n^Qfcukv zS$zHAN_f9SNkC@ztp|cb;JusmLjvr&Bczx4ZDgXfD>ER%^XtvIap!o*Ky)T}Ik{OS zKMLM%{&p1@9!HLE@L8_g|i?RPqEq5iH(eZ#NP6F)#t533bE_3 z(^@K|N0C~giQV*nn%)nw>ywc}>iQtwXs`NzQtunN!xs<4s}z z=JDtxEJk*pr@Nt$zEvg0w#^XIroS#FvGCp%XyW(M4_84hWGu~ES9+|4Maallgw^0u zmxn@sArcx)(V|VhY*+WKVw=CwplLiDuLH_|9x0>7U^JSRErESWk$IDoXbX zyucSWVavw$Iw4KS)q8&T7gD0ug~VXQB_dFSM#HDrRxv4&v2q(J2JG+DZrfJ zM8k&WX^kRHc-P&)HLqFbs!48O5=6ls{q3L+>%&_SWHce52sCNfmmt5#aSsJDd2ISmUeP-ZO>~q$8#22jYao~DO#eqlN~ag) zrU~gnFJ|iXMM!0A50@R5LaMNs2&pR(Qi!eU<&fDziuVXEinuMLA5q57m#-Gm!IBm` zA^F>O=bF?BNmb!%{cIJX-JRxwE?r{*k+J^7qoQiCM!{!-j5YuCQ;CN4r+|`|o9foM z3uX07EG$`eLVEm4&8PK>kZv8fiJ9dsB>X8Ry%2f{BI{N)6$)kaa{CW{93hcA4~`W# z2s9~}U5>vpJDE*Hr(hZ%Li8IJ3F(b@qeG3JgiLU}|4cyb0`E-0H4XwtBE^K^}e zoP;v@-E-V$ED%z>Txe(QF!G`e6Uv^ZvPa_o$>cHdC%fj+=kSSm z^4Q83tqRX}m>09r^Y{4uLSfmAg#BQmWeQU892vVA$ObX#|0uNN^^WE66&1*;I%Ffw zV7(+D>m_%p11$tZ!rXQB#haH3ypS-zo^?j+<_JkmV`<Dt1GYM!EoZ2(Aef_t{DHuaLr* zUcYG3C!~q`dr!O9a|ulvrV_xKlg{P+BR77gEp}NbvtD@q@Z}pq!o8eV7HYO{j(=UM zC?I*Pt7*I1*l9lDMIQ6D@Y{X7NJyfum>DyZgjA|$*IhHtN3zTDD;8sCZ}0GlP7nz@ z9L7_90&f&tM#3aMwtr1N!z74|$b>06U(`RK%I{HnYQ{owU2l!cnV<8$_G$1LxYUR)GAA_Z%y zFy>|V>#CaQ^;&7rv|ikV>Nm1bZ>BV4J!T?qx!;#gkUSRJpH`!Pg^%uaO2gGWHrsqF ze<24oV=|!G7`k|Q*9aY*J|hDW-e@Xm9%XS+!$TSH`ONq1K37;LGiznQ4u2DkC$iDy zFq#D9r~^SD+rmM;0J{p zcxbhlB4}Qe`=UsfXZ)kJGnXv?IYs|R=PX;$$rvFm^q1PZsai-0tx1Qhl7%!~MKe-Y zQt0JoEG2cyOehlxZOfI0%7P%C;_pRe*XIbS#8a-20_johoB>_2`T9%{#dpT0Jq1E)1P*%O}VAp7^fXJBZShk2`5{s}Ce6<5ggygp4 z>9W{mLfVJY&Ar8hbYVnqU-54tneq_CE!gk+BFx~H&ND8n;r*d(No zqZtWiH9`uIzLh1NCM5hm+jU;(C4P-av&}Xk)i~yMROurh`aW`io`{l zXzfh=-o0Csekbq)1LZ~G&);8Tm#;z7V9%U_%XZ$J@^n2#1c!!;jLOYMtyagg1Z63l!^?dFjHsOe;FR)#mj`Q2K{ z^Om!ai;FTuxOFJ1W-Nw55DnYh)cm|KpNDQgQU<$|6SmL2B-toETm|g>)i2QdbdYDGt6?0Vd2V&JHILb-=`|lR+Y<5NXyQ@$22d7(7n`SQ!TE-C zG;cr)B9uoOWJQNF(SIsSz~}v1&z9P`T=e0+Hsqqwc#F$a<)JeMOTo_EU$&=iA=K+4 zRsii-sc_Q#I34X)TnUuFNJ*!`3`R?k#o^NjrJ))lppA8p3Nb7{+T$>;EmhsdvgRb3Z^L4LeYku8?H&gzLu`o<0-@WN3nNr6)u_(%Jv(7m~j6nAS#xl z@!a6D2ba*KVj&K<&yOw>lDF51Ps^kPM8mc`h}Ag{t8F|24WrcrU%BvJK=PQ3-XaML zSRvsud2BHjZREaE^B?_IxswbZ8ml@< z7iXZamijQ-KQ8?aTWgr;%sZSUu1{V$qr^ggbk=}fi&6N(x_KPr+_DyC@aX6-+L;ap zL1Zj3uhnDkFDCk=Y5~z=CP5jy#ZYJZ?W%USBG$r#lU)q7kD@6UN_XeSG5>I zgo4EEih)5UdbV&Q*uCA7bb4|l8x6K^0+QK!>TwUBjeIT}gXSkIg|`S>AU-d&8u0y^n_!K{YZEuBm!T-6z{SsEUH%$604@K^A z2Tl9@pfeRK;2CT30G}b^Gb9SNIB49`3rJ&Q?o%UWHp<&|5WL{0LZ{xc(3F8U*e$Yk zOX+|$N`{IrM6Wcd?&0(@(6_EbV7F|^UlmdU8@)c~2bv$r4GI!ab;pT<{aKq7^TU;m zZY}T!yYw-}cj+J|`dJYGG!$TC)#%AUep1K4E<5izm8{P|llyTB9cfQImq$m`@gSIa z?#243*G z&4ULlbm$XKYG{0mDpdWfl}k9ujZsp2!l&x?QG(V*M<@k~7 zex{Ji8?|V-3mzLwyYXl|bnZ(uL|7l-y5v0Uh;X7}jtLsik}q+P``j2Ha^s#NZD$T@ zToVhLf4 zG96?U2}zLkvm*{0BUUg7B4MEl_uIb&G6|v(&jWrsfNRQQHqpzHqGdJFt!&)xWq?A$ zE;x6>iHt@6kyc{D+Xg52{;hj%S)~s5H4qi+Ej<{$396Gg(Xgx)wBCiV?paxe%QS5g z-nq%anMV0#u&d`(Z{Wb!LNzZLG;0HnlIFvDsdIA*Xx`LJRMDRWdy>Qyi0&)5x+4!( zGj-LeAUonZrRoH$yxnbSkmr+;8PQv%VDGy<9c0tv-9pl=IVjjS9qblIto82>p%X;H zbi9pAx-cd|6zqM=1_k~on;k204ad2B+OO)b&jBSY2|$g_%5__VH?jkXnK zz|2=m-Wl;e%tVpfGl8h%JPmnxW8y@^uD88fPzU>WmE$PhJNv?JLTDjMVT@ z>+p5(;?;iA^QI#g{ms7tnvSc#-I9fqpSHrAK>OSVt1b<(P^(lnM9BVT7nZ?>)5j&* zkY_sYdT;t}CP9!(jNi)mFfJ;)oDH*NtbR&X3+@S=s07u+NpI4RZ1CBfm7s2r!a{Fj za)8XDE#@V@W+4xkT$mU9?6WGEjUFw%4PLgE++$3@sstwz_IvFgZ3S4n;Y7w9BDc|w zcC!egV4df$s5q7g?5LP&@oI7(R8??I8n#k7WO0}-k097<*BmRlUj?V!mvNtsqw61S z|II+jDYqfdkLGSx-=QYAxHb=>_eaJ$h%baYgmnkTeyO-(Q1U9+?Ys-odwA<-#lbAC zm&*q)2{&aL`%e^8sf4T( z4nHLwy2C^!_bb8Yr4?<<0-+*-6BVn7wmrKz9@f{Hk3o~Cka*^6B@3N3tpZIEIdA_@ zKj{QPe+W=E)Ykm`t010cy|c~UXeKIsSPfoYY|oILZNovoK2(F3OC`Y;GoG>tqG5*T zzSYkv;3A19H89tjbHCd#y16Jzt`73V&rsMSEOg~}JzQytJbOLWJAAb3QUgS&2x_=| z0^Y5M^`Aj57shNq4Xe?S-LGfh#b@TRI&~HgF}Y3P#m9e*6&<=hzLhnBtYjnauN|9* zuAO-fv`6Xb#mx`>u_gL*1rZ#Kc%{vbx?ua^zAi7 zNa-;fJkI7KbJsQ)QKL^-w-cQF{5|v*ymXcfmCS}}jAqPR7~ixJ$uT_+bdmtU8q7Cp zk>?8z%DD3ue10A}G}a7V7*9X8LuUH#80#bo=*WKGdyp+;B~R{$-Srx^50DGKLfQbU zAX{z+Xx{(b)ZRXmK@ijc+cI8Vgsu;7^A3pTP;s@^9M-fsb3OviJH__ea~Sq`2R?$A z!!bwJ_dkHUp!Xwq!S`btin9nA1@qlOj<9+v$p1%i*S43KkMfa~_GgIa()L&1h(kw_>`oY; z-Y|X0H4@Hme{_N@s~|1bPlHYn1#69qc~RUakjZ1EYl5vT*YnZdBVRzXb+scmz!;v} z8C@{fT8$~{y?{tB8(lRMPZVV7ks)eRJQT<@zHoY9_~(*v3- z=C+KcmcZ^>q!;ETcXn*1(g)}u>ch!r>}Lgigo~P6zrlz)MV5_Ene!3pMIYELDVbG3 z+Q~wFv%bTeos|D{#VeFS5ZLWzd6XXKAaHd)@+XCXJ0U!XZRD(@ zYH>g}9gWT!1TXXdE)Mzy`{@j~LAWYb0eKWFR}PAr7z8`KE5ap{jS8v1LDPcyb-f|f z*>Qr?|2N;&9z(xIrpj;7OsJk?FI~e%Z{mML1h0PYG3o<4TJYdE_`J}WH=R3=i_V4( zftM!rLyz-CSg3Q|AJCMXIu?9;E(1xt`~zNUO~+`)Ob$}~FbtYACZrQjRWs3^CnG@3 zmdl!2nNW4890j}866!ld&(H}XW1EL~-J7A-h09>|v?2T6>y_{}9U23gsmBU6hIa^K z`Z##Os{p-+SxAFB4ziszr@;AYJm`#?fIM4Tj3s=bt7UokBwVq=+Hd(Tj(qgYa0+PO z+1ThbrY3qnJ_TOdm{{QO6cZImh>F4)JKABBew%^nZ-|Lfup^;%zYj>j&Zbx#=$Fq> zeh7t)ux4>lD)v)V{ZAp(m(Ijb1NyMx-fA9)i^MCZi^^m7yhT2LQR5>cof#m*&w74u z=c2TM84zKa<(0H@sAQ}=GZV;+@zFYTSU_-A{zQE{S(!t~sF)~oARgN)AR6}Be(x_y zSnu8`nkh=gbWS{T?OF+IjpkXRa#-fw`nn74bb`njb1X2)vLD``AcM-foJ5*4^g-NK zmIO`lpkK#s!kz>tc`VsrPqwEvAN97+hFthxkFE^;1?%1aq`=Pkj^Qkm%^Va^E+b0A zMxO*r9)K=3s^=W=Qup%xGM|sImjR*>=eIKAY^0Dj2QvFo^660OX%;f}kOiN|A8H?5 zxPpn+M$ZMGH|ldz+AX+fX}uh1zFz-VqalEWPK1#mf|Acw>%=@R>iS552p4@NlIGUa z38E19wGOOl#YCwfJK$O{Qge)sC@X0|J@z(le?OulF|IuL>~@!kVCX^JMpgl21FPpo z;Po?=9t9ZfHRkxVM-lLq=~M(6b5n0+4fN@7Ta-YS`BLxk#VI(c->3pvH{@q)ErUw% z{ZtjmdiwNro-U~z^!La-813KTxMvg7Ip}oXe6YK2x994Ua6W32Qw5SroY&Q`h>z?o z)F2m?(vb0IuV9z_YXQif2koc4O6H>+8+C|&|KqRRpOtjvbyOYV*$3*X9lyvx&t7Uk zJV(YF%ct!e)agZsc=f!v3J+-xN_A%dE&Ao0{KyD;HT#+1bJehE@K5L;)KFuC%vw$F zV|*&C{9D)%!GHX!RW%RRgZ3QA<%d@HC(dIyQF+XPxlUDjU$Ny5tZ6ws(4@FpH-3<$ zquFJA$i>0oai%d;2pp)3K+`j9Zs>Ry7rk7h1rh3Ah@LKa3jJ;qTHrI2GIC41n~7v< z7XxL#?x^#-$wGf^w86`b;ZLO*P^I!W(}4(;12roRTId9ku{x!u?QNim%cxkG5puJK z&H$X`v1rA&AEm$WQGOq8cOd+&!e!Vq<3z%^_K7w>;9kNB<|VyxMmb^=1S_kKL38nN zE$7&7roNOV6<*} zJ=5`8-ikV$q*iggr7D>O3A)1`!j33Wd<}Rhs20n5Ig5|Bty%+KW}a7g`V{U7@e6BV zcDtgnh`Ev+bZ?IVzf;NF>oBg%=?ou9-%rSzQ-@e@3 zAhVQ#)V7(x>}qLv(08OzDMP8`jJp= z+P7dEcu9@?NtV}RqC+MKsBQUO{`TceWcn3BF6&>AckVjFM~~KZX$s=Ym6?}X8|9Z()yTh2nK0_{NaeMg^j?mGtZ zklh92>-G2hay}UD7fA;oFPD%-hB|bDNSH19LsAN6{v(oaT}y@w^r?a- z^n;Cx``g1#11Aa=DWbP&Q;5I|6&tMco?8Q7AY4Yn@XEeDsRq z4Dm7(mL~j3p`$+gy)bsXFMTQ4Ex5W5znY!W=P$tPUU~qCffUhTCnpad@enfM}TGz==N-$Doh8@c=~U$t>No&X14Q z*SLe`#o-nAHT>cGSKiA^iOy>AA-yh#LiFJJcp`-#$nLR z(R$Y$v5t%OF7SgfT;T4pN`aHm>mQGR-D9nx=GrCDhimB%@gmC=zHfNXLz^oCK$f$^ zafj?E1C{R%1TSe_PCdby@N9k!gk1Dyj;-q%;iIA%$6*XX{T?x<|G_>$F$id6f9p?4 z2dGP_1%qb!{55sa%9`k@XE69|UsNXRVZ}tQDIq}k4rR$qCYnwUgXs8^lb+2*Q6*sz z4?n|P@`Hnvd6}9xpTpJ--x$(Yloemugph?04 z7oS+V^fGiX5R_*|zo5BLNHS8|!+wqeO)~Z|%g12p4CrnEO)@r=`zxWhS4gxP z^6i}xhmgTZ(^TY>X;6E@?cgas*>G=dGLIm64DYKHhrZwB>$t2@G{|r@^vp8qqaZrt z zY`?~XEbE21mlBzS%KZ~y=KZ*Rh8+m{=Y21LZ0nT=N9&+ZM`H@7_|0xf(Hl6ZwDclq zYRZNd?D@z+o?=Om=lu|R!@je8RCYQEB3N8vlxzrP5+si$+L(0yfqOph`z4U6))j6D z3x^X_AbIRytnS*z*?d&-@iNGEB=0+G^q7G*olXH#7d6g^>4!6-fmDdckz3Of3bmQT z!)cIVH$ zs;VYK*BmZ`9sBcbDjUaG1W~a)oWV_bhq(mNu(XEh6_Hn=%D6TYqIXlvM?Y=npn8w1 zK-#K9f2To({M6_*@PePx?16Ji+k06+?5W0!S=*pKF`5O@t<1N-dXr8^H`}j6W_Zt# z2y{j0th)hOfA;UR{ys|+g{a*GyC=>g*k$OV3Aut3{r3r!cHX7XC8O z$p+F`k$7bPO*WdbD+jz3Y4CJzTJTZF$y|sq9Ef4tp#o|7F&Fa0_fGA{V5NHVHprY! zSC$XM&PH}y9?0T+$J9u_IVk+Y9gq#Z+ntNA-PR@C1=+@nerg@sP!~?f2fACJJ0_;X zMQwWvfG*nA%-jY2y8?>(Yj=KXl0y*eLay!CMbWS;2TdB$@12vwLq{$aK%P|Td7Am~ z*GB*A0vN+^TAW<#H&`dD6~Zh{ZNB3j_KS`}*o9zcwRz%XPc;{v7%KvLQl56FZ3Pd# z2)GA?zXwxod1!`pG30VJT&Jlaii6_klz^9Om_xtkB|1Sg?BhZoXKn-!4cL}Kykya~ zixKCbYiIL)7;RiT=hvwO2J)hmf!(VYuk}piV66`X&wKWx#gAvR2!g*8CYKtmX=4)v zt5~HHBi(ckK{QNyQhiI02#+AB1bfnUwZa#@AhHZH^D;ZGoe~6fKEHDCdBNeklP~PK zR%$+g=zB*eZFAP%MpclRduT~loiP`gJg9=JvfBQhqFx3c9XeeNJ}*g!y_Q@Ot2<5l zV0Q*Y!w%Y}s(jweL-FP{;1geo$7#R`&f;3o?EJJOjv_|JN|~rv_Tzy7l?!{`NPZ={#$oVj6T1Sqc)4ivsOZ!M77B9z0J61Z1D_tO z4Im#yn#p^<7_p-d=R@It{&rgNa{Y=0zjE%iEshAZAz>*rjP_%gtT{_u!7N zFotfq9r-c-OjL5S2jbm}MwX7oJhXGN2V}Df-?=#XFbRSR0mb=9-FG%YG@{ecAD$dp z^ zUmR4vfS@l3H2YHp1gBS^xo#<(&GWy(OgxOKjEX(ZMjo|&5S=d>wBQx=N9I-ZgRE;4@HoD*c1Fi$!r?W4Lk1{^|1kLUzQZ7gDvygt_FNkNg`0I($TO368`42SxzOUp& zJ>jF!=>tFq=5jPwUWTus=>TLs+wSb!U2p;Z_rIj+wc?zvIc=}De7-EuV*9b!$Ht=;x?w+!XFl1D~7Wr3!=kL1I{h(!q>2&0Jn=W zNG=Fp%to;XM`omLoXcXf8cCvYLasvxp>l%fz<8`UwwQ!2~XAG$2Wnsa3 zX+Fx9`3vKlrtY6O*#duX>HY;f{11y;1x&Q6_b-e=Zjnv8E9}m4NB%-)7p+6OKAzy9 zXW`>O>yxsU|IXl`3z_4P3*NcX0Au(_nE)@wy~|p|*Rqi3-3f@WiL_5w@+7>bDU5*8PU4iqHu$jQ5C{Da>=Cba diff --git a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt b/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/checksums.txt deleted file mode 100644 index a72e2bdb3706bcc08ec2f7d6ff976c10e56f75bd..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cMSjn%S#tikUGF#tP8}kE# z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOBB}>nT0?1>@KmfG%gdU6%j|WMOn+E@EkJxd)

ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcnq1^RmKF(8EcDh KC_ym>w$1=Amu)2g diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt deleted file mode 100644 index 193d3123896..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/columns.txt +++ /dev/null @@ -1,12 +0,0 @@ -columns format version: 1 -10 columns: -`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt deleted file mode 100644 index 7813681f5b4..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -5 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx deleted file mode 100644 index 4b6c49f8d50..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -…H…H \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/partition.dat deleted file mode 100644 index b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0VWo7^X0I>iw diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx deleted file mode 100644 index 99ab3ce7835..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202010_1_4_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -…H°-œ_…H!.œ_ \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/checksums.txt deleted file mode 100644 index 49d83aebd914bce7c8ebb35018eb910ea5337b95..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyUJ56aG_hdC|uFP3IrAaDDV{y1~G} za1Mw++Olyb=a=S{=#^BIFy_n&-TQLZLq37-DI9oG#gHUryHuyxGq-0z*~@5 zRFYW&bhjQ*5z9{=eeu7Cnoj8~>F6<*FiB_NE-1>(O)RPe>E>AzQ2A(+8nc=5L>~VI Iw&B?f0H_RU;s5{u diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt deleted file mode 100644 index 193d3123896..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/columns.txt +++ /dev/null @@ -1,12 +0,0 @@ -columns format version: 1 -10 columns: -`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx deleted file mode 100644 index d3715e50d46..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -¿H¿H \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/partition.dat deleted file mode 100644 index 7af4421fcb5ebc50aa06a5f4dee6e454020552a6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0UWo7^X0J#7) diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx deleted file mode 100644 index 0156d831c6a..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_35_35_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -¿Hˆ©ç_¿Hˆ©ç_ \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/checksums.txt deleted file mode 100644 index f45270c50bd34e0f4400456a9a0ac4304ebae307..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 261 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyXp3#UiEg{lBXdkG?GYv@f%}=_UgM z!+9Y7XvfBxoL`z(qE}K;!W{Rh;=-DOtwxG>m6dsRwjN>NNJ%V7)Jw|DJCdSa{OCi# zm(z|B#fI`%l-4CPUb)Y{Vnx12R!7vt_w^?Z&Do&ziBq59LtbuCw($j~;%~BCO{aYv z_O55y$Dp`Sl0h^#GcPx>B0jY&HLoNdXl|-rW=aK%{)#M*%!oA;KkjQ)eIr+F%D`KY zSX7c(0(7_@P!Y>d9)0n@hni05Ea~VmmM}?Y;4Uc2%uOt+1nK5c-7Q?0BT~9Ltm4Vj Jxg9xO3;;OHZSw#C diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt deleted file mode 100644 index 193d3123896..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/columns.txt +++ /dev/null @@ -1,12 +0,0 @@ -columns format version: 1 -10 columns: -`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt deleted file mode 100644 index a21cae3648e..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/count.txt +++ /dev/null @@ -1 +0,0 @@ -454 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx deleted file mode 100644 index 4fedf1265b2..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -¾H¿H \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/partition.dat deleted file mode 100644 index 7af4421fcb5ebc50aa06a5f4dee6e454020552a6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0UWo7^X0J#7) diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx deleted file mode 100644 index 698b75ec43a..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202012_5_34_6/primary.idx +++ /dev/null @@ -1 +0,0 @@ -¾H˜ç_¿HÆ¥ç_ \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/checksums.txt deleted file mode 100644 index 5d9d47463d2dc3c588c2a68beb24126184e5955d..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cvWrlG#xSW(BO;NigHQ+OX z+5i9m&;S4c@jnI$V{dhCbS`vwbOb;rV4Zj{6Wu0P+wm(Y%$fiQWMOn+E@EkJsoeoZ z83XkA_OM)y9%i<(>8`Z_;I@6CZZ2eDbOfQp0V$_ea0t>|RE;3Dk?sHsaB^vFVRCscX=Hc|1aaTY#PfTb M%D$$}R*TMQ0I^wb#{d8T diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt deleted file mode 100644 index 193d3123896..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/columns.txt +++ /dev/null @@ -1,12 +0,0 @@ -columns format version: 1 -10 columns: -`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt deleted file mode 100644 index 0ac440f96a1..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/count.txt +++ /dev/null @@ -1 +0,0 @@ -1378 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx deleted file mode 100644 index 4c89b29a856..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ËHÙH \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/partition.dat deleted file mode 100644 index a713aecd075daae7071deceaa93147871ef28fbf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRFWo7^X0v!O3 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx deleted file mode 100644 index 6180f063fac..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202101_36_115_17/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ËH×Õ÷_ÙHjì ` \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/checksums.txt deleted file mode 100644 index b91b7faa4f06a1664ebb74109586cac5fd23a1bf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 261 zcmV+g0s8)9Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bE+#qTQ)Lu{YCP+gXf%IE~ z*#H0l&j0`b@jeC#V{dhCbS`vwbOU(kK>xQg&QB-oQa%ok*L?s8WMOn+E@EkJi8YXH zVhIrlaZ>%T4I&riEH(kO-vz`k)u>G8-l<@lxx5?vo?Zzr0Pt>Ya%(ft0?3Luk)Hm4 z)H$e;_u%sC>HrmOX>M&{cwc39Wo~p|WMOn=E@@2I=Kk=xD!%mjDZJa%pX0a(OOkWOxk4<8|vG32}9| Lb!HVccCGgS;Baoy diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt deleted file mode 100644 index 193d3123896..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/columns.txt +++ /dev/null @@ -1,12 +0,0 @@ -columns format version: 1 -10 columns: -`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt deleted file mode 100644 index 0e92c3c0fc9..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/count.txt +++ /dev/null @@ -1 +0,0 @@ -306 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx deleted file mode 100644 index 0e3b7fb75b8..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -êHëH \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx deleted file mode 100644 index b8d049dbb19..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_116_127_3/primary.idx +++ /dev/null @@ -1 +0,0 @@ -êH«C!`ëH©¢"` \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/checksums.txt deleted file mode 100644 index f448a75b6063fba410c431980059309d4b9a6aa1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$a$Hb&?2dEW#ESpcY+$ktzi z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOCIfRlVt&lYFUDR9N&B1^)mDWMOn+E@EkJ_5<}3 z3}5ITzipTtZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnsN(E!5h!GNyDe K$;I8+;Fkb~3T$=& diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt deleted file mode 100644 index 193d3123896..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/columns.txt +++ /dev/null @@ -1,12 +0,0 @@ -columns format version: 1 -10 columns: -`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 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/partition.dat b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx deleted file mode 100644 index 9f363c02b40..00000000000 --- a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_128_128_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëHG£"`ëHG£"` \ No newline at end of file diff --git a/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/checksums.txt b/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a/202102_129_129_0/checksums.txt deleted file mode 100644 index d331cb1fcc1616a320c584208bbbe060c20c1947..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 261 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyV5)|94mWJa#3U_`UDuPj{Tybd!OB z;XDw3v}5B;&M(a?(JQGaVLEni-X{0LThekT`8{LRP8u?Bq$HLk>Lq37bqTze&kSbS z^7D+CCQnYxcHy^-OLj7TynlAv5BGeAS4nH{ichgE;?!sOke6GOZG3^LKys6eD>t{J z*eAbZOH(qk8ANk4^Kug_;#12~^Gf1@=BDaprc|&b9%Ngvb93e*TLDw!M-#K08F&j4 zi%K#}fDYFKDq?YzOsi}P`os3}X~c_+{jJj(xC@Fha}$dyLArVRV>iAT$d33qg;0jf$7<_1PzbQF>&8 z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOPvZe4NH(<2?+h%j6?;@(ut9WMOn+E@EkJgdK+V z5ZhW2+=>to#{T20Lq-9{RA`+*_6V{cl3!Qme$Qq=nF%fc@NR8#Ycuc(P`_DAmgVQ_ z!Oxz{GfIzl02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcns^T1OQK`i$9(N KipbM-MdtvM<#0p* diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt deleted file mode 100644 index 532baa6cba6..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/columns.txt +++ /dev/null @@ -1,41 +0,0 @@ -columns format version: 1 -39 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`memory_usage` Int64 -`peak_memory_usage` Int64 -`thread_name` String -`thread_id` UInt64 -`master_thread_id` UInt64 -`query` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt deleted file mode 100644 index 3cacc0b93c9..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -12 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx deleted file mode 100644 index 4b6c49f8d50..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -…H…H \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/partition.dat deleted file mode 100644 index b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0VWo7^X0I>iw diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx deleted file mode 100644 index d599340cc82..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202010_1_5_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -…H!.œ_…Hm.œ_ \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/checksums.txt deleted file mode 100644 index 4398dc6a7abbe10202dcc9ff464b250d9453cfcf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$aI-o22|e$OHz$TackS;rWH z+5i9m&;S4c@jnI$V{dhCbS`vwbOTsTaBK6JI5}pr1~GpWy)OU=WMOn+E@EkJsmK8g z4bfp!g8FEMWeVZj)V}Wlv(^fr`M!nXj-ryTzn4HBVbGchFaYpwZE|Zf@CkwPt9t01 zsfP|9Ri`pFb>9FLZE0?8VR&C@6CZZ2eDbOiYfFBkp7g32tUh>SBBG-m({aB^vFVRCscX=Hc|vV`+0{_>ZK M%eyNa13TFQ07qJGTL1t6 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt deleted file mode 100644 index 532baa6cba6..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/columns.txt +++ /dev/null @@ -1,41 +0,0 @@ -columns format version: 1 -39 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`memory_usage` Int64 -`peak_memory_usage` Int64 -`thread_name` String -`thread_id` UInt64 -`master_thread_id` UInt64 -`query` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt deleted file mode 100644 index cbd6012bc6f..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/count.txt +++ /dev/null @@ -1 +0,0 @@ -204 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx deleted file mode 100644 index 4fedf1265b2..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -¾H¿H \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/partition.dat deleted file mode 100644 index 7af4421fcb5ebc50aa06a5f4dee6e454020552a6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0UWo7^X0J#7) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx deleted file mode 100644 index 7f33001d913..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202012_6_45_8/primary.idx +++ /dev/null @@ -1 +0,0 @@ -¾Hb˜ç_¿H»©ç_ \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/checksums.txt deleted file mode 100644 index cc8c216e1b7494e413c0da9a5b31486c5d90c403..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyWf3AW-@%fo-?_t?&Cpwzzz3y2Zf2 zZ~=%v+Ou&c=a=S{=#^BIFgFFyXHxB3fAoMC=byk-#h(luDTyVCdP$ji9rGB|rzA|U zJX$kL_GQ{`uZTS%j8ms`sB}#X6EH4R>Y^4rRJ5y0}W2q%S@?YIU{wz_k+a5P6o-x>)Eq|k1_BT zBo>uqmH=I@2UNte@Gzt1@~{GqOEDq63R@?>W8f|*%FInHss!ogDczYWTxYG5;(xo& KrZ}f&2Lk}qd}`$Y diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt deleted file mode 100644 index 532baa6cba6..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/columns.txt +++ /dev/null @@ -1,41 +0,0 @@ -columns format version: 1 -39 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`memory_usage` Int64 -`peak_memory_usage` Int64 -`thread_name` String -`thread_id` UInt64 -`master_thread_id` UInt64 -`query` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt deleted file mode 100644 index aa59885c836..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/count.txt +++ /dev/null @@ -1 +0,0 @@ -143 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx deleted file mode 100644 index 4c89b29a856..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ËHÙH \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/partition.dat deleted file mode 100644 index a713aecd075daae7071deceaa93147871ef28fbf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRFWo7^X0v!O3 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx deleted file mode 100644 index f166214418a..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202101_46_72_6/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ËHéÕ÷_ÙHåë ` \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/checksums.txt deleted file mode 100644 index 0921ff99748c1a04a7fd95cb8943ecfec846c1a0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 261 zcmV+g0s8)9Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$aAz*8X&k@J-6kO<{X=LkxI z*#H0l&j0`b@jeC#V{dhCbS`vwbOJ{BwXp-jZa9E;ck*GPUV8utWMOn+E@EkJ+%Aqj zHZie?gISC4ZtT~N;D7=A*a4?auCebz5f>&NuopMmmh}lQ0Pt>Ya%(g23Fi<6sEDe~ zu$pGM;@RW~KL8bNX>M&{cwc39Wo~p|WMOn=E@@2I=Kk=xD!%mjDZJa%pX0a(OOkWOxh%C+Pf(wa}}N LsJMgcdn4umo7ixJ diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt deleted file mode 100644 index 532baa6cba6..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/columns.txt +++ /dev/null @@ -1,41 +0,0 @@ -columns format version: 1 -39 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`memory_usage` Int64 -`peak_memory_usage` Int64 -`thread_name` String -`thread_id` UInt64 -`master_thread_id` UInt64 -`query` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt deleted file mode 100644 index 978b4e8e518..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -26 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx deleted file mode 100644 index 0e3b7fb75b8..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -êHëH \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx deleted file mode 100644 index 67f45d29597..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_73_77_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -êHÍC!`ëHG£"` \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/checksums.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/checksums.txt deleted file mode 100644 index 2c30b88770877a5ddaa75c2f3daf2dbc321be19f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cbNRM_*!6JF+5+jxa^3W%O z*Z=?k&Hw-a@jV6!V{dhCbS`vwbON#VDF3f}{FN>b+}S`mO_BfzWMOn+E@EkJz#hw3 zOT%v^;N`ZYj_KCHG`s=1SwsOGv?YnRsHJA*@;B-H;|VSR@NR8#Ycuc(Kw~dMnFgB~ zmXR7wn|S>302OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnl8Xz`TJd7NP!7 K53t!rdd~o;%5H-I diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt deleted file mode 100644 index 532baa6cba6..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/columns.txt +++ /dev/null @@ -1,41 +0,0 @@ -columns format version: 1 -39 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`query_start_time` DateTime -`query_start_time_microseconds` DateTime64(6) -`query_duration_ms` UInt64 -`read_rows` UInt64 -`read_bytes` UInt64 -`written_rows` UInt64 -`written_bytes` UInt64 -`memory_usage` Int64 -`peak_memory_usage` Int64 -`thread_name` String -`thread_id` UInt64 -`master_thread_id` UInt64 -`query` String -`is_initial_query` UInt8 -`user` String -`query_id` String -`address` IPv6 -`port` UInt16 -`initial_user` String -`initial_query_id` String -`initial_address` IPv6 -`initial_port` UInt16 -`interface` UInt8 -`os_user` String -`client_hostname` String -`client_name` String -`client_revision` UInt32 -`client_version_major` UInt32 -`client_version_minor` UInt32 -`client_version_patch` UInt32 -`http_method` UInt8 -`http_user_agent` String -`quota_key` String -`revision` UInt32 -`ProfileEvents.Names` Array(String) -`ProfileEvents.Values` Array(UInt64) diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt deleted file mode 100644 index 9a037142aa3..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/partition.dat b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx deleted file mode 100644 index 8f3c7c8809e..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/202102_78_78_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëH®©"`ëH¯©"` \ No newline at end of file diff --git a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt b/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/checksums.txt deleted file mode 100644 index e9f3f5dbfb1363c5743ef536fa11535cbb312446..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$c>P9c|kkD-grFaYpwZE|Zfumc7>wrI~> z!G@D=hNwi5yodl5ZE0?8VR&C_3*EI_1(J_Im&g zaA9(EX>@6CZZ2eDbOiRJGt`jAL&ph4JQlLni7WsMaB^vFVRCscX=Hc|qQj~+&$gS+ M9BY!1)str&0ESs^mH+?% diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt deleted file mode 100644 index 37cb59062b4..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/columns.txt +++ /dev/null @@ -1,15 +0,0 @@ -columns format version: 1 -13 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`microseconds` UInt32 -`thread_name` LowCardinality(String) -`thread_id` UInt64 -`level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8) -`query_id` String -`logger_name` LowCardinality(String) -`message` String -`revision` UInt32 -`source_file` LowCardinality(String) -`source_line` UInt64 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt deleted file mode 100644 index 19afd4ebd92..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/count.txt +++ /dev/null @@ -1 +0,0 @@ -527 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx deleted file mode 100644 index 4b6c49f8d50..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -…H…H \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/partition.dat b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/partition.dat deleted file mode 100644 index b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0VWo7^X0I>iw diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx deleted file mode 100644 index 714ba773856..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202010_1_29_6/primary.idx +++ /dev/null @@ -1 +0,0 @@ -…H®-œ_…H€.œ_ \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202012_2358_2361_1/checksums.txt deleted file mode 100644 index e0dd83d152b7ad470cb7e6dc01063da79250d71e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyYGj&^g1ob@95*pY+^0H=Su~y1~G} za1Mw++Olyb=a=S{=#^BIFijMHvvzKnnEIOFX9gz}^Zqk%q$HLk>Lq37{ZYO&OJI4M zjCAP3lOg(izXaSEH$*n5CK&i9%nMq!Y`X9HpGOaI>M?xC%Pq<_-oU(R)d$Ag!s`2H zEpFbZu;!i*0UP(OA*i^mDlnNGyw=9*XUwBkGJZ(#GEfAi{z*~@5 zRFYW&bhjQ*5z9{=eeu7Cnoj8~>F6<*FiB_NE-1>(O)RPe>E?-ZOk)-2I#_?b=Uju7Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$b_qY0(p<%vUw$Fz9=x*t@6 z+W-In(EtDd@jwO%V{dhCbS`vwbOp+-iJ0^6C!0o57CJpSfd~KyWMOn+E@EkJ)Phz1 z%k1aRq;w4zWD2{8chZf$aFGnfz4lqGhN z^F@@yAM#diPMbsk6>Vv5ZDDv{Wp-t5bYEm)bY(7SWOxKobNpT`{e-ssQD+6%72KTw z4RB#{bZK;HZ*DGRVRQuf3@;b`!h*^yq=<|&7&K=93vhC2ZDDeGE@@u7Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$aUySV8>?*YX%1ymm|vajoc z+W-In(EtDd@jwO%V{dhCbS`vwbOk!)l;t<$46HHJYX_ISYDxeIWMOn+E@EkJvza1} zQyBilEl5*9x)JNQ3ivDm(!!zvxK+z2v32q!r+DVSDoo(32{8chZf$aFGvEe@okB*~ zbTR%3Y-@*T4j5|y6>Vv5ZDDv{Wp-t5bYEm)bY(7SWOxM38o*8P7?Ftp8RM@9Yg5Jm z4RB#{bZK;HZ*DGRVRQtc!vQI$R&WT?TvUx9wUO=s3vhC2ZDDeGE@@ diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/checksums.txt deleted file mode 100644 index 7c01d242d7ed0b91099869d3ec8d9393edff1e66..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$b;Du41;5Zw_V+h#FX$e?_J z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOOj1`)FV*dCjdq%k=Y^lq&!TWMOn+E@EkJ%pskK z4+T*^fU(%s4!?yZDEI--N?tRe=#`t*XPo$DrKeG@MhPwe@NR8#YcsF|50od+F4&OI z%2q0?M&%lG02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=%oGj0*JiNNvOS&DmLLRW01a?q za&&2QX>V>WWMOm!p~C?wr&e$X(p*%HAhnV101I$(X>DP0c`j*WcntT(8?D{|1xXDg K$kqa`ii-e@x@&s? diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt deleted file mode 100644 index 37cb59062b4..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/columns.txt +++ /dev/null @@ -1,15 +0,0 @@ -columns format version: 1 -13 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`microseconds` UInt32 -`thread_name` LowCardinality(String) -`thread_id` UInt64 -`level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8) -`query_id` String -`logger_name` LowCardinality(String) -`message` String -`revision` UInt32 -`source_file` LowCardinality(String) -`source_line` UInt64 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt deleted file mode 100644 index b2412e34dff..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -62 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx deleted file mode 100644 index 76b5229cbee..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ÙHÙH \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/partition.dat b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/partition.dat deleted file mode 100644 index a713aecd075daae7071deceaa93147871ef28fbf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRFWo7^X0v!O3 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx deleted file mode 100644 index e19de0711bb..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202101_3537_3539_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ÙHÁì `ÙHÑì ` \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/checksums.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_3540_4637_415/checksums.txt deleted file mode 100644 index 6ff207deeb3ac1e0093ae21d37c1b81c3a9d2d5b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 263 zcmV+i0r>u7Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$cnp1>605;0weWScx&VIRYS z+W-In(EtDd@jwO%V{dhCbS`vwbOlE0NoHh|IjQ9=nBu&yL8kx+WMOn+E@EkJ-sK!V z{>Uu-LH|6^^@SRDj@mx~`{{rIlayO075KqB0u)m?sefCr2{8chZf$aFGvEeBYTdjB z7Ei**<+~!{Vky7?6>Vv5ZDDv{Wp-t5bYEm)bY(7SWOxJvIhh$>@v-l*wq3;?bvV`l z4RB#{bZK;HZ*DGRVRQsR8D@EcQv3$#iBAM}yn N|2B1HzutRv6aZJdZV>iAT$b7Fo=lt*g5R6KtA;xU-PVj z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOH&t)oZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnlzQ8dORAdLOmq K{L?d@g`fZn8gaG& diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt deleted file mode 100644 index 37cb59062b4..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/columns.txt +++ /dev/null @@ -1,15 +0,0 @@ -columns format version: 1 -13 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`microseconds` UInt32 -`thread_name` LowCardinality(String) -`thread_id` UInt64 -`level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8) -`query_id` String -`logger_name` LowCardinality(String) -`message` String -`revision` UInt32 -`source_file` LowCardinality(String) -`source_line` UInt64 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt deleted file mode 100644 index dce6588ca14..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/partition.dat b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx deleted file mode 100644 index 3b0e472e2cc..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/202102_4638_4638_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëH²©"`ëH¹©"` \ No newline at end of file diff --git a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt b/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql deleted file mode 100644 index 43ff67fb63d..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/asynchronous_metric_log.sql +++ /dev/null @@ -1,12 +0,0 @@ -ATTACH TABLE _ UUID 'e6e8fcae-2f46-4616-a056-74fe935c46b1' -( - `event_date` Date, - `event_time` DateTime, - `event_time_microseconds` DateTime64(6), - `name` String, - `value` Float64 -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(event_date) -ORDER BY (event_date, event_time) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql deleted file mode 100644 index 6cb4dcdc376..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/crash_log.sql +++ /dev/null @@ -1,17 +0,0 @@ -ATTACH TABLE _ UUID '5ec3f88a-cd1b-467a-9769-7ff65a09c665' -( - `event_date` Date, - `event_time` DateTime, - `timestamp_ns` UInt64, - `signal` Int32, - `thread_id` UInt64, - `query_id` String, - `trace` Array(UInt64), - `trace_full` Array(String), - `version` String, - `revision` UInt32, - `build_id` String -) -ENGINE = MergeTree -ORDER BY (event_date, event_time) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql deleted file mode 100644 index fbdbe3c0592..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/metric_log.sql +++ /dev/null @@ -1,264 +0,0 @@ -ATTACH TABLE _ UUID '057dfcea-531d-4525-a19a-6720aa9ea4b0' -( - `event_date` Date, - `event_time` DateTime, - `event_time_microseconds` DateTime64(6), - `milliseconds` UInt64, - `ProfileEvent_Query` UInt64, - `ProfileEvent_SelectQuery` UInt64, - `ProfileEvent_InsertQuery` UInt64, - `ProfileEvent_FailedQuery` UInt64, - `ProfileEvent_FailedSelectQuery` UInt64, - `ProfileEvent_FailedInsertQuery` UInt64, - `ProfileEvent_QueryTimeMicroseconds` UInt64, - `ProfileEvent_SelectQueryTimeMicroseconds` UInt64, - `ProfileEvent_InsertQueryTimeMicroseconds` UInt64, - `ProfileEvent_FileOpen` UInt64, - `ProfileEvent_Seek` UInt64, - `ProfileEvent_ReadBufferFromFileDescriptorRead` UInt64, - `ProfileEvent_ReadBufferFromFileDescriptorReadFailed` UInt64, - `ProfileEvent_ReadBufferFromFileDescriptorReadBytes` UInt64, - `ProfileEvent_WriteBufferFromFileDescriptorWrite` UInt64, - `ProfileEvent_WriteBufferFromFileDescriptorWriteFailed` UInt64, - `ProfileEvent_WriteBufferFromFileDescriptorWriteBytes` UInt64, - `ProfileEvent_ReadBufferAIORead` UInt64, - `ProfileEvent_ReadBufferAIOReadBytes` UInt64, - `ProfileEvent_WriteBufferAIOWrite` UInt64, - `ProfileEvent_WriteBufferAIOWriteBytes` UInt64, - `ProfileEvent_ReadCompressedBytes` UInt64, - `ProfileEvent_CompressedReadBufferBlocks` UInt64, - `ProfileEvent_CompressedReadBufferBytes` UInt64, - `ProfileEvent_UncompressedCacheHits` UInt64, - `ProfileEvent_UncompressedCacheMisses` UInt64, - `ProfileEvent_UncompressedCacheWeightLost` UInt64, - `ProfileEvent_IOBufferAllocs` UInt64, - `ProfileEvent_IOBufferAllocBytes` UInt64, - `ProfileEvent_ArenaAllocChunks` UInt64, - `ProfileEvent_ArenaAllocBytes` UInt64, - `ProfileEvent_FunctionExecute` UInt64, - `ProfileEvent_TableFunctionExecute` UInt64, - `ProfileEvent_MarkCacheHits` UInt64, - `ProfileEvent_MarkCacheMisses` UInt64, - `ProfileEvent_CreatedReadBufferOrdinary` UInt64, - `ProfileEvent_CreatedReadBufferAIO` UInt64, - `ProfileEvent_CreatedReadBufferAIOFailed` UInt64, - `ProfileEvent_CreatedReadBufferMMap` UInt64, - `ProfileEvent_CreatedReadBufferMMapFailed` UInt64, - `ProfileEvent_CreatedWriteBufferOrdinary` UInt64, - `ProfileEvent_CreatedWriteBufferAIO` UInt64, - `ProfileEvent_CreatedWriteBufferAIOFailed` UInt64, - `ProfileEvent_DiskReadElapsedMicroseconds` UInt64, - `ProfileEvent_DiskWriteElapsedMicroseconds` UInt64, - `ProfileEvent_NetworkReceiveElapsedMicroseconds` UInt64, - `ProfileEvent_NetworkSendElapsedMicroseconds` UInt64, - `ProfileEvent_ThrottlerSleepMicroseconds` UInt64, - `ProfileEvent_QueryMaskingRulesMatch` UInt64, - `ProfileEvent_ReplicatedPartFetches` UInt64, - `ProfileEvent_ReplicatedPartFailedFetches` UInt64, - `ProfileEvent_ObsoleteReplicatedParts` UInt64, - `ProfileEvent_ReplicatedPartMerges` UInt64, - `ProfileEvent_ReplicatedPartFetchesOfMerged` UInt64, - `ProfileEvent_ReplicatedPartMutations` UInt64, - `ProfileEvent_ReplicatedPartChecks` UInt64, - `ProfileEvent_ReplicatedPartChecksFailed` UInt64, - `ProfileEvent_ReplicatedDataLoss` UInt64, - `ProfileEvent_InsertedRows` UInt64, - `ProfileEvent_InsertedBytes` UInt64, - `ProfileEvent_DelayedInserts` UInt64, - `ProfileEvent_RejectedInserts` UInt64, - `ProfileEvent_DelayedInsertsMilliseconds` UInt64, - `ProfileEvent_DuplicatedInsertedBlocks` UInt64, - `ProfileEvent_ZooKeeperInit` UInt64, - `ProfileEvent_ZooKeeperTransactions` UInt64, - `ProfileEvent_ZooKeeperList` UInt64, - `ProfileEvent_ZooKeeperCreate` UInt64, - `ProfileEvent_ZooKeeperRemove` UInt64, - `ProfileEvent_ZooKeeperExists` UInt64, - `ProfileEvent_ZooKeeperGet` UInt64, - `ProfileEvent_ZooKeeperSet` UInt64, - `ProfileEvent_ZooKeeperMulti` UInt64, - `ProfileEvent_ZooKeeperCheck` UInt64, - `ProfileEvent_ZooKeeperClose` UInt64, - `ProfileEvent_ZooKeeperWatchResponse` UInt64, - `ProfileEvent_ZooKeeperUserExceptions` UInt64, - `ProfileEvent_ZooKeeperHardwareExceptions` UInt64, - `ProfileEvent_ZooKeeperOtherExceptions` UInt64, - `ProfileEvent_ZooKeeperWaitMicroseconds` UInt64, - `ProfileEvent_ZooKeeperBytesSent` UInt64, - `ProfileEvent_ZooKeeperBytesReceived` UInt64, - `ProfileEvent_DistributedConnectionFailTry` UInt64, - `ProfileEvent_DistributedConnectionMissingTable` UInt64, - `ProfileEvent_DistributedConnectionStaleReplica` UInt64, - `ProfileEvent_DistributedConnectionFailAtAll` UInt64, - `ProfileEvent_CompileAttempt` UInt64, - `ProfileEvent_CompileSuccess` UInt64, - `ProfileEvent_CompileFunction` UInt64, - `ProfileEvent_CompiledFunctionExecute` UInt64, - `ProfileEvent_CompileExpressionsMicroseconds` UInt64, - `ProfileEvent_CompileExpressionsBytes` UInt64, - `ProfileEvent_ExternalSortWritePart` UInt64, - `ProfileEvent_ExternalSortMerge` UInt64, - `ProfileEvent_ExternalAggregationWritePart` UInt64, - `ProfileEvent_ExternalAggregationMerge` UInt64, - `ProfileEvent_ExternalAggregationCompressedBytes` UInt64, - `ProfileEvent_ExternalAggregationUncompressedBytes` UInt64, - `ProfileEvent_SlowRead` UInt64, - `ProfileEvent_ReadBackoff` UInt64, - `ProfileEvent_ReplicaPartialShutdown` UInt64, - `ProfileEvent_SelectedParts` UInt64, - `ProfileEvent_SelectedRanges` UInt64, - `ProfileEvent_SelectedMarks` UInt64, - `ProfileEvent_SelectedRows` UInt64, - `ProfileEvent_SelectedBytes` UInt64, - `ProfileEvent_Merge` UInt64, - `ProfileEvent_MergedRows` UInt64, - `ProfileEvent_MergedUncompressedBytes` UInt64, - `ProfileEvent_MergesTimeMilliseconds` UInt64, - `ProfileEvent_MergeTreeDataWriterRows` UInt64, - `ProfileEvent_MergeTreeDataWriterUncompressedBytes` UInt64, - `ProfileEvent_MergeTreeDataWriterCompressedBytes` UInt64, - `ProfileEvent_MergeTreeDataWriterBlocks` UInt64, - `ProfileEvent_MergeTreeDataWriterBlocksAlreadySorted` UInt64, - `ProfileEvent_CannotRemoveEphemeralNode` UInt64, - `ProfileEvent_RegexpCreated` UInt64, - `ProfileEvent_ContextLock` UInt64, - `ProfileEvent_StorageBufferFlush` UInt64, - `ProfileEvent_StorageBufferErrorOnFlush` UInt64, - `ProfileEvent_StorageBufferPassedAllMinThresholds` UInt64, - `ProfileEvent_StorageBufferPassedTimeMaxThreshold` UInt64, - `ProfileEvent_StorageBufferPassedRowsMaxThreshold` UInt64, - `ProfileEvent_StorageBufferPassedBytesMaxThreshold` UInt64, - `ProfileEvent_DictCacheKeysRequested` UInt64, - `ProfileEvent_DictCacheKeysRequestedMiss` UInt64, - `ProfileEvent_DictCacheKeysRequestedFound` UInt64, - `ProfileEvent_DictCacheKeysExpired` UInt64, - `ProfileEvent_DictCacheKeysNotFound` UInt64, - `ProfileEvent_DictCacheKeysHit` UInt64, - `ProfileEvent_DictCacheRequestTimeNs` UInt64, - `ProfileEvent_DictCacheRequests` UInt64, - `ProfileEvent_DictCacheLockWriteNs` UInt64, - `ProfileEvent_DictCacheLockReadNs` UInt64, - `ProfileEvent_DistributedSyncInsertionTimeoutExceeded` UInt64, - `ProfileEvent_DataAfterMergeDiffersFromReplica` UInt64, - `ProfileEvent_DataAfterMutationDiffersFromReplica` UInt64, - `ProfileEvent_PolygonsAddedToPool` UInt64, - `ProfileEvent_PolygonsInPoolAllocatedBytes` UInt64, - `ProfileEvent_RWLockAcquiredReadLocks` UInt64, - `ProfileEvent_RWLockAcquiredWriteLocks` UInt64, - `ProfileEvent_RWLockReadersWaitMilliseconds` UInt64, - `ProfileEvent_RWLockWritersWaitMilliseconds` UInt64, - `ProfileEvent_DNSError` UInt64, - `ProfileEvent_RealTimeMicroseconds` UInt64, - `ProfileEvent_UserTimeMicroseconds` UInt64, - `ProfileEvent_SystemTimeMicroseconds` UInt64, - `ProfileEvent_SoftPageFaults` UInt64, - `ProfileEvent_HardPageFaults` UInt64, - `ProfileEvent_VoluntaryContextSwitches` UInt64, - `ProfileEvent_InvoluntaryContextSwitches` UInt64, - `ProfileEvent_OSIOWaitMicroseconds` UInt64, - `ProfileEvent_OSCPUWaitMicroseconds` UInt64, - `ProfileEvent_OSCPUVirtualTimeMicroseconds` UInt64, - `ProfileEvent_OSReadBytes` UInt64, - `ProfileEvent_OSWriteBytes` UInt64, - `ProfileEvent_OSReadChars` UInt64, - `ProfileEvent_OSWriteChars` UInt64, - `ProfileEvent_PerfCpuCycles` UInt64, - `ProfileEvent_PerfInstructions` UInt64, - `ProfileEvent_PerfCacheReferences` UInt64, - `ProfileEvent_PerfCacheMisses` UInt64, - `ProfileEvent_PerfBranchInstructions` UInt64, - `ProfileEvent_PerfBranchMisses` UInt64, - `ProfileEvent_PerfBusCycles` UInt64, - `ProfileEvent_PerfStalledCyclesFrontend` UInt64, - `ProfileEvent_PerfStalledCyclesBackend` UInt64, - `ProfileEvent_PerfRefCpuCycles` UInt64, - `ProfileEvent_PerfCpuClock` UInt64, - `ProfileEvent_PerfTaskClock` UInt64, - `ProfileEvent_PerfContextSwitches` UInt64, - `ProfileEvent_PerfCpuMigrations` UInt64, - `ProfileEvent_PerfAlignmentFaults` UInt64, - `ProfileEvent_PerfEmulationFaults` UInt64, - `ProfileEvent_PerfMinEnabledTime` UInt64, - `ProfileEvent_PerfMinEnabledRunningTime` UInt64, - `ProfileEvent_PerfDataTLBReferences` UInt64, - `ProfileEvent_PerfDataTLBMisses` UInt64, - `ProfileEvent_PerfInstructionTLBReferences` UInt64, - `ProfileEvent_PerfInstructionTLBMisses` UInt64, - `ProfileEvent_PerfLocalMemoryReferences` UInt64, - `ProfileEvent_PerfLocalMemoryMisses` UInt64, - `ProfileEvent_CreatedHTTPConnections` UInt64, - `ProfileEvent_CannotWriteToWriteBufferDiscard` UInt64, - `ProfileEvent_QueryProfilerSignalOverruns` UInt64, - `ProfileEvent_CreatedLogEntryForMerge` UInt64, - `ProfileEvent_NotCreatedLogEntryForMerge` UInt64, - `ProfileEvent_CreatedLogEntryForMutation` UInt64, - `ProfileEvent_NotCreatedLogEntryForMutation` UInt64, - `ProfileEvent_S3ReadMicroseconds` UInt64, - `ProfileEvent_S3ReadBytes` UInt64, - `ProfileEvent_S3ReadRequestsCount` UInt64, - `ProfileEvent_S3ReadRequestsErrors` UInt64, - `ProfileEvent_S3ReadRequestsThrottling` UInt64, - `ProfileEvent_S3ReadRequestsRedirects` UInt64, - `ProfileEvent_S3WriteMicroseconds` UInt64, - `ProfileEvent_S3WriteBytes` UInt64, - `ProfileEvent_S3WriteRequestsCount` UInt64, - `ProfileEvent_S3WriteRequestsErrors` UInt64, - `ProfileEvent_S3WriteRequestsThrottling` UInt64, - `ProfileEvent_S3WriteRequestsRedirects` UInt64, - `ProfileEvent_QueryMemoryLimitExceeded` UInt64, - `CurrentMetric_Query` Int64, - `CurrentMetric_Merge` Int64, - `CurrentMetric_PartMutation` Int64, - `CurrentMetric_ReplicatedFetch` Int64, - `CurrentMetric_ReplicatedSend` Int64, - `CurrentMetric_ReplicatedChecks` Int64, - `CurrentMetric_BackgroundPoolTask` Int64, - `CurrentMetric_BackgroundMovePoolTask` Int64, - `CurrentMetric_BackgroundSchedulePoolTask` Int64, - `CurrentMetric_BackgroundBufferFlushSchedulePoolTask` Int64, - `CurrentMetric_BackgroundDistributedSchedulePoolTask` Int64, - `CurrentMetric_BackgroundMessageBrokerSchedulePoolTask` Int64, - `CurrentMetric_CacheDictionaryUpdateQueueBatches` Int64, - `CurrentMetric_CacheDictionaryUpdateQueueKeys` Int64, - `CurrentMetric_DiskSpaceReservedForMerge` Int64, - `CurrentMetric_DistributedSend` Int64, - `CurrentMetric_QueryPreempted` Int64, - `CurrentMetric_TCPConnection` Int64, - `CurrentMetric_MySQLConnection` Int64, - `CurrentMetric_HTTPConnection` Int64, - `CurrentMetric_InterserverConnection` Int64, - `CurrentMetric_PostgreSQLConnection` Int64, - `CurrentMetric_OpenFileForRead` Int64, - `CurrentMetric_OpenFileForWrite` Int64, - `CurrentMetric_Read` Int64, - `CurrentMetric_Write` Int64, - `CurrentMetric_SendScalars` Int64, - `CurrentMetric_SendExternalTables` Int64, - `CurrentMetric_QueryThread` Int64, - `CurrentMetric_ReadonlyReplica` Int64, - `CurrentMetric_MemoryTracking` Int64, - `CurrentMetric_EphemeralNode` Int64, - `CurrentMetric_ZooKeeperSession` Int64, - `CurrentMetric_ZooKeeperWatch` Int64, - `CurrentMetric_ZooKeeperRequest` Int64, - `CurrentMetric_DelayedInserts` Int64, - `CurrentMetric_ContextLockWait` Int64, - `CurrentMetric_StorageBufferRows` Int64, - `CurrentMetric_StorageBufferBytes` Int64, - `CurrentMetric_DictCacheRequests` Int64, - `CurrentMetric_Revision` Int64, - `CurrentMetric_VersionInteger` Int64, - `CurrentMetric_RWLockWaitingReaders` Int64, - `CurrentMetric_RWLockWaitingWriters` Int64, - `CurrentMetric_RWLockActiveReaders` Int64, - `CurrentMetric_RWLockActiveWriters` Int64, - `CurrentMetric_GlobalThread` Int64, - `CurrentMetric_GlobalThreadActive` Int64, - `CurrentMetric_LocalThread` Int64, - `CurrentMetric_LocalThreadActive` Int64, - `CurrentMetric_DistributedFilesToInsert` Int64 -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(event_date) -ORDER BY (event_date, event_time) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql deleted file mode 100644 index ea3aabd51e4..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/part_log.sql +++ /dev/null @@ -1,25 +0,0 @@ -ATTACH TABLE _ UUID '721d6adf-4b1c-49bc-8fbe-1599ac4e048d' -( - `event_type` Enum8('NewPart' = 1, 'MergeParts' = 2, 'DownloadPart' = 3, 'RemovePart' = 4, 'MutatePart' = 5, 'MovePart' = 6), - `event_date` Date, - `event_time` DateTime, - `duration_ms` UInt64, - `database` String, - `table` String, - `part_name` String, - `partition_id` String, - `path_on_disk` String, - `rows` UInt64, - `size_in_bytes` UInt64, - `merged_from` Array(String), - `bytes_uncompressed` UInt64, - `read_rows` UInt64, - `read_bytes` UInt64, - `peak_memory_usage` UInt64, - `error` UInt16, - `exception` String -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(event_date) -ORDER BY (event_date, event_time) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql deleted file mode 100644 index bcafc4b0530..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_log.sql +++ /dev/null @@ -1,52 +0,0 @@ -ATTACH TABLE _ UUID '3f558312-ca2a-49dc-a6bf-a5daea1869df' -( - `type` Enum8('QueryStart' = 1, 'QueryFinish' = 2, 'ExceptionBeforeStart' = 3, 'ExceptionWhileProcessing' = 4), - `event_date` Date, - `event_time` DateTime, - `event_time_microseconds` DateTime64(6), - `query_start_time` DateTime, - `query_start_time_microseconds` DateTime64(6), - `query_duration_ms` UInt64, - `read_rows` UInt64, - `read_bytes` UInt64, - `written_rows` UInt64, - `written_bytes` UInt64, - `result_rows` UInt64, - `result_bytes` UInt64, - `memory_usage` UInt64, - `current_database` String, - `query` String, - `exception_code` Int32, - `exception` String, - `stack_trace` String, - `is_initial_query` UInt8, - `user` String, - `query_id` String, - `address` IPv6, - `port` UInt16, - `initial_user` String, - `initial_query_id` String, - `initial_address` IPv6, - `initial_port` UInt16, - `interface` UInt8, - `os_user` String, - `client_hostname` String, - `client_name` String, - `client_revision` UInt32, - `client_version_major` UInt32, - `client_version_minor` UInt32, - `client_version_patch` UInt32, - `http_method` UInt8, - `http_user_agent` String, - `quota_key` String, - `revision` UInt32, - `thread_ids` Array(UInt64), - `ProfileEvents.Names` Array(String), - `ProfileEvents.Values` Array(UInt64), - `Settings.Names` Array(String), - `Settings.Values` Array(String) -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(event_date) -ORDER BY (event_date, event_time) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql deleted file mode 100644 index 56ff3366702..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/query_thread_log.sql +++ /dev/null @@ -1,46 +0,0 @@ -ATTACH TABLE _ UUID '8b2738c3-4bea-49f7-b547-0a92673830cc' -( - `event_date` Date, - `event_time` DateTime, - `event_time_microseconds` DateTime64(6), - `query_start_time` DateTime, - `query_start_time_microseconds` DateTime64(6), - `query_duration_ms` UInt64, - `read_rows` UInt64, - `read_bytes` UInt64, - `written_rows` UInt64, - `written_bytes` UInt64, - `memory_usage` Int64, - `peak_memory_usage` Int64, - `thread_name` String, - `thread_id` UInt64, - `master_thread_id` UInt64, - `query` String, - `is_initial_query` UInt8, - `user` String, - `query_id` String, - `address` IPv6, - `port` UInt16, - `initial_user` String, - `initial_query_id` String, - `initial_address` IPv6, - `initial_port` UInt16, - `interface` UInt8, - `os_user` String, - `client_hostname` String, - `client_name` String, - `client_revision` UInt32, - `client_version_major` UInt32, - `client_version_minor` UInt32, - `client_version_patch` UInt32, - `http_method` UInt8, - `http_user_agent` String, - `quota_key` String, - `revision` UInt32, - `ProfileEvents.Names` Array(String), - `ProfileEvents.Values` Array(UInt64) -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(event_date) -ORDER BY (event_date, event_time) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql deleted file mode 100644 index 874466c610f..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/text_log.sql +++ /dev/null @@ -1,20 +0,0 @@ -ATTACH TABLE _ UUID 'b619f0a8-9836-4824-9594-a722b0a7d500' -( - `event_date` Date, - `event_time` DateTime, - `event_time_microseconds` DateTime64(6), - `microseconds` UInt32, - `thread_name` LowCardinality(String), - `thread_id` UInt64, - `level` Enum8('Fatal' = 1, 'Critical' = 2, 'Error' = 3, 'Warning' = 4, 'Notice' = 5, 'Information' = 6, 'Debug' = 7, 'Trace' = 8), - `query_id` String, - `logger_name` LowCardinality(String), - `message` String, - `revision` UInt32, - `source_file` LowCardinality(String), - `source_line` UInt64 -) -ENGINE = MergeTree -PARTITION BY toYYYYMM(event_date) -ORDER BY (event_date, event_time) -SETTINGS index_granularity = 8192 diff --git a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql b/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql deleted file mode 100644 index 04517208a2e..00000000000 --- a/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681/trace_log.sql +++ /dev/null @@ -1,17 +0,0 @@ -ATTACH TABLE _ UUID '72dc35e3-6e64-44bf-af3f-4d66fb70834a' -( - `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 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/checksums.txt deleted file mode 100644 index 15f590ec5a9f148c51c40e5f8643b93ed6312e48..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bUKSS==f{FYCNyEFtWbCto z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOT5}}ui?1Ks zvg>S#d$7rg02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=Yhv{9L4)i+mY_Q2(jfMG01a?q za&&2QX>V>WWMOm!_Mvet<#01I$(X>DP0c`j*Wcnm83Pgz_C>>6$e Kar3Qw+64edhHlyb diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt deleted file mode 100644 index 5bfe4600a4e..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/columns.txt +++ /dev/null @@ -1,7 +0,0 @@ -columns format version: 1 -5 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`name` String -`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt deleted file mode 100644 index afbe847262c..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/count.txt +++ /dev/null @@ -1 +0,0 @@ -126 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx deleted file mode 100644 index 4b6c49f8d50..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -…H…H \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/partition.dat deleted file mode 100644 index b9ea5569e3a3b34e3a1b6a1c4cbcb46f5e2d3fac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0VWo7^X0I>iw diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx deleted file mode 100644 index 38e2c642aeb..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202010_1_3_1/primary.idx +++ /dev/null @@ -1 +0,0 @@ -…HÖ-œ_…HN.œ_ \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/checksums.txt deleted file mode 100644 index 288379f59b4d0506587d2b915da29b5429f3c32c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyVUf?3K)m;9pkOXWjEYr54|4y2Zf2 zZ~=%v+Ou&c=a=S{=#^BIupV9eg5{F*3qhl-g^$@cNz7y5NJ%V7)Jw|DJHCbKd3ghe zFgEolo@6;#*(z-(X1n4kgRx0RBc}nwhrHaPY~v25bs?^=WxQQ@ znPqo7Se%>sj6pOvGcPx>B0jY&HLoNdXmF}tW=aK1VDXg_AS<=yCEMbz)z+F(3nVVQt3DPa}a>cj&7s0hFg}*8) KC3BoV%>V$TlWN`o diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt deleted file mode 100644 index 5bfe4600a4e..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/columns.txt +++ /dev/null @@ -1,7 +0,0 @@ -columns format version: 1 -5 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`name` String -`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt deleted file mode 100644 index 23644ad9685..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/count.txt +++ /dev/null @@ -1 +0,0 @@ -12390 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx deleted file mode 100644 index 07abc658bdb..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -¾HÁH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/partition.dat deleted file mode 100644 index 7af4421fcb5ebc50aa06a5f4dee6e454020552a6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 Lcmb0UWo7^X0J#7) diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx deleted file mode 100644 index d57044faf98..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202012_4_295_59/primary.idx +++ /dev/null @@ -1 +0,0 @@ -¾H>˜ç_¿HÇç_ÁH’}ê_ \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/checksums.txt deleted file mode 100644 index d702cbf79d2a5a91c07163c9680526ae0c503ceb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$bUW>V4jhooyz_~f{Za(pN8Q&6FaYpwZE|ZfumSZKokr7~ z|403vu$n{)-iQDdZE0?8VR&C@6CZZ2eDbOfQp0V$_ea0t>|RE;3Dk?sHsaB^vFVRCscX=Hc|m=Z;zemg0s MudjjE3#Lvc0M|-wEC2ui diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt deleted file mode 100644 index 5bfe4600a4e..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/columns.txt +++ /dev/null @@ -1,7 +0,0 @@ -columns format version: 1 -5 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`name` String -`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt deleted file mode 100644 index 721ca6f552a..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/count.txt +++ /dev/null @@ -1 +0,0 @@ -6048 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx deleted file mode 100644 index 4c89b29a856..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ËHÙH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/partition.dat deleted file mode 100644 index a713aecd075daae7071deceaa93147871ef28fbf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRFWo7^X0v!O3 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx deleted file mode 100644 index 8e5b4736172..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202101_296_439_30/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ËHÖ÷_ÙHšì ` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_440_574_27/checksums.txt deleted file mode 100644 index 318b8a518b960630dc1e834f27ce63e27b6593ab..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 262 zcmV+h0r~!8Xk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$c7oqGLyZE0?8VR&C@6CZZ2eDbOb>eW_f~A{08aeSmV>iAT$cC2%NmSkKqG{?=Jq@?bnHd z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOJne3tc?*x$p?c>F_{T(_H`vWMOn+E@EkJfCsQb zD@h%ngTaZ|pyvL@;Y0zh4$4=AdlcKPVg5TpMxB@3V+k$*@NR8#YcsF`21}@_f-!EI z40m1$`G2OI02OU%Zf#+BUuAY>ZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*Wcnrlp=b;Ym@z8Dp Kp(2L^K0p9kBW<++ diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt deleted file mode 100644 index 5bfe4600a4e..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/columns.txt +++ /dev/null @@ -1,7 +0,0 @@ -columns format version: 1 -5 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`name` String -`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt deleted file mode 100644 index f70d7bba4ae..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -42 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx deleted file mode 100644 index 47460b95be9..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_575_575_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëHª¥"`ëHª¥"` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/checksums.txt deleted file mode 100644 index 705c91fa699e37af9a5bc17585f859123272e27b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmYe!NKMWzF3l}gNXswEO)OC;OD!tS%+Iq@FyZ=8e&PS-lZH>%6zMI~Ocl~>y1~G} za1Mw++Olyb=a=S{=#^BIFxixH$J%_|`GMoa%MT9Wmtz??QW8rN^^!94dfBVyP4w8E z^d;5x=sNDHm&BhluH*BS+J9T=m`+Tsp_!zbhWTwyJ%$f?xkcH=8yJhzv;sK_cCKB% z@k(c3X6Zc!(cH|u+{B9b)UwpPl6au8sd||y6)cGd*%s{FoVmzWz|{EB#4Kk9-h#xU zlFSmIyY+yISR5tODw~4-u)Ta5@gie?>vRV0f}+ga#G*=&Zk}DNanDuX@+;a)C2mzy I?byly0Nr6}0ssI2 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt deleted file mode 100644 index 5bfe4600a4e..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/columns.txt +++ /dev/null @@ -1,7 +0,0 @@ -columns format version: 1 -5 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`name` String -`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt deleted file mode 100644 index f70d7bba4ae..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -42 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx deleted file mode 100644 index dc2b3df8c14..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_576_576_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëHæ¥"`ëHæ¥"` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/checksums.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/checksums.txt deleted file mode 100644 index a79136dea2ae4694404058e307f30c58f2c95216..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 260 zcmV+f0sH=AXk}w-b9HTVAZBlJZDDjEc4cyNX>V>iAT$b+KNoPO=+Yv2+Llc&Eg)Ee z*Z=?k&Hw-a@jV6!V{dhCbS`vwbOJne3tc?*x$p?c>F_{T(_H`vWMOn+E@EkJk_VW( zJ28oFC&TW$V{^28W&8oI4&bJ;l7ZNHLvKrW15Q;okqIsU@NR8#YcsF`wn)byrXbCF zZggK{VRU6KX=Hc=VZjEVxwC1aJrFfB;*n}Y01a?q za&&2QX>V>WWMOm!K^bOwf>Qhj>E&4HXupM*01I$(X>DP0c`j*WcnoZhRO7$)udRQM Kfs2=!gM9!_8*V`W diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt deleted file mode 100644 index 5bfe4600a4e..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/columns.txt +++ /dev/null @@ -1,7 +0,0 @@ -columns format version: 1 -5 columns: -`event_date` Date -`event_time` DateTime -`event_time_microseconds` DateTime64(6) -`name` String -`value` Float64 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt deleted file mode 100644 index f70d7bba4ae..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/count.txt +++ /dev/null @@ -1 +0,0 @@ -42 \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt deleted file mode 100644 index 061d1280b89..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/default_compression_codec.txt +++ /dev/null @@ -1 +0,0 @@ -CODEC(LZ4) \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx deleted file mode 100644 index 16cc6680505..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/minmax_event_date.idx +++ /dev/null @@ -1 +0,0 @@ -ëHëH \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/partition.dat b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/partition.dat deleted file mode 100644 index da540c2d1a825be3ef80d13baa834834b2248491..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 4 LcmXRDWo7^X0wDm8 diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx deleted file mode 100644 index bd60e38174b..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/202102_577_577_0/primary.idx +++ /dev/null @@ -1 +0,0 @@ -ëH"¦"`ëH"¦"` \ No newline at end of file diff --git a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt b/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt deleted file mode 100644 index 56a6051ca2b..00000000000 --- a/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1/format_version.txt +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file From 4510911881b696d7c02cec88516b5d68845bbbd5 Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 21:51:41 +0300 Subject: [PATCH 074/358] remove another unnessesary files --- programs/server/data/system/asynchronous_metric_log | 1 - programs/server/data/system/crash_log | 1 - programs/server/data/system/metric_log | 1 - programs/server/data/system/part_log | 1 - programs/server/data/system/query_log | 1 - programs/server/data/system/query_thread_log | 1 - programs/server/data/system/text_log | 1 - programs/server/data/system/trace_log | 1 - programs/server/data/test/hits | 1 - programs/server/data/test/visits | 1 - programs/server/metadata/system | 1 - programs/server/metadata/test | 1 - 12 files changed, 12 deletions(-) delete mode 120000 programs/server/data/system/asynchronous_metric_log delete mode 120000 programs/server/data/system/crash_log delete mode 120000 programs/server/data/system/metric_log delete mode 120000 programs/server/data/system/part_log delete mode 120000 programs/server/data/system/query_log delete mode 120000 programs/server/data/system/query_thread_log delete mode 120000 programs/server/data/system/text_log delete mode 120000 programs/server/data/system/trace_log delete mode 120000 programs/server/data/test/hits delete mode 120000 programs/server/data/test/visits delete mode 120000 programs/server/metadata/system delete mode 120000 programs/server/metadata/test diff --git a/programs/server/data/system/asynchronous_metric_log b/programs/server/data/system/asynchronous_metric_log deleted file mode 120000 index e065d7bf424..00000000000 --- a/programs/server/data/system/asynchronous_metric_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/e6e/e6e8fcae-2f46-4616-a056-74fe935c46b1 \ No newline at end of file diff --git a/programs/server/data/system/crash_log b/programs/server/data/system/crash_log deleted file mode 120000 index 7c05d325fb8..00000000000 --- a/programs/server/data/system/crash_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/5ec/5ec3f88a-cd1b-467a-9769-7ff65a09c665 \ No newline at end of file diff --git a/programs/server/data/system/metric_log b/programs/server/data/system/metric_log deleted file mode 120000 index cf9f177d49f..00000000000 --- a/programs/server/data/system/metric_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/057/057dfcea-531d-4525-a19a-6720aa9ea4b0 \ No newline at end of file diff --git a/programs/server/data/system/part_log b/programs/server/data/system/part_log deleted file mode 120000 index b19efce9efd..00000000000 --- a/programs/server/data/system/part_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/721/721d6adf-4b1c-49bc-8fbe-1599ac4e048d \ No newline at end of file diff --git a/programs/server/data/system/query_log b/programs/server/data/system/query_log deleted file mode 120000 index 4222415ce96..00000000000 --- a/programs/server/data/system/query_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/3f5/3f558312-ca2a-49dc-a6bf-a5daea1869df \ No newline at end of file diff --git a/programs/server/data/system/query_thread_log b/programs/server/data/system/query_thread_log deleted file mode 120000 index 6108d6eab87..00000000000 --- a/programs/server/data/system/query_thread_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/8b2/8b2738c3-4bea-49f7-b547-0a92673830cc \ No newline at end of file diff --git a/programs/server/data/system/text_log b/programs/server/data/system/text_log deleted file mode 120000 index 4ae519959fe..00000000000 --- a/programs/server/data/system/text_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/b61/b619f0a8-9836-4824-9594-a722b0a7d500 \ No newline at end of file diff --git a/programs/server/data/system/trace_log b/programs/server/data/system/trace_log deleted file mode 120000 index 6dfdc4802b3..00000000000 --- a/programs/server/data/system/trace_log +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/72d/72dc35e3-6e64-44bf-af3f-4d66fb70834a \ No newline at end of file diff --git a/programs/server/data/test/hits b/programs/server/data/test/hits deleted file mode 120000 index 28b16b0be67..00000000000 --- a/programs/server/data/test/hits +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/726/72667623-6371-4258-be57-799522ee4e64 \ No newline at end of file diff --git a/programs/server/data/test/visits b/programs/server/data/test/visits deleted file mode 120000 index 49e67e1c13b..00000000000 --- a/programs/server/data/test/visits +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/044/044d7000-c91c-482d-9f0f-719f226eff0f \ No newline at end of file diff --git a/programs/server/metadata/system b/programs/server/metadata/system deleted file mode 120000 index 4ff5d3274d6..00000000000 --- a/programs/server/metadata/system +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/dfe/dfe46a8e-9eec-4e9d-955b-685b1278c681 \ No newline at end of file diff --git a/programs/server/metadata/test b/programs/server/metadata/test deleted file mode 120000 index b4ed8633adf..00000000000 --- a/programs/server/metadata/test +++ /dev/null @@ -1 +0,0 @@ -/home/max/github/ClickHouse/programs/server/store/2c6/2c6da17b-121a-4775-b287-b0d80cd04080 \ No newline at end of file From b693652f22b000d62d6c390924dc3c8680905e5e Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 23:41:42 +0300 Subject: [PATCH 075/358] revert changes made to cube transform --- src/Processors/Transforms/CubeTransform.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/Processors/Transforms/CubeTransform.cpp b/src/Processors/Transforms/CubeTransform.cpp index 13e4963573c..c64f39af5de 100644 --- a/src/Processors/Transforms/CubeTransform.cpp +++ b/src/Processors/Transforms/CubeTransform.cpp @@ -45,7 +45,7 @@ Chunk CubeTransform::generate() consumed_chunks.clear(); auto num_rows = cube_chunk.getNumRows(); - mask = (UInt64(1) << keys.size()); + mask = (UInt64(1) << keys.size()) - 1; current_columns = cube_chunk.getColumns(); current_zero_columns.clear(); @@ -55,11 +55,11 @@ Chunk CubeTransform::generate() current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows)); } - // auto gen_chunk = std::move(cube_chunk); + auto gen_chunk = std::move(cube_chunk); - if (mask > 1) + if (mask) { - mask = mask >> 1; + --mask; auto columns = current_columns; auto size = keys.size(); @@ -72,7 +72,6 @@ Chunk CubeTransform::generate() chunks.emplace_back(std::move(columns), current_columns.front()->size()); cube_chunk = merge(std::move(chunks), false); } - auto gen_chunk = std::move(cube_chunk); finalizeChunk(gen_chunk); return gen_chunk; From 66e74a89074af28f851c5c67e93c26b98577ba46 Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 23:43:03 +0300 Subject: [PATCH 076/358] fix typo --- src/Parsers/ParserSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index f77fff658d6..6c2889d2768 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -200,7 +200,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - /// WITH ROLLUP, CUBE, GROU PING SETS or TOTALS + /// WITH ROLLUP, CUBE, GROUPING SETS or TOTALS if (s_with.ignore(pos, expected)) { if (s_rollup.ignore(pos, expected)) From b5bca809e1596778e62a5a28c65d0549277f8c08 Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 23:44:56 +0300 Subject: [PATCH 077/358] erase blank line to restore initial state --- src/Processors/Transforms/AggregatingTransform.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index c9d516f8a11..2a515fdf3be 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -15,7 +15,6 @@ public: AggregatedArenasChunkInfo(Arenas arenas_) : arenas(std::move(arenas_)) {} - }; class AggregatedChunkInfo : public ChunkInfo From 391a6019d17d4f4388991bcac007608514488bc4 Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 23:50:35 +0300 Subject: [PATCH 078/358] fixes --- src/Parsers/ParserSelectQuery.cpp | 2 ++ src/Processors/Transforms/AggregatingTransform.h | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 6c2889d2768..6a17993a704 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -207,6 +207,8 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->group_by_with_rollup = true; else if (s_cube.ignore(pos, expected)) select_query->group_by_with_cube = true; + else if (s_grouping_sets.ignore(pos, expected)) + select_query->group_by_with_grouping_sets = true; else if (s_totals.ignore(pos, expected)) select_query->group_by_with_totals = true; else diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 2a515fdf3be..4a55e8b9b5f 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -14,7 +14,7 @@ public: Arenas arenas; AggregatedArenasChunkInfo(Arenas arenas_) : arenas(std::move(arenas_)) - {} + {} }; class AggregatedChunkInfo : public ChunkInfo From ff0bc94055fabca23d4ad474ae22a9c157dd8d7b Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Tue, 9 Feb 2021 23:52:12 +0300 Subject: [PATCH 079/358] fix typos --- src/Processors/Transforms/AggregatingTransform.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 4a55e8b9b5f..2a515fdf3be 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -14,7 +14,7 @@ public: Arenas arenas; AggregatedArenasChunkInfo(Arenas arenas_) : arenas(std::move(arenas_)) - {} + {} }; class AggregatedChunkInfo : public ChunkInfo From f05aaa465143725160c5e6545eb94304fdb5ceae Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Thu, 25 Nov 2021 14:28:39 +0100 Subject: [PATCH 080/358] Add test case for base64Encode returning corrupted data Sometimes, base64Encode returns data that belongs to the next string in the buffer. This test, when failing, shows something like that: -SELECT * FROM tabl_1 SETTINGS log_comment = ?; -SELECT * FROM tabl_2 SETTINGS log_comment = ?; +SELECT * FROM tabl_1 SETTINGS log_comment = ?;\0S +SELECT * FROM tabl_2 SETTINGS log_comment = ?;\0c Where the S at the end of the first string is the first S of the next string in the buffer. I don't know exactly why, but reading from query_log is how I manage to reliably trigger the bug. The same code on literals, or even on a different table, did not trigger the bug, something related to alignment maybe. --- .../02113_base64encode_trailing_bytes.reference | 2 ++ .../02113_base64encode_trailing_bytes.sql | 13 +++++++++++++ 2 files changed, 15 insertions(+) create mode 100644 tests/queries/0_stateless/02113_base64encode_trailing_bytes.reference create mode 100644 tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql diff --git a/tests/queries/0_stateless/02113_base64encode_trailing_bytes.reference b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.reference new file mode 100644 index 00000000000..61911f9540a --- /dev/null +++ b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.reference @@ -0,0 +1,2 @@ +SELECT * FROM tabl_1 SETTINGS log_comment = ?; +SELECT * FROM tabl_2 SETTINGS log_comment = ?; diff --git a/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql new file mode 100644 index 00000000000..11582242a13 --- /dev/null +++ b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql @@ -0,0 +1,13 @@ +SET log_queries=1; + +CREATE TABLE tabl_1 (key String) ENGINE MergeTree ORDER BY key; +CREATE TABLE tabl_2 (key String) ENGINE MergeTree ORDER BY key; +SELECT * FROM tabl_1 SETTINGS log_comment = 'ad15a651'; +SELECT * FROM tabl_2 SETTINGS log_comment = 'ad15a651'; +SYSTEM FLUSH LOGS; + +SELECT base64Decode(base64Encode(normalizeQuery(query))) + FROM system.query_log + WHERE type='QueryFinish' AND log_comment = 'ad15a651' + GROUP BY normalizeQuery(query) + ORDER BY normalizeQuery(query); From 8356d44c7097f68235678c713b13ec20ebc5d79d Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Thu, 25 Nov 2021 14:32:06 +0100 Subject: [PATCH 081/358] Fix base64Encode returning corrupted data The "fast" mode of turbo base64 seem to go too fast on small strings, disabling the AVX2 optimisation for small string fixes the problem. See the test named 02113_base64encode_trailing_bytes.sql for an example of a failing case (only applies when running on a CPU with the AVX2 instruction set). Ref. https://github.com/powturbo/Turbo-Base64/blob/15e044269d747bb30c1692cec921ce6ca2128720/turbob64.h#L106-L111 --- src/Functions/base64Decode.cpp | 2 +- src/Functions/base64Encode.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/base64Decode.cpp b/src/Functions/base64Decode.cpp index 66e0969783e..027fef73911 100644 --- a/src/Functions/base64Decode.cpp +++ b/src/Functions/base64Decode.cpp @@ -8,7 +8,7 @@ namespace DB { void registerFunctionBase64Decode(FunctionFactory & factory) { - tb64ini(0, 1); + tb64ini(0, 0); factory.registerFunction>(); /// MysQL compatibility alias. diff --git a/src/Functions/base64Encode.cpp b/src/Functions/base64Encode.cpp index 0c038eb5fca..63e57434ac6 100644 --- a/src/Functions/base64Encode.cpp +++ b/src/Functions/base64Encode.cpp @@ -12,7 +12,7 @@ namespace DB { void registerFunctionBase64Encode(FunctionFactory & factory) { - tb64ini(0, 1); + tb64ini(0, 0); factory.registerFunction>(); /// MysQL compatibility alias. From 2bd07ef338adc2663bc1239976fe68ada5649b0e Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Thu, 13 May 2021 17:51:07 +0300 Subject: [PATCH 082/358] feat grouping-sets: initial changes --- src/Core/ColumnNumbers.h | 1 + src/Core/NamesAndTypes.h | 2 + src/Interpreters/Aggregator.h | 22 ++++ src/Interpreters/ExpressionAnalyzer.cpp | 11 ++ src/Interpreters/ExpressionAnalyzer.h | 4 + src/Interpreters/InterpreterSelectQuery.cpp | 106 +++++++++++++++++- src/Interpreters/InterpreterSelectQuery.h | 3 +- .../Transforms/GroupingSetsTransform.cpp | 9 +- .../Transforms/GroupingSetsTransform.h | 2 + .../Transforms/TotalsHavingTransform.cpp | 10 ++ .../Transforms/TotalsHavingTransform.h | 4 + 11 files changed, 167 insertions(+), 7 deletions(-) diff --git a/src/Core/ColumnNumbers.h b/src/Core/ColumnNumbers.h index 9441f6485a7..82868ef3cd3 100644 --- a/src/Core/ColumnNumbers.h +++ b/src/Core/ColumnNumbers.h @@ -8,5 +8,6 @@ namespace DB { using ColumnNumbers = std::vector; +using ColumnNumbersTwoDimension = std::vector; } diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 58b5189db63..787dd629190 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -107,6 +107,8 @@ public: std::optional tryGetByName(const std::string & name) const; }; +using TwoDimensionNamesAndTypesList = std::list; + } namespace std diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 3c53769e128..91eca14d483 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -879,6 +879,7 @@ public: /// What to count. const ColumnNumbers keys; + const ColumnNumbersTwoDimension keys_vector; const AggregateDescriptions aggregates; const size_t keys_size; const size_t aggregates_size; @@ -938,6 +939,27 @@ public: { } + /// two dimensional vector of aggregating keys in params + Params( + const Block & src_header_, + const ColumnNumbersTwoDimension & keys_vector_, const AggregateDescriptions & aggregates_, + bool overflow_row_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, + size_t group_by_two_level_threshold_, size_t group_by_two_level_threshold_bytes_, + size_t max_bytes_before_external_group_by_, + bool empty_result_for_aggregation_by_empty_set_, + VolumePtr tmp_volume_, size_t max_threads_, + size_t min_free_disk_space_) + : src_header(src_header_), + keys_vector(keys_vector_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), + overflow_row(overflow_row_), max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_), + group_by_two_level_threshold(group_by_two_level_threshold_), group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_), + max_bytes_before_external_group_by(max_bytes_before_external_group_by_), + empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_), + tmp_volume(tmp_volume_), max_threads(max_threads_), + min_free_disk_space(min_free_disk_space_) + { + } + /// Only parameters that matter during merge. Params(const Block & intermediate_header_, const ColumnNumbers & keys_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_threads_) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d40a92972b2..c09bf92a7da 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -345,6 +345,12 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) if (getContext()->getSettingsRef().enable_positional_arguments) replaceForPositionalArguments(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY); + if (select_query->group_by_with_grouping_sets) + { + LOG_DEBUG(poco_log, "analyzeAggregation: detect group by with grouping sets"); + /// TODO + } + getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); const auto & column_name = group_asts[i]->getColumnName(); @@ -380,6 +386,11 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) /// Aggregation keys are uniqued. if (!unique_keys.count(key.name)) { + if (select_query->group_by_with_grouping_sets) + { + aggregation_keys_list.push_back({key}); + } + unique_keys.insert(key.name); aggregation_keys.push_back(key); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index b6bb3c5fad5..7598efa40c6 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -65,6 +65,7 @@ struct ExpressionAnalyzerData bool has_aggregation = false; NamesAndTypesList aggregation_keys; bool has_const_aggregation_keys = false; + TwoDimensionNamesAndTypesList aggregation_keys_list; AggregateDescriptions aggregate_descriptions; WindowDescriptions window_descriptions; @@ -94,6 +95,8 @@ private: explicit ExtractedSettings(const Settings & settings_); }; + Poco::Logger * poco_log = &Poco::Logger::get("ExpressionAnalyzer"); + public: /// Ctor for non-select queries. Generally its usage is: /// auto actions = ExpressionAnalyzer(query, syntax, context).getActions(); @@ -321,6 +324,7 @@ public: const NamesAndTypesList & aggregationKeys() const { return aggregation_keys; } bool hasConstAggregationKeys() const { return has_const_aggregation_keys; } + const TwoDimensionNamesAndTypesList & aggregationKeysList() const { return aggregation_keys_list; } const AggregateDescriptions & aggregates() const { return aggregate_descriptions; } const PreparedSets & getPreparedSets() const { return prepared_sets; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8d824d85d37..705191e3ef8 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1182,8 +1182,18 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aggregationKeys()) + { keys.push_back(header_before_aggregation.getPositionByName(key.name)); + LOG_DEBUG(log, "executeAggregation pushed back key with name {} and number {}", key.name, header_before_aggregation.getPositionByName(key.name)); + } AggregateDescriptions aggregates = query_analyzer->aggregates(); for (auto & descr : aggregates) @@ -2203,14 +2214,18 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific query_plan.addStep(std::move(step)); } -void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan) +void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { + /* const auto & header_before_transform = query_plan.getCurrentDataStream().header; ColumnNumbers keys; for (const auto & key : query_analyzer->aggregationKeys()) + { keys.push_back(header_before_transform.getPositionByName(key.name)); + LOG_DEBUG(log, "executeGroupingSets pushed back key with name {} and number {}", key.name, header_before_transform.getPositionByName(key.name)); + } const Settings & settings = context->getSettingsRef(); @@ -2226,6 +2241,87 @@ void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan) step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); query_plan.addStep(std::move(step)); + */ + + auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), expression); + expression_before_aggregation->setStepDescription("Before GROUP BY"); + query_plan.addStep(std::move(expression_before_aggregation)); + + const auto & header_before_aggregation = query_plan.getCurrentDataStream().header; + ColumnNumbers keys; + ColumnNumbersTwoDimension keys_vector; + for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) + { + keys.clear(); + for (const auto & key : aggregation_keys) + { + keys.push_back(header_before_aggregation.getPositionByName(key.name)); + LOG_DEBUG( + log, + "executeGroupingSets add key with name {} and number {}", + key.name, + header_before_aggregation.getPositionByName(key.name)); + } + keys_vector.push_back(keys); + } + + AggregateDescriptions aggregates = query_analyzer->aggregates(); + for (auto & descr : aggregates) + if (descr.arguments.empty()) + for (const auto & name : descr.argument_names) + { + descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); + LOG_DEBUG( + log, + "executeGroupingSets add descr.atruments with name {} and number {}", + name, + header_before_aggregation.getPositionByName(name)); + } + + const Settings & settings = context->getSettingsRef(); + + Aggregator::Params params( + header_before_aggregation, + keys_vector, + aggregates, + overflow_row, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + settings.group_by_two_level_threshold, + settings.group_by_two_level_threshold_bytes, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set, + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data); + + SortDescription group_by_sort_description; + + if (group_by_info && settings.optimize_aggregation_in_order) + group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery()); + else + group_by_info = nullptr; + + auto merge_threads = max_streams; + auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads + ? static_cast(settings.aggregation_memory_efficient_merge_threads) + : static_cast(settings.max_threads); + + bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead(); + + auto aggregating_step = std::make_unique( + query_plan.getCurrentDataStream(), + params, + final, + settings.max_block_size, + settings.aggregation_in_order_max_block_bytes, + merge_threads, + temporary_data_merge_threads, + storage_has_evenly_distributed_read, + std::move(group_by_info), + std::move(group_by_sort_description)); + + query_plan.addStep(std::move(aggregating_step)); } void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 8ea04cf7b27..db77a8bd8cb 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -14,6 +14,7 @@ #include #include +#include "Interpreters/ActionsDAG.h" namespace Poco { @@ -160,7 +161,7 @@ private: }; void executeRollupOrCube(QueryPlan & query_plan, Modificator modificator); - void executeGroupingSets(QueryPlan & query_plan); + void executeGroupingSets(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. * diff --git a/src/Processors/Transforms/GroupingSetsTransform.cpp b/src/Processors/Transforms/GroupingSetsTransform.cpp index 637f244ff96..ccbafac8e3e 100644 --- a/src/Processors/Transforms/GroupingSetsTransform.cpp +++ b/src/Processors/Transforms/GroupingSetsTransform.cpp @@ -19,6 +19,7 @@ GroupingSetsTransform::GroupingSetsTransform(Block header, AggregatingTransformP Chunk GroupingSetsTransform::merge(Chunks && chunks, bool final) { + LOG_DEBUG(log, "merge {} blocks", chunks.size()); BlocksList rollup_blocks; for (auto & chunk : chunks) rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); @@ -31,12 +32,15 @@ Chunk GroupingSetsTransform::merge(Chunks && chunks, bool final) void GroupingSetsTransform::consume(Chunk chunk) { consumed_chunks.emplace_back(std::move(chunk)); + LOG_DEBUG(log, "consumed block, now consumed_chunks size is {}", consumed_chunks.size()); } Chunk GroupingSetsTransform::generate() { + LOG_DEBUG(log, "generate start, mask = {}", mask); if (!consumed_chunks.empty()) { + LOG_DEBUG(log, "consumed_chunks not empty, size is {}", consumed_chunks.size()); if (consumed_chunks.size() > 1) grouping_sets_chunk = merge(std::move(consumed_chunks), false); else @@ -46,6 +50,7 @@ Chunk GroupingSetsTransform::generate() auto num_rows = grouping_sets_chunk.getNumRows(); mask = (UInt64(1) << keys.size()); + LOG_DEBUG(log, "changed mask, mask = {}", mask); current_columns = grouping_sets_chunk.getColumns(); current_zero_columns.clear(); @@ -56,9 +61,10 @@ Chunk GroupingSetsTransform::generate() } // auto gen_chunk = std::move(cube_chunk); - + LOG_DEBUG(log, "before if mask"); if (mask > 1) { + LOG_DEBUG(log, "in if mask > 1"); mask = mask >> 1; auto columns = current_columns; @@ -72,6 +78,7 @@ Chunk GroupingSetsTransform::generate() chunks.emplace_back(std::move(columns), current_columns.front()->size()); grouping_sets_chunk = merge(std::move(chunks), false); } + LOG_DEBUG(log, "before gen_chunk"); auto gen_chunk = std::move(grouping_sets_chunk); finalizeChunk(gen_chunk); diff --git a/src/Processors/Transforms/GroupingSetsTransform.h b/src/Processors/Transforms/GroupingSetsTransform.h index 0e300453215..10108d3c1a7 100644 --- a/src/Processors/Transforms/GroupingSetsTransform.h +++ b/src/Processors/Transforms/GroupingSetsTransform.h @@ -29,6 +29,8 @@ private: UInt64 mask = 0; + Poco::Logger * log = &Poco::Logger::get("GroupingSetsTransform"); + Chunk merge(Chunks && chunks, bool final); }; diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 0b7797da24f..a636ba70ef9 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -119,6 +119,7 @@ TotalsHavingTransform::TotalsHavingTransform( IProcessor::Status TotalsHavingTransform::prepare() { + LOG_DEBUG(log, "TotalsHavingTransform::prepare()"); if (!finished_transform) { auto status = ISimpleTransform::prepare(); @@ -143,20 +144,24 @@ IProcessor::Status TotalsHavingTransform::prepare() totals_output.push(std::move(totals)); totals_output.finish(); + LOG_DEBUG(log, "exit TotalsHavingTransform::prepare()"); return Status::Finished; } void TotalsHavingTransform::work() { + LOG_DEBUG(log, "TotalsHavingTransform::work()"); if (finished_transform) prepareTotals(); else ISimpleTransform::work(); + LOG_DEBUG(log, "exit TotalsHavingTransform::work()"); } void TotalsHavingTransform::transform(Chunk & chunk) { /// Block with values not included in `max_rows_to_group_by`. We'll postpone it. + LOG_DEBUG(log, "TotalsHavingTransform::transform()"); if (overflow_row) { const auto & info = chunk.getChunkInfo(); @@ -249,10 +254,12 @@ void TotalsHavingTransform::transform(Chunk & chunk) } passed_keys += chunk.getNumRows(); + LOG_DEBUG(log, "exit TotalsHavingTransform::transform()"); } void TotalsHavingTransform::addToTotals(const Chunk & chunk, const IColumn::Filter * filter) { + LOG_DEBUG(log, "TotalsHavingTransform::addToTotals()"); auto num_columns = chunk.getNumColumns(); for (size_t col = 0; col < num_columns; ++col) { @@ -284,10 +291,12 @@ void TotalsHavingTransform::addToTotals(const Chunk & chunk, const IColumn::Filt } } } + LOG_DEBUG(log, "exit TotalsHavingTransform::addToTotals()"); } void TotalsHavingTransform::prepareTotals() { + LOG_DEBUG(log, "TotalsHavingTransform::prepareTotals()"); /// If totals_mode == AFTER_HAVING_AUTO, you need to decide whether to add aggregates to TOTALS for strings, /// not passed max_rows_to_group_by. if (overflow_aggregates) @@ -312,6 +321,7 @@ void TotalsHavingTransform::prepareTotals() /// Note: after expression totals may have several rows if `arrayJoin` was used in expression. totals = Chunk(block.getColumns(), num_rows); } + LOG_DEBUG(log, "exit TotalsHavingTransform::prepareTotals()"); } } diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index 03635054c65..59fa58edfbc 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -2,6 +2,8 @@ #include #include +#include // to be removed +// #include namespace DB { @@ -74,6 +76,8 @@ private: /// Here, total values are accumulated. After the work is finished, they will be placed in totals. MutableColumns current_totals; + + Poco::Logger * log = &Poco::Logger::get("TotalsHavingTransform"); }; void finalizeChunk(Chunk & chunk); From 3418de337a3195e52f4889fa5ab1e085818c7e1b Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Sat, 15 May 2021 15:14:58 +0300 Subject: [PATCH 083/358] development --- src/Interpreters/ExpressionAnalyzer.cpp | 30 +++++++++++++++++++------ src/Interpreters/TreeOptimizer.cpp | 9 +++++--- src/Parsers/ExpressionListParsers.cpp | 6 +++++ src/Parsers/ExpressionListParsers.h | 8 +++++++ src/Parsers/ParserSelectQuery.cpp | 14 ++++++++++-- 5 files changed, 55 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index c09bf92a7da..eb1bd592673 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -386,13 +386,16 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) /// Aggregation keys are uniqued. if (!unique_keys.count(key.name)) { + unique_keys.insert(key.name); if (select_query->group_by_with_grouping_sets) { aggregation_keys_list.push_back({key}); + LOG_DEBUG(poco_log, "pushed grouping set of 1 column: " + key.name); + } + else + { + aggregation_keys.push_back(key); } - - unique_keys.insert(key.name); - aggregation_keys.push_back(key); /// Key is no longer needed, therefore we can save a little by moving it. aggregated_columns.push_back(std::move(key)); @@ -1156,10 +1159,23 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain ExpressionActionsChain::Step & step = chain.lastStep(columns_after_join); ASTs asts = select_query->groupBy()->children; - for (const auto & ast : asts) + if (select_query->group_by_with_grouping_sets) { - step.addRequiredOutput(ast->getColumnName()); - getRootActions(ast, only_types, step.actions()); + for (const auto & inner_asts : asts) { + for (const auto & ast : inner_asts->children) + { + step.addRequiredOutput(ast->getColumnName()); + getRootActions(ast, only_types, step.actions()); + } + } + } + else + { + for (const auto & ast : asts) + { + step.addRequiredOutput(ast->getColumnName()); + getRootActions(ast, only_types, step.actions()); + } } if (optimize_aggregation_in_order) @@ -1650,7 +1666,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( /// TODO correct conditions optimize_aggregation_in_order = context->getSettingsRef().optimize_aggregation_in_order - && storage && query.groupBy(); + && storage && query.groupBy() && !query.group_by_with_grouping_sets; query_analyzer.appendGroupBy(chain, only_types || !first_stage, optimize_aggregation_in_order, group_by_elements_actions); query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 5e355cd52af..a76e05a8ad6 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -84,6 +84,9 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context) { const FunctionFactory & function_factory = FunctionFactory::instance(); + if (select_query->group_by_with_grouping_sets) + return; + if (!select_query->groupBy()) return; @@ -213,7 +216,7 @@ GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys) ///eliminate functions of other GROUP BY keys void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) { - if (!select_query->groupBy()) + if (!select_query->groupBy() || select_query->group_by_with_grouping_sets) return; auto group_by = select_query->groupBy(); @@ -243,7 +246,7 @@ void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) /// Eliminates min/max/any-aggregators of functions of GROUP BY keys void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query, ASTPtr & node) { - if (!select_query->groupBy()) + if (!select_query->groupBy() || select_query->group_by_with_grouping_sets) return; const auto & group_by_keys = select_query->groupBy()->children; @@ -413,7 +416,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context const Names & sorting_key_columns) { auto order_by = select_query->orderBy(); - if (!order_by) + if (!order_by || select_query->group_by_with_grouping_sets) return; for (const auto & child : order_by->children) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index b6e80d83777..a2be8d75e1b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -753,6 +753,12 @@ bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & .parse(pos, node, expected); } +bool ParserGroupingSetsExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList(std::make_unique(false), + std::make_unique(TokenType::Comma), false) + .parse(pos, node, expected); +} bool ParserTTLExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 17deec4e9e4..85ed9a81d83 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -511,6 +511,14 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserGroupingSetsExpressionList : public IParserBase +{ +protected: + const char * getName() const override { return "grouping sets expression"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + + /// Parser for key-value pair, where value can be list of pairs. class ParserKeyValuePair : public IParserBase diff --git a/src/Parsers/ParserSelectQuery.cpp b/src/Parsers/ParserSelectQuery.cpp index 6a17993a704..f767ee88b96 100644 --- a/src/Parsers/ParserSelectQuery.cpp +++ b/src/Parsers/ParserSelectQuery.cpp @@ -66,6 +66,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. ParserExpressionWithOptionalAlias exp_elem(false); ParserOrderByExpressionList order_list; + ParserGroupingSetsExpressionList grouping_sets_list; ParserToken open_bracket(TokenType::OpeningRoundBracket); ParserToken close_bracket(TokenType::ClosingRoundBracket); @@ -192,8 +193,17 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) !open_bracket.ignore(pos, expected)) return false; - if (!exp_list.parse(pos, group_expression_list, expected)) - return false; + if (select_query->group_by_with_grouping_sets) + { + if (!grouping_sets_list.parse(pos, group_expression_list, expected)) + return false; + } + else + { + if (!exp_list.parse(pos, group_expression_list, expected)) + return false; + } + if ((select_query->group_by_with_rollup || select_query->group_by_with_cube || select_query->group_by_with_grouping_sets) && !close_bracket.ignore(pos, expected)) From e60d1dd818c0a066bbfbe308e81260a1f5619b4a Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Sat, 15 May 2021 19:41:22 +0300 Subject: [PATCH 084/358] grouping sets development --- src/Interpreters/Aggregator.h | 10 +- src/Interpreters/InterpreterSelectQuery.cpp | 142 ++++++++++++++------ src/Interpreters/InterpreterSelectQuery.h | 1 - 3 files changed, 111 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 91eca14d483..cea420c6442 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -948,15 +948,21 @@ public: size_t max_bytes_before_external_group_by_, bool empty_result_for_aggregation_by_empty_set_, VolumePtr tmp_volume_, size_t max_threads_, - size_t min_free_disk_space_) + size_t min_free_disk_space_, + bool compile_aggregate_expressions_, + size_t min_count_to_compile_aggregate_expression_, + const Block & intermediate_header_ = {}) : src_header(src_header_), + intermediate_header(intermediate_header_), keys_vector(keys_vector_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), overflow_row(overflow_row_), max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_), group_by_two_level_threshold(group_by_two_level_threshold_), group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_), max_bytes_before_external_group_by(max_bytes_before_external_group_by_), empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_), tmp_volume(tmp_volume_), max_threads(max_threads_), - min_free_disk_space(min_free_disk_space_) + min_free_disk_space(min_free_disk_space_), + compile_aggregate_expressions(compile_aggregate_expressions_), + min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_) { } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 705191e3ef8..edf93c20fd9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1182,18 +1182,10 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aggregationKeys()) + ColumnNumbersTwoDimension keys_vector; + auto & query = getSelectQuery(); + if (query.group_by_with_grouping_sets) { - keys.push_back(header_before_aggregation.getPositionByName(key.name)); - LOG_DEBUG(log, "executeAggregation pushed back key with name {} and number {}", key.name, header_before_aggregation.getPositionByName(key.name)); + for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) + { + keys.clear(); + for (const auto & key : aggregation_keys) + { + keys.push_back(header_before_aggregation.getPositionByName(key.name)); + LOG_DEBUG( + log, + "GroupingSets add key with name {} and number {}", + key.name, + header_before_aggregation.getPositionByName(key.name)); + } + keys_vector.push_back(keys); + LOG_DEBUG( + log, + "GroupingSets add keys set of size {}", + keys.size()); + } + } + else + { + for (const auto & key : query_analyzer->aggregationKeys()) + { + keys.push_back(header_before_aggregation.getPositionByName(key.name)); + LOG_DEBUG(log, "executeAggregation pushed back key with name {} and number {}", key.name, header_before_aggregation.getPositionByName(key.name)); + } } AggregateDescriptions aggregates = query_analyzer->aggregates(); @@ -2078,29 +2096,72 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac const Settings & settings = context->getSettingsRef(); - Aggregator::Params params( - header_before_aggregation, - keys, - aggregates, - overflow_row, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - settings.group_by_two_level_threshold, - settings.group_by_two_level_threshold_bytes, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set - || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() - && query_analyzer->hasConstAggregationKeys()), - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); + // Aggregator::Params params( + // header_before_aggregation, + // keys, + // aggregates, + // overflow_row, + // settings.max_rows_to_group_by, + // settings.group_by_overflow_mode, + // settings.group_by_two_level_threshold, + // settings.group_by_two_level_threshold_bytes, + // settings.max_bytes_before_external_group_by, + // settings.empty_result_for_aggregation_by_empty_set + // || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() + // && query_analyzer->hasConstAggregationKeys()), + // context->getTemporaryVolume(), + // settings.max_threads, + // settings.min_free_disk_space_for_temporary_data, + // settings.compile_aggregate_expressions, + // settings.min_count_to_compile_aggregate_expression); + std::shared_ptr params_ptr; + if (query.group_by_with_grouping_sets) + { + params_ptr = std::make_shared( + header_before_aggregation, + keys_vector, + aggregates, + overflow_row, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + settings.group_by_two_level_threshold, + settings.group_by_two_level_threshold_bytes, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() + && query_analyzer->hasConstAggregationKeys()), + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); + } + else + { + params_ptr = std::make_shared( + header_before_aggregation, + keys, + aggregates, + overflow_row, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + settings.group_by_two_level_threshold, + settings.group_by_two_level_threshold_bytes, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() + && query_analyzer->hasConstAggregationKeys()), + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); + } SortDescription group_by_sort_description; - if (group_by_info && settings.optimize_aggregation_in_order) - group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery()); + if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) + group_by_sort_description = getSortDescriptionFromGroupBy(query); else group_by_info = nullptr; @@ -2113,7 +2174,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), - params, + *params_ptr, final, settings.max_block_size, settings.aggregation_in_order_max_block_bytes, @@ -2214,9 +2275,10 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific query_plan.addStep(std::move(step)); } -void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) +/* +void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { - /* + const auto & header_before_transform = query_plan.getCurrentDataStream().header; ColumnNumbers keys; @@ -2241,7 +2303,7 @@ void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan, const A step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); query_plan.addStep(std::move(step)); - */ + auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), expression); expression_before_aggregation->setStepDescription("Before GROUP BY"); @@ -2322,7 +2384,9 @@ void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan, const A std::move(group_by_sort_description)); query_plan.addStep(std::move(aggregating_step)); + } +*/ void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description) { diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index db77a8bd8cb..34d56a7354b 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -161,7 +161,6 @@ private: }; void executeRollupOrCube(QueryPlan & query_plan, Modificator modificator); - void executeGroupingSets(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. * From d2258decf5aa33ed7d1a3096ab682541a67e60fd Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Sat, 15 May 2021 22:58:41 +0300 Subject: [PATCH 085/358] grouping sets development --- src/Interpreters/Aggregator.h | 2 +- src/QueryPipeline/Pipe.cpp | 55 +++++++++++++++++++++- src/QueryPipeline/Pipe.h | 1 + src/QueryPipeline/QueryPipelineBuilder.cpp | 6 +++ src/QueryPipeline/QueryPipelineBuilder.h | 2 + 5 files changed, 64 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index cea420c6442..ae4986f5135 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -878,7 +878,7 @@ public: Block intermediate_header; /// What to count. - const ColumnNumbers keys; + ColumnNumbers keys; const ColumnNumbersTwoDimension keys_vector; const AggregateDescriptions aggregates; const size_t keys_size; diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 6cef7cc28bd..093491eed2a 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -432,6 +432,59 @@ void Pipe::addTransform(ProcessorPtr transform) addTransform(std::move(transform), static_cast(nullptr), static_cast(nullptr)); } +void Pipe::addParallelTransforms(Processors transforms) +{ + if (output_ports.empty()) + throw Exception("Cannot add parallel transforms to empty Pipe.", ErrorCodes::LOGICAL_ERROR); + + std::vector inputs; + std::vector outputs; + for (const auto & transform : transforms) + { + auto current_transform_inputs = transform->getInputs(); + if (current_transform_inputs.size() != 1) + throw Exception("Each parallel transform should have one input port", ErrorCodes::LOGICAL_ERROR); + + inputs.push_back(current_transform_inputs.front()); + + auto current_transform_outputs = transform->getOutputs(); + if (current_transform_outputs.size() != 1) + throw Exception("Each parallel transform should have one output port", ErrorCodes::LOGICAL_ERROR); + + outputs.push_back(current_transform_outputs.front()); + } + + if (inputs.size() != output_ports.size()) + throw Exception("Cannot add parallel transforms to Pipes because " + + std::to_string(transforms.size()) + " transforms were passed, " + "but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR); + + size_t next_output = 0; + for (auto & input : inputs) + { + connect(*output_ports[next_output], input); + ++next_output; + } + + output_ports.clear(); + output_ports.reserve(outputs.size()); + + for (auto & output : outputs) + { + output_ports.emplace_back(&output); + } + + /// do not check output formats because they are different in case of parallel aggregations + + if (collected_processors) + collected_processors->insert(collected_processors->end(), transforms.begin(), transforms.end()); + + processors.insert(processors.end(), transforms.begin(), transforms.end()); + + /// Should not change streams number, so maybe not need max_parallel_streams update + max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); +} + void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes) { if (output_ports.empty()) @@ -497,7 +550,7 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort for (size_t i = 1; i < output_ports.size(); ++i) assertBlocksHaveEqualStructure(header, output_ports[i]->getHeader(), "Pipes"); - // Temporarily skip this check. TotaslHavingTransform may return finalized totals but not finalized data. + // Temporarily skip this check. TotalsHavingTransform may return finalized totals but not finalized data. // if (totals_port) // assertBlocksHaveEqualStructure(header, totals_port->getHeader(), "Pipes"); diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 613e92a782d..e655f807ec8 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -63,6 +63,7 @@ public: /// Output ports should have same headers. /// If totals or extremes are not empty, transform shouldn't change header. void addTransform(ProcessorPtr transform); + void addParallelTransforms(Processors transform); void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes); void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 40c64046560..1377db34daf 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -155,6 +155,12 @@ void QueryPipelineBuilder::transform(const Transformer & transformer) pipe.transform(transformer); } +void QueryPipelineBuilder::addParallelTransforms(Processors transforms) +{ + checkInitializedAndNotCompleted(); + pipe.addParallelTransforms(transforms); +} + void QueryPipelineBuilder::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) { checkInitializedAndNotCompleted(); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 9e198f45e98..9428bbc5d0a 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -67,6 +67,8 @@ public: /// Transform pipeline in general way. void transform(const Transformer & transformer); + /// Add transform and connects it to outputs[stream_index] stream + void addParallelTransforms(Processors transform); /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); /// Add transform which calculates extremes. This transform adds extremes port and doesn't change inputs number. From ddd1799743bd7b2f8cc7824706423868a48782ab Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Sun, 16 May 2021 22:44:20 +0300 Subject: [PATCH 086/358] grouping sets development --- src/Interpreters/Aggregator.h | 4 +- src/Interpreters/ExpressionAnalyzer.cpp | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 45 +++++++++---------- .../Transforms/AggregatingTransform.cpp | 1 + src/QueryPipeline/Pipe.cpp | 22 ++++++++- src/QueryPipeline/Pipe.h | 3 ++ src/QueryPipeline/QueryPipelineBuilder.h | 2 +- 7 files changed, 52 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index ae4986f5135..87da0d7492d 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -881,7 +881,7 @@ public: ColumnNumbers keys; const ColumnNumbersTwoDimension keys_vector; const AggregateDescriptions aggregates; - const size_t keys_size; + size_t keys_size; const size_t aggregates_size; /// The settings of approximate calculation of GROUP BY. @@ -942,6 +942,7 @@ public: /// two dimensional vector of aggregating keys in params Params( const Block & src_header_, + const ColumnNumbers & keys_, const ColumnNumbersTwoDimension & keys_vector_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, size_t group_by_two_level_threshold_, size_t group_by_two_level_threshold_bytes_, @@ -954,6 +955,7 @@ public: const Block & intermediate_header_ = {}) : src_header(src_header_), intermediate_header(intermediate_header_), + keys(keys_), keys_vector(keys_vector_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), overflow_row(overflow_row_), max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_), group_by_two_level_threshold(group_by_two_level_threshold_), group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_), diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index eb1bd592673..8ac892d8834 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -390,6 +390,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) if (select_query->group_by_with_grouping_sets) { aggregation_keys_list.push_back({key}); + aggregation_keys.push_back(key); LOG_DEBUG(poco_log, "pushed grouping set of 1 column: " + key.name); } else diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index edf93c20fd9..f69e52364fc 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1049,11 +1049,13 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aggregates(); for (auto & descr : aggregates) if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); - + LOG_DEBUG(log, "GroupingSets debug 2"); const Settings & settings = context->getSettingsRef(); - - // Aggregator::Params params( - // header_before_aggregation, - // keys, - // aggregates, - // overflow_row, - // settings.max_rows_to_group_by, - // settings.group_by_overflow_mode, - // settings.group_by_two_level_threshold, - // settings.group_by_two_level_threshold_bytes, - // settings.max_bytes_before_external_group_by, - // settings.empty_result_for_aggregation_by_empty_set - // || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() - // && query_analyzer->hasConstAggregationKeys()), - // context->getTemporaryVolume(), - // settings.max_threads, - // settings.min_free_disk_space_for_temporary_data, - // settings.compile_aggregate_expressions, - // settings.min_count_to_compile_aggregate_expression); + LOG_DEBUG(log, "GroupingSets debug 3"); std::shared_ptr params_ptr; if (query.group_by_with_grouping_sets) { + LOG_DEBUG(log, "GroupingSets debug 4"); params_ptr = std::make_shared( header_before_aggregation, + all_keys, keys_vector, aggregates, overflow_row, @@ -2157,7 +2153,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression); } - + LOG_DEBUG(log, "GroupingSets debug 5"); SortDescription group_by_sort_description; if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) @@ -2171,7 +2167,8 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac : static_cast(settings.max_threads); bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead(); - + LOG_DEBUG(log, "GroupingSets debug 6"); + LOG_DEBUG(log, "GroupingSets step header structure: {}", query_plan.getCurrentDataStream().header.dumpStructure()); auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), *params_ptr, @@ -2183,8 +2180,10 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac storage_has_evenly_distributed_read, std::move(group_by_info), std::move(group_by_sort_description)); - + LOG_DEBUG(log, "GroupingSets step header structure: {}", aggregating_step->getOutputStream().header.dumpStructure()); + LOG_DEBUG(log, "GroupingSets debug 7"); query_plan.addStep(std::move(aggregating_step)); + LOG_DEBUG(log, "GroupingSets debug 8"); } void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 8357a997960..47f5f80857e 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -501,6 +501,7 @@ void AggregatingTransform::work() Processors AggregatingTransform::expandPipeline() { + LOG_DEBUG(log, "in AggregatingTransform::expandPipeline"); auto & out = processors.back()->getOutputs().front(); inputs.emplace_back(out.getHeader(), this); connect(out, inputs.back()); diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 093491eed2a..d9917b8636d 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -434,6 +434,8 @@ void Pipe::addTransform(ProcessorPtr transform) void Pipe::addParallelTransforms(Processors transforms) { + LOG_DEBUG(log, "Begin addParallelTransforms, have {} transforms", transforms.size()); + if (output_ports.empty()) throw Exception("Cannot add parallel transforms to empty Pipe.", ErrorCodes::LOGICAL_ERROR); @@ -452,6 +454,7 @@ void Pipe::addParallelTransforms(Processors transforms) throw Exception("Each parallel transform should have one output port", ErrorCodes::LOGICAL_ERROR); outputs.push_back(current_transform_outputs.front()); + LOG_DEBUG(log, "addParallelTransforms, added inputs and outputs for processor {}", transform->getName()); } if (inputs.size() != output_ports.size()) @@ -464,6 +467,7 @@ void Pipe::addParallelTransforms(Processors transforms) { connect(*output_ports[next_output], input); ++next_output; + LOG_DEBUG(log, "addParallelTransforms connect current output to new input {}", next_output); } output_ports.clear(); @@ -471,15 +475,22 @@ void Pipe::addParallelTransforms(Processors transforms) for (auto & output : outputs) { + LOG_DEBUG(log, "addParallelTransforms change outputs to new output"); + LOG_DEBUG(log, "addParallelTransforms is output connected: {}", output.isConnected()); output_ports.emplace_back(&output); } /// do not check output formats because they are different in case of parallel aggregations + LOG_DEBUG(log, "addParallelTransforms do not check format"); if (collected_processors) collected_processors->insert(collected_processors->end(), transforms.begin(), transforms.end()); - processors.insert(processors.end(), transforms.begin(), transforms.end()); + for (auto & transform_ptr : transforms) + { + processors.emplace_back(std::move(transform_ptr)); + } + LOG_DEBUG(log, "addParallelTransforms inserted processors, now processors is of size {}", processors.size()); /// Should not change streams number, so maybe not need max_parallel_streams update max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); @@ -512,6 +523,15 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort size_t next_output = 0; for (auto & input : inputs) { + LOG_DEBUG(log, "Pipe: is input connected {}", input.isConnected()); + LOG_DEBUG(log, "Pipe: is output connected {}", output_ports[next_output]->isConnected()); + if (output_ports[next_output]->isConnected()) + { + if (output_ports[next_output]->getHeader()) + LOG_DEBUG(log, "output header structure is: {}", output_ports[next_output]->getHeader().dumpStructure()); + else + LOG_DEBUG(log, "could not retrieve info about output"); + } connect(*output_ports[next_output], input); ++next_output; } diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index e655f807ec8..21e8fbfe039 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -137,6 +138,8 @@ private: /// It is needed for debug. See QueryPipelineProcessorsCollector. Processors * collected_processors = nullptr; + Poco::Logger * log = &Poco::Logger::get("Pipe"); + /// This methods are for QueryPipeline. It is allowed to complete graph only there. /// So, we may be sure that Pipe always has output port if not empty. bool isCompleted() const { return !empty() && output_ports.empty(); } diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 9428bbc5d0a..d920f8a7e81 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -67,7 +67,7 @@ public: /// Transform pipeline in general way. void transform(const Transformer & transformer); - /// Add transform and connects it to outputs[stream_index] stream + /// Add transforms and connect it to outputs streams void addParallelTransforms(Processors transform); /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); From e7711d953a630d854d8205c6dfbe4094d5742b5a Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Sun, 16 May 2021 23:14:21 +0300 Subject: [PATCH 087/358] grouping sets cleanup --- src/Interpreters/InterpreterSelectQuery.cpp | 134 +----------------- src/Processors/QueryPlan/GroupingSetsStep.cpp | 46 ------ src/Processors/QueryPlan/GroupingSetsStep.h | 25 ---- .../Transforms/GroupingSetsTransform.cpp | 88 ------------ .../Transforms/GroupingSetsTransform.h | 37 ----- .../Transforms/TotalsHavingTransform.cpp | 10 -- .../Transforms/TotalsHavingTransform.h | 4 - 7 files changed, 2 insertions(+), 342 deletions(-) delete mode 100644 src/Processors/QueryPlan/GroupingSetsStep.cpp delete mode 100644 src/Processors/QueryPlan/GroupingSetsStep.h delete mode 100644 src/Processors/Transforms/GroupingSetsTransform.cpp delete mode 100644 src/Processors/Transforms/GroupingSetsTransform.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f69e52364fc..c2fc074d47c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -54,7 +54,6 @@ #include #include #include -#include #include #include #include @@ -1049,13 +1048,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aggregates(); for (auto & descr : aggregates) if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); - LOG_DEBUG(log, "GroupingSets debug 2"); + const Settings & settings = context->getSettingsRef(); - LOG_DEBUG(log, "GroupingSets debug 3"); std::shared_ptr params_ptr; if (query.group_by_with_grouping_sets) { - LOG_DEBUG(log, "GroupingSets debug 4"); params_ptr = std::make_shared( header_before_aggregation, all_keys, @@ -2153,7 +2139,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression); } - LOG_DEBUG(log, "GroupingSets debug 5"); SortDescription group_by_sort_description; if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) @@ -2167,7 +2152,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac : static_cast(settings.max_threads); bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead(); - LOG_DEBUG(log, "GroupingSets debug 6"); + LOG_DEBUG(log, "GroupingSets step header structure: {}", query_plan.getCurrentDataStream().header.dumpStructure()); auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -2181,9 +2166,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac std::move(group_by_info), std::move(group_by_sort_description)); LOG_DEBUG(log, "GroupingSets step header structure: {}", aggregating_step->getOutputStream().header.dumpStructure()); - LOG_DEBUG(log, "GroupingSets debug 7"); query_plan.addStep(std::move(aggregating_step)); - LOG_DEBUG(log, "GroupingSets debug 8"); } void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final) @@ -2274,119 +2257,6 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific query_plan.addStep(std::move(step)); } -/* -void InterpreterSelectQuery::executeGroupingSets(QueryPlan & query_plan, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) -{ - - const auto & header_before_transform = query_plan.getCurrentDataStream().header; - - ColumnNumbers keys; - - for (const auto & key : query_analyzer->aggregationKeys()) - { - keys.push_back(header_before_transform.getPositionByName(key.name)); - LOG_DEBUG(log, "executeGroupingSets pushed back key with name {} and number {}", key.name, header_before_transform.getPositionByName(key.name)); - } - - const Settings & settings = context->getSettingsRef(); - - Aggregator::Params params(header_before_transform, keys, query_analyzer->aggregates(), - false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0, - settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, settings.min_count_to_compile_aggregate_expression); - - auto transform_params = std::make_shared(params, true); - - QueryPlanStepPtr step; - step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); - - query_plan.addStep(std::move(step)); - - - auto expression_before_aggregation = std::make_unique(query_plan.getCurrentDataStream(), expression); - expression_before_aggregation->setStepDescription("Before GROUP BY"); - query_plan.addStep(std::move(expression_before_aggregation)); - - const auto & header_before_aggregation = query_plan.getCurrentDataStream().header; - ColumnNumbers keys; - ColumnNumbersTwoDimension keys_vector; - for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) - { - keys.clear(); - for (const auto & key : aggregation_keys) - { - keys.push_back(header_before_aggregation.getPositionByName(key.name)); - LOG_DEBUG( - log, - "executeGroupingSets add key with name {} and number {}", - key.name, - header_before_aggregation.getPositionByName(key.name)); - } - keys_vector.push_back(keys); - } - - AggregateDescriptions aggregates = query_analyzer->aggregates(); - for (auto & descr : aggregates) - if (descr.arguments.empty()) - for (const auto & name : descr.argument_names) - { - descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); - LOG_DEBUG( - log, - "executeGroupingSets add descr.atruments with name {} and number {}", - name, - header_before_aggregation.getPositionByName(name)); - } - - const Settings & settings = context->getSettingsRef(); - - Aggregator::Params params( - header_before_aggregation, - keys_vector, - aggregates, - overflow_row, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - settings.group_by_two_level_threshold, - settings.group_by_two_level_threshold_bytes, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data); - - SortDescription group_by_sort_description; - - if (group_by_info && settings.optimize_aggregation_in_order) - group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery()); - else - group_by_info = nullptr; - - auto merge_threads = max_streams; - auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads - ? static_cast(settings.aggregation_memory_efficient_merge_threads) - : static_cast(settings.max_threads); - - bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead(); - - auto aggregating_step = std::make_unique( - query_plan.getCurrentDataStream(), - params, - final, - settings.max_block_size, - settings.aggregation_in_order_max_block_bytes, - merge_threads, - temporary_data_merge_threads, - storage_has_evenly_distributed_read, - std::move(group_by_info), - std::move(group_by_sort_description)); - - query_plan.addStep(std::move(aggregating_step)); - -} -*/ - void InterpreterSelectQuery::executeExpression(QueryPlan & query_plan, const ActionsDAGPtr & expression, const std::string & description) { if (!expression) diff --git a/src/Processors/QueryPlan/GroupingSetsStep.cpp b/src/Processors/QueryPlan/GroupingSetsStep.cpp deleted file mode 100644 index d409e0e061a..00000000000 --- a/src/Processors/QueryPlan/GroupingSetsStep.cpp +++ /dev/null @@ -1,46 +0,0 @@ -#include -#include -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits() -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = false, - .returns_single_stream = true, - .preserves_number_of_streams = false, - .preserves_sorting = false, - }, - { - .preserves_number_of_rows = false, - } - }; -} - -GroupingSetsStep::GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_) - : ITransformingStep(input_stream_, params_->getHeader(), getTraits()) - , params(std::move(params_)) -{ - /// Aggregation keys are distinct - for (auto key : params->params.keys) - output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name); -} - -void GroupingSetsStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*settings*/) -{ - pipeline.resize(1); - - pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr - { - if (stream_type == QueryPipelineBuilder::StreamType::Totals) - return nullptr; - - return std::make_shared(header, std::move(params)); - }); -} - -} diff --git a/src/Processors/QueryPlan/GroupingSetsStep.h b/src/Processors/QueryPlan/GroupingSetsStep.h deleted file mode 100644 index f27d7f1280e..00000000000 --- a/src/Processors/QueryPlan/GroupingSetsStep.h +++ /dev/null @@ -1,25 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -struct AggregatingTransformParams; -using AggregatingTransformParamsPtr = std::shared_ptr; - -/// WITH CUBE. See CubeTransform. -class GroupingSetsStep : public ITransformingStep -{ -public: - GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_); - - String getName() const override { return "Grouping Sets"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; - -private: - AggregatingTransformParamsPtr params; -}; - -} diff --git a/src/Processors/Transforms/GroupingSetsTransform.cpp b/src/Processors/Transforms/GroupingSetsTransform.cpp deleted file mode 100644 index ccbafac8e3e..00000000000 --- a/src/Processors/Transforms/GroupingSetsTransform.cpp +++ /dev/null @@ -1,88 +0,0 @@ -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -GroupingSetsTransform::GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params_) - : IAccumulatingTransform(std::move(header), params_->getHeader()) - , params(std::move(params_)) - , keys(params->params.keys) -{ -// if (keys.size() >= 8 * sizeof(mask)) -// throw Exception("Too many keys are used for CubeTransform.", ErrorCodes::LOGICAL_ERROR); -} - -Chunk GroupingSetsTransform::merge(Chunks && chunks, bool final) -{ - LOG_DEBUG(log, "merge {} blocks", chunks.size()); - BlocksList rollup_blocks; - for (auto & chunk : chunks) - rollup_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); - - auto rollup_block = params->aggregator.mergeBlocks(rollup_blocks, final); - auto num_rows = rollup_block.rows(); - return Chunk(rollup_block.getColumns(), num_rows); -} - -void GroupingSetsTransform::consume(Chunk chunk) -{ - consumed_chunks.emplace_back(std::move(chunk)); - LOG_DEBUG(log, "consumed block, now consumed_chunks size is {}", consumed_chunks.size()); -} - -Chunk GroupingSetsTransform::generate() -{ - LOG_DEBUG(log, "generate start, mask = {}", mask); - if (!consumed_chunks.empty()) - { - LOG_DEBUG(log, "consumed_chunks not empty, size is {}", consumed_chunks.size()); - if (consumed_chunks.size() > 1) - grouping_sets_chunk = merge(std::move(consumed_chunks), false); - else - grouping_sets_chunk = std::move(consumed_chunks.front()); - - consumed_chunks.clear(); - - auto num_rows = grouping_sets_chunk.getNumRows(); - mask = (UInt64(1) << keys.size()); - LOG_DEBUG(log, "changed mask, mask = {}", mask); - - current_columns = grouping_sets_chunk.getColumns(); - current_zero_columns.clear(); - current_zero_columns.reserve(keys.size()); - - for (auto key : keys) - current_zero_columns.emplace_back(current_columns[key]->cloneEmpty()->cloneResized(num_rows)); - } - - // auto gen_chunk = std::move(cube_chunk); - LOG_DEBUG(log, "before if mask"); - if (mask > 1) - { - LOG_DEBUG(log, "in if mask > 1"); - mask = mask >> 1; - - auto columns = current_columns; - auto size = keys.size(); - for (size_t i = 0; i < size; ++i) - /// Reverse bit order to support previous behaviour. - if ((mask & (UInt64(1) << (size - i - 1))) == 0) - columns[keys[i]] = current_zero_columns[i]; - - Chunks chunks; - chunks.emplace_back(std::move(columns), current_columns.front()->size()); - grouping_sets_chunk = merge(std::move(chunks), false); - } - LOG_DEBUG(log, "before gen_chunk"); - auto gen_chunk = std::move(grouping_sets_chunk); - - finalizeChunk(gen_chunk); - return gen_chunk; -} - -} diff --git a/src/Processors/Transforms/GroupingSetsTransform.h b/src/Processors/Transforms/GroupingSetsTransform.h deleted file mode 100644 index 10108d3c1a7..00000000000 --- a/src/Processors/Transforms/GroupingSetsTransform.h +++ /dev/null @@ -1,37 +0,0 @@ -#pragma once -#include -#include - - -namespace DB -{ - -/// Takes blocks after grouping, with non-finalized aggregate functions. -/// Calculates all subsets of columns and aggregates over them. -class GroupingSetsTransform : public IAccumulatingTransform -{ -public: - GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params); - String getName() const override { return "GroupingSetsTransform"; } - -protected: - void consume(Chunk chunk) override; - Chunk generate() override; - -private: - AggregatingTransformParamsPtr params; - ColumnNumbers keys; - - Chunks consumed_chunks; - Chunk grouping_sets_chunk; - Columns current_columns; - Columns current_zero_columns; - - UInt64 mask = 0; - - Poco::Logger * log = &Poco::Logger::get("GroupingSetsTransform"); - - Chunk merge(Chunks && chunks, bool final); -}; - -} diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index a636ba70ef9..0b7797da24f 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -119,7 +119,6 @@ TotalsHavingTransform::TotalsHavingTransform( IProcessor::Status TotalsHavingTransform::prepare() { - LOG_DEBUG(log, "TotalsHavingTransform::prepare()"); if (!finished_transform) { auto status = ISimpleTransform::prepare(); @@ -144,24 +143,20 @@ IProcessor::Status TotalsHavingTransform::prepare() totals_output.push(std::move(totals)); totals_output.finish(); - LOG_DEBUG(log, "exit TotalsHavingTransform::prepare()"); return Status::Finished; } void TotalsHavingTransform::work() { - LOG_DEBUG(log, "TotalsHavingTransform::work()"); if (finished_transform) prepareTotals(); else ISimpleTransform::work(); - LOG_DEBUG(log, "exit TotalsHavingTransform::work()"); } void TotalsHavingTransform::transform(Chunk & chunk) { /// Block with values not included in `max_rows_to_group_by`. We'll postpone it. - LOG_DEBUG(log, "TotalsHavingTransform::transform()"); if (overflow_row) { const auto & info = chunk.getChunkInfo(); @@ -254,12 +249,10 @@ void TotalsHavingTransform::transform(Chunk & chunk) } passed_keys += chunk.getNumRows(); - LOG_DEBUG(log, "exit TotalsHavingTransform::transform()"); } void TotalsHavingTransform::addToTotals(const Chunk & chunk, const IColumn::Filter * filter) { - LOG_DEBUG(log, "TotalsHavingTransform::addToTotals()"); auto num_columns = chunk.getNumColumns(); for (size_t col = 0; col < num_columns; ++col) { @@ -291,12 +284,10 @@ void TotalsHavingTransform::addToTotals(const Chunk & chunk, const IColumn::Filt } } } - LOG_DEBUG(log, "exit TotalsHavingTransform::addToTotals()"); } void TotalsHavingTransform::prepareTotals() { - LOG_DEBUG(log, "TotalsHavingTransform::prepareTotals()"); /// If totals_mode == AFTER_HAVING_AUTO, you need to decide whether to add aggregates to TOTALS for strings, /// not passed max_rows_to_group_by. if (overflow_aggregates) @@ -321,7 +312,6 @@ void TotalsHavingTransform::prepareTotals() /// Note: after expression totals may have several rows if `arrayJoin` was used in expression. totals = Chunk(block.getColumns(), num_rows); } - LOG_DEBUG(log, "exit TotalsHavingTransform::prepareTotals()"); } } diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index 59fa58edfbc..03635054c65 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -2,8 +2,6 @@ #include #include -#include // to be removed -// #include namespace DB { @@ -76,8 +74,6 @@ private: /// Here, total values are accumulated. After the work is finished, they will be placed in totals. MutableColumns current_totals; - - Poco::Logger * log = &Poco::Logger::get("TotalsHavingTransform"); }; void finalizeChunk(Chunk & chunk); From fff23945faddabb3ca90eb576b3f7d0bef231ebf Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Mon, 17 May 2021 15:03:07 +0300 Subject: [PATCH 088/358] grouping sets: fix 'Port already connected' error --- src/Processors/ya.make | 2 +- src/QueryPipeline/Pipe.cpp | 24 +++++++++++++----------- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index a1386acb6d0..a6f8e274e9e 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -102,7 +102,7 @@ SRCS( QueryPlan/FillingStep.cpp QueryPlan/FilterStep.cpp QueryPlan/FinishSortingStep.cpp - QueryPlan/GroupingSetsStep.cpp + QueryPlan/IQueryPlanStep.cpp QueryPlan/ISourceStep.cpp QueryPlan/ITransformingStep.cpp diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index d9917b8636d..25cdf2c8fa9 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -439,21 +439,21 @@ void Pipe::addParallelTransforms(Processors transforms) if (output_ports.empty()) throw Exception("Cannot add parallel transforms to empty Pipe.", ErrorCodes::LOGICAL_ERROR); - std::vector inputs; - std::vector outputs; + std::vector inputs; + std::vector outputs; for (const auto & transform : transforms) { - auto current_transform_inputs = transform->getInputs(); + auto & current_transform_inputs = transform->getInputs(); if (current_transform_inputs.size() != 1) throw Exception("Each parallel transform should have one input port", ErrorCodes::LOGICAL_ERROR); - inputs.push_back(current_transform_inputs.front()); + inputs.push_back(&(current_transform_inputs.front())); - auto current_transform_outputs = transform->getOutputs(); + auto & current_transform_outputs = transform->getOutputs(); if (current_transform_outputs.size() != 1) throw Exception("Each parallel transform should have one output port", ErrorCodes::LOGICAL_ERROR); - outputs.push_back(current_transform_outputs.front()); + outputs.push_back(&(current_transform_outputs.front())); LOG_DEBUG(log, "addParallelTransforms, added inputs and outputs for processor {}", transform->getName()); } @@ -463,9 +463,11 @@ void Pipe::addParallelTransforms(Processors transforms) "but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR); size_t next_output = 0; - for (auto & input : inputs) + for (auto * input : inputs) { - connect(*output_ports[next_output], input); + LOG_DEBUG(log, "is input connected {}", input->isConnected()); + LOG_DEBUG(log, "is output connected {}", output_ports[next_output]->isConnected()); + connect(*output_ports[next_output], *input); ++next_output; LOG_DEBUG(log, "addParallelTransforms connect current output to new input {}", next_output); } @@ -473,11 +475,11 @@ void Pipe::addParallelTransforms(Processors transforms) output_ports.clear(); output_ports.reserve(outputs.size()); - for (auto & output : outputs) + for (auto * output : outputs) { LOG_DEBUG(log, "addParallelTransforms change outputs to new output"); - LOG_DEBUG(log, "addParallelTransforms is output connected: {}", output.isConnected()); - output_ports.emplace_back(&output); + LOG_DEBUG(log, "addParallelTransforms is output connected: {}", output->isConnected()); + output_ports.emplace_back(std::move(output)); } /// do not check output formats because they are different in case of parallel aggregations From e32d58a36da4beac38e75943ac84d5e1615e242b Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Fri, 21 May 2021 02:35:49 +0300 Subject: [PATCH 089/358] grouping sets dev: fix errors, something works --- src/Interpreters/ExpressionAnalyzer.cpp | 3 ++- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++++-- src/Processors/QueryPlan/AggregatingStep.h | 2 ++ src/Processors/ya.make | 2 -- src/QueryPipeline/Pipe.cpp | 20 +------------------- 5 files changed, 9 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 8ac892d8834..27f5ed83fd7 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1162,7 +1162,8 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain ASTs asts = select_query->groupBy()->children; if (select_query->group_by_with_grouping_sets) { - for (const auto & inner_asts : asts) { + for (const auto & inner_asts : asts) + { for (const auto & ast : inner_asts->children) { step.addRequiredOutput(ast->getColumnName()); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c2fc074d47c..fab9b174dea 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -582,6 +582,8 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) /// /// But if it's a projection query, plan header does not match result_header. /// TODO: add special stage for InterpreterSelectQuery? + LOG_DEBUG(log, "query_plan header: {}", query_plan.getCurrentDataStream().header.dumpStructure()); + LOG_DEBUG(log, "result header: {}", result_header.dumpStructure()); if (!options.is_projection_query && !blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) { auto convert_actions_dag = ActionsDAG::makeConvertingActions( @@ -2153,7 +2155,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead(); - LOG_DEBUG(log, "GroupingSets step header structure: {}", query_plan.getCurrentDataStream().header.dumpStructure()); + LOG_DEBUG(log, "GroupingSets step header before step structure: {}", query_plan.getCurrentDataStream().header.dumpStructure()); auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), *params_ptr, @@ -2165,7 +2167,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac storage_has_evenly_distributed_read, std::move(group_by_info), std::move(group_by_sort_description)); - LOG_DEBUG(log, "GroupingSets step header structure: {}", aggregating_step->getOutputStream().header.dumpStructure()); + LOG_DEBUG(log, "GroupingSets step header after step structure: {}", aggregating_step->getOutputStream().header.dumpStructure()); query_plan.addStep(std::move(aggregating_step)); } diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 154ff0abc0a..4c476a2841f 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -56,6 +56,8 @@ private: Processors aggregating; + Poco::Logger * log = &Poco::Logger::get("AggregatingStep"); + }; } diff --git a/src/Processors/ya.make b/src/Processors/ya.make index a6f8e274e9e..b2f8b9ba7c2 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -102,7 +102,6 @@ SRCS( QueryPlan/FillingStep.cpp QueryPlan/FilterStep.cpp QueryPlan/FinishSortingStep.cpp - QueryPlan/IQueryPlanStep.cpp QueryPlan/ISourceStep.cpp QueryPlan/ITransformingStep.cpp @@ -141,7 +140,6 @@ SRCS( Transforms/FillingTransform.cpp Transforms/FilterTransform.cpp Transforms/FinishSortingTransform.cpp - Transforms/GroupingSetsTransform.cpp Transforms/JoiningTransform.cpp Transforms/LimitByTransform.cpp Transforms/LimitsCheckingTransform.cpp diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 25cdf2c8fa9..184cad18c17 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -455,6 +455,7 @@ void Pipe::addParallelTransforms(Processors transforms) outputs.push_back(&(current_transform_outputs.front())); LOG_DEBUG(log, "addParallelTransforms, added inputs and outputs for processor {}", transform->getName()); + LOG_DEBUG(log, "output structure: {}", transform->getOutputs().front().getHeader().dumpStructure()); } if (inputs.size() != output_ports.size()) @@ -465,22 +466,15 @@ void Pipe::addParallelTransforms(Processors transforms) size_t next_output = 0; for (auto * input : inputs) { - LOG_DEBUG(log, "is input connected {}", input->isConnected()); - LOG_DEBUG(log, "is output connected {}", output_ports[next_output]->isConnected()); connect(*output_ports[next_output], *input); ++next_output; - LOG_DEBUG(log, "addParallelTransforms connect current output to new input {}", next_output); } output_ports.clear(); output_ports.reserve(outputs.size()); for (auto * output : outputs) - { - LOG_DEBUG(log, "addParallelTransforms change outputs to new output"); - LOG_DEBUG(log, "addParallelTransforms is output connected: {}", output->isConnected()); output_ports.emplace_back(std::move(output)); - } /// do not check output formats because they are different in case of parallel aggregations LOG_DEBUG(log, "addParallelTransforms do not check format"); @@ -489,10 +483,7 @@ void Pipe::addParallelTransforms(Processors transforms) collected_processors->insert(collected_processors->end(), transforms.begin(), transforms.end()); for (auto & transform_ptr : transforms) - { processors.emplace_back(std::move(transform_ptr)); - } - LOG_DEBUG(log, "addParallelTransforms inserted processors, now processors is of size {}", processors.size()); /// Should not change streams number, so maybe not need max_parallel_streams update max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); @@ -525,15 +516,6 @@ void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort size_t next_output = 0; for (auto & input : inputs) { - LOG_DEBUG(log, "Pipe: is input connected {}", input.isConnected()); - LOG_DEBUG(log, "Pipe: is output connected {}", output_ports[next_output]->isConnected()); - if (output_ports[next_output]->isConnected()) - { - if (output_ports[next_output]->getHeader()) - LOG_DEBUG(log, "output header structure is: {}", output_ports[next_output]->getHeader().dumpStructure()); - else - LOG_DEBUG(log, "could not retrieve info about output"); - } connect(*output_ports[next_output], input); ++next_output; } From 4f07ec2659202c4d47b58b024450ccf54cfb62e0 Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Fri, 21 May 2021 17:56:54 +0300 Subject: [PATCH 090/358] grouping sets: make simple aggregation with grouping sets to work --- src/Interpreters/ExpressionAnalyzer.cpp | 8 ++------ src/Interpreters/InterpreterSelectQuery.cpp | 20 +++++++++++--------- src/Interpreters/TreeOptimizer.cpp | 10 +++++----- 3 files changed, 18 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 27f5ed83fd7..eb49d87f017 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -387,16 +387,12 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) if (!unique_keys.count(key.name)) { unique_keys.insert(key.name); + aggregation_keys.push_back(key); if (select_query->group_by_with_grouping_sets) { aggregation_keys_list.push_back({key}); - aggregation_keys.push_back(key); LOG_DEBUG(poco_log, "pushed grouping set of 1 column: " + key.name); } - else - { - aggregation_keys.push_back(key); - } /// Key is no longer needed, therefore we can save a little by moving it. aggregated_columns.push_back(std::move(key)); @@ -1668,7 +1664,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( /// TODO correct conditions optimize_aggregation_in_order = context->getSettingsRef().optimize_aggregation_in_order - && storage && query.groupBy() && !query.group_by_with_grouping_sets; + && storage && query.groupBy(); query_analyzer.appendGroupBy(chain, only_types || !first_stage, optimize_aggregation_in_order, group_by_elements_actions); query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fab9b174dea..7d47e204160 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -965,8 +965,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

QueryProcessingStage::WithMergeableState && - !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube && - !query.group_by_with_grouping_sets; + !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube; if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) { @@ -2070,14 +2069,14 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac all_keys.push_back(header_before_aggregation.getPositionByName(key.name)); LOG_DEBUG( log, - "GroupingSets add key with name {} and number {}", + "execute aggregation with grouping sets add key with name {} and number {}", key.name, header_before_aggregation.getPositionByName(key.name)); } keys_vector.push_back(keys); LOG_DEBUG( log, - "GroupingSets add keys set of size {}", + "execute aggregation with grouping sets add keys set of size {}", keys.size()); } } @@ -2086,7 +2085,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac for (const auto & key : query_analyzer->aggregationKeys()) { keys.push_back(header_before_aggregation.getPositionByName(key.name)); - LOG_DEBUG(log, "executeAggregation pushed back key with name {} and number {}", key.name, header_before_aggregation.getPositionByName(key.name)); + LOG_DEBUG(log, "execute aggregation without grouping sets pushed back key with name {} and number {}", key.name, header_before_aggregation.getPositionByName(key.name)); } } @@ -2143,10 +2142,13 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac } SortDescription group_by_sort_description; - if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) + if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) { group_by_sort_description = getSortDescriptionFromGroupBy(query); - else + LOG_DEBUG(log, "execute aggregation without grouping sets got group_by_sort_description"); + } else { group_by_info = nullptr; + LOG_DEBUG(log, "execute aggregation didn't get group_by_sort_description"); + } auto merge_threads = max_streams; auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads @@ -2155,7 +2157,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead(); - LOG_DEBUG(log, "GroupingSets step header before step structure: {}", query_plan.getCurrentDataStream().header.dumpStructure()); + LOG_DEBUG(log, "execute aggregation header structure before step: {}", query_plan.getCurrentDataStream().header.dumpStructure()); auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), *params_ptr, @@ -2167,7 +2169,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac storage_has_evenly_distributed_read, std::move(group_by_info), std::move(group_by_sort_description)); - LOG_DEBUG(log, "GroupingSets step header after step structure: {}", aggregating_step->getOutputStream().header.dumpStructure()); + LOG_DEBUG(log, "execute aggregation header structure after step: {}", aggregating_step->getOutputStream().header.dumpStructure()); query_plan.addStep(std::move(aggregating_step)); } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index a76e05a8ad6..beb920b734f 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -84,8 +84,8 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context) { const FunctionFactory & function_factory = FunctionFactory::instance(); - if (select_query->group_by_with_grouping_sets) - return; +// if (select_query->group_by_with_grouping_sets) +// return; if (!select_query->groupBy()) return; @@ -216,7 +216,7 @@ GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys) ///eliminate functions of other GROUP BY keys void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) { - if (!select_query->groupBy() || select_query->group_by_with_grouping_sets) + if (!select_query->groupBy()) return; auto group_by = select_query->groupBy(); @@ -246,7 +246,7 @@ void optimizeGroupByFunctionKeys(ASTSelectQuery * select_query) /// Eliminates min/max/any-aggregators of functions of GROUP BY keys void optimizeAggregateFunctionsOfGroupByKeys(ASTSelectQuery * select_query, ASTPtr & node) { - if (!select_query->groupBy() || select_query->group_by_with_grouping_sets) + if (!select_query->groupBy()) return; const auto & group_by_keys = select_query->groupBy()->children; @@ -416,7 +416,7 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, Context const Names & sorting_key_columns) { auto order_by = select_query->orderBy(); - if (!order_by || select_query->group_by_with_grouping_sets) + if (!order_by) return; for (const auto & child : order_by->children) From 2640f51843abc07887e93a97a93f477bf9f772c0 Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Fri, 21 May 2021 23:47:57 +0300 Subject: [PATCH 091/358] grouping sets: fix --- src/Interpreters/InterpreterSelectQuery.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7d47e204160..dc30bbb227a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -582,8 +582,6 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) /// /// But if it's a projection query, plan header does not match result_header. /// TODO: add special stage for InterpreterSelectQuery? - LOG_DEBUG(log, "query_plan header: {}", query_plan.getCurrentDataStream().header.dumpStructure()); - LOG_DEBUG(log, "result header: {}", result_header.dumpStructure()); if (!options.is_projection_query && !blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) { auto convert_actions_dag = ActionsDAG::makeConvertingActions( From 9953e1bc0a241edda9f81c502388c71eebea40a3 Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Thu, 27 May 2021 22:50:35 +0300 Subject: [PATCH 092/358] grouping sets: add tests, fix bug --- src/Interpreters/InterpreterSelectQuery.cpp | 3 +++ .../01883_with_grouping_sets.reference | 8 +++++++ .../0_stateless/01883_with_grouping_sets.sql | 22 +++++++++++++++++++ 3 files changed, 33 insertions(+) create mode 100644 tests/queries/0_stateless/01883_with_grouping_sets.reference create mode 100644 tests/queries/0_stateless/01883_with_grouping_sets.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index dc30bbb227a..0c775baab56 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -965,6 +965,9 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

QueryProcessingStage::WithMergeableState && !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube; + if (query.group_by_with_grouping_sets && query.group_by_with_totals) + throw Exception("WITH TOTALS and GROUPING SETS are not supported together", ErrorCodes::NOT_IMPLEMENTED); + if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) { query_info.projection->aggregate_overflow_row = aggregate_overflow_row; diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference new file mode 100644 index 00000000000..60cc348f671 --- /dev/null +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -0,0 +1,8 @@ + 1 40 4 + 2 80 4 +a 0 70 4 +b 0 50 4 + 1 40 4 + 2 80 4 +a 0 70 4 +b 0 50 4 diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.sql b/tests/queries/0_stateless/01883_with_grouping_sets.sql new file mode 100644 index 00000000000..71f03209cce --- /dev/null +++ b/tests/queries/0_stateless/01883_with_grouping_sets.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS grouping_sets; +CREATE TABLE grouping_sets(a String, b Int32, s Int32) ENGINE = Memory; + +INSERT INTO grouping_sets VALUES ('a', 1, 10), ('a', 1, 15), ('a', 2, 20); +INSERT INTO grouping_sets VALUES ('a', 2, 25), ('b', 1, 10), ('b', 1, 5); +INSERT INTO grouping_sets VALUES ('b', 2, 20), ('b', 2, 15); + +SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) ORDER BY a, b; + +-- doesn't work now +-- SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) WITH TOTALS ORDER BY a, b; + +SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS ORDER BY a, b; + +-- doesn't work now +-- SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS WITH TOTALS ORDER BY a, b; + +-- not sure that always works +-- SET group_by_two_level_threshold = 1; +-- SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS ORDER BY a, b; + +DROP TABLE grouping_sets; From b39f5bd45f753a95c46fc1dc712ff55cd7a26a4f Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Sun, 20 Jun 2021 12:36:00 +0300 Subject: [PATCH 093/358] grouping-sets: rearrange result columns so that resize is possible after grouping sets --- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0c775baab56..ff75512c91b 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -965,8 +965,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

QueryProcessingStage::WithMergeableState && !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube; - if (query.group_by_with_grouping_sets && query.group_by_with_totals) - throw Exception("WITH TOTALS and GROUPING SETS are not supported together", ErrorCodes::NOT_IMPLEMENTED); +// if (query.group_by_with_grouping_sets && query.group_by_with_totals) +// throw Exception("WITH TOTALS and GROUPING SETS are not supported together", ErrorCodes::NOT_IMPLEMENTED); if (query_info.projection && query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) { From d425cbe58827f2cc25989546252eaae8fc8b7a6e Mon Sep 17 00:00:00 2001 From: MaxTheHuman Date: Sun, 20 Jun 2021 13:04:18 +0300 Subject: [PATCH 094/358] make grouping sets work with total --- .../0_stateless/01883_with_grouping_sets.reference | 12 ++++++++++++ .../queries/0_stateless/01883_with_grouping_sets.sql | 6 ++---- 2 files changed, 14 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index 60cc348f671..fd128444ebc 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -6,3 +6,15 @@ b 0 50 4 2 80 4 a 0 70 4 b 0 50 4 + + 0 240 16 + 1 40 4 + 2 80 4 +a 0 70 4 +b 0 50 4 + 1 40 4 + 2 80 4 +a 0 70 4 +b 0 50 4 + + 0 240 16 diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.sql b/tests/queries/0_stateless/01883_with_grouping_sets.sql index 71f03209cce..6270b772328 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.sql +++ b/tests/queries/0_stateless/01883_with_grouping_sets.sql @@ -7,13 +7,11 @@ INSERT INTO grouping_sets VALUES ('b', 2, 20), ('b', 2, 15); SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) ORDER BY a, b; --- doesn't work now --- SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) WITH TOTALS ORDER BY a, b; +SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) WITH TOTALS ORDER BY a, b; SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS ORDER BY a, b; --- doesn't work now --- SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS WITH TOTALS ORDER BY a, b; +SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS WITH TOTALS ORDER BY a, b; -- not sure that always works -- SET group_by_two_level_threshold = 1; From 2912344896d53318b8418f2f3af2a85c9b0776a1 Mon Sep 17 00:00:00 2001 From: fanzhou Date: Wed, 28 Jul 2021 15:02:10 +0800 Subject: [PATCH 095/358] grouping sets support multiple group --- src/Interpreters/ActionsDAG.h | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 127 ++++++++++++++------ src/Interpreters/InterpreterSelectQuery.cpp | 14 ++- src/Interpreters/TreeOptimizer.cpp | 18 ++- src/Parsers/ExpressionListParsers.cpp | 47 +++++++- src/Parsers/ExpressionListParsers.h | 6 + 6 files changed, 164 insertions(+), 50 deletions(-) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 9a5ad01a252..cfb85cd5369 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -183,7 +183,7 @@ public: ActionsDAGPtr clone() const; /// Execute actions for header. Input block must have empty columns. - /// Result should be equal to the execution of ExpressionActions build form this DAG. + /// Result should be equal to the execution of ExpressionActions built from this DAG. /// Actions are not changed, no expressions are compiled. /// /// In addition, check that result constants are constants according to DAG. diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index eb49d87f017..ba38ee10d8c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -40,6 +40,7 @@ #include #include +#include "Core/NamesAndTypes.h" #include @@ -345,57 +346,103 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) if (getContext()->getSettingsRef().enable_positional_arguments) replaceForPositionalArguments(group_asts[i], select_query, ASTSelectQuery::Expression::GROUP_BY); - if (select_query->group_by_with_grouping_sets) - { - LOG_DEBUG(poco_log, "analyzeAggregation: detect group by with grouping sets"); - /// TODO - } - getRootActionsNoMakeSet(group_asts[i], true, temp_actions, false); - const auto & column_name = group_asts[i]->getColumnName(); - - const auto * node = temp_actions->tryFindInIndex(column_name); - if (!node) - throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); - - /// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled. - if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0) + if (select_query->group_by_with_grouping_sets) { - /// Constant expressions have non-null column pointer at this stage. - if (node->column && isColumnConst(*node->column)) + ASTs group_elements_ast; + if (group_asts[i]->as()) { - select_query->group_by_with_constant_keys = true; + group_elements_ast = group_asts[i]->as()->children; + } + else + { + const auto id_ast = group_asts[i]->as(); + group_elements_ast.push_back(std::make_shared(id_ast)); + } - /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. - if (!aggregate_descriptions.empty() || size > 1) + NamesAndTypesList grouping_set_list; + + for (ssize_t j = 0; j < ssize_t(group_elements_ast.size()); ++j) + { + const auto & column_name = group_elements_ast[j]->getColumnName(); + const auto * node = temp_actions->tryFindInIndex(column_name); + if (!node) + throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); + + /// Constant expressions have non-null column pointer at this stage. + if (node->column && isColumnConst(*node->column)) { - if (i + 1 < static_cast(size)) - group_asts[i] = std::move(group_asts.back()); + /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. + if (!aggregate_descriptions.empty() || size > 1) + { + if (j + 1 < static_cast(size)) + group_asts[i] = std::move(group_asts.back()); - group_asts.pop_back(); + group_asts.pop_back(); - --i; - continue; + --j; + continue; + } + } + + NameAndTypePair key{column_name, node->result_type}; + + grouping_set_list.push_back(key); + + /// Aggregation keys are unique. + if (!unique_keys.count(key.name)) + { + unique_keys.insert(key.name); + aggregation_keys.push_back(key); + + /// Key is no longer needed, therefore we can save a little by moving it. + aggregated_columns.push_back(std::move(key)); } } + + aggregation_keys_list.push_back(grouping_set_list); } - - NameAndTypePair key{column_name, node->result_type}; - - /// Aggregation keys are uniqued. - if (!unique_keys.count(key.name)) + else { - unique_keys.insert(key.name); - aggregation_keys.push_back(key); - if (select_query->group_by_with_grouping_sets) + const auto & column_name = group_asts[i]->getColumnName(); + const auto * node = temp_actions->tryFindInIndex(column_name); + if (!node) + throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); + + /// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled. + if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0) { - aggregation_keys_list.push_back({key}); - LOG_DEBUG(poco_log, "pushed grouping set of 1 column: " + key.name); + /// Constant expressions have non-null column pointer at this stage. + if (node->column && isColumnConst(*node->column)) + { + select_query->group_by_with_constant_keys = true; + + /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. + if (!aggregate_descriptions.empty() || size > 1) + { + if (i + 1 < static_cast(size)) + group_asts[i] = std::move(group_asts.back()); + + group_asts.pop_back(); + + --i; + continue; + } + } } - /// Key is no longer needed, therefore we can save a little by moving it. - aggregated_columns.push_back(std::move(key)); + NameAndTypePair key{column_name, node->result_type}; + + /// Aggregation keys are uniqued. + if (!unique_keys.count(key.name)) + { + unique_keys.insert(key.name); + aggregation_keys.push_back(key); + + /// Key is no longer needed, therefore we can save a little by moving it. + aggregated_columns.push_back(std::move(key)); + } } } @@ -1158,12 +1205,12 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain ASTs asts = select_query->groupBy()->children; if (select_query->group_by_with_grouping_sets) { - for (const auto & inner_asts : asts) + for (const auto & ast : asts) { - for (const auto & ast : inner_asts->children) + for(const auto & ast_element : ast->children) { - step.addRequiredOutput(ast->getColumnName()); - getRootActions(ast, only_types, step.actions()); + step.addRequiredOutput(ast_element->getColumnName()); + getRootActions(ast_element, only_types, step.actions()); } } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ff75512c91b..5cd00b6d495 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2061,13 +2061,18 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac auto & query = getSelectQuery(); if (query.group_by_with_grouping_sets) { + std::set keys_set; for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) { keys.clear(); for (const auto & key : aggregation_keys) { - keys.push_back(header_before_aggregation.getPositionByName(key.name)); - all_keys.push_back(header_before_aggregation.getPositionByName(key.name)); + size_t key_name_pos = header_before_aggregation.getPositionByName(key.name); + if (!keys_set.count(key_name_pos)) + { + keys_set.insert(key_name_pos); + } + keys.push_back(key_name_pos); LOG_DEBUG( log, "execute aggregation with grouping sets add key with name {} and number {}", @@ -2080,6 +2085,11 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac "execute aggregation with grouping sets add keys set of size {}", keys.size()); } + all_keys.assign(keys_set.begin(), keys_set.end()); + LOG_DEBUG( + log, + "execute aggregation with grouping sets add all keys of size {}", + all_keys.size()); } else { diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index beb920b734f..e221b6da601 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -204,10 +204,22 @@ GroupByKeysInfo getGroupByKeysInfo(const ASTs & group_by_keys) /// filling set with short names of keys for (const auto & group_key : group_by_keys) { - if (group_key->as()) - data.has_function = true; + /// for grouping sets case + if (group_key->as()) + { + const auto express_list_ast = group_key->as(); + for (const auto & group_elem : express_list_ast.children) + { + data.key_names.insert(group_elem->getColumnName()); + } + } + else + { + if (group_key->as()) + data.has_function = true; - data.key_names.insert(group_key->getColumnName()); + data.key_names.insert(group_key->getColumnName()); + } } return data; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index a2be8d75e1b..12c8747a43b 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -746,18 +746,57 @@ bool ParserNotEmptyExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected return nested_parser.parse(pos, node, expected) && !node->children.empty(); } - bool ParserOrderByExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) .parse(pos, node, expected); } +bool ParserGroupingSetsExpressionListElements::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto command_list = std::make_shared(); + node = command_list; + + ParserToken s_comma(TokenType::Comma); + ParserToken s_open(TokenType::OpeningRoundBracket); + ParserToken s_close(TokenType::ClosingRoundBracket); + ParserExpressionWithOptionalAlias p_expression(false); + ParserList p_command(std::make_unique(false), + std::make_unique(TokenType::Comma), false); + + do + { + Pos begin = pos; + ASTPtr command; + if (!s_open.ignore(pos, expected)) + { + pos = begin; + if (!p_expression.parse(pos, command, expected)) + { + return false; + } + } + else + { + if (!p_command.parse(pos, command, expected)) + return false; + + if (!s_close.ignore(pos, expected)) + break; + } + + command_list->children.push_back(command); + } + while (s_comma.ignore(pos, expected)); + + return true; +} + bool ParserGroupingSetsExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - return ParserList(std::make_unique(false), - std::make_unique(TokenType::Comma), false) - .parse(pos, node, expected); + ParserGroupingSetsExpressionListElements grouping_sets_elements; + return grouping_sets_elements.parse(pos, node, expected); + } bool ParserTTLExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) diff --git a/src/Parsers/ExpressionListParsers.h b/src/Parsers/ExpressionListParsers.h index 85ed9a81d83..e75d773cc90 100644 --- a/src/Parsers/ExpressionListParsers.h +++ b/src/Parsers/ExpressionListParsers.h @@ -518,6 +518,12 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +class ParserGroupingSetsExpressionListElements : public IParserBase +{ +protected: + const char * getName() const override { return "grouping sets expression elements"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; /// Parser for key-value pair, where value can be list of pairs. From 534160b0d4b13b626da6d4f72241779cbfe7b40d Mon Sep 17 00:00:00 2001 From: fanzhou Date: Mon, 23 Aug 2021 14:28:49 +0800 Subject: [PATCH 096/358] grouing sets transformer instead of fork processor in aggregating transformer --- src/Interpreters/InterpreterSelectQuery.cpp | 108 ++++++++++++++---- src/Interpreters/InterpreterSelectQuery.h | 3 +- src/Processors/QueryPlan/GroupingSetsStep.cpp | 46 ++++++++ src/Processors/QueryPlan/GroupingSetsStep.h | 25 ++++ .../Transforms/GroupingSetsTransform.cpp | 76 ++++++++++++ .../Transforms/GroupingSetsTransform.h | 35 ++++++ .../01883_with_grouping_sets.reference | 85 +++++++++++--- .../0_stateless/01883_with_grouping_sets.sql | 40 +++++-- 8 files changed, 365 insertions(+), 53 deletions(-) create mode 100644 src/Processors/QueryPlan/GroupingSetsStep.cpp create mode 100644 src/Processors/QueryPlan/GroupingSetsStep.h create mode 100644 src/Processors/Transforms/GroupingSetsTransform.cpp create mode 100644 src/Processors/Transforms/GroupingSetsTransform.h diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5cd00b6d495..088f8eb1fd0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -44,6 +44,7 @@ #include #include #include +#include #include #include #include @@ -963,7 +964,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

QueryProcessingStage::WithMergeableState && - !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube; + !query.group_by_with_totals && !query.group_by_with_rollup && !query.group_by_with_cube && !query.group_by_with_grouping_sets; // if (query.group_by_with_grouping_sets && query.group_by_with_totals) // throw Exception("WITH TOTALS and GROUPING SETS are not supported together", ErrorCodes::NOT_IMPLEMENTED); @@ -1258,6 +1259,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

aggregationKeys()) - keys.push_back(header_before_transform.getPositionByName(key.name)); + ColumnNumbers all_keys; + ColumnNumbersTwoDimension keys_vector; + auto & query = getSelectQuery(); + if (query.group_by_with_grouping_sets) + { + std::set keys_set; + for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) + { + keys.clear(); + for (const auto & key : aggregation_keys) + { + size_t key_name_pos = header_before_transform.getPositionByName(key.name); + if (!keys_set.count(key_name_pos)) + { + keys_set.insert(key_name_pos); + } + keys.push_back(key_name_pos); + } + keys_vector.push_back(keys); + LOG_DEBUG( + log, + "execute aggregation with grouping sets add keys set of size {}", + keys.size()); + } + all_keys.assign(keys_set.begin(), keys_set.end()); + LOG_DEBUG( + log, + "execute aggregation with grouping sets add all keys of size {}", + all_keys.size()); + } + else + { + for (const auto & key : query_analyzer->aggregationKeys()) + { + keys.push_back(header_before_transform.getPositionByName(key.name)); + } + } const Settings & settings = context->getSettingsRef(); - Aggregator::Params params( - header_before_transform, - keys, - query_analyzer->aggregates(), - false, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - 0, - 0, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); + std::shared_ptr params_ptr; + if (query.group_by_with_grouping_sets) + { + params_ptr = std::make_shared( + header_before_transform, + all_keys, + keys_vector, + query_analyzer->aggregates(), + false, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + 0, + 0, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set, + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); + } + else + { + params_ptr = std::make_shared( + header_before_transform, + keys, + query_analyzer->aggregates(), + false, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + 0, + 0, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set, + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); + } - auto transform_params = std::make_shared(params, true); + auto transform_params = std::make_shared(*params_ptr, true); QueryPlanStepPtr step; if (modificator == Modificator::ROLLUP) step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); - else + else if (modificator == Modificator::CUBE) step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); + else + step = std::make_unique(query_plan.getCurrentDataStream(), std::move(transform_params)); query_plan.addStep(std::move(step)); } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 34d56a7354b..71a8ac5e1d5 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -157,7 +157,8 @@ private: enum class Modificator { ROLLUP = 0, - CUBE = 1 + CUBE = 1, + GROUPING_SETS = 2 }; void executeRollupOrCube(QueryPlan & query_plan, Modificator modificator); diff --git a/src/Processors/QueryPlan/GroupingSetsStep.cpp b/src/Processors/QueryPlan/GroupingSetsStep.cpp new file mode 100644 index 00000000000..383ecf5ea1b --- /dev/null +++ b/src/Processors/QueryPlan/GroupingSetsStep.cpp @@ -0,0 +1,46 @@ +#include +#include +#include "QueryPipeline/QueryPipelineBuilder.h" + +namespace DB +{ + +static ITransformingStep::Traits getTraits() +{ + return ITransformingStep::Traits + { + { + .preserves_distinct_columns = false, + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = false, + } + }; +} + +GroupingSetsStep::GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_) + : ITransformingStep(input_stream_, params_->getHeader(), getTraits()) + , params(std::move(params_)) +{ + /// Aggregation keys are distinct + for (auto key : params->params.keys) + output_stream->distinct_columns.insert(params->params.src_header.getByPosition(key).name); +} + +void GroupingSetsStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + pipeline.resize(1); + + pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr + { + if (stream_type == QueryPipelineBuilder::StreamType::Totals) + return nullptr; + + return std::make_shared(header, std::move(params)); + }); +} + +} diff --git a/src/Processors/QueryPlan/GroupingSetsStep.h b/src/Processors/QueryPlan/GroupingSetsStep.h new file mode 100644 index 00000000000..3dc7c3b5a99 --- /dev/null +++ b/src/Processors/QueryPlan/GroupingSetsStep.h @@ -0,0 +1,25 @@ +#pragma once +#include +#include +#include "QueryPipeline/QueryPipelineBuilder.h" + +namespace DB +{ + +struct AggregatingTransformParams; +using AggregatingTransformParamsPtr = std::shared_ptr; + +class GroupingSetsStep : public ITransformingStep +{ +public: + GroupingSetsStep(const DataStream & input_stream_, AggregatingTransformParamsPtr params_); + + String getName() const override { return "GroupingSets"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + +private: + AggregatingTransformParamsPtr params; +}; + +} diff --git a/src/Processors/Transforms/GroupingSetsTransform.cpp b/src/Processors/Transforms/GroupingSetsTransform.cpp new file mode 100644 index 00000000000..9672b8e32c2 --- /dev/null +++ b/src/Processors/Transforms/GroupingSetsTransform.cpp @@ -0,0 +1,76 @@ +#include +#include + +namespace DB +{ + +GroupingSetsTransform::GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params_) + : IAccumulatingTransform(std::move(header), params_->getHeader()) + , params(std::move(params_)) + , keys(params->params.keys) + , keys_vector(params->params.keys_vector) + , keys_vector_idx(0) +{ +} + +void GroupingSetsTransform::consume(Chunk chunk) +{ + consumed_chunks.emplace_back(std::move(chunk)); +} + +Chunk GroupingSetsTransform::merge(Chunks && chunks, bool final) +{ + BlocksList grouping_sets_blocks; + for (auto & chunk : chunks) + grouping_sets_blocks.emplace_back(getInputPort().getHeader().cloneWithColumns(chunk.detachColumns())); + + auto grouping_sets_block = params->aggregator.mergeBlocks(grouping_sets_blocks, final); + auto num_rows = grouping_sets_block.rows(); + return Chunk(grouping_sets_block.getColumns(), num_rows); +} + +Chunk GroupingSetsTransform::generate() +{ + if (!consumed_chunks.empty()) + { + if (consumed_chunks.size() > 1) + grouping_sets_chunk = merge(std::move(consumed_chunks), false); + else + grouping_sets_chunk = std::move(consumed_chunks.front()); + + consumed_chunks.clear(); + + auto num_rows = grouping_sets_chunk.getNumRows(); + + current_columns = grouping_sets_chunk.getColumns(); + current_zero_columns.clear(); + + for (auto key : keys) + current_zero_columns.emplace(key, current_columns[key]->cloneEmpty()->cloneResized(num_rows)); + } + + Chunk gen_chunk; + + if (keys_vector_idx < keys_vector.size()) + { + auto columns = current_columns; + std::set key_vector(keys_vector[keys_vector_idx].begin(), keys_vector[keys_vector_idx].end()); + + for (auto key : keys) + { + if (!key_vector.contains(key)) + columns[key] = current_zero_columns[key]; + } + + Chunks chunks; + chunks.emplace_back(std::move(columns), current_columns.front()->size()); + gen_chunk = merge(std::move(chunks), false); + + ++keys_vector_idx; + } + + finalizeChunk(gen_chunk); + return gen_chunk; +} + +} diff --git a/src/Processors/Transforms/GroupingSetsTransform.h b/src/Processors/Transforms/GroupingSetsTransform.h new file mode 100644 index 00000000000..4019b2ffa52 --- /dev/null +++ b/src/Processors/Transforms/GroupingSetsTransform.h @@ -0,0 +1,35 @@ +#pragma once +#include +#include + +namespace DB +{ + +class GroupingSetsTransform : public IAccumulatingTransform +{ +public: + GroupingSetsTransform(Block header, AggregatingTransformParamsPtr params); + String getName() const override { return "GroupingSetsTransform"; } + +protected: + void consume(Chunk chunk) override; + Chunk generate() override; + +private: + AggregatingTransformParamsPtr params; + ColumnNumbers keys; + ColumnNumbersTwoDimension keys_vector; + + Chunks consumed_chunks; + Chunk grouping_sets_chunk; + Columns current_columns; + std::unordered_map current_zero_columns; + + UInt64 keys_vector_idx = 0; + + Poco::Logger * log = &Poco::Logger::get("GroupingSetsTransform"); + + Chunk merge(Chunks && chunks, bool final); +}; + +} diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index fd128444ebc..258fa6f5abd 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -1,20 +1,69 @@ - 1 40 4 - 2 80 4 -a 0 70 4 -b 0 50 4 - 1 40 4 - 2 80 4 -a 0 70 4 -b 0 50 4 +1 0 1 4500 +1 0 3 4700 +1 0 5 4900 +1 0 7 5100 +1 0 9 5300 +1 1 0 4500 +1 2 0 5100 +1 3 0 4700 +1 4 0 5300 +1 5 0 4900 +2 0 2 4600 +2 0 4 4800 +2 0 6 5000 +2 0 8 5200 +2 0 10 5400 +2 1 0 5000 +2 2 0 4600 +2 3 0 5200 +2 4 0 4800 +2 5 0 5400 - 0 240 16 - 1 40 4 - 2 80 4 -a 0 70 4 -b 0 50 4 - 1 40 4 - 2 80 4 -a 0 70 4 -b 0 50 4 +0 0 1 1 4500 +0 0 2 2 4600 +0 0 3 3 4700 +0 0 4 4 4800 +0 0 5 5 4900 +0 0 6 6 5000 +0 0 7 7 5100 +0 0 8 8 5200 +0 0 9 9 5300 +0 0 10 10 5400 +1 1 0 0 4500 +1 2 0 0 5100 +1 3 0 0 4700 +1 4 0 0 5300 +1 5 0 0 4900 +2 1 0 0 5000 +2 2 0 0 4600 +2 3 0 0 5200 +2 4 0 0 4800 +2 5 0 0 5400 - 0 240 16 +1 0 24500 +1 1 4500 +1 3 4700 +1 5 4900 +1 7 5100 +1 9 5300 +2 0 25000 +2 2 4600 +2 4 4800 +2 6 5000 +2 8 5200 +2 10 5400 +0 0 49500 + +1 0 24500 +1 1 4500 +1 3 4700 +1 5 4900 +1 7 5100 +1 9 5300 +2 0 25000 +2 2 4600 +2 4 4800 +2 6 5000 +2 8 5200 +2 10 5400 +0 0 49500 diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.sql b/tests/queries/0_stateless/01883_with_grouping_sets.sql index 6270b772328..274eebef5ed 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.sql +++ b/tests/queries/0_stateless/01883_with_grouping_sets.sql @@ -1,20 +1,38 @@ DROP TABLE IF EXISTS grouping_sets; -CREATE TABLE grouping_sets(a String, b Int32, s Int32) ENGINE = Memory; -INSERT INTO grouping_sets VALUES ('a', 1, 10), ('a', 1, 15), ('a', 2, 20); -INSERT INTO grouping_sets VALUES ('a', 2, 25), ('b', 1, 10), ('b', 1, 5); -INSERT INTO grouping_sets VALUES ('b', 2, 20), ('b', 2, 15); +CREATE TABLE grouping_sets(fact_1_id Int32, fact_2_id Int32, fact_3_id Int32, fact_4_id Int32, sales_value Int32) ENGINE = Memory; -SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) ORDER BY a, b; +INSERT INTO grouping_sets +SELECT + number % 2 + 1 AS fact_1_id, + number % 5 + 1 AS fact_2_id, + number % 10 + 1 AS fact_3_id, + number % 10 + 1 AS fact_4_id, + number % 100 AS sales_value +FROM system.numbers limit 1000; -SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) WITH TOTALS ORDER BY a, b; +SELECT fact_1_id, fact_2_id, fact_3_id, SUM(sales_value) AS sales_value from grouping_sets +GROUP BY GROUPING SETS((fact_1_id, fact_2_id), (fact_1_id, fact_3_id)) +ORDER BY fact_1_id, fact_2_id, fact_3_id; -SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS ORDER BY a, b; +SELECT fact_1_id, fact_2_id, fact_3_id, fact_4_id, SUM(sales_value) AS sales_value from grouping_sets +GROUP BY GROUPING SETS((fact_1_id, fact_2_id), (fact_3_id, fact_4_id)) +ORDER BY fact_1_id, fact_2_id, fact_3_id, fact_4_id; -SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS WITH TOTALS ORDER BY a, b; +SELECT + fact_1_id, + fact_3_id, + SUM(sales_value) AS sales_value +FROM grouping_sets +GROUP BY grouping sets((fact_1_id), (fact_1_id, fact_3_id)) WITH TOTALS +ORDER BY fact_1_id, fact_3_id; --- not sure that always works --- SET group_by_two_level_threshold = 1; --- SELECT a, b, sum(s), count() from grouping_sets GROUP BY a, b WITH GROUPING SETS ORDER BY a, b; +SELECT + fact_1_id, + fact_3_id, + SUM(sales_value) AS sales_value +FROM grouping_sets +GROUP BY grouping sets(fact_1_id, (fact_1_id, fact_3_id)) WITH TOTALS +ORDER BY fact_1_id, fact_3_id; DROP TABLE grouping_sets; From 07a986fbf1888ec63d570367e429b34d0de4e0b8 Mon Sep 17 00:00:00 2001 From: fanzhou Date: Sat, 28 Aug 2021 12:10:41 +0800 Subject: [PATCH 097/358] fix bug when no data in grouping sets transformer --- src/Processors/Transforms/GroupingSetsTransform.cpp | 2 +- tests/queries/0_stateless/01883_with_grouping_sets.reference | 2 ++ tests/queries/0_stateless/01883_with_grouping_sets.sql | 4 ++++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Processors/Transforms/GroupingSetsTransform.cpp b/src/Processors/Transforms/GroupingSetsTransform.cpp index 9672b8e32c2..960ea0e5d76 100644 --- a/src/Processors/Transforms/GroupingSetsTransform.cpp +++ b/src/Processors/Transforms/GroupingSetsTransform.cpp @@ -51,7 +51,7 @@ Chunk GroupingSetsTransform::generate() Chunk gen_chunk; - if (keys_vector_idx < keys_vector.size()) + if (!current_columns.empty() && keys_vector_idx < keys_vector.size()) { auto columns = current_columns; std::set key_vector(keys_vector[keys_vector_idx].begin(), keys_vector[keys_vector_idx].end()); diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index 258fa6f5abd..5d5341dd093 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -67,3 +67,5 @@ 2 8 5200 2 10 5400 0 0 49500 + + diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.sql b/tests/queries/0_stateless/01883_with_grouping_sets.sql index 274eebef5ed..24350143849 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.sql +++ b/tests/queries/0_stateless/01883_with_grouping_sets.sql @@ -35,4 +35,8 @@ FROM grouping_sets GROUP BY grouping sets(fact_1_id, (fact_1_id, fact_3_id)) WITH TOTALS ORDER BY fact_1_id, fact_3_id; +truncate grouping_sets; + +SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) ORDER BY a, b; + DROP TABLE grouping_sets; From ac6f08edf3742c14a138ff079b08b0f5f2dc5694 Mon Sep 17 00:00:00 2001 From: fanzhou Date: Sun, 29 Aug 2021 15:13:53 +0800 Subject: [PATCH 098/358] correct test case results --- .../queries/0_stateless/01883_with_grouping_sets.reference | 7 ++----- tests/queries/0_stateless/01883_with_grouping_sets.sql | 4 ++-- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index 5d5341dd093..a1d55448f3f 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -18,7 +18,6 @@ 2 3 0 5200 2 4 0 4800 2 5 0 5400 - 0 0 1 1 4500 0 0 2 2 4600 0 0 3 3 4700 @@ -39,7 +38,6 @@ 2 3 0 0 5200 2 4 0 0 4800 2 5 0 0 5400 - 1 0 24500 1 1 4500 1 3 4700 @@ -52,8 +50,8 @@ 2 6 5000 2 8 5200 2 10 5400 -0 0 49500 +0 0 49500 1 0 24500 1 1 4500 1 3 4700 @@ -66,6 +64,5 @@ 2 6 5000 2 8 5200 2 10 5400 -0 0 49500 - +0 0 49500 \ No newline at end of file diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.sql b/tests/queries/0_stateless/01883_with_grouping_sets.sql index 24350143849..fffe4314706 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.sql +++ b/tests/queries/0_stateless/01883_with_grouping_sets.sql @@ -4,7 +4,7 @@ CREATE TABLE grouping_sets(fact_1_id Int32, fact_2_id Int32, fact_3_id Int32, fa INSERT INTO grouping_sets SELECT - number % 2 + 1 AS fact_1_id, + number % 2 + 1 AS fact_1_id, number % 5 + 1 AS fact_2_id, number % 10 + 1 AS fact_3_id, number % 10 + 1 AS fact_4_id, @@ -37,6 +37,6 @@ ORDER BY fact_1_id, fact_3_id; truncate grouping_sets; -SELECT a, b, sum(s), count() from grouping_sets GROUP BY GROUPING SETS(a, b) ORDER BY a, b; +SELECT fact_1_id, fact_3_id, sum(sales_value), count() from grouping_sets GROUP BY GROUPING SETS(fact_1_id, fact_3_id) ORDER BY fact_1_id, fact_3_id; DROP TABLE grouping_sets; From 43db4594baec2ce607f87a5378b6476fb2462a51 Mon Sep 17 00:00:00 2001 From: fanzhou Date: Mon, 6 Sep 2021 10:18:42 +0800 Subject: [PATCH 099/358] some changes --- src/Core/ColumnNumbers.h | 2 +- src/Core/NamesAndTypes.h | 2 +- src/Interpreters/Aggregator.h | 4 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.h | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 79 ++++++------------- .../Transforms/GroupingSetsTransform.h | 2 +- src/QueryPipeline/Pipe.cpp | 4 +- .../01883_with_grouping_sets.reference | 7 +- 9 files changed, 40 insertions(+), 66 deletions(-) diff --git a/src/Core/ColumnNumbers.h b/src/Core/ColumnNumbers.h index 82868ef3cd3..29b4c49dc83 100644 --- a/src/Core/ColumnNumbers.h +++ b/src/Core/ColumnNumbers.h @@ -8,6 +8,6 @@ namespace DB { using ColumnNumbers = std::vector; -using ColumnNumbersTwoDimension = std::vector; +using ColumnNumbersList = std::vector; } diff --git a/src/Core/NamesAndTypes.h b/src/Core/NamesAndTypes.h index 787dd629190..c636b1b5cf3 100644 --- a/src/Core/NamesAndTypes.h +++ b/src/Core/NamesAndTypes.h @@ -107,7 +107,7 @@ public: std::optional tryGetByName(const std::string & name) const; }; -using TwoDimensionNamesAndTypesList = std::list; +using NamesAndTypesLists = std::vector; } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 87da0d7492d..d64a181f3ca 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -879,7 +879,7 @@ public: /// What to count. ColumnNumbers keys; - const ColumnNumbersTwoDimension keys_vector; + const ColumnNumbersList keys_vector; const AggregateDescriptions aggregates; size_t keys_size; const size_t aggregates_size; @@ -943,7 +943,7 @@ public: Params( const Block & src_header_, const ColumnNumbers & keys_, - const ColumnNumbersTwoDimension & keys_vector_, const AggregateDescriptions & aggregates_, + const ColumnNumbersList & keys_vector_, const AggregateDescriptions & aggregates_, bool overflow_row_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, size_t group_by_two_level_threshold_, size_t group_by_two_level_threshold_bytes_, size_t max_bytes_before_external_group_by_, diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index ba38ee10d8c..cfe187babf6 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -401,7 +401,7 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) } } - aggregation_keys_list.push_back(grouping_set_list); + aggregation_keys_list.push_back(std::move(grouping_set_list)); } else { diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 7598efa40c6..19054e795a6 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -64,8 +64,8 @@ struct ExpressionAnalyzerData bool has_aggregation = false; NamesAndTypesList aggregation_keys; + NamesAndTypesLists aggregation_keys_list; bool has_const_aggregation_keys = false; - TwoDimensionNamesAndTypesList aggregation_keys_list; AggregateDescriptions aggregate_descriptions; WindowDescriptions window_descriptions; @@ -324,7 +324,7 @@ public: const NamesAndTypesList & aggregationKeys() const { return aggregation_keys; } bool hasConstAggregationKeys() const { return has_const_aggregation_keys; } - const TwoDimensionNamesAndTypesList & aggregationKeysList() const { return aggregation_keys_list; } + const NamesAndTypesLists & aggregationKeysList() const { return aggregation_keys_list; } const AggregateDescriptions & aggregates() const { return aggregate_descriptions; } const PreparedSets & getPreparedSets() const { return prepared_sets; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 088f8eb1fd0..0a72b99fa1c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2058,61 +2058,35 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac return; const auto & header_before_aggregation = query_plan.getCurrentDataStream().header; - ColumnNumbers keys; - ColumnNumbers all_keys; - ColumnNumbersTwoDimension keys_vector; - auto & query = getSelectQuery(); - if (query.group_by_with_grouping_sets) - { - std::set keys_set; - for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) - { - keys.clear(); - for (const auto & key : aggregation_keys) - { - size_t key_name_pos = header_before_aggregation.getPositionByName(key.name); - if (!keys_set.count(key_name_pos)) - { - keys_set.insert(key_name_pos); - } - keys.push_back(key_name_pos); - LOG_DEBUG( - log, - "execute aggregation with grouping sets add key with name {} and number {}", - key.name, - header_before_aggregation.getPositionByName(key.name)); - } - keys_vector.push_back(keys); - LOG_DEBUG( - log, - "execute aggregation with grouping sets add keys set of size {}", - keys.size()); - } - all_keys.assign(keys_set.begin(), keys_set.end()); - LOG_DEBUG( - log, - "execute aggregation with grouping sets add all keys of size {}", - all_keys.size()); - } - else - { - for (const auto & key : query_analyzer->aggregationKeys()) - { - keys.push_back(header_before_aggregation.getPositionByName(key.name)); - LOG_DEBUG(log, "execute aggregation without grouping sets pushed back key with name {} and number {}", key.name, header_before_aggregation.getPositionByName(key.name)); - } - } AggregateDescriptions aggregates = query_analyzer->aggregates(); for (auto & descr : aggregates) if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); - const Settings & settings = context->getSettingsRef(); std::shared_ptr params_ptr; + + auto & query = getSelectQuery(); if (query.group_by_with_grouping_sets) { + ColumnNumbers keys; + ColumnNumbers all_keys; + ColumnNumbersList keys_vector; + std::unordered_set keys_set; + for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) + { + keys.clear(); + for (const auto & key : aggregation_keys) + { + size_t key_name_pos = header_before_aggregation.getPositionByName(key.name); + keys_set.insert(key_name_pos); + keys.push_back(key_name_pos); + } + keys_vector.push_back(keys); + } + all_keys.assign(keys_set.begin(), keys_set.end()); + params_ptr = std::make_shared( header_before_aggregation, all_keys, @@ -2135,6 +2109,10 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac } else { + ColumnNumbers keys; + for (const auto & key : query_analyzer->aggregationKeys()) + keys.push_back(header_before_aggregation.getPositionByName(key.name)); + params_ptr = std::make_shared( header_before_aggregation, keys, @@ -2156,13 +2134,10 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac } SortDescription group_by_sort_description; - if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) { + if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) group_by_sort_description = getSortDescriptionFromGroupBy(query); - LOG_DEBUG(log, "execute aggregation without grouping sets got group_by_sort_description"); - } else { + else group_by_info = nullptr; - LOG_DEBUG(log, "execute aggregation didn't get group_by_sort_description"); - } auto merge_threads = max_streams; auto temporary_data_merge_threads = settings.aggregation_memory_efficient_merge_threads @@ -2171,7 +2146,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac bool storage_has_evenly_distributed_read = storage && storage->hasEvenlyDistributedRead(); - LOG_DEBUG(log, "execute aggregation header structure before step: {}", query_plan.getCurrentDataStream().header.dumpStructure()); auto aggregating_step = std::make_unique( query_plan.getCurrentDataStream(), *params_ptr, @@ -2183,7 +2157,6 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac storage_has_evenly_distributed_read, std::move(group_by_info), std::move(group_by_sort_description)); - LOG_DEBUG(log, "execute aggregation header structure after step: {}", aggregating_step->getOutputStream().header.dumpStructure()); query_plan.addStep(std::move(aggregating_step)); } @@ -2241,7 +2214,7 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific const auto & header_before_transform = query_plan.getCurrentDataStream().header; ColumnNumbers keys; ColumnNumbers all_keys; - ColumnNumbersTwoDimension keys_vector; + ColumnNumbersList keys_vector; auto & query = getSelectQuery(); if (query.group_by_with_grouping_sets) { diff --git a/src/Processors/Transforms/GroupingSetsTransform.h b/src/Processors/Transforms/GroupingSetsTransform.h index 4019b2ffa52..f9c688b2cc8 100644 --- a/src/Processors/Transforms/GroupingSetsTransform.h +++ b/src/Processors/Transforms/GroupingSetsTransform.h @@ -18,7 +18,7 @@ protected: private: AggregatingTransformParamsPtr params; ColumnNumbers keys; - ColumnNumbersTwoDimension keys_vector; + ColumnNumbersList keys_vector; Chunks consumed_chunks; Chunk grouping_sets_chunk; diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 184cad18c17..dd4a2cb6c77 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -464,9 +464,9 @@ void Pipe::addParallelTransforms(Processors transforms) "but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR); size_t next_output = 0; - for (auto * input : inputs) + for (size_t i = 0; i < inputs.size(); ++i) { - connect(*output_ports[next_output], *input); + connect(*output_ports[next_output], *inputs[i]); ++next_output; } diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index a1d55448f3f..258fa6f5abd 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -18,6 +18,7 @@ 2 3 0 5200 2 4 0 4800 2 5 0 5400 + 0 0 1 1 4500 0 0 2 2 4600 0 0 3 3 4700 @@ -38,6 +39,7 @@ 2 3 0 0 5200 2 4 0 0 4800 2 5 0 0 5400 + 1 0 24500 1 1 4500 1 3 4700 @@ -50,8 +52,8 @@ 2 6 5000 2 8 5200 2 10 5400 - 0 0 49500 + 1 0 24500 1 1 4500 1 3 4700 @@ -64,5 +66,4 @@ 2 6 5000 2 8 5200 2 10 5400 - -0 0 49500 \ No newline at end of file +0 0 49500 From e725630fbfbf1dede22d6e8daed3a60dd3e270b2 Mon Sep 17 00:00:00 2001 From: fanzhou Date: Mon, 6 Sep 2021 11:05:05 +0800 Subject: [PATCH 100/358] refactor --- src/Interpreters/InterpreterSelectQuery.cpp | 42 +++++++-------------- 1 file changed, 13 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0a72b99fa1c..68d4b5fec7c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2212,50 +2212,30 @@ void InterpreterSelectQuery::executeTotalsAndHaving( void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modificator modificator) { const auto & header_before_transform = query_plan.getCurrentDataStream().header; - ColumnNumbers keys; - ColumnNumbers all_keys; - ColumnNumbersList keys_vector; + + const Settings & settings = context->getSettingsRef(); + std::shared_ptr params_ptr; + auto & query = getSelectQuery(); if (query.group_by_with_grouping_sets) { - std::set keys_set; + ColumnNumbers keys; + ColumnNumbers all_keys; + ColumnNumbersList keys_vector; + std::unordered_set keys_set; for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) { keys.clear(); for (const auto & key : aggregation_keys) { size_t key_name_pos = header_before_transform.getPositionByName(key.name); - if (!keys_set.count(key_name_pos)) - { - keys_set.insert(key_name_pos); - } + keys_set.insert(key_name_pos); keys.push_back(key_name_pos); } keys_vector.push_back(keys); - LOG_DEBUG( - log, - "execute aggregation with grouping sets add keys set of size {}", - keys.size()); } all_keys.assign(keys_set.begin(), keys_set.end()); - LOG_DEBUG( - log, - "execute aggregation with grouping sets add all keys of size {}", - all_keys.size()); - } - else - { - for (const auto & key : query_analyzer->aggregationKeys()) - { - keys.push_back(header_before_transform.getPositionByName(key.name)); - } - } - const Settings & settings = context->getSettingsRef(); - - std::shared_ptr params_ptr; - if (query.group_by_with_grouping_sets) - { params_ptr = std::make_shared( header_before_transform, all_keys, @@ -2276,6 +2256,10 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modific } else { + ColumnNumbers keys; + for (const auto & key : query_analyzer->aggregationKeys()) + keys.push_back(header_before_transform.getPositionByName(key.name)); + params_ptr = std::make_shared( header_before_transform, keys, From b94f8878ff2ef423dda8ebeabd8b781966b8d287 Mon Sep 17 00:00:00 2001 From: fanzhou Date: Sat, 11 Sep 2021 11:08:22 +0800 Subject: [PATCH 101/358] 1. refactor ExpressionAnalyzer 2. Update formatAST to support grouping sets query with distributed table 3. modify astExpression to support function in grouping sets --- src/Interpreters/ExpressionAnalyzer.cpp | 13 +- src/Interpreters/InterpreterSelectQuery.cpp | 213 +++++++----------- src/Interpreters/InterpreterSelectQuery.h | 7 + src/Parsers/ASTExpressionList.cpp | 87 +++++-- src/Parsers/ASTSelectQuery.cpp | 15 +- src/Parsers/ExpressionListParsers.cpp | 5 +- .../01883_with_grouping_sets.reference | 24 ++ .../0_stateless/01883_with_grouping_sets.sql | 18 +- 8 files changed, 214 insertions(+), 168 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index cfe187babf6..72a85bbfa0f 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -351,15 +351,10 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) if (select_query->group_by_with_grouping_sets) { ASTs group_elements_ast; - if (group_asts[i]->as()) - { - group_elements_ast = group_asts[i]->as()->children; - } - else - { - const auto id_ast = group_asts[i]->as(); - group_elements_ast.push_back(std::make_shared(id_ast)); - } + const ASTExpressionList * group_ast_element = group_asts[i]->as(); + if (!group_ast_element) + throw Exception("Grouping Sets element " + group_asts[i]->getColumnName() + " should be an expression type", ErrorCodes::UNKNOWN_IDENTIFIER); + group_elements_ast = group_ast_element->children; NamesAndTypesList grouping_set_list; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 68d4b5fec7c..90df6a47495 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2037,7 +2037,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc } } - void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter) { auto where_step = std::make_unique( @@ -2047,6 +2046,79 @@ void InterpreterSelectQuery::executeWhere(QueryPlan & query_plan, const ActionsD query_plan.addStep(std::move(where_step)); } +void InterpreterSelectQuery::initAggregatorParams( + const Block & current_data_stream_header, + AggregatorParamsPtr & params_ptr, + const AggregateDescriptions & aggregates, + bool overflow_row, const Settings & settings, + size_t group_by_two_level_threshold, size_t group_by_two_level_threshold_bytes) +{ + auto & query = getSelectQuery(); + if (query.group_by_with_grouping_sets) + { + ColumnNumbers keys; + ColumnNumbers all_keys; + ColumnNumbersList keys_vector; + std::unordered_set keys_set; + for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) + { + keys.clear(); + for (const auto & key : aggregation_keys) + { + size_t key_name_pos = current_data_stream_header.getPositionByName(key.name); + keys_set.insert(key_name_pos); + keys.push_back(key_name_pos); + } + keys_vector.push_back(keys); + } + all_keys.assign(keys_set.begin(), keys_set.end()); + + params_ptr = std::make_unique( + current_data_stream_header, + all_keys, + keys_vector, + aggregates, + overflow_row, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + group_by_two_level_threshold, + group_by_two_level_threshold_bytes, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() + && query_analyzer->hasConstAggregationKeys()), + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); + } + else + { + ColumnNumbers keys; + for (const auto & key : query_analyzer->aggregationKeys()) + keys.push_back(current_data_stream_header.getPositionByName(key.name)); + + params_ptr = std::make_unique( + current_data_stream_header, + keys, + aggregates, + overflow_row, + settings.max_rows_to_group_by, + settings.group_by_overflow_mode, + group_by_two_level_threshold, + group_by_two_level_threshold_bytes, + settings.max_bytes_before_external_group_by, + settings.empty_result_for_aggregation_by_empty_set + || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() + && query_analyzer->hasConstAggregationKeys()), + context->getTemporaryVolume(), + settings.max_threads, + settings.min_free_disk_space_for_temporary_data, + settings.compile_aggregate_expressions, + settings.min_count_to_compile_aggregate_expression); + } +} void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { @@ -2064,78 +2136,17 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); + const Settings & settings = context->getSettingsRef(); - std::shared_ptr params_ptr; - auto & query = getSelectQuery(); - if (query.group_by_with_grouping_sets) - { - ColumnNumbers keys; - ColumnNumbers all_keys; - ColumnNumbersList keys_vector; - std::unordered_set keys_set; - for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) - { - keys.clear(); - for (const auto & key : aggregation_keys) - { - size_t key_name_pos = header_before_aggregation.getPositionByName(key.name); - keys_set.insert(key_name_pos); - keys.push_back(key_name_pos); - } - keys_vector.push_back(keys); - } - all_keys.assign(keys_set.begin(), keys_set.end()); + AggregatorParamsPtr params_ptr; + initAggregatorParams(header_before_aggregation, params_ptr, aggregates, overflow_row, settings, + settings.group_by_two_level_threshold, settings.group_by_two_level_threshold_bytes); - params_ptr = std::make_shared( - header_before_aggregation, - all_keys, - keys_vector, - aggregates, - overflow_row, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - settings.group_by_two_level_threshold, - settings.group_by_two_level_threshold_bytes, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set - || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() - && query_analyzer->hasConstAggregationKeys()), - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); - } - else - { - ColumnNumbers keys; - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header_before_aggregation.getPositionByName(key.name)); - - params_ptr = std::make_shared( - header_before_aggregation, - keys, - aggregates, - overflow_row, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - settings.group_by_two_level_threshold, - settings.group_by_two_level_threshold_bytes, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set - || (settings.empty_result_for_aggregation_by_constant_keys_on_empty_set && keys.empty() - && query_analyzer->hasConstAggregationKeys()), - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); - } SortDescription group_by_sort_description; - if (group_by_info && settings.optimize_aggregation_in_order && !query.group_by_with_grouping_sets) - group_by_sort_description = getSortDescriptionFromGroupBy(query); + if (group_by_info && settings.optimize_aggregation_in_order) + group_by_sort_description = getSortDescriptionFromGroupBy(getSelectQuery()); else group_by_info = nullptr; @@ -2208,76 +2219,14 @@ void InterpreterSelectQuery::executeTotalsAndHaving( query_plan.addStep(std::move(totals_having_step)); } - void InterpreterSelectQuery::executeRollupOrCube(QueryPlan & query_plan, Modificator modificator) { const auto & header_before_transform = query_plan.getCurrentDataStream().header; const Settings & settings = context->getSettingsRef(); - std::shared_ptr params_ptr; - - auto & query = getSelectQuery(); - if (query.group_by_with_grouping_sets) - { - ColumnNumbers keys; - ColumnNumbers all_keys; - ColumnNumbersList keys_vector; - std::unordered_set keys_set; - for (const auto & aggregation_keys : query_analyzer->aggregationKeysList()) - { - keys.clear(); - for (const auto & key : aggregation_keys) - { - size_t key_name_pos = header_before_transform.getPositionByName(key.name); - keys_set.insert(key_name_pos); - keys.push_back(key_name_pos); - } - keys_vector.push_back(keys); - } - all_keys.assign(keys_set.begin(), keys_set.end()); - - params_ptr = std::make_shared( - header_before_transform, - all_keys, - keys_vector, - query_analyzer->aggregates(), - false, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - 0, - 0, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); - } - else - { - ColumnNumbers keys; - for (const auto & key : query_analyzer->aggregationKeys()) - keys.push_back(header_before_transform.getPositionByName(key.name)); - - params_ptr = std::make_shared( - header_before_transform, - keys, - query_analyzer->aggregates(), - false, - settings.max_rows_to_group_by, - settings.group_by_overflow_mode, - 0, - 0, - settings.max_bytes_before_external_group_by, - settings.empty_result_for_aggregation_by_empty_set, - context->getTemporaryVolume(), - settings.max_threads, - settings.min_free_disk_space_for_temporary_data, - settings.compile_aggregate_expressions, - settings.min_count_to_compile_aggregate_expression); - } + AggregatorParamsPtr params_ptr; + initAggregatorParams(header_before_transform, params_ptr, query_analyzer->aggregates(), false, settings, 0, 0); auto transform_params = std::make_shared(*params_ptr, true); QueryPlanStepPtr step; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 71a8ac5e1d5..7f75c4ea41f 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -30,6 +30,7 @@ class QueryPlan; struct TreeRewriterResult; using TreeRewriterResultPtr = std::shared_ptr; +using AggregatorParamsPtr = std::unique_ptr; /** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage. @@ -127,6 +128,12 @@ private: /// Different stages of query execution. + void initAggregatorParams( + const Block & current_data_stream_header, + AggregatorParamsPtr & params_ptr, + const AggregateDescriptions & aggregates, + bool overflow_row, const Settings & settings, + size_t group_by_two_level_threshold, size_t group_by_two_level_threshold_bytes); void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan); void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); void executeAggregation( diff --git a/src/Parsers/ASTExpressionList.cpp b/src/Parsers/ASTExpressionList.cpp index 2724465537f..453624aa794 100644 --- a/src/Parsers/ASTExpressionList.cpp +++ b/src/Parsers/ASTExpressionList.cpp @@ -17,16 +17,38 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState if (frame.expression_list_prepend_whitespace) settings.ostr << ' '; - for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + if (frame.need_parens) { - if (it != children.begin()) + settings.ostr << "("; + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { - if (separator) - settings.ostr << separator; - settings.ostr << ' '; - } + if (it != children.begin()) + { + if (separator) + settings.ostr << separator; + settings.ostr << ' '; + } - (*it)->formatImpl(settings, state, frame); + settings.ostr << "("; + FormatStateStacked frame_nested = frame; + frame_nested.need_parens = false; + (*it)->formatImpl(settings, state, frame_nested); + settings.ostr << ")"; + } + settings.ostr << ")"; + } + else + { + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + { + if (it != children.begin()) + { + if (separator) + settings.ostr << separator; + settings.ostr << ' '; + } + (*it)->formatImpl(settings, state, frame); + } } } @@ -41,20 +63,53 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For } ++frame.indent; - for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + + if (frame.need_parens) { - if (it != children.begin()) + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { - if (separator) - settings.ostr << separator; + if (it != children.begin()) + { + if (separator) + settings.ostr << separator; + } + + if (children.size() > 1 || frame.expression_list_always_start_on_new_line) + settings.ostr << indent_str; + + if (it == children.begin()) + { + settings.ostr << "("; + } + + FormatStateStacked frame_nested = frame; + frame_nested.expression_list_always_start_on_new_line = false; + frame_nested.expression_list_prepend_whitespace = false; + frame_nested.need_parens = false; + settings.ostr << "("; + (*it)->formatImpl(settings, state, frame_nested); + settings.ostr << ")"; } + settings.ostr << ")"; + } + else + { + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + { + if (it != children.begin()) + { + if (separator) + settings.ostr << separator; + } - if (children.size() > 1 || frame.expression_list_always_start_on_new_line) - settings.ostr << indent_str; + if (children.size() > 1 || frame.expression_list_always_start_on_new_line) + settings.ostr << indent_str; - FormatStateStacked frame_nested = frame; - frame_nested.expression_list_always_start_on_new_line = false; - (*it)->formatImpl(settings, state, frame_nested); + FormatStateStacked frame_nested = frame; + frame_nested.expression_list_always_start_on_new_line = false; + + (*it)->formatImpl(settings, state, frame_nested); + } } } diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index f07a4a328e2..3a4f8d79662 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -114,9 +114,12 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (groupBy()) { s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "GROUP BY" << (s.hilite ? hilite_none : ""); - s.one_line + if (!group_by_with_grouping_sets) + { + s.one_line ? groupBy()->formatImpl(s, state, frame) : groupBy()->as().formatImplMultiline(s, state, frame); + } } if (group_by_with_rollup) @@ -126,7 +129,15 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH CUBE" << (s.hilite ? hilite_none : ""); if (group_by_with_grouping_sets) - s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH GROUPING SETS" << (s.hilite ? hilite_none : ""); + { + bool tmp_need_parens = frame.need_parens; + frame.need_parens = true; + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "GROUPING SETS" << (s.hilite ? hilite_none : ""); + s.one_line + ? groupBy()->formatImpl(s, state, frame) + : groupBy()->as().formatImplMultiline(s, state, frame); + frame.need_parens = tmp_need_parens; + } if (group_by_with_totals) s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "WITH TOTALS" << (s.hilite ? hilite_none : ""); diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 12c8747a43b..bd7eb9caa4d 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -762,7 +762,7 @@ bool ParserGroupingSetsExpressionListElements::parseImpl(Pos & pos, ASTPtr & nod ParserToken s_close(TokenType::ClosingRoundBracket); ParserExpressionWithOptionalAlias p_expression(false); ParserList p_command(std::make_unique(false), - std::make_unique(TokenType::Comma), false); + std::make_unique(TokenType::Comma), true); do { @@ -775,6 +775,9 @@ bool ParserGroupingSetsExpressionListElements::parseImpl(Pos & pos, ASTPtr & nod { return false; } + auto list = std::make_shared(','); + list->children.push_back(command); + command = std::move(list); } else { diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index 258fa6f5abd..82fa6f9e57d 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -1,3 +1,5 @@ + + 1 0 1 4500 1 0 3 4700 1 0 5 4900 @@ -40,6 +42,28 @@ 2 4 0 0 4800 2 5 0 0 5400 +0 0 0 49500 +0 0 1 4500 +0 0 2 4600 +0 0 3 4700 +0 0 4 4800 +0 0 5 4900 +0 0 6 5000 +0 0 7 5100 +0 0 8 5200 +0 0 9 5300 +0 0 10 5400 +1 1 0 4500 +1 2 0 5100 +1 3 0 4700 +1 4 0 5300 +1 5 0 4900 +2 1 0 5000 +2 2 0 4600 +2 3 0 5200 +2 4 0 4800 +2 5 0 5400 + 1 0 24500 1 1 4500 1 3 4700 diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.sql b/tests/queries/0_stateless/01883_with_grouping_sets.sql index fffe4314706..92a56be0a3f 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.sql +++ b/tests/queries/0_stateless/01883_with_grouping_sets.sql @@ -2,6 +2,8 @@ DROP TABLE IF EXISTS grouping_sets; CREATE TABLE grouping_sets(fact_1_id Int32, fact_2_id Int32, fact_3_id Int32, fact_4_id Int32, sales_value Int32) ENGINE = Memory; +SELECT fact_1_id, fact_3_id, sum(sales_value), count() from grouping_sets GROUP BY GROUPING SETS(fact_1_id, fact_3_id) ORDER BY fact_1_id, fact_3_id; + INSERT INTO grouping_sets SELECT number % 2 + 1 AS fact_1_id, @@ -12,19 +14,23 @@ SELECT FROM system.numbers limit 1000; SELECT fact_1_id, fact_2_id, fact_3_id, SUM(sales_value) AS sales_value from grouping_sets -GROUP BY GROUPING SETS((fact_1_id, fact_2_id), (fact_1_id, fact_3_id)) +GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_1_id, fact_3_id)) ORDER BY fact_1_id, fact_2_id, fact_3_id; SELECT fact_1_id, fact_2_id, fact_3_id, fact_4_id, SUM(sales_value) AS sales_value from grouping_sets -GROUP BY GROUPING SETS((fact_1_id, fact_2_id), (fact_3_id, fact_4_id)) +GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_3_id, fact_4_id)) ORDER BY fact_1_id, fact_2_id, fact_3_id, fact_4_id; +SELECT fact_1_id, fact_2_id, fact_3_id, SUM(sales_value) AS sales_value from grouping_sets +GROUP BY GROUPING SETS ((fact_1_id, fact_2_id), (fact_3_id), ()) +ORDER BY fact_1_id, fact_2_id, fact_3_id; + SELECT fact_1_id, fact_3_id, SUM(sales_value) AS sales_value FROM grouping_sets -GROUP BY grouping sets((fact_1_id), (fact_1_id, fact_3_id)) WITH TOTALS +GROUP BY grouping sets ((fact_1_id), (fact_1_id, fact_3_id)) WITH TOTALS ORDER BY fact_1_id, fact_3_id; SELECT @@ -32,11 +38,7 @@ SELECT fact_3_id, SUM(sales_value) AS sales_value FROM grouping_sets -GROUP BY grouping sets(fact_1_id, (fact_1_id, fact_3_id)) WITH TOTALS +GROUP BY grouping sets (fact_1_id, (fact_1_id, fact_3_id)) WITH TOTALS ORDER BY fact_1_id, fact_3_id; -truncate grouping_sets; - -SELECT fact_1_id, fact_3_id, sum(sales_value), count() from grouping_sets GROUP BY GROUPING SETS(fact_1_id, fact_3_id) ORDER BY fact_1_id, fact_3_id; - DROP TABLE grouping_sets; From 5526c1cc350a1e647ea419e3896866e6432c91db Mon Sep 17 00:00:00 2001 From: fanzhou Date: Mon, 13 Sep 2021 15:47:28 +0800 Subject: [PATCH 102/358] update test reference --- .../0_stateless/01883_with_grouping_sets.reference | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index 82fa6f9e57d..7a69064f567 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -1,5 +1,3 @@ - - 1 0 1 4500 1 0 3 4700 1 0 5 4900 @@ -20,7 +18,6 @@ 2 3 0 5200 2 4 0 4800 2 5 0 5400 - 0 0 1 1 4500 0 0 2 2 4600 0 0 3 3 4700 @@ -41,7 +38,6 @@ 2 3 0 0 5200 2 4 0 0 4800 2 5 0 0 5400 - 0 0 0 49500 0 0 1 4500 0 0 2 4600 @@ -63,7 +59,6 @@ 2 3 0 5200 2 4 0 4800 2 5 0 5400 - 1 0 24500 1 1 4500 1 3 4700 @@ -76,8 +71,8 @@ 2 6 5000 2 8 5200 2 10 5400 -0 0 49500 +0 0 49500 1 0 24500 1 1 4500 1 3 4700 @@ -90,4 +85,5 @@ 2 6 5000 2 8 5200 2 10 5400 -0 0 49500 + +0 0 49500 \ No newline at end of file From 3bcf9155417abe613257985b7b75040debf6fae7 Mon Sep 17 00:00:00 2001 From: fanzhou Date: Mon, 13 Sep 2021 16:31:46 +0800 Subject: [PATCH 103/358] update test reference --- tests/queries/0_stateless/01883_with_grouping_sets.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01883_with_grouping_sets.reference b/tests/queries/0_stateless/01883_with_grouping_sets.reference index 7a69064f567..e52f72b84b9 100644 --- a/tests/queries/0_stateless/01883_with_grouping_sets.reference +++ b/tests/queries/0_stateless/01883_with_grouping_sets.reference @@ -86,4 +86,4 @@ 2 8 5200 2 10 5400 -0 0 49500 \ No newline at end of file +0 0 49500 From f0691e21b174cd362ae2c3376f34fe8e37ad19b7 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 Dec 2021 18:25:36 +0300 Subject: [PATCH 104/358] fix test --- .../0_stateless/02117_show_create_table_system.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 2b391cd292e..35de7f8e82c 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -34,7 +34,7 @@ CREATE TABLE system.numbers_mt\n(\n `number` UInt64\n)\nENGINE = SystemNumber CREATE TABLE system.one\n(\n `dummy` UInt8\n)\nENGINE = SystemOne()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.part_moves_between_shards\n(\n `database` String,\n `table` String,\n `task_name` String,\n `task_uuid` UUID,\n `create_time` DateTime,\n `part_name` String,\n `part_uuid` UUID,\n `to_shard` String,\n `dst_part_name` String,\n `update_time` DateTime,\n `state` String,\n `rollback` UInt8,\n `num_tries` UInt32,\n `last_exception` String\n)\nENGINE = SystemShardMoves()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `serialization_kind` String,\n `subcolumns.names` Array(String),\n `subcolumns.types` Array(String),\n `subcolumns.serializations` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.privileges\n(\n `privilege` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.projection_parts\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' From bb12e064af4c5278364d1d12a8c59fa56f30191a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 Dec 2021 21:21:38 +0300 Subject: [PATCH 105/358] fix test --- .../queries/0_stateless/01780_column_sparse_alter.reference | 4 ++-- tests/queries/0_stateless/01780_column_sparse_alter.sql | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.reference b/tests/queries/0_stateless/01780_column_sparse_alter.reference index 61e6ee38c93..cec7af647b3 100644 --- a/tests/queries/0_stateless/01780_column_sparse_alter.reference +++ b/tests/queries/0_stateless/01780_column_sparse_alter.reference @@ -1,7 +1,7 @@ id Default u Sparse s Sparse -20 11 +182 155 id Default t Sparse -20 +182 diff --git a/tests/queries/0_stateless/01780_column_sparse_alter.sql b/tests/queries/0_stateless/01780_column_sparse_alter.sql index d350086a598..444a1f9cf43 100644 --- a/tests/queries/0_stateless/01780_column_sparse_alter.sql +++ b/tests/queries/0_stateless/01780_column_sparse_alter.sql @@ -8,9 +8,9 @@ SETTINGS ratio_of_defaults_for_sparse_serialization = 0.5; INSERT INTO t_sparse_alter SELECT number, - if (number % 10 = 0, number, 0), - if (number % 20 = 0, toString(number), '') -FROM numbers(200); + if (number % 11 = 0, number, 0), + if (number % 13 = 0, toString(number), '') +FROM numbers(2000); SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_sparse_alter' AND active ORDER BY name; From 7c90d4f8e3d38cfdeb2962105ab71ee6819041af Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Fri, 26 Nov 2021 16:00:09 +0200 Subject: [PATCH 106/358] refactor CI tests --- tests/ci/ast_fuzzer_check.py | 16 ++--- tests/ci/build_check.py | 14 +++-- tests/ci/build_report_check.py | 22 +++---- tests/ci/cherry_pick.py | 5 +- tests/ci/commit_status_helper.py | 5 +- tests/ci/compatibility_check.py | 11 ++-- tests/ci/docker_images_check.py | 10 ++-- tests/ci/docs_check.py | 12 ++-- tests/ci/docs_release.py | 11 ++-- tests/ci/env_helper.py | 18 ++++++ tests/ci/fast_test_check.py | 10 ++-- tests/ci/finish_check.py | 11 ++-- tests/ci/functional_test_check.py | 11 ++-- tests/ci/integration_test_check.py | 11 ++-- tests/ci/pr_info.py | 92 ++++++++++++++--------------- tests/ci/pvs_check.py | 17 ++++-- tests/ci/run_check.py | 10 ++-- tests/ci/s3_helper.py | 62 ++++++++++++++----- tests/ci/split_build_smoke_check.py | 11 ++-- tests/ci/stress_check.py | 11 ++-- tests/ci/style_check.py | 10 ++-- tests/ci/unit_tests_check.py | 11 ++-- tests/ci/upload_result_helper.py | 19 +++--- 23 files changed, 246 insertions(+), 164 deletions(-) create mode 100644 tests/ci/env_helper.py diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 656e9fdbe50..10b5de7014e 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -7,9 +7,11 @@ import sys from github import Github +from env_helper import GITHUB_REPOSITORY, TEMP_PATH, REPO_COPY, REPORTS_PATH, GITHUB_SERVER_URL, \ + GITHUB_RUN_ID from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo, get_event +from pr_info import PRInfo from build_download_helper import get_build_name_for_check, get_build_urls from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status @@ -26,7 +28,7 @@ def get_run_command(pr_number, sha, download_url, workspace_path, image): f'{image}' def get_commit(gh, commit_sha): - repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + repo = gh.get_repo(GITHUB_REPOSITORY) commit = repo.get_commit(commit_sha) return commit @@ -35,16 +37,16 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) - reports_path = os.getenv("REPORTS_PATH", "./reports") + temp_path = TEMP_PATH + repo_path = REPO_COPY + reports_path = REPORTS_PATH check_name = sys.argv[1] if not os.path.exists(temp_path): os.makedirs(temp_path) - pr_info = PRInfo(get_event()) + pr_info = PRInfo() gh = Github(get_best_robot_token()) @@ -106,7 +108,7 @@ if __name__ == "__main__": logging.info("Exception uploading file %s text %s", f, ex) paths[f] = '' - report_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + report_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}" if paths['runlog.log']: report_url = paths['runlog.log'] if paths['main.log']: diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 28974662745..a18d0f8a242 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -7,8 +7,10 @@ import os import sys import time from github import Github + +from env_helper import REPO_COPY, TEMP_PATH, CACHES_PATH, IMAGES_PATH from s3_helper import S3Helper -from pr_info import PRInfo, get_event +from pr_info import PRInfo from get_robot_token import get_best_robot_token from version_helper import get_version_from_repo, update_version_local from ccache_utils import get_ccache_if_not_exists, upload_ccache @@ -111,9 +113,9 @@ def create_json_artifact(temp_path, build_name, log_url, build_urls, build_confi if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - caches_path = os.getenv("CACHES_PATH", temp_path) + repo_path = REPO_COPY + temp_path = TEMP_PATH + caches_path = CACHES_PATH build_check_name = sys.argv[1] build_name = sys.argv[2] @@ -123,7 +125,7 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) - pr_info = PRInfo(get_event()) + pr_info = PRInfo() logging.info("Repo copy path %s", repo_path) @@ -160,7 +162,7 @@ if __name__ == "__main__": sys.exit(0) image_name = get_image_name(build_config) - docker_image = get_image_with_version(os.getenv("IMAGES_PATH"), image_name) + docker_image = get_image_with_version(IMAGES_PATH, image_name) image_version = docker_image.version logging.info("Got version from repo %s", version.get_version_string()) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index f807dcc7cc2..8d63ef38b89 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -5,11 +5,13 @@ import logging import os import sys from github import Github + +from env_helper import REPORTS_PATH, TEMP_PATH, GITHUB_REPOSITORY, GITHUB_SERVER_URL, GITHUB_RUN_ID from report import create_build_html_report from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo, get_event -from commit_status_helper import get_commit +from pr_info import PRInfo +from commit_status_helper import get_commit from ci_config import CI_CONFIG from rerun_helper import RerunHelper @@ -75,8 +77,8 @@ def get_build_name_from_file_name(file_name): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - reports_path = os.getenv("REPORTS_PATH", "./reports") - temp_path = os.path.join(os.getenv("TEMP_PATH", ".")) + reports_path = REPORTS_PATH + temp_path = TEMP_PATH logging.info("Reports path %s", reports_path) if not os.path.exists(temp_path): @@ -85,7 +87,7 @@ if __name__ == "__main__": build_check_name = sys.argv[1] gh = Github(get_best_robot_token()) - pr_info = PRInfo(get_event()) + pr_info = PRInfo() rerun_helper = RerunHelper(gh, pr_info, build_check_name) if rerun_helper.is_already_finished_by_status(): logging.info("Check is already finished according to github status, exiting") @@ -124,15 +126,15 @@ if __name__ == "__main__": s3_helper = S3Helper('https://s3.amazonaws.com') - pr_info = PRInfo(get_event()) + pr_info = PRInfo() - branch_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/commits/master" + branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commits/master" branch_name = "master" if pr_info.number != 0: branch_name = "PR #{}".format(pr_info.number) - branch_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/pull/{pr_info.number}" - commit_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/commit/{pr_info.sha}" - task_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID', '0')}" + branch_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/pull/{pr_info.number}" + commit_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/commit/{pr_info.sha}" + task_url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID or '0'}" report = create_build_html_report( build_check_name, build_results, diff --git a/tests/ci/cherry_pick.py b/tests/ci/cherry_pick.py index 112b58ef1cf..91a018f158f 100644 --- a/tests/ci/cherry_pick.py +++ b/tests/ci/cherry_pick.py @@ -5,6 +5,7 @@ import logging import os import subprocess +from env_helper import GITHUB_WORKSPACE, TEMP_PATH from get_robot_token import get_parameter_from_ssm from ssh import SSHKey from cherry_pick_utils.backport import Backport @@ -13,8 +14,8 @@ from cherry_pick_utils.cherrypick import CherryPick if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../"))) - temp_path = os.path.join(os.getenv("TEMP_PATH")) + repo_path = GITHUB_WORKSPACE + temp_path = TEMP_PATH if not os.path.exists(temp_path): os.makedirs(temp_path) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 5bdbf634715..c420b76aaf3 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -1,9 +1,10 @@ #!/usr/bin/env python3 -import os +from env_helper import GITHUB_REPOSITORY + def get_commit(gh, commit_sha): - repo = gh.get_repo(os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse")) + repo = gh.get_repo(GITHUB_REPOSITORY) commit = repo.get_commit(commit_sha) return commit diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 665f399b040..72626bd6364 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -8,9 +8,10 @@ import sys from github import Github +from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo, get_event +from pr_info import PRInfo from build_download_helper import download_builds_filter from upload_result_helper import upload_results from docker_pull_helper import get_images_with_versions @@ -103,11 +104,11 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) - reports_path = os.getenv("REPORTS_PATH", "./reports") + temp_path = TEMP_PATH + repo_path = REPO_COPY + reports_path = REPORTS_PATH - pr_info = PRInfo(get_event()) + pr_info = PRInfo() gh = Github(get_best_robot_token()) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 5e05cbaecd7..e389d612f44 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -6,8 +6,10 @@ import os import time import shutil from github import Github + +from env_helper import GITHUB_WORKSPACE, RUNNER_TEMP from s3_helper import S3Helper -from pr_info import PRInfo, get_event +from pr_info import PRInfo from get_robot_token import get_best_robot_token, get_parameter_from_ssm from upload_result_helper import upload_results from commit_status_helper import get_commit @@ -157,8 +159,8 @@ if __name__ == "__main__": stopwatch = Stopwatch() - repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) - temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'docker_images_check') + repo_path = GITHUB_WORKSPACE + temp_path = os.path.join(RUNNER_TEMP, 'docker_images_check') dockerhub_password = get_parameter_from_ssm('dockerhub_robot_password') if os.path.exists(temp_path): @@ -167,7 +169,7 @@ if __name__ == "__main__": if not os.path.exists(temp_path): os.makedirs(temp_path) - pr_info = PRInfo(get_event(), need_changed_files=True) + pr_info = PRInfo(need_changed_files=True) changed_images, dockerhub_repo_name = get_changed_docker_images(pr_info, repo_path, "docker/images.json") logging.info("Has changed images %s", ', '.join([str(image[0]) for image in changed_images])) pr_commit_version = str(pr_info.number) + '-' + pr_info.sha diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index 0430d566b38..c1854500192 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -4,8 +4,10 @@ import subprocess import os import sys from github import Github + +from env_helper import TEMP_PATH, REPO_COPY from s3_helper import S3Helper -from pr_info import PRInfo, get_event +from pr_info import PRInfo from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version @@ -22,10 +24,10 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.path.join(os.getenv("TEMP_PATH")) - repo_path = os.path.join(os.getenv("REPO_COPY")) + temp_path = TEMP_PATH + repo_path = REPO_COPY - pr_info = PRInfo(get_event(), need_changed_files=True) + pr_info = PRInfo(need_changed_files=True) gh = Github(get_best_robot_token()) @@ -35,7 +37,7 @@ if __name__ == "__main__": sys.exit(0) if not pr_info.has_changes_in_documentation(): - logging.info ("No changes in documentation") + logging.info("No changes in documentation") commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description="No changes in docs", state="success") sys.exit(0) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 04922e8c5ab..feabb9777d3 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -6,8 +6,9 @@ import sys from github import Github +from env_helper import TEMP_PATH, REPO_COPY, CLOUDFLARE_TOKEN from s3_helper import S3Helper -from pr_info import PRInfo, get_event +from pr_info import PRInfo from get_robot_token import get_best_robot_token from ssh import SSHKey from upload_result_helper import upload_results @@ -19,10 +20,10 @@ NAME = "Docs Release (actions)" if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - temp_path = os.path.join(os.getenv("TEMP_PATH")) - repo_path = os.path.join(os.getenv("REPO_COPY")) + temp_path = TEMP_PATH + repo_path = REPO_COPY - pr_info = PRInfo(get_event(), need_changed_files=True) + pr_info = PRInfo(need_changed_files=True) gh = Github(get_best_robot_token()) if not pr_info.has_changes_in_documentation(): @@ -42,7 +43,7 @@ if __name__ == "__main__": if not os.path.exists(test_output): os.makedirs(test_output) - token = os.getenv('CLOUDFLARE_TOKEN') + token = CLOUDFLARE_TOKEN cmd = "docker run --cap-add=SYS_PTRACE --volume=$SSH_AUTH_SOCK:/ssh-agent -e SSH_AUTH_SOCK=/ssh-agent " \ f"-e CLOUDFLARE_TOKEN={token} --volume={repo_path}:/repo_path --volume={test_output}:/output_path {docker_image}" diff --git a/tests/ci/env_helper.py b/tests/ci/env_helper.py new file mode 100644 index 00000000000..90178e5c56a --- /dev/null +++ b/tests/ci/env_helper.py @@ -0,0 +1,18 @@ +import os + +CI = bool(os.getenv("CI")) +TEMP_PATH = os.getenv("TEMP_PATH", os.path.abspath(".")) + +CACHES_PATH = os.getenv("CACHES_PATH", TEMP_PATH) +CLOUDFLARE_TOKEN = os.getenv("CLOUDFLARE_TOKEN") +GITHUB_EVENT_PATH = os.getenv("GITHUB_EVENT_PATH") +GITHUB_REPOSITORY = os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse") +GITHUB_RUN_ID = os.getenv("GITHUB_RUN_ID") +GITHUB_SERVER_URL = os.getenv("GITHUB_SERVER_URL", "https://github.com") +GITHUB_WORKSPACE = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) +IMAGES_PATH = os.getenv("IMAGES_PATH") +REPORTS_PATH = os.getenv("REPORTS_PATH", "./reports") +REPO_COPY = os.getenv("REPO_COPY", os.path.abspath("../../")) +RUNNER_TEMP = os.getenv("RUNNER_TEMP", os.path.abspath("./tmp")) +S3_BUILDS_BUCKET = os.getenv("S3_BUILDS_BUCKET", "clickhouse-builds") +S3_TEST_REPORTS_BUCKET = os.getenv("S3_TEST_REPORTS_BUCKET", "clickhouse-test-reports") diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 7953dcdf5d3..35a5d3f0228 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -7,7 +7,9 @@ import csv import sys from github import Github -from pr_info import PRInfo, get_event + +from env_helper import CACHES_PATH, TEMP_PATH +from pr_info import PRInfo from s3_helper import S3Helper from get_robot_token import get_best_robot_token from upload_result_helper import upload_results @@ -58,13 +60,13 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - caches_path = os.getenv("CACHES_PATH", temp_path) + temp_path = TEMP_PATH + caches_path = CACHES_PATH if not os.path.exists(temp_path): os.makedirs(temp_path) - pr_info = PRInfo(get_event()) + pr_info = PRInfo() gh = Github(get_best_robot_token()) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 576b97058c7..72f26daf4cd 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -1,13 +1,15 @@ #!/usr/bin/env python3 import logging -import os from github import Github -from pr_info import PRInfo, get_event + +from env_helper import GITHUB_SERVER_URL, GITHUB_REPOSITORY, GITHUB_RUN_ID +from pr_info import PRInfo from get_robot_token import get_best_robot_token from commit_status_helper import get_commit NAME = 'Run Check (actions)' + def filter_statuses(statuses): """ Squash statuses to latest state @@ -23,14 +25,15 @@ def filter_statuses(statuses): filt[status.context] = status return filt + if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - pr_info = PRInfo(get_event(), need_orgs=True) + pr_info = PRInfo(need_orgs=True) gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) - url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}" statuses = filter_statuses(list(commit.get_statuses())) if NAME in statuses and statuses[NAME].state == "pending": commit.create_status(context=NAME, description="All checks finished", state="success", target_url=url) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 8ab4fc3c9c4..9723989144c 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -8,9 +8,10 @@ import sys from github import Github +from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo, get_event +from pr_info import PRInfo from build_download_helper import download_all_deb_packages from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version @@ -110,16 +111,16 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) - reports_path = os.getenv("REPORTS_PATH", "./reports") + temp_path = TEMP_PATH + repo_path = REPO_COPY + reports_path = REPORTS_PATH check_name = sys.argv[1] kill_timeout = int(sys.argv[2]) flaky_check = 'flaky' in check_name.lower() gh = Github(get_best_robot_token()) - pr_info = PRInfo(get_event(), need_changed_files=flaky_check) + pr_info = PRInfo(need_changed_files=flaky_check) rerun_helper = RerunHelper(gh, pr_info, check_name) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index c4950c93422..92221c376f1 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -9,9 +9,10 @@ import csv from github import Github +from env_helper import TEMP_PATH, REPO_COPY, REPORTS_PATH from s3_helper import S3Helper from get_robot_token import get_best_robot_token -from pr_info import PRInfo, get_event +from pr_info import PRInfo from build_download_helper import download_all_deb_packages from upload_result_helper import upload_results from docker_pull_helper import get_images_with_versions @@ -100,9 +101,9 @@ if __name__ == "__main__": stopwatch = Stopwatch() - temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) - reports_path = os.getenv("REPORTS_PATH", "./reports") + temp_path = TEMP_PATH + repo_path = REPO_COPY + reports_path = REPORTS_PATH check_name = sys.argv[1] @@ -110,7 +111,7 @@ if __name__ == "__main__": os.makedirs(temp_path) is_flaky_check = 'flaky' in check_name - pr_info = PRInfo(get_event(), need_changed_files=is_flaky_check) + pr_info = PRInfo(need_changed_files=is_flaky_check) gh = Github(get_best_robot_token()) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index a26b19f4bc9..74cc4752677 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -6,11 +6,13 @@ import urllib import requests from unidiff import PatchSet +from env_helper import GITHUB_REPOSITORY, GITHUB_SERVER_URL, GITHUB_RUN_ID, GITHUB_EVENT_PATH -DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", ".jpg", ".py", ".sh", ".json"] +DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", + ".jpg", ".py", ".sh", ".json"] def get_pr_for_commit(sha, ref): - try_get_pr_url = f"https://api.github.com/repos/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}/commits/{sha}/pulls" + try_get_pr_url = f"https://api.github.com/repos/{GITHUB_REPOSITORY}/commits/{sha}/pulls" try: response = requests.get(try_get_pr_url) response.raise_for_status() @@ -22,7 +24,7 @@ def get_pr_for_commit(sha, ref): # refs for RPs looks like XX if pr['head']['ref'] in ref: return pr - print ("Cannot find PR with required ref", ref, "returning first one") + print("Cannot find PR with required ref", ref, "returning first one") first_pr = data[0] return first_pr except Exception as ex: @@ -30,24 +32,27 @@ def get_pr_for_commit(sha, ref): return None -def get_event(): - with open(os.getenv('GITHUB_EVENT_PATH'), 'r', encoding='utf-8') as ef: - return json.load(ef) - - class PRInfo: - def __init__(self, github_event, need_orgs=False, need_changed_files=False): - if 'pull_request' in github_event: # pull request and other similar events + def __init__(self, github_event=None, need_orgs=False, need_changed_files=False): + if not github_event: + if GITHUB_EVENT_PATH: + with open(GITHUB_EVENT_PATH, 'r', encoding='utf-8') as event_file: + github_event = json.load(event_file) + else: + github_event = {'commits': 1, 'after': 'HEAD', 'ref': None} + self.event = github_event + self.changed_files = set([]) + if 'pull_request' in github_event: # pull request and other similar events self.number = github_event['number'] if 'after' in github_event: self.sha = github_event['after'] else: self.sha = github_event['pull_request']['head']['sha'] - repo_prefix = f"{os.getenv('GITHUB_SERVER_URL', 'https://github.com')}/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}" - self.task_url = f"{repo_prefix}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + repo_prefix = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}" + self.task_url = f"{repo_prefix}/actions/runs/{GITHUB_RUN_ID or '0'}" - self.repo_full_name = os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse') + self.repo_full_name = GITHUB_REPOSITORY self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" self.pr_html_url = f"{repo_prefix}/pull/{self.number}" @@ -56,7 +61,7 @@ class PRInfo: self.head_ref = github_event['pull_request']['head']['ref'] self.head_name = github_event['pull_request']['head']['repo']['full_name'] - self.labels = { l['name'] for l in github_event['pull_request']['labels'] } + self.labels = {l['name'] for l in github_event['pull_request']['labels']} self.user_login = github_event['pull_request']['user']['login'] self.user_orgs = set([]) if need_orgs: @@ -65,21 +70,15 @@ class PRInfo: response_json = user_orgs_response.json() self.user_orgs = set(org['id'] for org in response_json) - self.changed_files = set([]) - if need_changed_files: - diff_url = github_event['pull_request']['diff_url'] - diff = urllib.request.urlopen(diff_url) - diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) - self.changed_files = { f.path for f in diff_object } - + self.diff_url = github_event['pull_request']['diff_url'] elif 'commits' in github_event: self.sha = github_event['after'] pull_request = get_pr_for_commit(self.sha, github_event['ref']) - repo_prefix = f"{os.getenv('GITHUB_SERVER_URL', 'https://github.com')}/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}" - self.task_url = f"{repo_prefix}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + repo_prefix = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}" + self.task_url = f"{repo_prefix}/actions/runs/{GITHUB_RUN_ID or '0'}" self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" - self.repo_full_name = os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse') - if pull_request is None or pull_request['state'] == 'closed': # it's merged PR to master + self.repo_full_name = GITHUB_REPOSITORY + if pull_request is None or pull_request['state'] == 'closed': # it's merged PR to master self.number = 0 self.labels = {} self.pr_html_url = f"{repo_prefix}/commits/master" @@ -87,40 +86,37 @@ class PRInfo: self.base_name = self.repo_full_name self.head_ref = "master" self.head_name = self.repo_full_name + self.diff_url = \ + f"https://api.github.com/repos/{GITHUB_REPOSITORY}/compare/{github_event['before']}...{self.sha}" else: self.number = pull_request['number'] - self.labels = { l['name'] for l in pull_request['labels'] } + self.labels = {l['name'] for l in pull_request['labels']} self.base_ref = pull_request['base']['ref'] self.base_name = pull_request['base']['repo']['full_name'] self.head_ref = pull_request['head']['ref'] self.head_name = pull_request['head']['repo']['full_name'] self.pr_html_url = pull_request['html_url'] - - if need_changed_files: - if self.number == 0: - commit_before = github_event['before'] - response = requests.get(f"https://api.github.com/repos/{os.getenv('GITHUB_REPOSITORY')}/compare/{commit_before}...{self.sha}") - response.raise_for_status() - diff = response.json() - - if 'files' in diff: - self.changed_files = [f['filename'] for f in diff['files']] - else: - self.changed_files = set([]) + if 'pr-backport' in self.labels: + self.diff_url = f"https://github.com/{GITHUB_REPOSITORY}/compare/master...{self.head_ref}.diff" else: - if 'pr-backport' in self.labels: - diff_url = f"https://github.com/{os.getenv('GITHUB_REPOSITORY')}/compare/master...{self.head_ref}.diff" - else: - diff_url = pull_request['diff_url'] - - diff = urllib.request.urlopen(diff_url) - diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) - self.changed_files = { f.path for f in diff_object } - else: - self.changed_files = set([]) + self.diff_url = pull_request['diff_url'] else: raise Exception("Cannot detect type of event") + if need_changed_files: + self.fetch_changed_files() + def fetch_changed_files(self): + if 'commits' in self.event and self.number == 0: + response = requests.get(self.diff_url) + response.raise_for_status() + diff = response.json() + + if 'files' in diff: + self.changed_files = [f['filename'] for f in diff['files']] + else: + diff = urllib.request.urlopen(self.diff_url) + diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) + self.changed_files = {f.path for f in diff_object} def get_dict(self): return { diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 1ce1c5839f4..e9deef65ec5 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -8,8 +8,10 @@ import json import logging import sys from github import Github + +from env_helper import REPO_COPY, TEMP_PATH, GITHUB_RUN_ID, GITHUB_REPOSITORY, GITHUB_SERVER_URL from s3_helper import S3Helper -from pr_info import PRInfo, get_event +from pr_info import PRInfo from get_robot_token import get_best_robot_token, get_parameter_from_ssm from upload_result_helper import upload_results from commit_status_helper import get_commit @@ -22,6 +24,7 @@ LICENCE_NAME = 'Free license: ClickHouse, Yandex' HTML_REPORT_FOLDER = 'pvs-studio-html-report' TXT_REPORT_NAME = 'pvs-studio-task-report.txt' + def _process_txt_report(path): warnings = [] errors = [] @@ -37,15 +40,16 @@ def _process_txt_report(path): return warnings, errors + if __name__ == "__main__": logging.basicConfig(level=logging.INFO) stopwatch = Stopwatch() - repo_path = os.path.join(os.getenv("REPO_COPY", os.path.abspath("../../"))) - temp_path = os.path.join(os.getenv("TEMP_PATH")) + repo_path = REPO_COPY + temp_path = TEMP_PATH - pr_info = PRInfo(get_event()) + pr_info = PRInfo() # this check modify repository so copy it to the temp directory logging.info("Repo copy path %s", repo_path) @@ -76,7 +80,8 @@ if __name__ == "__main__": try: subprocess.check_output(cmd, shell=True) except: - commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") + commit.create_status(context=NAME, description='PVS report failed to build', state='failure', + target_url=f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}") sys.exit(1) try: @@ -91,7 +96,7 @@ if __name__ == "__main__": if not index_html: commit.create_status(context=NAME, description='PVS report failed to build', state='failure', - target_url=f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}") + target_url=f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}") sys.exit(1) txt_report = os.path.join(temp_path, TXT_REPORT_NAME) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 3739cf88248..fa72b85d2e1 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -1,9 +1,10 @@ #!/usr/bin/env python3 -import os import sys import logging from github import Github -from pr_info import PRInfo, get_event + +from env_helper import GITHUB_RUN_ID, GITHUB_REPOSITORY, GITHUB_SERVER_URL +from pr_info import PRInfo from get_robot_token import get_best_robot_token from commit_status_helper import get_commit @@ -102,14 +103,15 @@ def should_run_checks_for_pr(pr_info): return True, "No special conditions apply" + if __name__ == "__main__": logging.basicConfig(level=logging.INFO) - pr_info = PRInfo(get_event(), need_orgs=True) + pr_info = PRInfo(need_orgs=True) can_run, description = should_run_checks_for_pr(pr_info) gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) - url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + url = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}/actions/runs/{GITHUB_RUN_ID}" if not can_run: print("::notice ::Cannot run") commit.create_status(context=NAME, description=description, state="failure", target_url=url) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 7c1ee8ad9ee..27a613f7787 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -2,10 +2,16 @@ import hashlib import logging import os +import re +import shutil from multiprocessing.dummy import Pool + import boto3 + +from env_helper import S3_TEST_REPORTS_BUCKET, S3_BUILDS_BUCKET, RUNNER_TEMP, CI from compress_files import compress_file_fast + def _md5(fname): hash_md5 = hashlib.md5() with open(fname, "rb") as f: @@ -25,7 +31,7 @@ def _flatten_list(lst): return result -class S3Helper(): +class S3Helper: def __init__(self, host): self.session = boto3.session.Session(region_name='us-east-1') self.client = self.session.client('s3', endpoint_url=host) @@ -49,9 +55,7 @@ class S3Helper(): else: logging.info("No content type provied for %s", file_path) else: - is_log = s3_path.endswith("log") or ".log." in s3_path - is_text = s3_path.endswith("txt") or is_log or s3_path.endswith("err") or s3_path.endswith("out") - if not s3_path.endswith('.gz') and (is_text or is_log): + if re.search(r'\.(txt|log|err|out)$', s3_path) or re.search(r'\.log\..*(? Date: Tue, 7 Dec 2021 16:44:09 +0300 Subject: [PATCH 107/358] fix GROUPING SETS formating --- src/Parsers/ASTExpressionList.cpp | 94 +++++++++---------------------- src/Parsers/ASTSelectQuery.cpp | 7 ++- src/Parsers/IAST.h | 1 + 3 files changed, 33 insertions(+), 69 deletions(-) diff --git a/src/Parsers/ASTExpressionList.cpp b/src/Parsers/ASTExpressionList.cpp index 453624aa794..2590c6b2941 100644 --- a/src/Parsers/ASTExpressionList.cpp +++ b/src/Parsers/ASTExpressionList.cpp @@ -17,38 +17,24 @@ void ASTExpressionList::formatImpl(const FormatSettings & settings, FormatState if (frame.expression_list_prepend_whitespace) settings.ostr << ' '; - if (frame.need_parens) + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { - settings.ostr << "("; - for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + if (it != children.begin()) { - if (it != children.begin()) - { - if (separator) - settings.ostr << separator; - settings.ostr << ' '; - } + if (separator) + settings.ostr << separator; + settings.ostr << ' '; + } + if (frame.surround_each_list_element_with_parens) settings.ostr << "("; - FormatStateStacked frame_nested = frame; - frame_nested.need_parens = false; - (*it)->formatImpl(settings, state, frame_nested); + + FormatStateStacked frame_nested = frame; + frame_nested.surround_each_list_element_with_parens = false; + (*it)->formatImpl(settings, state, frame_nested); + + if (frame.surround_each_list_element_with_parens) settings.ostr << ")"; - } - settings.ostr << ")"; - } - else - { - for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) - { - if (it != children.begin()) - { - if (separator) - settings.ostr << separator; - settings.ostr << ' '; - } - (*it)->formatImpl(settings, state, frame); - } } } @@ -64,52 +50,28 @@ void ASTExpressionList::formatImplMultiline(const FormatSettings & settings, For ++frame.indent; - if (frame.need_parens) + for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) { - for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) + if (it != children.begin()) { - if (it != children.begin()) - { - if (separator) - settings.ostr << separator; - } + if (separator) + settings.ostr << separator; + } - if (children.size() > 1 || frame.expression_list_always_start_on_new_line) - settings.ostr << indent_str; + if (children.size() > 1 || frame.expression_list_always_start_on_new_line) + settings.ostr << indent_str; - if (it == children.begin()) - { - settings.ostr << "("; - } + FormatStateStacked frame_nested = frame; + frame_nested.expression_list_always_start_on_new_line = false; + frame_nested.surround_each_list_element_with_parens = false; - FormatStateStacked frame_nested = frame; - frame_nested.expression_list_always_start_on_new_line = false; - frame_nested.expression_list_prepend_whitespace = false; - frame_nested.need_parens = false; + if (frame.surround_each_list_element_with_parens) settings.ostr << "("; - (*it)->formatImpl(settings, state, frame_nested); + + (*it)->formatImpl(settings, state, frame_nested); + + if (frame.surround_each_list_element_with_parens) settings.ostr << ")"; - } - settings.ostr << ")"; - } - else - { - for (ASTs::const_iterator it = children.begin(); it != children.end(); ++it) - { - if (it != children.begin()) - { - if (separator) - settings.ostr << separator; - } - - if (children.size() > 1 || frame.expression_list_always_start_on_new_line) - settings.ostr << indent_str; - - FormatStateStacked frame_nested = frame; - frame_nested.expression_list_always_start_on_new_line = false; - - (*it)->formatImpl(settings, state, frame_nested); - } } } diff --git a/src/Parsers/ASTSelectQuery.cpp b/src/Parsers/ASTSelectQuery.cpp index 3a4f8d79662..5a96e1d4df9 100644 --- a/src/Parsers/ASTSelectQuery.cpp +++ b/src/Parsers/ASTSelectQuery.cpp @@ -130,13 +130,14 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F if (group_by_with_grouping_sets) { - bool tmp_need_parens = frame.need_parens; - frame.need_parens = true; + frame.surround_each_list_element_with_parens = true; s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << (s.one_line ? "" : " ") << "GROUPING SETS" << (s.hilite ? hilite_none : ""); + s.ostr << " ("; s.one_line ? groupBy()->formatImpl(s, state, frame) : groupBy()->as().formatImplMultiline(s, state, frame); - frame.need_parens = tmp_need_parens; + s.ostr << ")"; + frame.surround_each_list_element_with_parens = false; } if (group_by_with_totals) diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 1c9c81ad378..b3b979e24d0 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -208,6 +208,7 @@ public: bool need_parens = false; bool expression_list_always_start_on_new_line = false; /// Line feed and indent before expression list even if it's of single element. bool expression_list_prepend_whitespace = false; /// Prepend whitespace (if it is required) + bool surround_each_list_element_with_parens = false; const IAST * current_select = nullptr; }; From 4afaa8697c8177f5874ca6364525480a2bfc5cd1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 7 Dec 2021 16:47:42 +0300 Subject: [PATCH 108/358] remove ya.make --- src/Processors/ya.make | 158 ----------------------------------------- 1 file changed, 158 deletions(-) delete mode 100644 src/Processors/ya.make diff --git a/src/Processors/ya.make b/src/Processors/ya.make deleted file mode 100644 index b2f8b9ba7c2..00000000000 --- a/src/Processors/ya.make +++ /dev/null @@ -1,158 +0,0 @@ -# This file is generated automatically, do not edit. See 'ya.make.in' and use 'utils/generate-ya-make' to regenerate it. -LIBRARY() - -PEERDIR( - clickhouse/src/Common - contrib/libs/msgpack - contrib/libs/protobuf -) - -CFLAGS(-g0) - -SRCS( - Chunk.cpp - ConcatProcessor.cpp - DelayedPortsProcessor.cpp - Executors/ExecutingGraph.cpp - Executors/PipelineExecutingBlockInputStream.cpp - Executors/PipelineExecutor.cpp - Executors/PullingAsyncPipelineExecutor.cpp - Executors/PullingPipelineExecutor.cpp - ForkProcessor.cpp - Formats/IInputFormat.cpp - Formats/Impl/BinaryRowInputFormat.cpp - Formats/Impl/BinaryRowOutputFormat.cpp - Formats/Impl/ConstantExpressionTemplate.cpp - Formats/Impl/CSVRowInputFormat.cpp - Formats/Impl/CSVRowOutputFormat.cpp - Formats/Impl/JSONAsStringRowInputFormat.cpp - Formats/Impl/JSONCompactEachRowRowInputFormat.cpp - Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp - Formats/Impl/JSONCompactRowOutputFormat.cpp - Formats/Impl/JSONEachRowRowInputFormat.cpp - Formats/Impl/JSONEachRowRowOutputFormat.cpp - Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp - Formats/Impl/JSONRowOutputFormat.cpp - Formats/Impl/LineAsStringRowInputFormat.cpp - Formats/Impl/MarkdownRowOutputFormat.cpp - Formats/Impl/MsgPackRowInputFormat.cpp - Formats/Impl/MsgPackRowOutputFormat.cpp - Formats/Impl/MySQLOutputFormat.cpp - Formats/Impl/NativeFormat.cpp - Formats/Impl/NullFormat.cpp - Formats/Impl/ODBCDriver2BlockOutputFormat.cpp - Formats/Impl/PostgreSQLOutputFormat.cpp - Formats/Impl/PrettyBlockOutputFormat.cpp - Formats/Impl/PrettyCompactBlockOutputFormat.cpp - Formats/Impl/PrettySpaceBlockOutputFormat.cpp - Formats/Impl/ProtobufRowInputFormat.cpp - Formats/Impl/ProtobufRowOutputFormat.cpp - Formats/Impl/RawBLOBRowInputFormat.cpp - Formats/Impl/RawBLOBRowOutputFormat.cpp - Formats/Impl/RegexpRowInputFormat.cpp - Formats/Impl/TabSeparatedRowInputFormat.cpp - Formats/Impl/TabSeparatedRowOutputFormat.cpp - Formats/Impl/TemplateBlockOutputFormat.cpp - Formats/Impl/TemplateRowInputFormat.cpp - Formats/Impl/TSKVRowInputFormat.cpp - Formats/Impl/TSKVRowOutputFormat.cpp - Formats/Impl/ValuesBlockInputFormat.cpp - Formats/Impl/ValuesRowOutputFormat.cpp - Formats/Impl/VerticalRowOutputFormat.cpp - Formats/Impl/XMLRowOutputFormat.cpp - Formats/IOutputFormat.cpp - Formats/IRowInputFormat.cpp - Formats/IRowOutputFormat.cpp - Formats/LazyOutputFormat.cpp - Formats/OutputStreamToOutputFormat.cpp - Formats/PullingOutputFormat.cpp - Formats/RowInputFormatWithDiagnosticInfo.cpp - IAccumulatingTransform.cpp - IInflatingTransform.cpp - IProcessor.cpp - ISimpleTransform.cpp - ISink.cpp - ISource.cpp - LimitTransform.cpp - Merges/Algorithms/AggregatingSortedAlgorithm.cpp - Merges/Algorithms/CollapsingSortedAlgorithm.cpp - Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp - Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.cpp - Merges/Algorithms/IMergingAlgorithmWithSharedChunks.cpp - Merges/Algorithms/MergingSortedAlgorithm.cpp - Merges/Algorithms/ReplacingSortedAlgorithm.cpp - Merges/Algorithms/SummingSortedAlgorithm.cpp - Merges/Algorithms/VersionedCollapsingAlgorithm.cpp - Merges/IMergingTransform.cpp - Merges/MergingSortedTransform.cpp - OffsetTransform.cpp - Pipe.cpp - Port.cpp - printPipeline.cpp - QueryPipeline.cpp - QueryPlan/AddingDelayedSourceStep.cpp - QueryPlan/AggregatingStep.cpp - QueryPlan/ArrayJoinStep.cpp - QueryPlan/ConvertingStep.cpp - QueryPlan/CreatingSetsStep.cpp - QueryPlan/CubeStep.cpp - QueryPlan/DistinctStep.cpp - QueryPlan/ExpressionStep.cpp - QueryPlan/ExtremesStep.cpp - QueryPlan/FillingStep.cpp - QueryPlan/FilterStep.cpp - QueryPlan/FinishSortingStep.cpp - QueryPlan/IQueryPlanStep.cpp - QueryPlan/ISourceStep.cpp - QueryPlan/ITransformingStep.cpp - QueryPlan/LimitByStep.cpp - QueryPlan/LimitStep.cpp - QueryPlan/MergeSortingStep.cpp - QueryPlan/MergingAggregatedStep.cpp - QueryPlan/MergingSortedStep.cpp - QueryPlan/OffsetStep.cpp - QueryPlan/PartialSortingStep.cpp - QueryPlan/QueryPlan.cpp - QueryPlan/ReadFromPreparedSource.cpp - QueryPlan/ReadFromStorageStep.cpp - QueryPlan/ReadNothingStep.cpp - QueryPlan/RollupStep.cpp - QueryPlan/TotalsHavingStep.cpp - QueryPlan/UnionStep.cpp - ResizeProcessor.cpp - Sources/DelayedSource.cpp - Sources/RemoteSource.cpp - Sources/SinkToOutputStream.cpp - Sources/SourceFromInputStream.cpp - Sources/SourceWithProgress.cpp - Transforms/AddingMissedTransform.cpp - Transforms/AddingSelectorTransform.cpp - Transforms/AggregatingInOrderTransform.cpp - Transforms/AggregatingTransform.cpp - Transforms/ArrayJoinTransform.cpp - Transforms/ConvertingTransform.cpp - Transforms/CopyTransform.cpp - Transforms/CreatingSetsTransform.cpp - Transforms/CubeTransform.cpp - Transforms/DistinctTransform.cpp - Transforms/ExpressionTransform.cpp - Transforms/ExtremesTransform.cpp - Transforms/FillingTransform.cpp - Transforms/FilterTransform.cpp - Transforms/FinishSortingTransform.cpp - Transforms/JoiningTransform.cpp - Transforms/LimitByTransform.cpp - Transforms/LimitsCheckingTransform.cpp - Transforms/MaterializingTransform.cpp - Transforms/MergeSortingTransform.cpp - Transforms/MergingAggregatedMemoryEfficientTransform.cpp - Transforms/MergingAggregatedTransform.cpp - Transforms/PartialSortingTransform.cpp - Transforms/ReverseTransform.cpp - Transforms/RollupTransform.cpp - Transforms/SortingTransform.cpp - Transforms/TotalsHavingTransform.cpp - -) - -END() From c9c69ce8a1fc7ecf9cbcf4d21eeb94560c6509ac Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Tue, 7 Dec 2021 14:24:57 +0000 Subject: [PATCH 109/358] Add documentation for insert_quorum_parallel. --- docs/en/operations/settings/settings.md | 32 ++++++++++++++++++++----- 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index fa4cc41e8ff..5e0394ecc45 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1707,18 +1707,17 @@ Quorum writes `INSERT` succeeds only when ClickHouse manages to correctly write data to the `insert_quorum` of replicas during the `insert_quorum_timeout`. If for any reason the number of replicas with successful writes does not reach the `insert_quorum`, the write is considered failed and ClickHouse will delete the inserted block from all the replicas where data has already been written. -All the replicas in the quorum are consistent, i.e., they contain data from all previous `INSERT` queries. The `INSERT` sequence is linearized. +When `insert_quorum_parallel` is disabled, all replicas in the quorum are consistent, i.e. they contain data from all previous `INSERT` queries (the `INSERT` sequence is linearized). When reading data written using `insert_quorum` and `insert_quorum_parallel` is disabled, you can turn on sequential consistency for `SELECT` queries using [select_sequential_consistency](#settings-select_sequential_consistency). -When reading the data written from the `insert_quorum`, you can use the [select_sequential_consistency](#settings-select_sequential_consistency) option. - -ClickHouse generates an exception +ClickHouse generates an exception: - If the number of available replicas at the time of the query is less than the `insert_quorum`. -- At an attempt to write data when the previous block has not yet been inserted in the `insert_quorum` of replicas. This situation may occur if the user tries to perform an `INSERT` before the previous one with the `insert_quorum` is completed. +- When `insert_quorum_parallel` is disabled and an attempt to write data is made when the previous block has not yet been inserted in `insert_quorum` of replicas. This situation may occur if the user tries to perform an `INSERT` before the previous one with `insert_quorum` is completed. See also: - [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) - [select_sequential_consistency](#settings-select_sequential_consistency) ## insert_quorum_timeout {#settings-insert_quorum_timeout} @@ -1730,11 +1729,29 @@ Default value: 600 000 milliseconds (ten minutes). See also: - [insert_quorum](#settings-insert_quorum) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) +- [select_sequential_consistency](#settings-select_sequential_consistency) + +## insert_quorum_parallel {#settings-insert_quorum_parallel} + +Enables or disables parallelism for quorum `INSERT` queries. If enabled, additional `INSERT` queries can be sent while previous queries have not yet finished. If disabled, additional queries will be rejected. + +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: 1. + +See also: + +- [insert_quorum](#settings-insert_quorum) +- [insert_quorum_timeout](#settings-insert_quorum_timeout) - [select_sequential_consistency](#settings-select_sequential_consistency) ## select_sequential_consistency {#settings-select_sequential_consistency} -Enables or disables sequential consistency for `SELECT` queries: +Enables or disables sequential consistency for `SELECT` queries. Requires `insert_quorum_parallel` to be disabled (enabled by default). Possible values: @@ -1747,10 +1764,13 @@ Usage When sequential consistency is enabled, ClickHouse allows the client to execute the `SELECT` query only for those replicas that contain data from all previous `INSERT` queries executed with `insert_quorum`. If the client refers to a partial replica, ClickHouse will generate an exception. The SELECT query will not include data that has not yet been written to the quorum of replicas. +When `insert_quorum_parallel` is enabled (the default), then `select_sequential_consistency` does not work. This is because parallel `INSERT` queries can be written to different sets of quorum replicas so there is no guarantee a single replica will have received all writes. + See also: - [insert_quorum](#settings-insert_quorum) - [insert_quorum_timeout](#settings-insert_quorum_timeout) +- [insert_quorum_parallel](#settings-insert_quorum_parallel) ## insert_deduplicate {#settings-insert-deduplicate} From a6f42de167fce3b4baacab1b6381f52f79d720ef Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Tue, 7 Dec 2021 17:34:01 +0000 Subject: [PATCH 110/358] Clarify blocking of parallel inserts. --- docs/en/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5e0394ecc45..e458f8109fe 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1712,7 +1712,7 @@ When `insert_quorum_parallel` is disabled, all replicas in the quorum are consis ClickHouse generates an exception: - If the number of available replicas at the time of the query is less than the `insert_quorum`. -- When `insert_quorum_parallel` is disabled and an attempt to write data is made when the previous block has not yet been inserted in `insert_quorum` of replicas. This situation may occur if the user tries to perform an `INSERT` before the previous one with `insert_quorum` is completed. +- When `insert_quorum_parallel` is disabled and an attempt to write data is made when the previous block has not yet been inserted in `insert_quorum` of replicas. This situation may occur if the user tries to perform another `INSERT` query to the same table before the previous one with `insert_quorum` is completed. See also: @@ -1734,7 +1734,7 @@ See also: ## insert_quorum_parallel {#settings-insert_quorum_parallel} -Enables or disables parallelism for quorum `INSERT` queries. If enabled, additional `INSERT` queries can be sent while previous queries have not yet finished. If disabled, additional queries will be rejected. +Enables or disables parallelism for quorum `INSERT` queries. If enabled, additional `INSERT` queries can be sent while previous queries have not yet finished. If disabled, additional writes to the same table will be rejected. Possible values: From 19a49f69340ced2b9d27f28d740db3ffd1a46e05 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Wed, 8 Dec 2021 00:21:54 +0300 Subject: [PATCH 111/358] Added ACL system to CH Keeper --- docs/en/operations/clickhouse-keeper.md | 13 +++++++++++-- docs/ru/operations/clickhouse-keeper.md | 10 ++++++++-- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 58c59ce9f79..34dbaf522e3 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -16,12 +16,14 @@ ZooKeeper is one of the first well-known open-source coordination systems. It's By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but `clickhouse-keeper-converter` tool allows to convert ZooKeeper data to ClickHouse Keeper snapshot. Interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so mixed ZooKeeper / ClickHouse Keeper cluster is impossible. +ClickHouse Keeper supports Access Control List the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth`, `digest`, `host` and `ip`. Digest authentication scheme uses pair username:password. Password is encoded in Base64. + ## Configuration ClickHouse Keeper can be used as a standalone replacement for ZooKeeper or as an internal part of the ClickHouse server, but in both cases configuration is almost the same `.xml` file. The main ClickHouse Keeper configuration tag is ``. Keeper configuration has the following parameters: - `tcp_port` — Port for a client to connect (default for ZooKeeper is `2181`). -- `tcp_port_secure` — Secure port for a client to connect. +- `tcp_port_secure` — Secure port for an SSL connection between client and keeper-server. - `server_id` — Unique server id, each participant of the ClickHouse Keeper cluster must have a unique number (1, 2, 3, and so on). - `log_storage_path` — Path to coordination logs, better to store logs on the non-busy device (same for ZooKeeper). - `snapshot_storage_path` — Path to coordination snapshots. @@ -50,7 +52,11 @@ Internal coordination settings are located in `..` section and contain servers description. The only parameter for the whole quorum is `secure`, which enables encrypted connection for communication between quorum participants. The main parameters for each `` are: +Quorum configuration is located in `.` section and contain servers description. + +The only parameter for the whole quorum is `secure`, which enables encrypted connection for communication between quorum participants. The parameter can be set `true` if SSL connection is required for internal communication between nodes, or left unspecified otherwise. + +The main parameters for each `` are: - `id` — Server identifier in a quorum. - `hostname` — Hostname where this server is placed. @@ -117,3 +123,6 @@ clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 -- 4. Copy snapshot to ClickHouse server nodes with a configured `keeper` or start ClickHouse Keeper instead of ZooKeeper. The snapshot must persist on all nodes, otherwise, empty nodes can be faster and one of them can become a leader. [Original article](https://clickhouse.com/docs/en/operations/clickhouse-keeper/) + + + diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index 14d95ebae68..de3faea9d56 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -16,12 +16,14 @@ ZooKeeper — один из первых широко извеÑтных Ñер По умолчанию ClickHouse Keeper предоÑтавлÑет те же гарантии, что и ZooKeeper (линеаризуемоÑÑ‚ÑŒ запиÑей, поÑÐ»ÐµÐ´Ð¾Ð²Ð°Ñ‚ÐµÐ»ÑŒÐ½Ð°Ñ ÑоглаÑованноÑÑ‚ÑŒ чтений). У него еÑÑ‚ÑŒ ÑовмеÑтимый клиент-Ñерверный протокол, поÑтому любой Ñтандартный клиент ZooKeeper может иÑпользоватьÑÑ Ð´Ð»Ñ Ð²Ð·Ð°Ð¸Ð¼Ð¾Ð´ÐµÐ¹ÑÑ‚Ð²Ð¸Ñ Ñ ClickHouse Keeper. СнÑпшоты и журналы имеют неÑовмеÑтимый Ñ ZooKeeper формат, однако можно конвертировать данные Zookeeper в ÑнÑпшот ClickHouse Keeper Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ `clickhouse-keeper-converter`. МежÑерверный протокол ClickHouse Keeper также неÑовмеÑтим Ñ ZooKeeper, поÑтому Ñоздание Ñмешанного клаÑтера ZooKeeper / ClickHouse Keeper невозможно. +СиÑтема ÑƒÐ¿Ñ€Ð°Ð²Ð»ÐµÐ½Ð¸Ñ Ð´Ð¾Ñтупом (ACL) ClickHouse Keeper реализована точно так же, как в [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl). ClickHouse Keeper поддерживает тот же набор разрешений и имеет идентичные Ñхемы: `world`, `auth`, `digest`, `host` и `ip`. Digest Ð´Ð»Ñ Ð°ÑƒÑ‚ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ†Ð¸Ð¸ иÑпользует пару значений username:password. Пароль кодируетÑÑ Ð² Base64. + ## ÐšÐ¾Ð½Ñ„Ð¸Ð³ÑƒÑ€Ð°Ñ†Ð¸Ñ ClickHouse Keeper может иÑпользоватьÑÑ ÐºÐ°Ðº Ñ€Ð°Ð²Ð½Ð¾Ñ†ÐµÐ½Ð½Ð°Ñ Ð·Ð°Ð¼ÐµÐ½Ð° ZooKeeper или как внутреннÑÑ Ñ‡Ð°ÑÑ‚ÑŒ Ñервера ClickHouse, но в обоих ÑлучаÑÑ… ÐºÐ¾Ð½Ñ„Ð¸Ð³ÑƒÑ€Ð°Ñ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ñтавлена файлом `.xml`. Главный тег конфигурации ClickHouse Keeper — Ñто ``. Параметры конфигурации: - `tcp_port` — порт Ð´Ð»Ñ Ð¿Ð¾Ð´ÐºÐ»ÑŽÑ‡ÐµÐ½Ð¸Ñ ÐºÐ»Ð¸ÐµÐ½Ñ‚Ð° (по умолчанию Ð´Ð»Ñ ZooKeeper: `2181`). -- `tcp_port_secure` — зашифрованный порт Ð´Ð»Ñ Ð¿Ð¾Ð´ÐºÐ»ÑŽÑ‡ÐµÐ½Ð¸Ñ ÐºÐ»Ð¸ÐµÐ½Ñ‚Ð°. +- `tcp_port_secure` — зашифрованный порт Ð´Ð»Ñ SSL-ÑÐ¾ÐµÐ´Ð¸Ð½ÐµÐ½Ð¸Ñ Ð¼ÐµÐ¶Ð´Ñƒ клиентом и Ñервером ÑервиÑа. - `server_id` — уникальный идентификатор Ñервера, каждый учаÑтник клаÑтера должен иметь уникальный номер (1, 2, 3 Ð¸ Ñ‚. Ð´.). - `log_storage_path` — путь к журналам координации, лучше хранить их на незанÑтом уÑтройÑтве (актуально и Ð´Ð»Ñ ZooKeeper). - `snapshot_storage_path` — путь к ÑнÑпшотам координации. @@ -50,7 +52,11 @@ ClickHouse Keeper может иÑпользоватьÑÑ ÐºÐ°Ðº равноце - `shutdown_timeout` — Ð²Ñ€ÐµÐ¼Ñ Ð¾Ð¶Ð¸Ð´Ð°Ð½Ð¸Ñ Ð·Ð°Ð²ÐµÑ€ÑˆÐµÐ½Ð¸Ñ Ð²Ð½ÑƒÑ‚Ñ€ÐµÐ½Ð½Ð¸Ñ… подключений и выключениÑ, в миллиÑекундах (по умолчанию: 5000). - `startup_timeout` — Ð²Ñ€ÐµÐ¼Ñ Ð¾Ñ‚ÐºÐ»ÑŽÑ‡ÐµÐ½Ð¸Ñ Ñервера, еÑли он не подключаетÑÑ Ðº другим учаÑтникам кворума, в миллиÑекундах (по умолчанию: 30000). -ÐšÐ¾Ð½Ñ„Ð¸Ð³ÑƒÑ€Ð°Ñ†Ð¸Ñ ÐºÐ²Ð¾Ñ€ÑƒÐ¼Ð° находитÑÑ Ð² `.` и Ñодержит опиÑание Ñерверов. ЕдинÑтвенный параметр Ð´Ð»Ñ Ð²Ñего кворума — `secure`, который включает зашифрованное Ñоединение Ð´Ð»Ñ ÑвÑзи между учаÑтниками кворума. Параметры Ð´Ð»Ñ ÐºÐ°Ð¶Ð´Ð¾Ð³Ð¾ ``: +ÐšÐ¾Ð½Ñ„Ð¸Ð³ÑƒÑ€Ð°Ñ†Ð¸Ñ ÐºÐ²Ð¾Ñ€ÑƒÐ¼Ð° находитÑÑ Ð² `.` и Ñодержит опиÑание Ñерверов. + +ЕдинÑтвенный параметр Ð´Ð»Ñ Ð²Ñего кворума — `secure`, который включает зашифрованное Ñоединение Ð´Ð»Ñ ÑвÑзи между учаÑтниками кворума. Параметру можно задать значение `true`, еÑли Ð´Ð»Ñ Ð²Ð½ÑƒÑ‚Ñ€ÐµÐ½Ð½ÐµÐ¹ коммуникации между узлами требуетÑÑ SSL-Ñоединение, в ином Ñлучае не указывайте ничего. + +Параметры Ð´Ð»Ñ ÐºÐ°Ð¶Ð´Ð¾Ð³Ð¾ ``: - `id` — идентификатор Ñервера в кворуме. - `hostname` — Ð¸Ð¼Ñ Ñ…Ð¾Ñта, на котором размещен Ñервер. From 368fa9144a3c66ad6db49c66450a2afeb1ccd046 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Wed, 8 Dec 2021 14:52:21 +0800 Subject: [PATCH 112/358] 1. opt keeper sizeInBytes caculation. --- src/Coordination/KeeperSnapshotManager.cpp | 3 ++ src/Coordination/KeeperStorage.cpp | 33 +++++++++++++--------- src/Coordination/KeeperStorage.h | 13 ++++++++- 3 files changed, 34 insertions(+), 15 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index be6d4db4219..2bff7234066 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -76,6 +76,8 @@ namespace writeBinary(node.stat.pzxid, out); writeBinary(node.seq_num, out); + + writeBinary(node.size_bytes, out); } void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) @@ -124,6 +126,7 @@ namespace readBinary(node.stat.numChildren, in); readBinary(node.stat.pzxid, in); readBinary(node.seq_num, in); + readBinary(node.size_bytes, in); } void serializeSnapshotMetadata(const SnapshotMetadataPtr & snapshot_meta, WriteBuffer & out) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 81bb3d0dd7d..146cb162cbd 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -133,20 +133,20 @@ static bool fixupACL( return valid_found; } -uint64_t KeeperStorage::Node::sizeInBytes() const -{ - uint64_t total_size{0}; - for (const auto & child : children) - total_size += child.size(); - - total_size += data.size(); - - total_size += sizeof(acl_id); - total_size += sizeof(is_sequental); - total_size += sizeof(stat); - total_size += sizeof(seq_num); - return total_size; -} +//uint64_t KeeperStorage::Node::sizeInBytes() const +//{ +// uint64_t total_size{0}; +// for (const auto & child : children) +// total_size += child.size(); +// +// total_size += data.size(); +// +// total_size += sizeof(acl_id); +// total_size += sizeof(is_sequental); +// total_size += sizeof(stat); +// total_size += sizeof(seq_num); +// return total_size; +//} static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type) { @@ -354,6 +354,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr { parent.children.insert(child_path); + parent.size_bytes += child_path.size(); prev_parent_cversion = parent.stat.cversion; prev_parent_zxid = parent.stat.pzxid; @@ -391,6 +392,7 @@ struct KeeperStorageCreateRequestProcessor final : public KeeperStorageRequestPr undo_parent.stat.cversion = prev_parent_cversion; undo_parent.stat.pzxid = prev_parent_zxid; undo_parent.children.erase(child_path); + undo_parent.size_bytes -= child_path.size(); }); }; @@ -524,6 +526,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr --parent.stat.numChildren; ++parent.stat.cversion; parent.children.erase(child_basename); + parent.size_bytes -= child_basename.size(); }); response.error = Coordination::Error::ZOK; @@ -543,6 +546,7 @@ struct KeeperStorageRemoveRequestProcessor final : public KeeperStorageRequestPr ++parent.stat.numChildren; --parent.stat.cversion; parent.children.insert(child_basename); + parent.size_bytes += child_basename.size(); }); }; } @@ -1110,6 +1114,7 @@ KeeperStorage::ResponsesForSessions KeeperStorage::processRequest(const Coordina --parent.stat.numChildren; ++parent.stat.cversion; parent.children.erase(getBaseName(ephemeral_path)); + parent.size_bytes -= getBaseName(ephemeral_path).size(); }); auto responses = processWatchesImpl(ephemeral_path, watches, list_watches, Coordination::Event::DELETED); diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 44dc1b2b43b..9f086f1ba69 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -35,9 +35,20 @@ public: Coordination::Stat stat{}; int32_t seq_num = 0; ChildrenSet children{}; + uint64_t size_bytes; // save size to avoid caculate every time + Node() { + size_bytes = sizeof(size_bytes); + size_bytes += data.size(); + size_bytes += sizeof(acl_id); + size_bytes += sizeof(is_sequental); + size_bytes += sizeof(stat); + size_bytes += sizeof(seq_num); + } /// Object memory size - uint64_t sizeInBytes() const; + uint64_t sizeInBytes() const { + return size_bytes; + } }; struct ResponseForSession From 61a5f8a61adc53729d7f794d3a75a666a6ac4997 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 8 Dec 2021 18:29:00 +0300 Subject: [PATCH 113/358] add comments --- src/DataTypes/IDataType.h | 2 +- src/DataTypes/Serializations/SerializationInfo.cpp | 14 +++++++++++--- src/DataTypes/Serializations/SerializationInfo.h | 10 +++++----- src/Interpreters/InterpreterInsertQuery.cpp | 2 ++ src/Processors/Chunk.h | 4 ++++ src/Processors/Transforms/DistinctTransform.cpp | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++-- src/Storages/MergeTree/IMergeTreeDataPart.h | 6 ++++-- .../MergeTree/IMergedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergedBlockOutputStream.cpp | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 2 +- 12 files changed, 34 insertions(+), 17 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 894571e114e..864dde0e86e 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -107,7 +107,7 @@ public: SerializationPtr getDefaultSerialization() const; SerializationPtr getSparseSerialization() const; - /// Chooses serialziation according to serialization kind. + /// Chooses serialization according to serialization kind. SerializationPtr getSerialization(ISerialization::Kind kind) const; /// Chooses serialization according to collected information about content of column. diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 9b683b6b0aa..08badd276b1 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -119,7 +119,7 @@ void SerializationInfo::fromJSON(const Poco::JSON::Object & object) ISerialization::Kind SerializationInfo::chooseKind(const Data & data, const Settings & settings) { double ratio = data.num_rows ? std::min(static_cast(data.num_defaults) / data.num_rows, 1.0) : 0.0; - return ratio > settings.ratio_for_sparse ? ISerialization::Kind::SPARSE : ISerialization::Kind::DEFAULT; + return ratio > settings.ratio_of_defaults_for_sparse ? ISerialization::Kind::SPARSE : ISerialization::Kind::DEFAULT; } SerializationInfoByName::SerializationInfoByName( @@ -158,7 +158,7 @@ void SerializationInfoByName::add(const SerializationInfoByName & other) } } -void SerializationInfoByName::writeText(WriteBuffer & out) const +void SerializationInfoByName::writeJSON(WriteBuffer & out) const { Poco::JSON::Object object; object.set(KEY_VERSION, SERIALIZATION_INFO_VERSION); @@ -180,7 +180,7 @@ void SerializationInfoByName::writeText(WriteBuffer & out) const return writeString(oss.str(), out); } -void SerializationInfoByName::readText(ReadBuffer & in) +void SerializationInfoByName::readJSON(ReadBuffer & in) { String json_str; readString(json_str, in); @@ -188,6 +188,14 @@ void SerializationInfoByName::readText(ReadBuffer & in) Poco::JSON::Parser parser; auto object = parser.parse(json_str).extract(); + if (object->has(KEY_VERSION)) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Missed version of serialization infos"); + + if (object->getValue(KEY_VERSION) > SERIALIZATION_INFO_VERSION) + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Unknown version of serialization infos ({}). Should be less or equal than {}", + object->getValue(KEY_VERSION), SERIALIZATION_INFO_VERSION); + if (object->has(KEY_COLUMNS)) { auto array = object->getArray(KEY_COLUMNS); diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index 55faa965145..f7af5d77217 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -13,7 +13,7 @@ class WriteBuffer; class NamesAndTypesList; class Block; -constexpr auto SERIALIZATION_INFO_VERSION = 1; +constexpr auto SERIALIZATION_INFO_VERSION = 0; /** Contains information about kind of serialization of column and its subcolumns. * Also contains information about content of columns, @@ -38,10 +38,10 @@ public: struct Settings { - const double ratio_for_sparse = 1.0; + const double ratio_of_defaults_for_sparse = 1.0; const bool choose_kind = false; - bool isAlwaysDefault() const { return ratio_for_sparse >= 1.0; } + bool isAlwaysDefault() const { return ratio_of_defaults_for_sparse >= 1.0; } }; SerializationInfo(ISerialization::Kind kind_, const Settings & settings_); @@ -89,8 +89,8 @@ public: void add(const Block & block); void add(const SerializationInfoByName & other); - void writeText(WriteBuffer & out) const; - void readText(ReadBuffer & in); + void writeJSON(WriteBuffer & out) const; + void readJSON(ReadBuffer & in); }; } diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index ca334e2e1e1..6caf7145b2a 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -407,6 +407,8 @@ BlockIO InterpreterInsertQuery::execute() return std::make_shared(in_header, actions); }); + /// We need to convert Sparse columns to full, because it's destination storage + /// may not support it may have different settings for applying Sparse serialization. pipeline.addSimpleTransform([&](const Block & in_header) -> ProcessorPtr { return std::make_shared(in_header); diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index 0aa7004af1f..e70ba57a267 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -131,6 +131,10 @@ private: RowsMaskByColumnId rows_mask_by_column_id; }; +/// Converts all columns to full serialization in chunk. +/// It's needed, when you have to access to the internals of the column, +/// or when you need to perform operation with two columns +/// and their structure must be equal (e.g. compareAt). void convertToFullIfSparse(Chunk & chunk); } diff --git a/src/Processors/Transforms/DistinctTransform.cpp b/src/Processors/Transforms/DistinctTransform.cpp index 5cff91eabde..cf6a128aa40 100644 --- a/src/Processors/Transforms/DistinctTransform.cpp +++ b/src/Processors/Transforms/DistinctTransform.cpp @@ -54,6 +54,7 @@ void DistinctTransform::buildFilter( void DistinctTransform::transform(Chunk & chunk) { + /// Convert to full column, because SetVariant for sparse column is not implemented. convertToFullIfSparse(chunk); auto num_rows = chunk.getNumRows(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index e9b77be973e..e69584a41f3 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1073,14 +1073,14 @@ void IMergeTreeDataPart::loadColumns(bool require) SerializationInfo::Settings settings = { - .ratio_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .ratio_of_defaults_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, .choose_kind = false, }; SerializationInfoByName infos(loaded_columns, settings); path = getFullRelativePath() + SERIALIZATION_FILE_NAME; if (volume->getDisk()->exists(path)) - infos.readText(*volume->getDisk()->readFile(path)); + infos.readJSON(*volume->getDisk()->readFile(path)); setColumns(loaded_columns, infos); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 07ce8919be3..f5cadf56760 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -393,6 +393,9 @@ public: static inline constexpr auto UUID_FILE_NAME = "uuid.txt"; + /// File that contains information about kinds of serialization of columns + /// and information that helps to choose kind of serialization later during merging + /// (number of rows, number of rows with default values, etc). static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.txt"; /// Checks that all TTLs (table min/max, column ttls, so on) for part @@ -447,10 +450,9 @@ private: /// In compact parts order of columns is necessary NameToNumber column_name_to_position; + /// Map from name of column to its serialization info. SerializationInfoByName serialization_infos; - SerializationByName serializations; - /// Reads part unique identifier (if exists) from uuid.txt void loadUUID(); diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 6e00a8d6a6d..5393d71ff86 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -20,7 +20,7 @@ IMergedBlockOutputStream::IMergedBlockOutputStream( { SerializationInfo::Settings info_settings = { - .ratio_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, + .ratio_of_defaults_for_sparse = storage.getSettings()->ratio_of_defaults_for_sparse_serialization, .choose_kind = false, }; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index ec0c13f5f92..9feba8f5a0a 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -167,7 +167,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() SerializationInfo::Settings info_settings = { - .ratio_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, + .ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization, .choose_kind = true, }; diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index a78c840fdac..cbdbb2339df 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -177,7 +177,7 @@ void MergedBlockOutputStream::finalizePartOnDisk( { auto out = volume->getDisk()->writeFile(part_path + IMergeTreeDataPart::SERIALIZATION_FILE_NAME, 4096); HashingWriteBuffer out_hashing(*out); - serialization_infos.writeText(out_hashing); + serialization_infos.writeJSON(out_hashing); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_size = out_hashing.count(); checksums.files[IMergeTreeDataPart::SERIALIZATION_FILE_NAME].file_hash = out_hashing.getHash(); out->finalize(); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 6b46e65e2c1..eabd901eb24 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -104,7 +104,7 @@ IMergeTreeDataPart::Checksums checkDataPart( if (disk->exists(serialization_path)) { auto serialization_file = disk->readFile(serialization_path); - serialization_infos.readText(*serialization_file); + serialization_infos.readJSON(*serialization_file); } auto get_serialization = [&serialization_infos](const auto & column) From ea67abf3f06dedb18e2b72804409e53d2093e541 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 8 Dec 2021 21:59:36 +0300 Subject: [PATCH 114/358] fix custom serializations in native protocol --- src/Columns/ColumnConst.h | 5 +++++ src/Core/ProtocolDefines.h | 4 ++-- src/DataTypes/Serializations/SerializationInfo.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 4ca97df7f74..1ddc8789e7d 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -261,6 +261,11 @@ public: } } + SerializationInfoPtr getSerializationInfo() const override + { + return data->getSerializationInfo(); + } + bool isNullable() const override { return isColumnNullable(*data); } bool onlyNull() const override { return data->isNullAt(0); } bool isNumeric() const override { return data->isNumeric(); } diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index f225055da67..04adeef224e 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -48,8 +48,8 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54452 +#define DBMS_TCP_PROTOCOL_VERSION 54453 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 -#define DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION 54452 +#define DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION 54453 diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 08badd276b1..42d3d14b672 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -188,7 +188,7 @@ void SerializationInfoByName::readJSON(ReadBuffer & in) Poco::JSON::Parser parser; auto object = parser.parse(json_str).extract(); - if (object->has(KEY_VERSION)) + if (!object->has(KEY_VERSION)) throw Exception(ErrorCodes::CORRUPTED_DATA, "Missed version of serialization infos"); if (object->getValue(KEY_VERSION) > SERIALIZATION_INFO_VERSION) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index f5cadf56760..c5b942b0388 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -396,7 +396,7 @@ public: /// File that contains information about kinds of serialization of columns /// and information that helps to choose kind of serialization later during merging /// (number of rows, number of rows with default values, etc). - static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.txt"; + static inline constexpr auto SERIALIZATION_FILE_NAME = "serialization.json"; /// Checks that all TTLs (table min/max, column ttls, so on) for part /// calculated. Part without calculated TTL may exist if TTL was added after From 21f50f95b2135e51bcf3936122d1eac937a7ecf3 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 9 Dec 2021 18:04:29 +0800 Subject: [PATCH 115/358] Update KeeperStorage.h fix code style --- src/Coordination/KeeperStorage.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 9f086f1ba69..dfe0a3d550e 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -37,7 +37,8 @@ public: ChildrenSet children{}; uint64_t size_bytes; // save size to avoid caculate every time - Node() { + Node() + { size_bytes = sizeof(size_bytes); size_bytes += data.size(); size_bytes += sizeof(acl_id); @@ -46,7 +47,8 @@ public: size_bytes += sizeof(seq_num); } /// Object memory size - uint64_t sizeInBytes() const { + uint64_t sizeInBytes() const + { return size_bytes; } }; From 359257483a89013152b6da8da63ab8f01c26ffb3 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 9 Dec 2021 18:05:11 +0800 Subject: [PATCH 116/358] Update KeeperStorage.h --- src/Coordination/KeeperStorage.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index dfe0a3d550e..9b1c1b018e4 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -35,7 +35,7 @@ public: Coordination::Stat stat{}; int32_t seq_num = 0; ChildrenSet children{}; - uint64_t size_bytes; // save size to avoid caculate every time + uint64_t size_bytes; // save size to avoid calculate every time Node() { From 91f26edecaaf1472def26427d56002a887a623f4 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 9 Dec 2021 18:15:53 +0800 Subject: [PATCH 117/358] Update KeeperStorage.h delete trailing whitespaces. --- src/Coordination/KeeperStorage.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 9b1c1b018e4..f61b17a88a6 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -37,7 +37,7 @@ public: ChildrenSet children{}; uint64_t size_bytes; // save size to avoid calculate every time - Node() + Node() { size_bytes = sizeof(size_bytes); size_bytes += data.size(); @@ -47,7 +47,7 @@ public: size_bytes += sizeof(seq_num); } /// Object memory size - uint64_t sizeInBytes() const + uint64_t sizeInBytes() const { return size_bytes; } From 2509e879db4bc7f66531d08bf4acdc302e13216a Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 9 Dec 2021 23:09:56 +0800 Subject: [PATCH 118/358] 1. snapshot add size of Node. --- src/Coordination/KeeperSnapshotManager.cpp | 15 +++++++++++---- src/Coordination/KeeperSnapshotManager.h | 3 ++- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 2bff7234066..518d569ca67 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -55,7 +55,7 @@ namespace return "/"; } - void writeNode(const KeeperStorage::Node & node, WriteBuffer & out) + void writeNode(const KeeperStorage::Node & node, SnapshotVersion version, WriteBuffer & out) { writeBinary(node.data, out); @@ -77,7 +77,10 @@ namespace writeBinary(node.seq_num, out); - writeBinary(node.size_bytes, out); + if (version >= SnapshotVersion::V4) + { + writeBinary(node.size_bytes, out); + } } void readNode(KeeperStorage::Node & node, ReadBuffer & in, SnapshotVersion version, ACLMap & acl_map) @@ -126,7 +129,11 @@ namespace readBinary(node.stat.numChildren, in); readBinary(node.stat.pzxid, in); readBinary(node.seq_num, in); - readBinary(node.size_bytes, in); + + if (version >= SnapshotVersion::V4) + { + readBinary(node.size_bytes, in); + } } void serializeSnapshotMetadata(const SnapshotMetadataPtr & snapshot_meta, WriteBuffer & out) @@ -179,7 +186,7 @@ void KeeperStorageSnapshot::serialize(const KeeperStorageSnapshot & snapshot, Wr throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to serialize node with mzxid {}, but last snapshot index {}", node.stat.mzxid, snapshot.snapshot_meta->get_last_log_idx()); writeBinary(path, out); - writeNode(node, out); + writeNode(node, snapshot.version, out); /// Last iteration: check and exit here without iterator increment. Otherwise /// false positive race condition on list end is possible. diff --git a/src/Coordination/KeeperSnapshotManager.h b/src/Coordination/KeeperSnapshotManager.h index 2889ec493df..174864a0ceb 100644 --- a/src/Coordination/KeeperSnapshotManager.h +++ b/src/Coordination/KeeperSnapshotManager.h @@ -18,9 +18,10 @@ enum SnapshotVersion : uint8_t V1 = 1, /// with ACL map V2 = 2, /// with 64 bit buffer header V3 = 3, /// compress snapshots with ZSTD codec + V4 = 4, /// add Node size to snapshots }; -static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V3; +static constexpr auto CURRENT_SNAPSHOT_VERSION = SnapshotVersion::V4; /// What is stored in binary shapsnot struct SnapshotDeserializationResult From 30af5d5341611e53aeeece37613890e7cdeddff7 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Fri, 10 Dec 2021 00:02:26 +0300 Subject: [PATCH 119/358] update-external-integrations --- docs/en/operations/clickhouse-keeper.md | 5 ++++- docs/ru/operations/clickhouse-keeper.md | 3 +++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 34dbaf522e3..4f2117a3fbf 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -16,7 +16,10 @@ ZooKeeper is one of the first well-known open-source coordination systems. It's By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but `clickhouse-keeper-converter` tool allows to convert ZooKeeper data to ClickHouse Keeper snapshot. Interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so mixed ZooKeeper / ClickHouse Keeper cluster is impossible. -ClickHouse Keeper supports Access Control List the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth`, `digest`, `host` and `ip`. Digest authentication scheme uses pair username:password. Password is encoded in Base64. +ClickHouse Keeper supports Access Control List the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth`, `digest`, `host` and `ip`. Digest authentication scheme uses pair username:password. Password is encoded in Base64. + +!!! info "Note" + External integrations are not supported. ## Configuration diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index de3faea9d56..bf0344894d9 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -18,6 +18,9 @@ ZooKeeper — один из первых широко извеÑтных Ñер СиÑтема ÑƒÐ¿Ñ€Ð°Ð²Ð»ÐµÐ½Ð¸Ñ Ð´Ð¾Ñтупом (ACL) ClickHouse Keeper реализована точно так же, как в [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl). ClickHouse Keeper поддерживает тот же набор разрешений и имеет идентичные Ñхемы: `world`, `auth`, `digest`, `host` и `ip`. Digest Ð´Ð»Ñ Ð°ÑƒÑ‚ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ†Ð¸Ð¸ иÑпользует пару значений username:password. Пароль кодируетÑÑ Ð² Base64. +!!! info "Примечание" + Внешние интеграции не поддерживаютÑÑ. + ## ÐšÐ¾Ð½Ñ„Ð¸Ð³ÑƒÑ€Ð°Ñ†Ð¸Ñ ClickHouse Keeper может иÑпользоватьÑÑ ÐºÐ°Ðº Ñ€Ð°Ð²Ð½Ð¾Ñ†ÐµÐ½Ð½Ð°Ñ Ð·Ð°Ð¼ÐµÐ½Ð° ZooKeeper или как внутреннÑÑ Ñ‡Ð°ÑÑ‚ÑŒ Ñервера ClickHouse, но в обоих ÑлучаÑÑ… ÐºÐ¾Ð½Ñ„Ð¸Ð³ÑƒÑ€Ð°Ñ†Ð¸Ñ Ð¿Ñ€ÐµÐ´Ñтавлена файлом `.xml`. Главный тег конфигурации ClickHouse Keeper — Ñто ``. Параметры конфигурации: From de3aad085d35e7cf0d404d944400c3b98595ab85 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Fri, 10 Dec 2021 10:26:28 +0800 Subject: [PATCH 120/358] 1. fix ut fail. --- src/Coordination/tests/gtest_coordination.cpp | 12 ++++++------ src/Parsers/tests/gtest_Parser.cpp | 2 +- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index d5498a1bc13..434236a9b52 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -979,24 +979,24 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) world.disableSnapshotMode(); world.insert("world", n1); - EXPECT_EQ(world.getApproximateDataSize(), 94); + EXPECT_EQ(world.getApproximateDataSize(), 98); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 96); + EXPECT_EQ(world.getApproximateDataSize(), 98); world.erase("world"); EXPECT_EQ(world.getApproximateDataSize(), 0); world.enableSnapshotMode(); world.insert("world", n1); - EXPECT_EQ(world.getApproximateDataSize(), 94); + EXPECT_EQ(world.getApproximateDataSize(), 98); world.updateValue("world", [&](Node & value) { value = n2; }); - EXPECT_EQ(world.getApproximateDataSize(), 190); + EXPECT_EQ(world.getApproximateDataSize(), 196); world.clearOutdatedNodes(); - EXPECT_EQ(world.getApproximateDataSize(), 96); + EXPECT_EQ(world.getApproximateDataSize(), 98); world.erase("world"); - EXPECT_EQ(world.getApproximateDataSize(), 96); + EXPECT_EQ(world.getApproximateDataSize(), 98); world.clear(); EXPECT_EQ(world.getApproximateDataSize(), 0); diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 5ebea834a91..00c13a5f342 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -157,7 +157,7 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) LAYOUT(FLAT()) LIFETIME(MIN 0 MAX 1000) COMMENT 'Test dictionary with comment'; -)sql", +sql", R"sql(CREATE DICTIONARY `2024_dictionary_with_comment` ( `id` UInt64, From 505fbc78c77a18e9592013a9ddb1e3840812005f Mon Sep 17 00:00:00 2001 From: Dao Date: Sat, 11 Dec 2021 00:18:11 +0800 Subject: [PATCH 121/358] fix groupBitmapAnd function on distributed table --- .../AggregateFunctionGroupBitmap.h | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index 77dcede4c20..f3322cf15c8 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -105,9 +105,17 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).rbs.write(buf); } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + DB::writeBoolText(this->data(place).init, buf); + this->data(place).rbs.write(buf); + } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).rbs.read(buf); } + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + DB::readBoolText(this->data(place).init, buf); + this->data(place).rbs.read(buf); + } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { From 9aeaa339c60a0e40b9edaca6740539cddf19accd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 10 Dec 2021 10:28:49 +0300 Subject: [PATCH 122/358] Make 01675_distributed_bytes_to_delay_insert less flaky (use http over client) client is too heavy in debug builds, switching to http reduces test time from 22s to 9s overall. And also drop `_long` suffix form file name, since `clickhouse-test` support `Tags` in tests. CI: https://s3.amazonaws.com/clickhouse-test-reports/32303/24751e7d45d94541be854c86ce46d65c2e0f66da/stateless_tests__debug__actions_.html v2: relax pattern --- ...stributed_bytes_to_delay_insert.reference} | 1 + ...01675_distributed_bytes_to_delay_insert.sh | 125 +++++++++++++++++ ..._distributed_bytes_to_delay_insert_long.sh | 130 ------------------ 3 files changed, 126 insertions(+), 130 deletions(-) rename tests/queries/0_stateless/{01675_distributed_bytes_to_delay_insert_long.reference => 01675_distributed_bytes_to_delay_insert.reference} (61%) create mode 100755 tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert.sh delete mode 100755 tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert_long.sh diff --git a/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert_long.reference b/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert.reference similarity index 61% rename from tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert_long.reference rename to tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert.reference index 343d1f3639f..63972b90eda 100644 --- a/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert_long.reference +++ b/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert.reference @@ -1,2 +1,3 @@ max_delay_to_insert will throw +Too many bytes pending for async INSERT max_delay_to_insert will succeed diff --git a/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert.sh b/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert.sh new file mode 100755 index 00000000000..a02589dde94 --- /dev/null +++ b/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert.sh @@ -0,0 +1,125 @@ +#!/usr/bin/env bash +# Tags: long, distributed + +# NOTE: $SECONDS accuracy is second, so we need some delta, hence -1 in time conditions. + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function drop_tables() +{ + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"drop table if exists dist_01675" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"drop table if exists data_01675" +} + +# +# Case 1: max_delay_to_insert will throw. +# +function test_max_delay_to_insert_will_throw() +{ + echo "max_delay_to_insert will throw" + + local max_delay_to_insert=2 + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"create table data_01675 (key Int) engine=Null()" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"system stop distributed sends dist_01675" + + local start_seconds=$SECONDS + # first batch is always OK, since there is no pending bytes yet + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0" + # second will fail, because of bytes_to_delay_insert=1 and max_delay_to_insert>0, + # while distributed sends is stopped. + # + # (previous block definitelly takes more, since it has header) + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0" |& grep -o 'Too many bytes pending for async INSERT' + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"system flush distributed dist_01675" + local end_seconds=$SECONDS + + if (( (end_seconds-start_seconds)<(max_delay_to_insert-1) )); then + echo "max_delay_to_insert was not satisfied ($end_seconds-$start_seconds)" + fi +} + +# +# Case 2: max_delay_to_insert will finally finished. +# +function test_max_delay_to_insert_will_succeed_once() +{ + local max_delay_to_insert=4 + local flush_delay=2 + + drop_tables + + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"create table data_01675 (key Int) engine=Null()" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert" + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"system stop distributed sends dist_01675" + + function flush_distributed_worker() + { + sleep $flush_delay + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -d @- <<<"system flush distributed dist_01675" + } + flush_distributed_worker & + + local start_seconds=$SECONDS + { + # NOTE: + # ignore stderr, since it may produce exception if flushing thread will be too slow + # (this is possible on CI) + + # first batch is always OK, since there is no pending bytes yet + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1" -d @- <<<"insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0" >& /dev/null + # second will succeed, due to SYSTEM FLUSH DISTRIBUTED in background. + ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&extremes=1" -d @- <<<"insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0" >& /dev/null + } + local end_seconds=$SECONDS + + wait + + local diff=$(( end_seconds-start_seconds )) + + if (( diff<(flush_delay-1) )); then + # this is fatal error, that should not be retriable + echo "max_delay_to_insert was not wait flush_delay ($diff)" + exit 1 + fi + + # retry the test until the diff will be satisfied + # (since we should not assume that there will be no other lags) + if (( diff>=(max_delay_to_insert-1) )); then + return 1 + fi + + return 0 +} +function test_max_delay_to_insert_will_succeed() +{ + echo "max_delay_to_insert will succeed" + + local retries=20 i=0 + while (( (i++) < retries )); do + if test_max_delay_to_insert_will_succeed_once; then + return + fi + done + + echo failed +} + +function run_test() +{ + local test_case=$1 && shift + + drop_tables + $test_case +} + +function main() +{ + run_test test_max_delay_to_insert_will_throw + run_test test_max_delay_to_insert_will_succeed + + drop_tables +} +main "$@" diff --git a/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert_long.sh b/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert_long.sh deleted file mode 100755 index e373f632155..00000000000 --- a/tests/queries/0_stateless/01675_distributed_bytes_to_delay_insert_long.sh +++ /dev/null @@ -1,130 +0,0 @@ -#!/usr/bin/env bash -# Tags: long, distributed - -# NOTE: $SECONDS accuracy is second, so we need some delta, hence -1 in time conditions. - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -function drop_tables() -{ - ${CLICKHOUSE_CLIENT} -nq " - drop table if exists dist_01675; - drop table if exists data_01675; - " -} - -# -# Case 1: max_delay_to_insert will throw. -# -function test_max_delay_to_insert_will_throw() -{ - echo "max_delay_to_insert will throw" - - local max_delay_to_insert=2 - ${CLICKHOUSE_CLIENT} -nq " - create table data_01675 (key Int) engine=Null(); - create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert; - system stop distributed sends dist_01675; - " - - local start_seconds=$SECONDS - ${CLICKHOUSE_CLIENT} --testmode -nq " - -- first batch is always OK, since there is no pending bytes yet - insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0; - -- second will fail, because of bytes_to_delay_insert=1 and max_delay_to_insert>0, - -- while distributed sends is stopped. - -- - -- (previous block definitelly takes more, since it has header) - insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0; -- { serverError 574 } - system flush distributed dist_01675; - " - local end_seconds=$SECONDS - - if (( (end_seconds-start_seconds)<(max_delay_to_insert-1) )); then - echo "max_delay_to_insert was not satisfied ($end_seconds-$start_seconds)" - fi -} - -# -# Case 2: max_delay_to_insert will finally finished. -# -function test_max_delay_to_insert_will_succeed_once() -{ - local max_delay_to_insert=4 - local flush_delay=2 - - drop_tables - - ${CLICKHOUSE_CLIENT} -nq " - create table data_01675 (key Int) engine=Null(); - create table dist_01675 (key Int) engine=Distributed(test_shard_localhost, currentDatabase(), data_01675) settings bytes_to_delay_insert=1, max_delay_to_insert=$max_delay_to_insert; - system stop distributed sends dist_01675; - " - - function flush_distributed_worker() - { - sleep $flush_delay - ${CLICKHOUSE_CLIENT} -q "system flush distributed dist_01675" - } - flush_distributed_worker & - - local start_seconds=$SECONDS - # ignore stderr, since it may produce exception if flushing thread will be too slow - # (this is possible on CI) - ${CLICKHOUSE_CLIENT} --testmode -nq " - -- first batch is always OK, since there is no pending bytes yet - insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0; - -- second will succeed, due to SYSTEM FLUSH DISTRIBUTED in background. - insert into dist_01675 select * from numbers(1) settings prefer_localhost_replica=0; - " >& /dev/null - local end_seconds=$SECONDS - wait - - local diff=$(( end_seconds-start_seconds )) - - if (( diff<(flush_delay-1) )); then - # this is fatal error, that should not be retriable - echo "max_delay_to_insert was not wait flush_delay ($diff)" - exit 1 - fi - - # retry the test until the diff will be satisfied - # (since we cannot assume that there will be no other lags) - if (( diff>=(max_delay_to_insert-1) )); then - return 1 - fi - - return 0 -} -function test_max_delay_to_insert_will_succeed() -{ - echo "max_delay_to_insert will succeed" - - local retries=20 i=0 - while (( (i++) < retries )); do - if test_max_delay_to_insert_will_succeed_once; then - return - fi - done - - echo failed -} - -function run_test() -{ - local test_case=$1 && shift - - drop_tables - $test_case -} - -function main() -{ - run_test test_max_delay_to_insert_will_throw - run_test test_max_delay_to_insert_will_succeed - - drop_tables -} -main "$@" From 933cc770478d4a139ef806d29d078adfc7b02803 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Sat, 11 Dec 2021 00:12:36 +0300 Subject: [PATCH 123/358] Apply suggestions from code review Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/operations/clickhouse-keeper.md | 2 +- docs/ru/operations/clickhouse-keeper.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 5cb0e858d2c..065223224c1 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -16,7 +16,7 @@ ZooKeeper is one of the first well-known open-source coordination systems. It's By default, ClickHouse Keeper provides the same guarantees as ZooKeeper (linearizable writes, non-linearizable reads). It has a compatible client-server protocol, so any standard ZooKeeper client can be used to interact with ClickHouse Keeper. Snapshots and logs have an incompatible format with ZooKeeper, but `clickhouse-keeper-converter` tool allows to convert ZooKeeper data to ClickHouse Keeper snapshot. Interserver protocol in ClickHouse Keeper is also incompatible with ZooKeeper so mixed ZooKeeper / ClickHouse Keeper cluster is impossible. -ClickHouse Keeper supports Access Control List the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth`, `digest`, `host` and `ip`. Digest authentication scheme uses pair username:password. Password is encoded in Base64. +ClickHouse Keeper supports Access Control List (ACL) the same way as [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) does. ClickHouse Keeper supports the same set of permissions and has the identical built-in schemes: `world`, `auth`, `digest`, `host` and `ip`. Digest authentication scheme uses pair `username:password`. Password is encoded in Base64. !!! info "Note" External integrations are not supported. diff --git a/docs/ru/operations/clickhouse-keeper.md b/docs/ru/operations/clickhouse-keeper.md index bf0344894d9..9d6c4799008 100644 --- a/docs/ru/operations/clickhouse-keeper.md +++ b/docs/ru/operations/clickhouse-keeper.md @@ -16,7 +16,7 @@ ZooKeeper — один из первых широко извеÑтных Ñер По умолчанию ClickHouse Keeper предоÑтавлÑет те же гарантии, что и ZooKeeper (линеаризуемоÑÑ‚ÑŒ запиÑей, поÑÐ»ÐµÐ´Ð¾Ð²Ð°Ñ‚ÐµÐ»ÑŒÐ½Ð°Ñ ÑоглаÑованноÑÑ‚ÑŒ чтений). У него еÑÑ‚ÑŒ ÑовмеÑтимый клиент-Ñерверный протокол, поÑтому любой Ñтандартный клиент ZooKeeper может иÑпользоватьÑÑ Ð´Ð»Ñ Ð²Ð·Ð°Ð¸Ð¼Ð¾Ð´ÐµÐ¹ÑÑ‚Ð²Ð¸Ñ Ñ ClickHouse Keeper. СнÑпшоты и журналы имеют неÑовмеÑтимый Ñ ZooKeeper формат, однако можно конвертировать данные Zookeeper в ÑнÑпшот ClickHouse Keeper Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ `clickhouse-keeper-converter`. МежÑерверный протокол ClickHouse Keeper также неÑовмеÑтим Ñ ZooKeeper, поÑтому Ñоздание Ñмешанного клаÑтера ZooKeeper / ClickHouse Keeper невозможно. -СиÑтема ÑƒÐ¿Ñ€Ð°Ð²Ð»ÐµÐ½Ð¸Ñ Ð´Ð¾Ñтупом (ACL) ClickHouse Keeper реализована точно так же, как в [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl). ClickHouse Keeper поддерживает тот же набор разрешений и имеет идентичные Ñхемы: `world`, `auth`, `digest`, `host` и `ip`. Digest Ð´Ð»Ñ Ð°ÑƒÑ‚ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ†Ð¸Ð¸ иÑпользует пару значений username:password. Пароль кодируетÑÑ Ð² Base64. +СиÑтема ÑƒÐ¿Ñ€Ð°Ð²Ð»ÐµÐ½Ð¸Ñ Ð´Ð¾Ñтупом (ACL) ClickHouse Keeper реализована так же, как в [ZooKeeper](https://zookeeper.apache.org/doc/r3.1.2/zookeeperProgrammers.html#sc_ZooKeeperAccessControl). ClickHouse Keeper поддерживает тот же набор разрешений и идентичные Ñхемы: `world`, `auth`, `digest`, `host` и `ip`. Digest Ð´Ð»Ñ Ð°ÑƒÑ‚ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ†Ð¸Ð¸ иÑпользует пару значений `username:password`. Пароль кодируетÑÑ Ð² Base64. !!! info "Примечание" Внешние интеграции не поддерживаютÑÑ. From 2af221df1814b629f7ad750c15e7c2d2a4f5952f Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Sat, 11 Dec 2021 22:24:59 +0800 Subject: [PATCH 124/358] 1. update node size in set processor. --- src/Coordination/KeeperStorage.cpp | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 146cb162cbd..93e4aecc1be 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -133,21 +133,6 @@ static bool fixupACL( return valid_found; } -//uint64_t KeeperStorage::Node::sizeInBytes() const -//{ -// uint64_t total_size{0}; -// for (const auto & child : children) -// total_size += child.size(); -// -// total_size += data.size(); -// -// total_size += sizeof(acl_id); -// total_size += sizeof(is_sequental); -// total_size += sizeof(stat); -// total_size += sizeof(seq_num); -// return total_size; -//} - static KeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, KeeperStorage::Watches & watches, KeeperStorage::Watches & list_watches, Coordination::Event event_type) { KeeperStorage::ResponsesForSessions result; @@ -625,11 +610,11 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce auto itr = container.updateValue(request.path, [zxid, request] (KeeperStorage::Node & value) { - value.data = request.data; value.stat.version++; value.stat.mzxid = zxid; value.stat.mtime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); value.stat.dataLength = request.data.length(); + value.size_bytes += request.data.size() - value.data.size(); value.data = request.data; }); From 8b7eeea1625f0dba774a7a94e69e618da14ee6a5 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sat, 11 Dec 2021 23:09:06 +0300 Subject: [PATCH 125/358] parsing enums in TSV/CSV --- docs/ru/interfaces/formats.md | 6 ++ docs/ru/operations/settings/settings.md | 83 +++++++++++++++++++------ 2 files changed, 69 insertions(+), 20 deletions(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 79d760271f5..996514aa068 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -129,6 +129,9 @@ world Каждый Ñлемент Ñтруктуры типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) предÑтавлÑетÑÑ ÐºÐ°Ðº отдельный маÑÑив. +Ð—Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑлениÑ, в качеÑтве входных данных, могут быть предÑтавлены как Ð¸Ð¼Ñ Ð¸Ð»Ð¸ как идентификаторы. Сначала мы пытаемÑÑ ÑопоÑтавить входное значение Ñ Ð¸Ð¼ÐµÐ½ÐµÐ¼ перечиÑлениÑ. Ð’ Ñлучае неудачи и при уÑловии, что входное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, мы пытаемÑÑ ÑопоÑтавить Ñто чиÑло Ñ Ð¸Ð´ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ‚Ð¾Ñ€Ð¾Ð¼ перечиÑлениÑ. +ЕÑли входные данные Ñодержат только ENUM идентификаторы, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга перечиÑлениÑ. + Ðапример: ``` sql @@ -362,6 +365,9 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR ЕÑли уÑтановлена наÑтройка [input_format_defaults_for_omitted_fields = 1](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) и тип Ñтолбца не `Nullable(T)`, то пуÑтые Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð±ÐµÐ· кавычек заменÑÑŽÑ‚ÑÑ Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñми по умолчанию Ð´Ð»Ñ Ñ‚Ð¸Ð¿Ð° данных Ñтолбца. +Ð—Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑлениÑ, в качеÑтве входных данных, могут быть предÑтавлены как Ð¸Ð¼Ñ Ð¸Ð»Ð¸ как идентификаторы. Сначала мы пытаемÑÑ ÑопоÑтавить входное значение Ñ Ð¸Ð¼ÐµÐ½ÐµÐ¼ перечиÑлениÑ. Ð’ Ñлучае неудачи и при уÑловии, что входное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, мы пытаемÑÑ ÑопоÑтавить Ñто чиÑло Ñ Ð¸Ð´ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ‚Ð¾Ñ€Ð¾Ð¼ перечиÑлениÑ. +ЕÑли входные данные Ñодержат только идентификаторы перечиÑлениÑ, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга перечиÑлениÑ. + Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`. ## CSVWithNames {#csvwithnames} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1b4da512c9f..267dae3ae2a 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -391,12 +391,12 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( ## input_format_tsv_enum_as_number {#settings-input_format_tsv_enum_as_number} -Включает или отключает парÑинг значений перечиÑлений как идентификаторов перечиÑлений Ð´Ð»Ñ Ð²Ñ…Ð¾Ð´Ð½Ð¾Ð³Ð¾ формата TSV. +При включенном режиме вÑегда обрабатывайте Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº идентификаторы перечиÑÐ»ÐµÐ½Ð¸Ñ Ð´Ð»Ñ Ð²Ñ…Ð¾Ð´Ð½Ð¾Ð³Ð¾ формата TSV. Ð”Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð°Ñ‚ÑŒ Ñтот параметр, еÑли данные Ñодержат только идентификаторы перечиÑлениÑ. Возможные значениÑ: -- 0 — парÑинг значений перечиÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº значений. -- 1 — парÑинг значений перечиÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº идентификаторов перечиÑлениÑ. +- 0 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ как Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¸Ð»Ð¸ как идентификаторы. +- 1 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ только как идентификаторы. Значение по умолчанию: 0. @@ -410,10 +410,39 @@ CREATE TABLE table_with_enum_column_for_tsv_insert (Id Int32,Value Enum('first' При включенной наÑтройке `input_format_tsv_enum_as_number`: +ЗапроÑ: + ```sql SET input_format_tsv_enum_as_number = 1; INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; -INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 1; +SELECT * FROM table_with_enum_column_for_tsv_insert; +``` + +Результат: + +```text +┌──Id─┬─Value──┠+│ 102 │ second │ +└─────┴────────┘ +``` + +ЗапроÑ: + +```sql +SET input_format_tsv_enum_as_number = 1; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 'first'; +``` + +Ñгенерирует иÑключение. + +При отключенной наÑтройке `input_format_tsv_enum_as_number`: + +ЗапроÑ: + +```sql +SET input_format_tsv_enum_as_number = 0; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; +INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 103 'first'; SELECT * FROM table_with_enum_column_for_tsv_insert; ``` @@ -428,15 +457,6 @@ SELECT * FROM table_with_enum_column_for_tsv_insert; └─────┴────────┘ ``` -При отключенной наÑтройке `input_format_tsv_enum_as_number` Ð·Ð°Ð¿Ñ€Ð¾Ñ `INSERT`: - -```sql -SET input_format_tsv_enum_as_number = 0; -INSERT INTO table_with_enum_column_for_tsv_insert FORMAT TSV 102 2; -``` - -Ñгенерирует иÑключение. - ## input_format_null_as_default {#settings-input-format-null-as-default} Включает или отключает инициализацию [значениÑми по умолчанию](../../sql-reference/statements/create/table.md#create-default-values) Ñчеек Ñ [NULL](../../sql-reference/syntax.md#null-literal), еÑли тип данных Ñтолбца не позволÑет [хранить NULL](../../sql-reference/data-types/nullable.md#data_type-nullable). @@ -1511,12 +1531,12 @@ SELECT area/period FROM account_orders FORMAT JSON; ## input_format_csv_enum_as_number {#settings-input_format_csv_enum_as_number} -Включает или отключает парÑинг значений перечиÑлений как идентификаторов перечиÑлений Ð´Ð»Ñ Ð²Ñ…Ð¾Ð´Ð½Ð¾Ð³Ð¾ формата CSV. +При включенном режиме вÑегда обрабатывайте Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº идентификаторы перечиÑÐ»ÐµÐ½Ð¸Ñ Ð´Ð»Ñ Ð²Ñ…Ð¾Ð´Ð½Ð¾Ð³Ð¾ формата CSV. Ð”Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð°Ñ‚ÑŒ Ñтот параметр, еÑли данные Ñодержат только идентификаторы перечиÑлениÑ. Возможные значениÑ: -- 0 — парÑинг значений перечиÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº значений. -- 1 — парÑинг значений перечиÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº идентификаторов перечиÑлениÑ. +- 0 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ как Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¸Ð»Ð¸ как идентификаторы. +- 1 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ только как идентификаторы. Значение по умолчанию: 0. @@ -1530,10 +1550,11 @@ CREATE TABLE table_with_enum_column_for_csv_insert (Id Int32,Value Enum('first' При включенной наÑтройке `input_format_csv_enum_as_number`: +ЗапроÑ: + ```sql SET input_format_csv_enum_as_number = 1; INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; -SELECT * FROM table_with_enum_column_for_csv_insert; ``` Результат: @@ -1544,15 +1565,37 @@ SELECT * FROM table_with_enum_column_for_csv_insert; └─────┴────────┘ ``` -При отключенной наÑтройке `input_format_csv_enum_as_number` Ð·Ð°Ð¿Ñ€Ð¾Ñ `INSERT`: +ЗапроÑ: ```sql -SET input_format_csv_enum_as_number = 0; -INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2; +SET input_format_csv_enum_as_number = 1; +INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 103,'first' ``` Ñгенерирует иÑключение. +При отключенной наÑтройке `input_format_csv_enum_as_number`: + +ЗапроÑ: + +```sql +SET input_format_csv_enum_as_number = 0; +INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 102,2 +INSERT INTO table_with_enum_column_for_csv_insert FORMAT CSV 103,'first' +SELECT * FROM table_with_enum_column_for_csv_insert; +``` + +Результат: + +```text +┌──Id─┬─Value──┠+│ 102 │ second │ +└─────┴────────┘ +┌──Id─┬─Value─┠+│ 103 │ first │ +└─────┴───────┘ +``` + ## output_format_csv_crlf_end_of_line {#settings-output-format-csv-crlf-end-of-line} ИÑпользовать в качеÑтве Ñ€Ð°Ð·Ð´ÐµÐ»Ð¸Ñ‚ÐµÐ»Ñ Ñтрок Ð´Ð»Ñ CSV формата CRLF (DOS/Windows Ñтиль) вмеÑто LF (Unix Ñтиль). From 61a1eb28281a5353f67d7c3a45dae3460664ed4e Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sun, 12 Dec 2021 00:21:05 +0300 Subject: [PATCH 126/358] Update RabbitMQ --- docs/en/engines/table-engines/integrations/rabbitmq.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/rabbitmq.md b/docs/en/engines/table-engines/integrations/rabbitmq.md index ebb42461204..78c144ac76f 100644 --- a/docs/en/engines/table-engines/integrations/rabbitmq.md +++ b/docs/en/engines/table-engines/integrations/rabbitmq.md @@ -37,6 +37,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] [rabbitmq_skip_broken_messages = N,] [rabbitmq_max_block_size = N,] [rabbitmq_flush_interval_ms = N] + [rabbitmq_queue_settings_list = 'x-dead-letter-exchange=my-dlx,x-max-length=10,x-overflow=reject-publish'] ``` Required parameters: @@ -59,6 +60,7 @@ Optional parameters: - `rabbitmq_skip_broken_messages` – RabbitMQ message parser tolerance to schema-incompatible messages per block. Default: `0`. If `rabbitmq_skip_broken_messages = N` then the engine skips *N* RabbitMQ messages that cannot be parsed (a message equals a row of data). - `rabbitmq_max_block_size` - `rabbitmq_flush_interval_ms` +- `rabbitmq_queue_settings_list` - allows to set RabbitMQ settings when creating a queue. Available settings: `x-max-length`, `x-max-length-bytes`, `x-message-ttl`, `x-expires`, `x-priority`, `x-max-priority`, `x-overflow`, `x-dead-letter-exchange`, `x-queue-type`. The `durable` setting is enabled automatically for the queue. SSL connection: From c62e425aa162f9708deb967ef05430a57548266c Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sun, 12 Dec 2021 00:32:10 +0300 Subject: [PATCH 127/358] stop referencing insert_sample_with_metadata --- docs/ru/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 996514aa068..82e2992df55 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -699,7 +699,7 @@ CREATE TABLE IF NOT EXISTS example_table - ЕÑли `input_format_defaults_for_omitted_fields = 1`, то значение по умолчанию Ð´Ð»Ñ `x` равно `0`, а значение по умолчанию `a` равно `x * 2`. !!! note "Предупреждение" - ЕÑли `input_format_defaults_for_omitted_fields = 1`, то при обработке запроÑов ClickHouse потреблÑет больше вычиÑлительных реÑурÑов, чем еÑли `input_format_defaults_for_omitted_fields = 0`. + При добавлении данных Ñ Ð¿Ð¾Ð¼Ð¾Ñ‰ÑŒÑŽ `input_format_defaults_for_omitted_fields = 1`, ClickHouse потреблÑет больше вычиÑлительных реÑурÑов по Ñравнению Ñ `input_format_defaults_for_omitted_fields = 0`. ### Выборка данных {#vyborka-dannykh} From 830e793bd1df1032a7d789cd87cc466043f55a10 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Sun, 12 Dec 2021 00:57:33 +0300 Subject: [PATCH 128/358] Update formats.md --- docs/ru/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 82e2992df55..b797c9ba6a7 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -130,7 +130,7 @@ world Каждый Ñлемент Ñтруктуры типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) предÑтавлÑетÑÑ ÐºÐ°Ðº отдельный маÑÑив. Ð—Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑлениÑ, в качеÑтве входных данных, могут быть предÑтавлены как Ð¸Ð¼Ñ Ð¸Ð»Ð¸ как идентификаторы. Сначала мы пытаемÑÑ ÑопоÑтавить входное значение Ñ Ð¸Ð¼ÐµÐ½ÐµÐ¼ перечиÑлениÑ. Ð’ Ñлучае неудачи и при уÑловии, что входное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, мы пытаемÑÑ ÑопоÑтавить Ñто чиÑло Ñ Ð¸Ð´ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ‚Ð¾Ñ€Ð¾Ð¼ перечиÑлениÑ. -ЕÑли входные данные Ñодержат только ENUM идентификаторы, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга перечиÑлениÑ. +ЕÑли входные данные Ñодержат только идентификаторы перечиÑлениÑ, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга перечиÑлениÑ. Ðапример: From 2c68192bb6f7f0deefce3a668064dc9959d3d6ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 12 Dec 2021 05:35:33 +0300 Subject: [PATCH 129/358] Fix strange code in TCPHandler --- src/Server/TCPHandler.cpp | 6 ++---- src/Server/TCPHandler.h | 1 + 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index cdf1838c06b..36d126559a7 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -233,8 +233,6 @@ void TCPHandler::runImpl() /// NOTE: these settings are applied only for current connection (not for distributed tables' connections) state.timeout_setter = std::make_unique(socket(), receive_timeout, send_timeout); - std::mutex fatal_error_mutex; - /// Should we send internal logs to client? const auto client_logs_level = query_context->getSettingsRef().send_logs_level; if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_SERVER_LOGS @@ -243,7 +241,7 @@ void TCPHandler::runImpl() state.logs_queue = std::make_shared(); state.logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString()); CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level); - CurrentThread::setFatalErrorCallback([this, &fatal_error_mutex] + CurrentThread::setFatalErrorCallback([this] { std::lock_guard lock(fatal_error_mutex); sendLogs(); @@ -351,7 +349,7 @@ void TCPHandler::runImpl() /// Should not check for cancel in case of input. if (!state.need_receive_data_for_input) { - auto callback = [this, &fatal_error_mutex]() + auto callback = [this]() { std::lock_guard lock(fatal_error_mutex); diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 4a340e328ed..54af44759e7 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -177,6 +177,7 @@ private: String cluster_secret; std::mutex task_callback_mutex; + std::mutex fatal_error_mutex; /// At the moment, only one ongoing query in the connection is supported at a time. QueryState state; From e4b405716f809b694a8b594306d93e8320daf7d1 Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Mon, 6 Dec 2021 23:23:17 +0800 Subject: [PATCH 130/358] fix multiply row policies on same column --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++- ...iply_row_policies_on_same_column.reference | 8 ++++ ...1_multiply_row_policies_on_same_column.sql | 47 +++++++++++++++++++ 3 files changed, 59 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference create mode 100644 tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c8f48f2ed1f..df79fdb443e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -113,8 +113,10 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co select_ast->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); auto expr_list = select_ast->select(); - // The first column is our filter expression. - expr_list->children.push_back(row_policy_filter); + /// The first column is our filter expression. + /// the row_policy_filter should be cloned, because it may be changed by TreeRewriter. + /// which make it possible an invalid expression, although it may be valid in whole select. + expr_list->children.push_back(row_policy_filter->clone()); /// Keep columns that are required after the filter actions. for (const auto & column_str : prerequisite_columns) diff --git a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference new file mode 100644 index 00000000000..2ea0833e92c --- /dev/null +++ b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference @@ -0,0 +1,8 @@ +4 +3 +4 +4 +4 +3 +2 +1 diff --git a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql new file mode 100644 index 00000000000..75fa3778797 --- /dev/null +++ b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS 02131_multiply_row_policies_on_same_column; +CREATE TABLE 02131_multiply_row_policies_on_same_column (x UInt8) ENGINE = MergeTree ORDER BY x; +INSERT INTO 02131_multiply_row_policies_on_same_column VALUES (1), (2), (3), (4); + + +DROP ROW POLICY IF EXISTS 02131_filter_1 ON 02131_multiply_row_policies_on_same_column; +DROP ROW POLICY IF EXISTS 02131_filter_2 ON 02131_multiply_row_policies_on_same_column; +DROP ROW POLICY IF EXISTS 02131_filter_3 ON 02131_multiply_row_policies_on_same_column; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; + + +CREATE ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column USING x=1 TO ALL; +CREATE ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column USING x=2 TO ALL; +CREATE ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column USING x=3 TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; + + +DROP ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column; +DROP ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column; +DROP ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; + + +DROP TABLE IF EXISTS 02131_multiply_row_policies_on_same_column_1; +CREATE TABLE 02131_multiply_row_policies_on_same_column_1 (x UInt8) ENGINE = MergeTree ORDER BY x; +INSERT INTO 02131_multiply_row_policies_on_same_column_1 VALUES (1), (2), (3), (4); + +DROP ROW POLICY IF EXISTS 02131_filter_0_1 ON 02131_multiply_row_policies_on_same_column_1; +DROP ROW POLICY IF EXISTS 02131_filter_1_1 ON 02131_multiply_row_policies_on_same_column_1; +DROP ROW POLICY IF EXISTS 02131_filter_2_1 ON 02131_multiply_row_policies_on_same_column_1; +DROP ROW POLICY IF EXISTS 02131_filter_3_1 ON 02131_multiply_row_policies_on_same_column_1; +SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; + +CREATE ROW POLICY 02131_filter_0_1 ON 02131_multiply_row_policies_on_same_column_1 USING 1 TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; +CREATE ROW POLICY 02131_filter_1_1 ON 02131_multiply_row_policies_on_same_column_1 USING x>1 AS RESTRICTIVE TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; +CREATE ROW POLICY 02131_filter_2_1 ON 02131_multiply_row_policies_on_same_column_1 USING x>2 AS RESTRICTIVE TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; +CREATE ROW POLICY 02131_filter_3_1 ON 02131_multiply_row_policies_on_same_column_1 USING x>3 AS RESTRICTIVE TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; + + +DROP TABLE 02131_multiply_row_policies_on_same_column_1; +DROP ROW POLICY 02131_filter_1_1 ON 02131_multiply_row_policies_on_same_column_1; +DROP ROW POLICY 02131_filter_2_1 ON 02131_multiply_row_policies_on_same_column_1; +DROP ROW POLICY 02131_filter_3_1 ON 02131_multiply_row_policies_on_same_column_1; From fab1e53489cf50e59dccc73162557dfd5c79e68e Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Mon, 6 Dec 2021 23:28:19 +0800 Subject: [PATCH 131/358] fix tests for multiply row policies on same column --- .../02131_multiply_row_policies_on_same_column.reference | 4 ++++ .../02131_multiply_row_policies_on_same_column.sql | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference index 2ea0833e92c..5049f94bd3e 100644 --- a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference +++ b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference @@ -1,4 +1,7 @@ 4 +1 +2 +3 3 4 4 @@ -6,3 +9,4 @@ 3 2 1 +1 diff --git a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql index 75fa3778797..ca02632aaca 100644 --- a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql +++ b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql @@ -10,9 +10,12 @@ SELECT count() FROM 02131_multiply_row_policies_on_same_column; CREATE ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column USING x=1 TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; CREATE ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column USING x=2 TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; CREATE ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column USING x=3 TO ALL; SELECT count() FROM 02131_multiply_row_policies_on_same_column; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; DROP ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column; @@ -39,7 +42,7 @@ CREATE ROW POLICY 02131_filter_2_1 ON 02131_multiply_row_policies_on_same_column SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; CREATE ROW POLICY 02131_filter_3_1 ON 02131_multiply_row_policies_on_same_column_1 USING x>3 AS RESTRICTIVE TO ALL; SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; - +SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; DROP TABLE 02131_multiply_row_policies_on_same_column_1; DROP ROW POLICY 02131_filter_1_1 ON 02131_multiply_row_policies_on_same_column_1; From 8f9b6710dffa9ab3bd14789eace4540ea73c500a Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Tue, 7 Dec 2021 08:38:38 +0800 Subject: [PATCH 132/358] remove useless multiply restrictive row policies test --- ...iply_row_policies_on_same_column.reference | 8 ++--- ...1_multiply_row_policies_on_same_column.sql | 32 ++++--------------- 2 files changed, 8 insertions(+), 32 deletions(-) diff --git a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference index 5049f94bd3e..3f71510f3a5 100644 --- a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference +++ b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.reference @@ -3,10 +3,6 @@ 2 3 3 -4 -4 -4 3 -2 -1 -1 +3 +4 diff --git a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql index ca02632aaca..75f7f737e85 100644 --- a/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql +++ b/tests/queries/0_stateless/02131_multiply_row_policies_on_same_column.sql @@ -18,33 +18,13 @@ SELECT count() FROM 02131_multiply_row_policies_on_same_column; SELECT count() FROM 02131_multiply_row_policies_on_same_column; +CREATE ROW POLICY 02131_filter_4 ON 02131_multiply_row_policies_on_same_column USING x<4 AS RESTRICTIVE TO ALL; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; +SELECT count() FROM 02131_multiply_row_policies_on_same_column; + DROP ROW POLICY 02131_filter_1 ON 02131_multiply_row_policies_on_same_column; DROP ROW POLICY 02131_filter_2 ON 02131_multiply_row_policies_on_same_column; DROP ROW POLICY 02131_filter_3 ON 02131_multiply_row_policies_on_same_column; +DROP ROW POLICY 02131_filter_4 ON 02131_multiply_row_policies_on_same_column; SELECT count() FROM 02131_multiply_row_policies_on_same_column; - - -DROP TABLE IF EXISTS 02131_multiply_row_policies_on_same_column_1; -CREATE TABLE 02131_multiply_row_policies_on_same_column_1 (x UInt8) ENGINE = MergeTree ORDER BY x; -INSERT INTO 02131_multiply_row_policies_on_same_column_1 VALUES (1), (2), (3), (4); - -DROP ROW POLICY IF EXISTS 02131_filter_0_1 ON 02131_multiply_row_policies_on_same_column_1; -DROP ROW POLICY IF EXISTS 02131_filter_1_1 ON 02131_multiply_row_policies_on_same_column_1; -DROP ROW POLICY IF EXISTS 02131_filter_2_1 ON 02131_multiply_row_policies_on_same_column_1; -DROP ROW POLICY IF EXISTS 02131_filter_3_1 ON 02131_multiply_row_policies_on_same_column_1; -SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; - -CREATE ROW POLICY 02131_filter_0_1 ON 02131_multiply_row_policies_on_same_column_1 USING 1 TO ALL; -SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; -CREATE ROW POLICY 02131_filter_1_1 ON 02131_multiply_row_policies_on_same_column_1 USING x>1 AS RESTRICTIVE TO ALL; -SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; -CREATE ROW POLICY 02131_filter_2_1 ON 02131_multiply_row_policies_on_same_column_1 USING x>2 AS RESTRICTIVE TO ALL; -SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; -CREATE ROW POLICY 02131_filter_3_1 ON 02131_multiply_row_policies_on_same_column_1 USING x>3 AS RESTRICTIVE TO ALL; -SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; -SELECT count() FROM 02131_multiply_row_policies_on_same_column_1; - -DROP TABLE 02131_multiply_row_policies_on_same_column_1; -DROP ROW POLICY 02131_filter_1_1 ON 02131_multiply_row_policies_on_same_column_1; -DROP ROW POLICY 02131_filter_2_1 ON 02131_multiply_row_policies_on_same_column_1; -DROP ROW POLICY 02131_filter_3_1 ON 02131_multiply_row_policies_on_same_column_1; +DROP TABLE 02131_multiply_row_policies_on_same_column; From 116451696a7c8b1d0e5eb823efcbd72065b49b06 Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Sun, 12 Dec 2021 17:13:31 +0300 Subject: [PATCH 133/358] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 191 ------------------------ 1 file changed, 191 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 065223224c1..65c28feca4d 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -54,7 +54,6 @@ Internal coordination settings are located in `..` section and contain servers description. @@ -110,196 +109,6 @@ ClickHouse Keeper is bundled into the ClickHouse server package, just add config clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` -## Four Letter Word Commands - -ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. - -The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro". - -You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. -``` -echo mntr | nc localhost 9181 -``` - -Bellow is the detailed 4lw commands: - -- ruok : Tests if server is running in a non-error state. The server will respond with imok if it is running. Otherwise it will not respond at all. A response of "imok" does not necessarily indicate that the server has joined the quorum, just that the server process is active and bound to the specified client port. Use "stat" for details on state wrt quorum and client connection information. - -``` -imok -``` - -- mntr : Outputs a list of variables that could be used for monitoring the health of the cluster. - -``` -zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -zk_avg_latency 0 -zk_max_latency 0 -zk_min_latency 0 -zk_packets_received 68 -zk_packets_sent 68 -zk_num_alive_connections 1 -zk_outstanding_requests 0 -zk_server_state leader -zk_znode_count 4 -zk_watch_count 1 -zk_ephemerals_count 0 -zk_approximate_data_size 723 -zk_open_file_descriptor_count 310 -zk_max_file_descriptor_count 10240 -zk_followers 0 -zk_synced_followers 0 -``` - -- srvr : Lists full details for the server. - -``` -ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -Latency min/avg/max: 0/0/0 - -Received: 2 -Sent : 2 -Connections: 1 -Outstanding: 0 -Zxid: 34 -Mode: leader -Node count: 4 -``` - -- stat : Lists brief details for the server and connected clients. - -``` -ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -Clients: - 192.168.1.1:52852(recved=0,sent=0) - 192.168.1.1:52042(recved=24,sent=48) - -Latency min/avg/max: 0/0/0 - -Received: 4 -Sent : 4 -Connections: 1 -Outstanding: 0 -Zxid: 36 -Mode: leader -Node count: 4 - -``` - -- srst : Reset server statistics. The command will affect the result of `srvr`, `mntr` and `stat`. - -``` -Server stats reset. -``` - -- conf : Print details about serving configuration. - -``` -server_id=1 -tcp_port=2181 -four_letter_word_white_list=* -log_storage_path=./coordination/logs -snapshot_storage_path=./coordination/snapshots -max_requests_batch_size=100 -session_timeout_ms=30000 -operation_timeout_ms=10000 -dead_session_check_period_ms=500 -heart_beat_interval_ms=500 -election_timeout_lower_bound_ms=1000 -election_timeout_upper_bound_ms=2000 -reserved_log_items=1000000000000000 -snapshot_distance=10000 -auto_forwarding=true -shutdown_timeout=5000 -startup_timeout=240000 -raft_logs_level=information -snapshots_to_keep=3 -rotate_log_storage_interval=100000 -stale_log_gap=10000 -fresh_log_gap=200 -max_requests_batch_size=100 -quorum_reads=false -force_sync=false -compress_logs=true -compress_snapshots_with_zstd_format=true -configuration_change_tries_count=20 -``` - -- cons : List full connection/session details for all clients connected to this server. Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... - -``` - 192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0) - 192.168.1.1:52042(recved=9,sent=18,sid=0x0000000000000001,lop=List,est=1636454739887,to=30000,lcxid=0x0000000000000005,lzxid=0x0000000000000005,lresp=1636454739892,llat=0,minlat=0,avglat=0,maxlat=0) -``` - -- crst : Reset connection/session statistics for all connections. - -``` -Connection stats reset. -``` - -- envi : Print details about serving environment - -``` -Environment: -clickhouse.keeper.version=v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 -host.name=ZBMAC-C02D4054M.local -os.name=Darwin -os.arch=x86_64 -os.version=19.6.0 -cpu.count=12 -user.name=root -user.home=/Users/JackyWoo/ -user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/ -user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ -``` - - -- dirs : Shows the total size of snapshot and log files in bytes - -``` -snapshot_dir_size: 0 -log_dir_size: 3875 -``` - -- isro: Tests if server is running in read-only mode. The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. - -``` -rw -``` - -- wchs : Lists brief information on watches for the server. - -``` -1 connections watching 1 paths -Total watches:1 -``` - -- wchc : Lists detailed information on watches for the server, by session. This outputs a list of sessions(connections) with associated watches (paths). Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. - -``` -0x0000000000000001 - /clickhouse/task_queue/ddl -``` - -- wchp : Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. - -``` -/clickhouse/task_queue/ddl - 0x0000000000000001 -``` - -- dump : Lists the outstanding sessions and ephemeral nodes. This only works on the leader. - -``` -Sessions dump (2): -0x0000000000000001 -0x0000000000000002 -Sessions with Ephemerals (1): -0x0000000000000001 - /clickhouse/task_queue/ddl -``` - ## [experimental] Migration from ZooKeeper Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: From c9e9bd78b2e863a27c8ac6c62ac6e3d1cd8131ff Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 13 Dec 2021 11:14:10 +0100 Subject: [PATCH 134/358] Remove old unused file --- .../PushingToViewsBlockOutputStream.cpp | 401 ------------------ 1 file changed, 401 deletions(-) delete mode 100644 src/DataStreams/PushingToViewsBlockOutputStream.cpp diff --git a/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/src/DataStreams/PushingToViewsBlockOutputStream.cpp deleted file mode 100644 index 76c378c07e0..00000000000 --- a/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ /dev/null @@ -1,401 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( - const StoragePtr & storage_, - const StorageMetadataPtr & metadata_snapshot_, - ContextPtr context_, - const ASTPtr & query_ptr_, - bool no_destination) - : WithContext(context_) - , storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , log(&Poco::Logger::get("PushingToViewsBlockOutputStream")) - , query_ptr(query_ptr_) -{ - checkStackSize(); - - /** TODO This is a very important line. At any insertion into the table one of streams should own lock. - * Although now any insertion into the table is done via PushingToViewsBlockOutputStream, - * but it's clear that here is not the best place for this functionality. - */ - addTableLock( - storage->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout)); - - /// If the "root" table deduplicates blocks, there are no need to make deduplication for children - /// Moreover, deduplication for AggregatingMergeTree children could produce false positives due to low size of inserting blocks - bool disable_deduplication_for_children = false; - if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views) - disable_deduplication_for_children = !no_destination && storage->supportsDeduplication(); - - auto table_id = storage->getStorageID(); - Dependencies dependencies = DatabaseCatalog::instance().getDependencies(table_id); - - /// We need special context for materialized views insertions - if (!dependencies.empty()) - { - select_context = Context::createCopy(context); - insert_context = Context::createCopy(context); - - const auto & insert_settings = insert_context->getSettingsRef(); - - // Do not deduplicate insertions into MV if the main insertion is Ok - if (disable_deduplication_for_children) - insert_context->setSetting("insert_deduplicate", Field{false}); - - // Separate min_insert_block_size_rows/min_insert_block_size_bytes for children - if (insert_settings.min_insert_block_size_rows_for_materialized_views) - insert_context->setSetting("min_insert_block_size_rows", insert_settings.min_insert_block_size_rows_for_materialized_views.value); - if (insert_settings.min_insert_block_size_bytes_for_materialized_views) - insert_context->setSetting("min_insert_block_size_bytes", insert_settings.min_insert_block_size_bytes_for_materialized_views.value); - } - - for (const auto & database_table : dependencies) - { - auto dependent_table = DatabaseCatalog::instance().getTable(database_table, getContext()); - auto dependent_metadata_snapshot = dependent_table->getInMemoryMetadataPtr(); - - ASTPtr query; - BlockOutputStreamPtr out; - - if (auto * materialized_view = dynamic_cast(dependent_table.get())) - { - addTableLock( - materialized_view->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout)); - - StoragePtr inner_table = materialized_view->getTargetTable(); - auto inner_table_id = inner_table->getStorageID(); - auto inner_metadata_snapshot = inner_table->getInMemoryMetadataPtr(); - query = dependent_metadata_snapshot->getSelectQuery().inner_query; - - std::unique_ptr insert = std::make_unique(); - insert->table_id = inner_table_id; - - /// Get list of columns we get from select query. - auto header = InterpreterSelectQuery(query, select_context, SelectQueryOptions().analyze()) - .getSampleBlock(); - - /// Insert only columns returned by select. - auto list = std::make_shared(); - const auto & inner_table_columns = inner_metadata_snapshot->getColumns(); - for (const auto & column : header) - { - /// But skip columns which storage doesn't have. - if (inner_table_columns.hasPhysical(column.name)) - list->children.emplace_back(std::make_shared(column.name)); - } - - insert->columns = std::move(list); - - ASTPtr insert_query_ptr(insert.release()); - InterpreterInsertQuery interpreter(insert_query_ptr, insert_context); - BlockIO io = interpreter.execute(); - out = io.out; - } - else if ( - dynamic_cast(dependent_table.get()) || dynamic_cast(dependent_table.get())) - out = std::make_shared( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr(), true); - else - out = std::make_shared( - dependent_table, dependent_metadata_snapshot, insert_context, ASTPtr()); - - views.emplace_back(ViewInfo{std::move(query), database_table, std::move(out), nullptr, 0 /* elapsed_ms */}); - } - - /// Do not push to destination table if the flag is set - if (!no_destination) - { - output = storage->write(query_ptr, storage->getInMemoryMetadataPtr(), getContext()); - replicated_output = dynamic_cast(output.get()); - } -} - - -Block PushingToViewsBlockOutputStream::getHeader() const -{ - /// If we don't write directly to the destination - /// then expect that we're inserting with precalculated virtual columns - if (output) - return metadata_snapshot->getSampleBlock(); - else - return metadata_snapshot->getSampleBlockWithVirtuals(storage->getVirtuals()); -} - - -void PushingToViewsBlockOutputStream::write(const Block & block) -{ - /** Throw an exception if the sizes of arrays - elements of nested data structures doesn't match. - * We have to make this assertion before writing to table, because storage engine may assume that they have equal sizes. - * NOTE It'd better to do this check in serialization of nested structures (in place when this assumption is required), - * but currently we don't have methods for serialization of nested structures "as a whole". - */ - Nested::validateArraySizes(block); - - if (auto * live_view = dynamic_cast(storage.get())) - { - StorageLiveView::writeIntoLiveView(*live_view, block, getContext()); - } - else if (auto * window_view = dynamic_cast(storage.get())) - { - StorageWindowView::writeIntoWindowView(*window_view, block, getContext()); - } - else - { - if (output) - /// TODO: to support virtual and alias columns inside MVs, we should return here the inserted block extended - /// with additional columns directly from storage and pass it to MVs instead of raw block. - output->write(block); - } - - /// Don't process materialized views if this block is duplicate - if (!getContext()->getSettingsRef().deduplicate_blocks_in_dependent_materialized_views && replicated_output && replicated_output->lastBlockIsDuplicate()) - return; - - // Insert data into materialized views only after successful insert into main table - const Settings & settings = getContext()->getSettingsRef(); - if (settings.parallel_view_processing && views.size() > 1) - { - // Push to views concurrently if enabled and more than one view is attached - ThreadPool pool(std::min(size_t(settings.max_threads), views.size())); - for (auto & view : views) - { - auto thread_group = CurrentThread::getGroup(); - pool.scheduleOrThrowOnError([=, &view, this] - { - setThreadName("PushingToViews"); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - process(block, view); - }); - } - // Wait for concurrent view processing - pool.wait(); - } - else - { - // Process sequentially - for (auto & view : views) - { - process(block, view); - - if (view.exception) - std::rethrow_exception(view.exception); - } - } -} - -void PushingToViewsBlockOutputStream::writePrefix() -{ - if (output) - output->writePrefix(); - - for (auto & view : views) - { - try - { - view.out->writePrefix(); - } - catch (Exception & ex) - { - ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); - throw; - } - } -} - -void PushingToViewsBlockOutputStream::writeSuffix() -{ - if (output) - output->writeSuffix(); - - std::exception_ptr first_exception; - - const Settings & settings = getContext()->getSettingsRef(); - bool parallel_processing = false; - - /// Run writeSuffix() for views in separate thread pool. - /// In could have been done in PushingToViewsBlockOutputStream::process, however - /// it is not good if insert into main table fail but into view succeed. - if (settings.parallel_view_processing && views.size() > 1) - { - parallel_processing = true; - - // Push to views concurrently if enabled and more than one view is attached - ThreadPool pool(std::min(size_t(settings.max_threads), views.size())); - auto thread_group = CurrentThread::getGroup(); - - for (auto & view : views) - { - if (view.exception) - continue; - - pool.scheduleOrThrowOnError([thread_group, &view, this] - { - setThreadName("PushingToViews"); - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - - Stopwatch watch; - try - { - view.out->writeSuffix(); - } - catch (...) - { - view.exception = std::current_exception(); - } - view.elapsed_ms += watch.elapsedMilliseconds(); - - LOG_TRACE(log, "Pushing from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.elapsed_ms); - }); - } - // Wait for concurrent view processing - pool.wait(); - } - - for (auto & view : views) - { - if (view.exception) - { - if (!first_exception) - first_exception = view.exception; - - continue; - } - - if (parallel_processing) - continue; - - Stopwatch watch; - try - { - view.out->writeSuffix(); - } - catch (Exception & ex) - { - ex.addMessage("while write prefix to view " + view.table_id.getNameForLogs()); - throw; - } - view.elapsed_ms += watch.elapsedMilliseconds(); - - LOG_TRACE(log, "Pushing from {} to {} took {} ms.", - storage->getStorageID().getNameForLogs(), - view.table_id.getNameForLogs(), - view.elapsed_ms); - } - - if (first_exception) - std::rethrow_exception(first_exception); - - UInt64 milliseconds = main_watch.elapsedMilliseconds(); - if (views.size() > 1) - { - LOG_DEBUG(log, "Pushing from {} to {} views took {} ms.", - storage->getStorageID().getNameForLogs(), views.size(), - milliseconds); - } -} - -void PushingToViewsBlockOutputStream::flush() -{ - if (output) - output->flush(); - - for (auto & view : views) - view.out->flush(); -} - -void PushingToViewsBlockOutputStream::process(const Block & block, ViewInfo & view) -{ - Stopwatch watch; - - try - { - BlockInputStreamPtr in; - - /// We need keep InterpreterSelectQuery, until the processing will be finished, since: - /// - /// - We copy Context inside InterpreterSelectQuery to support - /// modification of context (Settings) for subqueries - /// - InterpreterSelectQuery lives shorter than query pipeline. - /// It's used just to build the query pipeline and no longer needed - /// - ExpressionAnalyzer and then, Functions, that created in InterpreterSelectQuery, - /// **can** take a reference to Context from InterpreterSelectQuery - /// (the problem raises only when function uses context from the - /// execute*() method, like FunctionDictGet do) - /// - These objects live inside query pipeline (DataStreams) and the reference become dangling. - std::optional select; - - if (view.query) - { - /// We create a table with the same name as original table and the same alias columns, - /// but it will contain single block (that is INSERT-ed into main table). - /// InterpreterSelectQuery will do processing of alias columns. - - auto local_context = Context::createCopy(select_context); - local_context->addViewSource( - StorageValues::create(storage->getStorageID(), metadata_snapshot->getColumns(), block, storage->getVirtuals())); - select.emplace(view.query, local_context, SelectQueryOptions()); - in = std::make_shared(select->execute().getInputStream()); - - /// Squashing is needed here because the materialized view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - in = std::make_shared( - in, getContext()->getSettingsRef().min_insert_block_size_rows, getContext()->getSettingsRef().min_insert_block_size_bytes); - in = std::make_shared(in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name); - } - else - in = std::make_shared(block); - - in->readPrefix(); - - while (Block result_block = in->read()) - { - Nested::validateArraySizes(result_block); - view.out->write(result_block); - } - - in->readSuffix(); - } - catch (Exception & ex) - { - ex.addMessage("while pushing to view " + view.table_id.getNameForLogs()); - view.exception = std::current_exception(); - } - catch (...) - { - view.exception = std::current_exception(); - } - - view.elapsed_ms += watch.elapsedMilliseconds(); -} - -} From b25cfa0b4d4b3fae41f61691b97236754359465a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 13 Dec 2021 11:50:46 +0100 Subject: [PATCH 135/358] InterpreterInsertQuery::buildChain: Make sure to keep the context alive --- src/Interpreters/InterpreterInsertQuery.cpp | 3 +++ src/QueryPipeline/Chain.h | 2 ++ .../0_stateless/02137_mv_into_join.reference | 3 +++ .../queries/0_stateless/02137_mv_into_join.sql | 17 +++++++++++++++++ 4 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02137_mv_into_join.reference create mode 100644 tests/queries/0_stateless/02137_mv_into_join.sql diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index b7edf12e23f..d1b8a056053 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -196,6 +196,9 @@ Chain InterpreterInsertQuery::buildChainImpl( /// We create a pipeline of several streams, into which we will write data. Chain out; + /// Keep a reference to the context to make sure it stays alive until the chain is executed and destroyed + out.addInterpreterContext(context_ptr); + /// NOTE: we explicitly ignore bound materialized views when inserting into Kafka Storage. /// Otherwise we'll get duplicates when MV reads same rows again from Kafka. if (table->noPushingToViews() && !no_destination) diff --git a/src/QueryPipeline/Chain.h b/src/QueryPipeline/Chain.h index c5fdc34cecf..60dbad10131 100644 --- a/src/QueryPipeline/Chain.h +++ b/src/QueryPipeline/Chain.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include @@ -42,6 +43,7 @@ public: void addTableLock(TableLockHolder lock) { holder.table_locks.emplace_back(std::move(lock)); } void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } void attachResources(PipelineResourcesHolder holder_) { holder = std::move(holder_); } + void addInterpreterContext(ContextPtr context) { holder.interpreter_context.emplace_back(std::move(context)); } PipelineResourcesHolder detachResources() { return std::move(holder); } void reset(); diff --git a/tests/queries/0_stateless/02137_mv_into_join.reference b/tests/queries/0_stateless/02137_mv_into_join.reference new file mode 100644 index 00000000000..1228a2322e6 --- /dev/null +++ b/tests/queries/0_stateless/02137_mv_into_join.reference @@ -0,0 +1,3 @@ +sku_0001 black women nice shirt +sku_0001_black sku_0001 black women nice shirt +sku_0001_black sku_0001 black women nice shirt diff --git a/tests/queries/0_stateless/02137_mv_into_join.sql b/tests/queries/0_stateless/02137_mv_into_join.sql new file mode 100644 index 00000000000..cca896ac622 --- /dev/null +++ b/tests/queries/0_stateless/02137_mv_into_join.sql @@ -0,0 +1,17 @@ +CREATE TABLE main ( `id` String, `color` String, `section` String, `description` String) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE destination_join ( `key` String, `id` String, `color` String, `section` String, `description` String) ENGINE = Join(ANY, LEFT, key); +CREATE TABLE destination_set (`key` String) ENGINE = Set; + +CREATE MATERIALIZED VIEW mv_to_join TO `destination_join` AS SELECT concat(id, '_', color) AS key, * FROM main; +CREATE MATERIALIZED VIEW mv_to_set TO `destination_set` AS SELECT key FROM destination_join; + +INSERT INTO main VALUES ('sku_0001','black','women','nice shirt'); +SELECT * FROM main; +SELECT * FROM destination_join; +SELECT * FROM destination_join WHERE key in destination_set; + +DROP TABLE mv_to_set; +DROP TABLE destination_set; +DROP TABLE mv_to_join; +DROP TABLE destination_join; +DROP TABLE main; From 7ebd4498b607af25789a0bcafe8d2379b0b19578 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 13 Dec 2021 21:13:56 +0300 Subject: [PATCH 136/358] [docs] update development/tests.md according to recent changes --- docs/en/development/tests.md | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index ea32f608124..b61bfef50bb 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -106,7 +106,7 @@ Build ClickHouse. Run ClickHouse from the terminal: change directory to `program Note that all clickhouse tools (server, client, etc) are just symlinks to a single binary named `clickhouse`. You can find this binary at `programs/clickhouse`. All tools can also be invoked as `clickhouse tool` instead of `clickhouse-tool`. -Alternatively you can install ClickHouse package: either stable release from Yandex repository or you can build package for yourself with `./release` in ClickHouse sources root. Then start the server with `sudo service clickhouse-server start` (or stop to stop the server). Look for logs at `/etc/clickhouse-server/clickhouse-server.log`. +Alternatively you can install ClickHouse package: either stable release from ClickHouse repository or you can build package for yourself with `./release` in ClickHouse sources root. Then start the server with `sudo service clickhouse-server start` (or stop to stop the server). Look for logs at `/etc/clickhouse-server/clickhouse-server.log`. When ClickHouse is already installed on your system, you can build a new `clickhouse` binary and replace the existing binary: @@ -257,10 +257,6 @@ There are five variants (Debug, ASan, TSan, MSan, UBSan). Thread Fuzzer (please don't mix up with Thread Sanitizer) is another kind of fuzzing that allows to randomize thread order of execution. It helps to find even more special cases. -## Security Audit {#security-audit} - -People from Yandex Security Team do some basic overview of ClickHouse capabilities from the security standpoint. - ## Static Analyzers {#static-analyzers} We run `clang-tidy` and `PVS-Studio` on per-commit basis. `clang-static-analyzer` checks are also enabled. `clang-tidy` is also used for some style checks. @@ -326,15 +322,11 @@ There is automated check for flaky tests. It runs all new tests 100 times (for f ## Testflows -[Testflows](https://testflows.com/) is an enterprise-grade testing framework. It is used by Altinity for some of the tests and we run these tests in our CI. - -## Yandex Checks (only for Yandex employees) - -These checks are importing ClickHouse code into Yandex internal monorepository, so ClickHouse codebase can be used as a library by other products at Yandex (YT and YDB). Note that clickhouse-server itself is not being build from internal repo and unmodified open-source build is used for Yandex applications. +[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to implement a subset of ClickHouse. ## Test Automation {#test-automation} -We run tests with Yandex internal CI and job automation system named “Sandboxâ€. +We run tests with [GitHub Actions](https://github.com/features/actions). Build jobs and tests are run in Sandbox on per commit basis. Resulting packages and test results are published in GitHub and can be downloaded by direct links. Artifacts are stored for several months. When you send a pull request on GitHub, we tag it as “can be tested†and our CI system will build ClickHouse packages (release, debug, with address sanitizer, etc) for you. From 70c46920e3ef393526946017fbe8d721a94b79e5 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Mon, 13 Dec 2021 21:20:07 +0300 Subject: [PATCH 137/358] Update docs/ru/interfaces/formats.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index b797c9ba6a7..eef50eb6eaf 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -129,7 +129,7 @@ world Каждый Ñлемент Ñтруктуры типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) предÑтавлÑетÑÑ ÐºÐ°Ðº отдельный маÑÑив. -Ð—Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑлениÑ, в качеÑтве входных данных, могут быть предÑтавлены как Ð¸Ð¼Ñ Ð¸Ð»Ð¸ как идентификаторы. Сначала мы пытаемÑÑ ÑопоÑтавить входное значение Ñ Ð¸Ð¼ÐµÐ½ÐµÐ¼ перечиÑлениÑ. Ð’ Ñлучае неудачи и при уÑловии, что входное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, мы пытаемÑÑ ÑопоÑтавить Ñто чиÑло Ñ Ð¸Ð´ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ‚Ð¾Ñ€Ð¾Ð¼ перечиÑлениÑ. +ВходÑщие параметры типа "перечиÑление" (`ENUM`) могут передаватьÑÑ Ð² виде значений или порÑдковых номеров. Сначала переданное значение будет ÑопоÑтавлÑÑ‚ÑŒÑÑ Ñ Ñлементами перечиÑлениÑ. ЕÑли Ñовпадение не будет найдено и при Ñтом переданное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, оно будет трактоватьÑÑ ÐºÐ°Ðº порÑдковый номер в перечиÑлении. ЕÑли входные данные Ñодержат только идентификаторы перечиÑлениÑ, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга перечиÑлениÑ. Ðапример: From 27f5ade498dc44e2ee2b16d6e872eb88ea901dbf Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Mon, 13 Dec 2021 21:20:49 +0300 Subject: [PATCH 138/358] Update docs/ru/interfaces/formats.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index eef50eb6eaf..a158dc63074 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -130,7 +130,7 @@ world Каждый Ñлемент Ñтруктуры типа [Nested](../sql-reference/data-types/nested-data-structures/nested.md) предÑтавлÑетÑÑ ÐºÐ°Ðº отдельный маÑÑив. ВходÑщие параметры типа "перечиÑление" (`ENUM`) могут передаватьÑÑ Ð² виде значений или порÑдковых номеров. Сначала переданное значение будет ÑопоÑтавлÑÑ‚ÑŒÑÑ Ñ Ñлементами перечиÑлениÑ. ЕÑли Ñовпадение не будет найдено и при Ñтом переданное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, оно будет трактоватьÑÑ ÐºÐ°Ðº порÑдковый номер в перечиÑлении. -ЕÑли входные данные Ñодержат только идентификаторы перечиÑлениÑ, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга перечиÑлениÑ. +ЕÑли входÑщие параметры типа `ENUM` Ñодержат только порÑдковые номера, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ ÑƒÑÐºÐ¾Ñ€ÐµÐ½Ð¸Ñ Ð¿Ð°Ñ€Ñинга. Ðапример: From 9fbc0eb4b9e67bee88da4053e72424a5ab1cd606 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Mon, 13 Dec 2021 21:21:28 +0300 Subject: [PATCH 139/358] Update docs/ru/interfaces/formats.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/interfaces/formats.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index a158dc63074..a384776e519 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -365,8 +365,8 @@ $ clickhouse-client --format_csv_delimiter="|" --query="INSERT INTO test.csv FOR ЕÑли уÑтановлена наÑтройка [input_format_defaults_for_omitted_fields = 1](../operations/settings/settings.md#session_settings-input_format_defaults_for_omitted_fields) и тип Ñтолбца не `Nullable(T)`, то пуÑтые Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð±ÐµÐ· кавычек заменÑÑŽÑ‚ÑÑ Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñми по умолчанию Ð´Ð»Ñ Ñ‚Ð¸Ð¿Ð° данных Ñтолбца. -Ð—Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑлениÑ, в качеÑтве входных данных, могут быть предÑтавлены как Ð¸Ð¼Ñ Ð¸Ð»Ð¸ как идентификаторы. Сначала мы пытаемÑÑ ÑопоÑтавить входное значение Ñ Ð¸Ð¼ÐµÐ½ÐµÐ¼ перечиÑлениÑ. Ð’ Ñлучае неудачи и при уÑловии, что входное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, мы пытаемÑÑ ÑопоÑтавить Ñто чиÑло Ñ Ð¸Ð´ÐµÐ½Ñ‚Ð¸Ñ„Ð¸ÐºÐ°Ñ‚Ð¾Ñ€Ð¾Ð¼ перечиÑлениÑ. -ЕÑли входные данные Ñодержат только идентификаторы перечиÑлениÑ, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга перечиÑлениÑ. +ВходÑщие параметры типа "перечиÑление" (`ENUM`) могут передаватьÑÑ Ð² виде значений или порÑдковых номеров. Сначала переданное значение будет ÑопоÑтавлÑÑ‚ÑŒÑÑ Ñ Ñлементами перечиÑлениÑ. ЕÑли Ñовпадение не будет найдено и при Ñтом переданное значение ÑвлÑетÑÑ Ñ‡Ð¸Ñлом, оно будет трактоватьÑÑ ÐºÐ°Ðº порÑдковый номер в перечиÑлении. +ЕÑли входÑщие параметры типа `ENUM` Ñодержат только порÑдковые номера, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð¸Ñ‚ÑŒ наÑтройку [input_format_tsv_enum_as_number](../operations/settings/settings.md#settings-input_format_tsv_enum_as_number) Ð´Ð»Ñ ÑƒÑÐºÐ¾Ñ€ÐµÐ½Ð¸Ñ Ð¿Ð°Ñ€Ñинга. Формат CSV поддерживает вывод totals и extremes аналогично `TabSeparated`. From 7117e2c48972a1d8b881a729bef2c098c6b371e1 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Mon, 13 Dec 2021 21:21:41 +0300 Subject: [PATCH 140/358] Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 267dae3ae2a..c60d46705e3 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1531,7 +1531,8 @@ SELECT area/period FROM account_orders FORMAT JSON; ## input_format_csv_enum_as_number {#settings-input_format_csv_enum_as_number} -При включенном режиме вÑегда обрабатывайте Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº идентификаторы перечиÑÐ»ÐµÐ½Ð¸Ñ Ð´Ð»Ñ Ð²Ñ…Ð¾Ð´Ð½Ð¾Ð³Ð¾ формата CSV. Ð”Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð°Ñ‚ÑŒ Ñтот параметр, еÑли данные Ñодержат только идентификаторы перечиÑлениÑ. +Включает или отключает парÑинг значений перечиÑлений как порÑдковых номеров. +ЕÑли режим включен, то во входÑщих данных в формате `CSV` Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑÐ»ÐµÐ½Ð¸Ñ (тип `ENUM`) вÑегда трактуютÑÑ ÐºÐ°Ðº порÑдковые номера, а не как Ñлементы перечиÑлениÑ. Эту наÑтройку рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð°Ñ‚ÑŒ Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга, еÑли данные типа `ENUM` Ñодержат только порÑдковые номера, а не Ñами Ñлементы перечиÑлениÑ. Возможные значениÑ: From d3e5b97ac9e3c659cb00a20a709db8665daf14d5 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Mon, 13 Dec 2021 21:22:03 +0300 Subject: [PATCH 141/358] Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index c60d46705e3..e473c37986a 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1536,8 +1536,8 @@ SELECT area/period FROM account_orders FORMAT JSON; Возможные значениÑ: -- 0 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ как Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¸Ð»Ð¸ как идентификаторы. -- 1 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ только как идентификаторы. +- 0 — входÑщие Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ñ‚Ð¸Ð¿Ð° `ENUM` Ñначала ÑопоÑтавлÑÑŽÑ‚ÑÑ Ñ Ñлементами перечиÑлениÑ, а еÑли Ñовпадений не найдено, то трактуютÑÑ ÐºÐ°Ðº порÑдковые номера. +- 1 — входÑщие Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ñ‚Ð¸Ð¿Ð° `ENUM` Ñразу трактуютÑÑ ÐºÐ°Ðº порÑдковые номера. Значение по умолчанию: 0. From b928c551c60ec97b93cfeaf6f283a730962c0aa8 Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Mon, 13 Dec 2021 21:23:11 +0300 Subject: [PATCH 142/358] Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e473c37986a..680d868a536 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -391,7 +391,9 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( ## input_format_tsv_enum_as_number {#settings-input_format_tsv_enum_as_number} -При включенном режиме вÑегда обрабатывайте Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑÐ»ÐµÐ½Ð¸Ñ ÐºÐ°Ðº идентификаторы перечиÑÐ»ÐµÐ½Ð¸Ñ Ð´Ð»Ñ Ð²Ñ…Ð¾Ð´Ð½Ð¾Ð³Ð¾ формата TSV. Ð”Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга, рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð°Ñ‚ÑŒ Ñтот параметр, еÑли данные Ñодержат только идентификаторы перечиÑлениÑ. +Включает или отключает парÑинг значений перечиÑлений как порÑдковых номеров. + +ЕÑли режим включен, то во входÑщих данных в формате `TCV` Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¿ÐµÑ€ÐµÑ‡Ð¸ÑÐ»ÐµÐ½Ð¸Ñ (тип `ENUM`) вÑегда трактуютÑÑ ÐºÐ°Ðº порÑдковые номера, а не как Ñлементы перечиÑлениÑ. Эту наÑтройку рекомендуетÑÑ Ð²ÐºÐ»ÑŽÑ‡Ð°Ñ‚ÑŒ Ð´Ð»Ñ Ð¾Ð¿Ñ‚Ð¸Ð¼Ð¸Ð·Ð°Ñ†Ð¸Ð¸ парÑинга, еÑли данные типа `ENUM` Ñодержат только порÑдковые номера, а не Ñами Ñлементы перечиÑлениÑ. Возможные значениÑ: From 43a3d927f67774e7f18eac2b214cde010fb2d73b Mon Sep 17 00:00:00 2001 From: andrc1901 <92211164+andrc1901@users.noreply.github.com> Date: Mon, 13 Dec 2021 21:23:22 +0300 Subject: [PATCH 143/358] Update docs/ru/operations/settings/settings.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 680d868a536..94bd2078373 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -397,8 +397,8 @@ INSERT INTO test VALUES (lower('Hello')), (lower('world')), (lower('INSERT')), ( Возможные значениÑ: -- 0 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ как Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ð¸Ð»Ð¸ как идентификаторы. -- 1 — данные перечиÑÐ»ÐµÐ½Ð¸Ñ Ð¾Ð±Ñ€Ð°Ð±Ð¾Ñ‚Ð°Ð½Ñ‹ только как идентификаторы. +- 0 — входÑщие Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ñ‚Ð¸Ð¿Ð° `ENUM` Ñначала ÑопоÑтавлÑÑŽÑ‚ÑÑ Ñ Ñлементами перечиÑлениÑ, а еÑли Ñовпадений не найдено, то трактуютÑÑ ÐºÐ°Ðº порÑдковые номера. +- 1 — входÑщие Ð·Ð½Ð°Ñ‡ÐµÐ½Ð¸Ñ Ñ‚Ð¸Ð¿Ð° `ENUM` Ñразу трактуютÑÑ ÐºÐ°Ðº порÑдковые номера. Значение по умолчанию: 0. From f0d0714e479caab7f1a1999ccfcddd0e23decee7 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 13 Dec 2021 22:06:46 +0000 Subject: [PATCH 144/358] Pass timeouts for mysql --- base/mysqlxx/PoolWithFailover.cpp | 8 +++--- base/mysqlxx/PoolWithFailover.h | 5 +++- src/Core/Settings.h | 4 +++ src/Databases/DatabaseFactory.cpp | 5 +++- .../MySQL/ConnectionMySQLSettings.cpp | 2 +- src/Databases/MySQL/ConnectionMySQLSettings.h | 7 ++++- src/Dictionaries/MySQLDictionarySource.cpp | 14 +++++++--- .../ExternalDataSourceConfiguration.cpp | 1 + src/Storages/MySQL/MySQLHelpers.cpp | 26 +++++++++++++++++++ src/Storages/MySQL/MySQLHelpers.h | 18 +++++++++++++ src/Storages/MySQL/MySQLSettings.h | 2 ++ src/Storages/StorageMySQL.cpp | 9 ++----- src/TableFunctions/TableFunctionMySQL.cpp | 7 ++++- 13 files changed, 89 insertions(+), 19 deletions(-) create mode 100644 src/Storages/MySQL/MySQLHelpers.cpp create mode 100644 src/Storages/MySQL/MySQLHelpers.h diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index 14c0db9ecd5..9a5214454f5 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -82,7 +82,9 @@ PoolWithFailover::PoolWithFailover( unsigned default_connections_, unsigned max_connections_, size_t max_tries_, - uint64_t wait_timeout_) + uint64_t wait_timeout_, + size_t connect_timeout_, + size_t rw_timeout_) : max_tries(max_tries_) , shareable(false) , wait_timeout(wait_timeout_) @@ -93,8 +95,8 @@ PoolWithFailover::PoolWithFailover( replicas_by_priority[0].emplace_back(std::make_shared(database, host, user, password, port, /* socket_ = */ "", - MYSQLXX_DEFAULT_TIMEOUT, - MYSQLXX_DEFAULT_RW_TIMEOUT, + connect_timeout_, + rw_timeout_, default_connections_, max_connections_)); } diff --git a/base/mysqlxx/PoolWithFailover.h b/base/mysqlxx/PoolWithFailover.h index 2bd5ec9f30a..17870d141e1 100644 --- a/base/mysqlxx/PoolWithFailover.h +++ b/base/mysqlxx/PoolWithFailover.h @@ -6,6 +6,7 @@ #define MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS 1 #define MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS 16 #define MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3 +#define MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_CONNECTION_WAIT_TIMEOUT 5 /// in seconds namespace mysqlxx @@ -121,7 +122,9 @@ namespace mysqlxx unsigned default_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, unsigned max_connections_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_CONNECTIONS, size_t max_tries_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, - uint64_t wait_timeout_ = UINT64_MAX); + uint64_t wait_timeout_ = MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_CONNECTION_WAIT_TIMEOUT, + size_t connect_timeout = MYSQLXX_DEFAULT_TIMEOUT, + size_t rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT); PoolWithFailover(const PoolWithFailover & other); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 47b01655c26..964ba8ffc13 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -496,8 +496,12 @@ class IColumn; M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 1, "Version of DDL entry to write into ZooKeeper", 0) \ + \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ + M(UInt64, external_storage_connect_timeout, 60, "Connect timeout. Now supported only for MySQL", 0) \ + M(UInt64, external_storage_rw_timeout, 1800, "Read/write timeout. Now supported only for MySQL", 0) \ + \ M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index a4c8f3cdb77..49369086468 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -23,6 +23,8 @@ # include # include # include +# include +# include # include # include #endif @@ -198,7 +200,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine_name == "MySQL") { auto mysql_database_settings = std::make_unique(); - auto mysql_pool = mysqlxx::PoolWithFailover(configuration.database, configuration.addresses, configuration.username, configuration.password); + MySQLSettings mysql_settings; + auto mysql_pool = createMySQLPoolWithFailover(configuration, mysql_settings); mysql_database_settings->loadFromQueryContext(context); mysql_database_settings->loadFromQuery(*engine_define); /// higher priority diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.cpp b/src/Databases/MySQL/ConnectionMySQLSettings.cpp index 1026d14018b..87da701e481 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.cpp +++ b/src/Databases/MySQL/ConnectionMySQLSettings.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -IMPLEMENT_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_CONNECTION_MYSQL_SETTINGS) +IMPLEMENT_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_MYSQL_DATABASE_SETTINGS) void ConnectionMySQLSettings::loadFromQuery(ASTStorage & storage_def) { diff --git a/src/Databases/MySQL/ConnectionMySQLSettings.h b/src/Databases/MySQL/ConnectionMySQLSettings.h index f05985a0cda..8b17d4e7898 100644 --- a/src/Databases/MySQL/ConnectionMySQLSettings.h +++ b/src/Databases/MySQL/ConnectionMySQLSettings.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -17,7 +18,11 @@ class ASTStorage; #define APPLY_FOR_IMMUTABLE_CONNECTION_MYSQL_SETTINGS(M) \ M(mysql_datatypes_support_level) -DECLARE_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_CONNECTION_MYSQL_SETTINGS) +#define LIST_OF_MYSQL_DATABASE_SETTINGS(M) \ + LIST_OF_CONNECTION_MYSQL_SETTINGS(M) \ + LIST_OF_MYSQL_SETTINGS(M) + +DECLARE_SETTINGS_TRAITS(ConnectionMySQLSettingsTraits, LIST_OF_MYSQL_DATABASE_SETTINGS) /** Settings for the MySQL database engine. diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index f6de6ca0cc1..fd090f99136 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -14,6 +14,8 @@ #include #include #include +#include +#include namespace DB @@ -46,13 +48,17 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) auto settings_config_prefix = config_prefix + ".mysql"; std::shared_ptr pool; - ExternalDataSourceConfiguration configuration; + StorageMySQLConfiguration configuration; auto named_collection = created_from_ddl ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context) : std::nullopt; if (named_collection) { - configuration = *named_collection; - std::vector> addresses{std::make_pair(configuration.host, configuration.port)}; - pool = std::make_shared(configuration.database, addresses, configuration.username, configuration.password); + configuration.set(*named_collection); + configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; + MySQLSettings mysql_settings; + const auto & settings = global_context->getSettingsRef(); + mysql_settings.connect_timeout = settings.external_storage_connect_timeout; + mysql_settings.read_write_timeout = settings.external_storage_rw_timeout; + pool = std::make_shared(createMySQLPoolWithFailover(configuration, mysql_settings)); } else { diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index ade89ea7228..42b3b148551 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -54,6 +54,7 @@ void ExternalDataSourceConfiguration::set(const ExternalDataSourceConfiguration database = conf.database; table = conf.table; schema = conf.schema; + addresses = conf.addresses; addresses_expr = conf.addresses_expr; } diff --git a/src/Storages/MySQL/MySQLHelpers.cpp b/src/Storages/MySQL/MySQLHelpers.cpp new file mode 100644 index 00000000000..e7745e6c0bb --- /dev/null +++ b/src/Storages/MySQL/MySQLHelpers.cpp @@ -0,0 +1,26 @@ +#include "MySQLHelpers.h" + +#if USE_MYSQL +#include +#include +#include + +namespace DB +{ + +mysqlxx::PoolWithFailover +createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, const MySQLSettings & mysql_settings) +{ + return mysqlxx::PoolWithFailover( + configuration.database, configuration.addresses, configuration.username, configuration.password, + MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, + mysql_settings.connection_pool_size, + mysql_settings.connection_max_tries, + mysql_settings.connection_wait_timeout, + mysql_settings.connect_timeout, + mysql_settings.read_write_timeout); +} + +} + +#endif diff --git a/src/Storages/MySQL/MySQLHelpers.h b/src/Storages/MySQL/MySQLHelpers.h new file mode 100644 index 00000000000..3b047896eb1 --- /dev/null +++ b/src/Storages/MySQL/MySQLHelpers.h @@ -0,0 +1,18 @@ +#include "config_core.h" + +#if USE_MYSQL +#include + +namespace mysqlxx { class PoolWithFailover; } + +namespace DB +{ +struct StorageMySQLConfiguration; +struct MySQLSettings; + +mysqlxx::PoolWithFailover +createMySQLPoolWithFailover(const StorageMySQLConfiguration & configuration, const MySQLSettings & mysql_settings); + +} + +#endif diff --git a/src/Storages/MySQL/MySQLSettings.h b/src/Storages/MySQL/MySQLSettings.h index 872b0607e20..896141ae673 100644 --- a/src/Storages/MySQL/MySQLSettings.h +++ b/src/Storages/MySQL/MySQLSettings.h @@ -19,6 +19,8 @@ class ASTStorage; M(UInt64, connection_max_tries, 3, "Number of retries for pool with failover", 0) \ M(UInt64, connection_wait_timeout, 5, "Timeout (in seconds) for waiting for free connection (in case of there is already connection_pool_size active connections), 0 - do not wait.", 0) \ M(Bool, connection_auto_close, true, "Auto-close connection after query execution, i.e. disable connection reuse.", 0) \ + M(UInt64, connect_timeout, 60, "Connect timeout", 0) \ + M(UInt64, read_write_timeout, 1800, "Read/write timeout", 0) \ DECLARE_SETTINGS_TRAITS(MySQLSettingsTraits, LIST_OF_MYSQL_SETTINGS) diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 0b6095e033b..66adf3ae272 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -306,13 +307,7 @@ void registerStorageMySQL(StorageFactory & factory) if (!mysql_settings.connection_pool_size) throw Exception("connection_pool_size cannot be zero.", ErrorCodes::BAD_ARGUMENTS); - mysqlxx::PoolWithFailover pool( - configuration.database, configuration.addresses, - configuration.username, configuration.password, - MYSQLXX_POOL_WITH_FAILOVER_DEFAULT_START_CONNECTIONS, - mysql_settings.connection_pool_size, - mysql_settings.connection_max_tries, - mysql_settings.connection_wait_timeout); + mysqlxx::PoolWithFailover pool = createMySQLPoolWithFailover(configuration, mysql_settings); return StorageMySQL::create( args.table_id, diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index a9cecb11a1c..86abe352a7d 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -37,7 +38,11 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr throw Exception("Table function 'mysql' must have arguments.", ErrorCodes::LOGICAL_ERROR); configuration = StorageMySQL::getConfiguration(args_func.arguments->children, context); - pool.emplace(configuration->database, configuration->addresses, configuration->username, configuration->password); + MySQLSettings mysql_settings; + const auto & settings = context->getSettingsRef(); + mysql_settings.connect_timeout = settings.external_storage_connect_timeout; + mysql_settings.read_write_timeout = settings.external_storage_rw_timeout; + pool.emplace(createMySQLPoolWithFailover(*configuration, mysql_settings)); } ColumnsDescription TableFunctionMySQL::getActualTableStructure(ContextPtr context) const From c9ec664094242204248c48316559308168122ebd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 14 Dec 2021 01:25:53 +0300 Subject: [PATCH 145/358] Update MySQLHelpers.h --- src/Storages/MySQL/MySQLHelpers.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MySQL/MySQLHelpers.h b/src/Storages/MySQL/MySQLHelpers.h index 3b047896eb1..712c5a2c719 100644 --- a/src/Storages/MySQL/MySQLHelpers.h +++ b/src/Storages/MySQL/MySQLHelpers.h @@ -1,3 +1,4 @@ +#pragma once #include "config_core.h" #if USE_MYSQL From dbe57e10cdf7fc619738cdfabb78cc252c189e29 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Mon, 13 Dec 2021 15:28:43 -0700 Subject: [PATCH 146/358] Add v21.21 release blog post --- .../en/2021/clickhouse-v21.12-released.md | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 website/blog/en/2021/clickhouse-v21.12-released.md diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md new file mode 100644 index 00000000000..555229ff843 --- /dev/null +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -0,0 +1,60 @@ +--- +title: 'ClickHouse v21.12 Released' +image: 'https://blog-images.clickhouse.com/en/2021/clickhouse-v21-12/featured.jpg' +date: '2021-12-13' +author: '[Rich Raposa](https://github.com/rfraposa), [Alexey Milovidov](https://github.com/alexey-milovidov)' +tags: ['company', 'community'] +--- + +We're continuing our monthly release cadence and blog updates at[ ClickHouse, Inc](https://clickhouse.com/blog/en/2021/clickhouse-inc/). Let's highlight some of these new exciting new capabilities in 21.12: + +## Bool data type + +ClickHouse now natively supports a `bool` data type. Supported input values are “trueâ€/â€falseâ€, “TRUEâ€/â€FALSE†and “1â€/â€0â€. By default, the text representation for CSV/TSV output is “true†and “false†but can be adjusted to anything else using the settings `bool_true_representation` and `bool_false_representation` (for example, “yes†and “noâ€). + +**How does this help you?** + +Native boolean data types exist today in other databases that are often integrated with ClickHouse, such as PostgreSQL. The `bool` data type in ClickHouse will make it more compatible with existing code and ease migration from other databases. + + +## Partitions for File, URL, and HDFS storage + +When using the table engines file, url, and hdfs ClickHouse now supports partitions. When creating a table you can specify the partition key using the `PARTITION BY` clause e.g. `CREATE TABLE hits_files (<columns>) ENGINE=File(TabSeparated) PARTITION BY toYYYYMM(EventDate)`. + +Similarly, when exporting data from ClickHouse using the file, url, and hdfs table functions you can now specify that the data is to be partitioned into multiple files using a `PARTITION BY` clause. For example, `INSERT INTO TABLE FUNCTION file(’<path>/hits_{_partition_id}’, ‘TSV’, ‘<table_format>’) PARTITION BY toYYYYMM(EventDate) VALUES <values>’ will create as many files as there are unique month in the dataset. + +The S3 table function has supported partitioned writes since ClickHouse 21.10. + +**How does this help you?** + +This feature makes using tables backed by files on local file systems and remote file systems accessed over HTTP and HDFS more convenient and flexible. Especially for large datasets you might not want all data to be in one big file. This feature allows you to split one table into smaller files any way you want e.g. by month. + + +## Table constraints + +When creating tables, you can now (optionally) specify constraints. A constraint tells ClickHouse that a column has a specific relationship to another column in the same table. For example, a string column might store the prefix of another column. Then, when a select query is trying to calculate the prefix on the original column, ClickHouse will rewrite the query to use the prefix column. + +For now, constraints are implemented as assumptions, that is, ClickHouse does not validate that they are correct and will not reject new data that violates them. + +This feature is disabled by default. To turn it on, enable `optimize_using_constraints`, `optimize_substitute_columns` and/or `optimize_append_index`. + +**How does this help you?** + +Especially in large ClickHouse deployments with many complex tables it can be hard for users to always be up to date on the best way to query a given dataset. Constraints can help optimize queries without having to change the query structure itself. They can also make it easier to make changes to tables. For example, let’s say you have a table containing web requests and it includes a URL column that contains the full URL of each request. Many times, users will want to know the top level domain (.com, .co.uk, etc.), something ClickHouse provides the topLevelDomain function to calculate. If you discover that many people are using this function you might decide to create a new materialized column that pre-calculates the top level domain for each record. Rather than tell all your users to change their queries you can use a table constraint to tell ClickHouse that each time a user tries to call the topLevelDomain function the request should be rewritten to use the new materialized column. + + +## Read large remote files in chunks + +When reading large files in Parquet, ORC, and Arrow format using the s3, url, and hdfs table functions, ClickHouse will now automatically choose whether to read the entire file at once or read parts of it incrementally. This is now enabled by default and the setting `remote_read_min_bytes_for_seek` controls when to switch from reading it all to reading in chunks. The default is 1MiB. + +**How does this help our ClickHouse Users?** + +In previous versions, when reading files from remote locations with the s3, url, and hdfs table functions, ClickHouse would always read the entire file into memory. This works well when the files are small but will cause excessive memory usage or not work at all when the files are large. With this change, ClickHouse will read large files in chunks to keep memory usage in check and is now able to read even very large files. + +ClickHouse Keeper is 100% feature complete. More updates to come in the coming weeks around where and how you can test and provide feedback for us! + +Release 21.12 + +Release Date: 2021-12-13 + +Release Notes: [21.12](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) From 5c198525d22c77b9f51a5072dd72d20da0f064d5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Nov 2021 00:41:34 +0300 Subject: [PATCH 147/358] Disconnect connection on unknown packet. --- src/QueryPipeline/ConnectionCollector.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/QueryPipeline/ConnectionCollector.cpp b/src/QueryPipeline/ConnectionCollector.cpp index a6a0afb68d3..59865006a4e 100644 --- a/src/QueryPipeline/ConnectionCollector.cpp +++ b/src/QueryPipeline/ConnectionCollector.cpp @@ -90,6 +90,9 @@ void ConnectionCollector::drainConnections(IConnections & connections) noexcept break; default: + /// Connection should be closed in case of unknown packet, + /// since this means that the connection in some bad state. + is_drained = false; throw Exception( ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from one of the following replicas: {}", From eb9968babf0729218a9eeb2884f3fe8993ab58ae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Nov 2021 00:51:38 +0300 Subject: [PATCH 148/358] Add timeout into "Read timeout while draining from" message --- src/Client/IConnections.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Client/IConnections.cpp b/src/Client/IConnections.cpp index dc57cae61a4..1ab7ba6e5d8 100644 --- a/src/Client/IConnections.cpp +++ b/src/Client/IConnections.cpp @@ -25,7 +25,12 @@ struct PocoSocketWrapper : public Poco::Net::SocketImpl void IConnections::DrainCallback::operator()(int fd, Poco::Timespan, const std::string fd_description) const { if (!PocoSocketWrapper(fd).poll(drain_timeout, Poco::Net::Socket::SELECT_READ)) - throw Exception(ErrorCodes::SOCKET_TIMEOUT, "Read timeout while draining from {}", fd_description); + { + throw Exception(ErrorCodes::SOCKET_TIMEOUT, + "Read timeout ({} ms) while draining from {}", + drain_timeout.totalMilliseconds(), + fd_description); + } } } From 510eab25a1868c3c4669d27354f37ca79c6ef702 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 27 Nov 2021 01:00:48 +0300 Subject: [PATCH 149/358] Add timeout into "Timeout exceeded while reading from" message --- src/Client/MultiplexedConnections.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index c3000443a9c..37a372dfb45 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -395,17 +395,17 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead read_list.push_back(*connection->socket); } + auto timeout = is_draining ? drain_timeout : receive_timeout; int n = Poco::Net::Socket::select( read_list, write_list, except_list, - is_draining ? drain_timeout : receive_timeout); + timeout); /// We treat any error as timeout for simplicity. /// And we also check if read_list is still empty just in case. if (n <= 0 || read_list.empty()) { - auto err_msg = fmt::format("Timeout exceeded while reading from {}", dumpAddressesUnlocked()); for (ReplicaState & state : replica_states) { Connection * connection = state.connection; @@ -415,7 +415,10 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead invalidateReplica(state); } } - throw Exception(err_msg, ErrorCodes::TIMEOUT_EXCEEDED); + throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, + "Timeout ({} ms) exceeded while reading from {}", + timeout.totalMilliseconds(), + dumpAddressesUnlocked()); } } From 15bc4c0ca8903c46f95b3dbdf65dcf1d83865d9b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 29 Nov 2021 23:09:37 +0300 Subject: [PATCH 150/358] Add max_threads_for_connection_collector example and description into config --- programs/server/config.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/programs/server/config.xml b/programs/server/config.xml index 9a2a6d7729f..d5fdb114e5c 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -293,6 +293,10 @@ 10000 + + + 0.9 From 33ffd343f3fee716ff167f066f32510ecbc0ca39 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 29 Nov 2021 23:09:55 +0300 Subject: [PATCH 151/358] Add ability to drain connections synchronously This is mostly for testing, to avoid hiding errors. --- src/Core/Settings.h | 2 +- src/QueryPipeline/ConnectionCollector.cpp | 7 +++-- src/QueryPipeline/ConnectionCollector.h | 2 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 20 ++++++++++--- .../02127_connection_drain.reference | 2 ++ .../0_stateless/02127_connection_drain.sh | 29 +++++++++++++++++++ 6 files changed, 54 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02127_connection_drain.reference create mode 100755 tests/queries/0_stateless/02127_connection_drain.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 47b01655c26..3b0010976bf 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,7 +55,7 @@ class IColumn; M(Milliseconds, connect_timeout_with_failover_secure_ms, 100, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \ M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ - M(Seconds, drain_timeout, 3, "", 0) \ + M(Seconds, drain_timeout, 3, "Timeout for draining remote connections, -1 means synchronous drain w/o ignoring errors", 0) \ M(Seconds, tcp_keep_alive_timeout, 290 /* less than DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC */, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ M(Milliseconds, hedged_connection_timeout_ms, 100, "Connection timeout for establishing connection with replica for Hedged requests", 0) \ M(Milliseconds, receive_data_timeout_ms, 2000, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \ diff --git a/src/QueryPipeline/ConnectionCollector.cpp b/src/QueryPipeline/ConnectionCollector.cpp index 59865006a4e..c2cdd1a1133 100644 --- a/src/QueryPipeline/ConnectionCollector.cpp +++ b/src/QueryPipeline/ConnectionCollector.cpp @@ -46,7 +46,7 @@ struct AsyncDrainTask std::shared_ptr shared_connections; void operator()() const { - ConnectionCollector::drainConnections(*shared_connections); + ConnectionCollector::drainConnections(*shared_connections, /* throw_error= */ false); } // We don't have std::unique_function yet. Wrap it in shared_ptr to make the functor copyable. @@ -71,7 +71,7 @@ std::shared_ptr ConnectionCollector::enqueueConnectionCleanup( return connections; } -void ConnectionCollector::drainConnections(IConnections & connections) noexcept +void ConnectionCollector::drainConnections(IConnections & connections, bool throw_error) { bool is_drained = false; try @@ -114,6 +114,9 @@ void ConnectionCollector::drainConnections(IConnections & connections) noexcept tryLogCurrentException(&Poco::Logger::get("ConnectionCollector"), __PRETTY_FUNCTION__); } } + + if (throw_error) + throw; } } diff --git a/src/QueryPipeline/ConnectionCollector.h b/src/QueryPipeline/ConnectionCollector.h index 5b6e82d000e..44482607277 100644 --- a/src/QueryPipeline/ConnectionCollector.h +++ b/src/QueryPipeline/ConnectionCollector.h @@ -17,7 +17,7 @@ public: static ConnectionCollector & init(ContextMutablePtr global_context_, size_t max_threads); static std::shared_ptr enqueueConnectionCleanup(const ConnectionPoolWithFailoverPtr & pool, std::shared_ptr connections) noexcept; - static void drainConnections(IConnections & connections) noexcept; + static void drainConnections(IConnections & connections, bool throw_error); private: explicit ConnectionCollector(ContextMutablePtr global_context_, size_t max_threads); diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 653d9a2bbf8..142e56ceb25 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -495,14 +495,26 @@ void RemoteQueryExecutor::finish(std::unique_ptr * read_context) /// Send the request to abort the execution of the request, if not already sent. tryCancel("Cancelling query because enough data has been read", read_context); - /// Try to drain connections asynchronously. - if (auto conn = ConnectionCollector::enqueueConnectionCleanup(pool, connections)) + + if (context->getSettingsRef().drain_timeout != Poco::Timespan(-1000000)) { - /// Drain connections synchronously. + auto connections_left = ConnectionCollector::enqueueConnectionCleanup(pool, connections); + if (connections_left) + { + /// Drain connections synchronously and suppress errors. + CurrentMetrics::Increment metric_increment(CurrentMetrics::ActiveSyncDrainedConnections); + ConnectionCollector::drainConnections(*connections_left, /* throw_error= */ false); + CurrentMetrics::add(CurrentMetrics::SyncDrainedConnections, 1); + } + } + else + { + /// Drain connections synchronously w/o suppressing errors. CurrentMetrics::Increment metric_increment(CurrentMetrics::ActiveSyncDrainedConnections); - ConnectionCollector::drainConnections(*conn); + ConnectionCollector::drainConnections(*connections, /* throw_error= */ true); CurrentMetrics::add(CurrentMetrics::SyncDrainedConnections, 1); } + finished = true; } diff --git a/tests/queries/0_stateless/02127_connection_drain.reference b/tests/queries/0_stateless/02127_connection_drain.reference new file mode 100644 index 00000000000..c31f2f40f6d --- /dev/null +++ b/tests/queries/0_stateless/02127_connection_drain.reference @@ -0,0 +1,2 @@ +OK: sync drain +OK: async drain diff --git a/tests/queries/0_stateless/02127_connection_drain.sh b/tests/queries/0_stateless/02127_connection_drain.sh new file mode 100755 index 00000000000..50eff192c1c --- /dev/null +++ b/tests/queries/0_stateless/02127_connection_drain.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# sync drain +for _ in {1..100}; do + prev=$(curl -d@- -sS "${CLICKHOUSE_URL}" <<<"select value from system.metrics where metric = 'SyncDrainedConnections'") + curl -d@- -sS "${CLICKHOUSE_URL}" <<<"select * from remote('127.{2,3}', view(select * from numbers(1e6))) limit 100 settings drain_timeout=-1 format Null" + now=$(curl -d@- -sS "${CLICKHOUSE_URL}" <<<"select value from system.metrics where metric = 'SyncDrainedConnections'") + if [[ "$prev" != $(( now-2 )) ]]; then + continue + fi + echo "OK: sync drain" + break +done + +# async drain +for _ in {1..100}; do + prev=$(curl -d@- -sS "${CLICKHOUSE_URL}" <<<"select value from system.metrics where metric = 'AsyncDrainedConnections'") + curl -d@- -sS "${CLICKHOUSE_URL}" <<<"select * from remote('127.{2,3}', view(select * from numbers(1e6))) limit 100 settings drain_timeout=10 format Null" + now=$(curl -d@- -sS "${CLICKHOUSE_URL}" <<<"select value from system.metrics where metric = 'AsyncDrainedConnections'") + if [[ "$prev" != $(( now-2 )) ]]; then + continue + fi + echo "OK: async drain" + break +done From 03dbbaca3e012274579d43dda8a5fc5a092fee2a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 1 Dec 2021 09:21:11 +0300 Subject: [PATCH 152/358] Disable 02127_connection_drain to run in parallel --- tests/queries/0_stateless/02127_connection_drain.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02127_connection_drain.sh b/tests/queries/0_stateless/02127_connection_drain.sh index 50eff192c1c..523b02d9bd5 100755 --- a/tests/queries/0_stateless/02127_connection_drain.sh +++ b/tests/queries/0_stateless/02127_connection_drain.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 5c7b2b0aafa16bc4fbeb764691f95e90e414ea01 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 14 Dec 2021 07:59:47 +0000 Subject: [PATCH 153/358] Update test --- tests/queries/0_stateless/01293_show_settings.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index aa27ef83f52..367a0670acf 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -2,6 +2,7 @@ send_timeout Seconds 300 connect_timeout Seconds 10 connect_timeout_with_failover_ms Milliseconds 2000 connect_timeout_with_failover_secure_ms Milliseconds 3000 +external_storage_connect_timeout UInt64 60 max_memory_usage UInt64 10000000000 max_untracked_memory UInt64 1048576 memory_profiler_step UInt64 1048576 From 7b29377338f636ec13bbf9b31485642a9f9fbb97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Dec 2021 13:40:03 +0300 Subject: [PATCH 154/358] Better control build artifacts --- tests/ci/build_check.py | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 36db7d596c9..99c1ab56b04 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -76,15 +76,23 @@ def get_image_name(build_config): return 'clickhouse/deb-builder' -def build_clickhouse(packager_cmd, logs_path): +def build_clickhouse(packager_cmd, logs_path, build_output_path): build_log_path = os.path.join(logs_path, 'build_log.log') with TeePopen(packager_cmd, build_log_path) as process: retcode = process.wait() + if os.path.exists(build_output_path): + build_results = os.listdir(build_output_path) + else: + build_results = [] + if retcode == 0: - logging.info("Built successfully") + if len(build_results) != 0: + logging.info("Built successfully") + else: + logging.info("Success exit code, but no build artifacts => build failed") else: logging.info("Build failed") - return build_log_path, retcode == 0 + return build_log_path, retcode == 0 and len(build_results) > 0 def get_build_results_if_exists(s3_helper, s3_prefix): @@ -197,7 +205,7 @@ if __name__ == "__main__": os.makedirs(build_clickhouse_log) start = time.time() - log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log) + log_path, success = build_clickhouse(packager_cmd, build_clickhouse_log, build_output_path) elapsed = int(time.time() - start) subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {build_output_path}", shell=True) subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {ccache_path}", shell=True) From 14ca6aa92b5206981aa8f41b0c5fce39d78be836 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=9D=B3=E9=98=B3?= <260893248@qq.com> Date: Tue, 14 Dec 2021 19:21:12 +0800 Subject: [PATCH 155/358] name -> table_name fix possible wrong word --- docs/en/sql-reference/statements/create/table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index d64642704f5..c3e54545549 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -22,7 +22,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE = engine ``` -Creates a table named `name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine. +Creates a table named `table_name` in the `db` database or the current database if `db` is not set, with the structure specified in brackets and the `engine` engine. The structure of the table is a list of column descriptions, secondary indexes and constraints . If [primary key](#primary-key) is supported by the engine, it will be indicated as parameter for the table engine. A column description is `name type` in the simplest case. Example: `RegionID UInt32`. From 4223553508e2f76d063e6c7970b02f0f32b2a199 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Tue, 14 Dec 2021 14:57:13 +0300 Subject: [PATCH 156/358] Fixed test --- tests/queries/0_stateless/02048_views_with_comment.sql | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02048_views_with_comment.sql b/tests/queries/0_stateless/02048_views_with_comment.sql index 9b501b76669..a7c991d119b 100644 --- a/tests/queries/0_stateless/02048_views_with_comment.sql +++ b/tests/queries/0_stateless/02048_views_with_comment.sql @@ -1,5 +1,5 @@ -# Make sure that any kind of `VIEW` can be created with a `COMMENT` clause -# and value of that clause is visible as `comment` column of `system.tables` table. +-- Make sure that any kind of `VIEW` can be created with a `COMMENT` clause +-- and value of that clause is visible as `comment` column of `system.tables` table. CREATE VIEW view_comment_test AS (SELECT 1) COMMENT 'simple view'; CREATE MATERIALIZED VIEW materialized_view_comment_test TO test1 (a UInt64) AS (SELECT 1) COMMENT 'materialized view'; @@ -7,4 +7,6 @@ CREATE MATERIALIZED VIEW materialized_view_comment_test TO test1 (a UInt64) AS ( SET allow_experimental_live_view=1; CREATE LIVE VIEW live_view_comment_test AS (SELECT 1) COMMENT 'live view'; -SELECT name, engine, comment FROM system.tables WHERE name LIKE '%view_comment_test' ORDER BY name; \ No newline at end of file +SYSTEM FLUSH LOGS; + +SELECT name, engine, comment FROM system.tables WHERE database == currentDatabase() ORDER BY name; From 9974f1063136f5148049968b896bd5e22608feb4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Dec 2021 15:34:20 +0300 Subject: [PATCH 157/358] Fix clang tidy --- src/Common/getRandomASCIIString.cpp | 17 +++++++++++++++++ src/Common/getRandomASCIIString.h | 10 ++++++++++ src/Disks/BlobStorage/DiskBlobStorage.cpp | 3 ++- src/Disks/RemoteDisksCommon.cpp | 12 +----------- src/Disks/RemoteDisksCommon.h | 3 +-- src/Disks/S3/DiskS3.cpp | 3 ++- src/IO/ReadBufferFromBlobStorage.cpp | 2 +- src/IO/WriteBufferFromBlobStorage.cpp | 3 ++- src/Parsers/ASTTableOverrides.h | 2 +- 9 files changed, 37 insertions(+), 18 deletions(-) create mode 100644 src/Common/getRandomASCIIString.cpp create mode 100644 src/Common/getRandomASCIIString.h diff --git a/src/Common/getRandomASCIIString.cpp b/src/Common/getRandomASCIIString.cpp new file mode 100644 index 00000000000..788c0d05ff5 --- /dev/null +++ b/src/Common/getRandomASCIIString.cpp @@ -0,0 +1,17 @@ +#include +#include +#include + +namespace DB +{ + +String getRandomASCIIString(size_t len, char first, char last) +{ + std::uniform_int_distribution distribution(first, last); + String res(len, ' '); + for (auto & c : res) + c = distribution(thread_local_rng); + return res; +} + +} diff --git a/src/Common/getRandomASCIIString.h b/src/Common/getRandomASCIIString.h new file mode 100644 index 00000000000..69684a9bef2 --- /dev/null +++ b/src/Common/getRandomASCIIString.h @@ -0,0 +1,10 @@ +#pragma once +#include + +namespace DB +{ +/// Slow random string. Useful for random names and things like this. Not for +/// generating data. +String getRandomASCIIString(size_t len = 32, char first = 'a', char last = 'z'); + +} diff --git a/src/Disks/BlobStorage/DiskBlobStorage.cpp b/src/Disks/BlobStorage/DiskBlobStorage.cpp index 253df7a9cf6..f33e698a50d 100644 --- a/src/Disks/BlobStorage/DiskBlobStorage.cpp +++ b/src/Disks/BlobStorage/DiskBlobStorage.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -93,7 +94,7 @@ std::unique_ptr DiskBlobStorage::writeFile( WriteMode mode) { auto metadata = readOrCreateMetaForWriting(path, mode); - auto blob_path = path + "_" + getRandomName(8); /// NOTE: path contains the tmp_* prefix in the blob name + auto blob_path = path + "_" + getRandomASCIIString(8); /// NOTE: path contains the tmp_* prefix in the blob name LOG_TRACE(log, "{} to file by path: {}. Blob Storage path: {}", mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), blob_path); diff --git a/src/Disks/RemoteDisksCommon.cpp b/src/Disks/RemoteDisksCommon.cpp index 1cdf4b5d4ab..1402e3f62c8 100644 --- a/src/Disks/RemoteDisksCommon.cpp +++ b/src/Disks/RemoteDisksCommon.cpp @@ -1,4 +1,5 @@ #include +#include namespace DB { @@ -8,17 +9,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } - -String getRandomName(size_t len, char first, char last) -{ - std::uniform_int_distribution distribution(first, last); - String res(len, ' '); - for (auto & c : res) - c = distribution(thread_local_rng); - return res; -} - - std::shared_ptr wrapWithCache( std::shared_ptr disk, String cache_name, String cache_path, String metadata_path) { diff --git a/src/Disks/RemoteDisksCommon.h b/src/Disks/RemoteDisksCommon.h index 2fd17d191f5..0d057b44d18 100644 --- a/src/Disks/RemoteDisksCommon.h +++ b/src/Disks/RemoteDisksCommon.h @@ -6,13 +6,12 @@ #include #include #include +#include namespace DB { -String getRandomName(size_t len = 32, char first = 'a', char last = 'z'); - std::shared_ptr wrapWithCache( std::shared_ptr disk, String cache_name, String cache_path, String metadata_path); diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 6b70b85b623..0e2f75505fa 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -246,7 +247,7 @@ std::unique_ptr DiskS3::writeFile(const String & path, auto metadata = readOrCreateMetaForWriting(path, mode); /// Path to store new S3 object. - auto s3_path = getRandomName(); + auto s3_path = getRandomASCIIString(); std::optional object_metadata; if (settings->send_metadata) diff --git a/src/IO/ReadBufferFromBlobStorage.cpp b/src/IO/ReadBufferFromBlobStorage.cpp index 67b33e65239..ada462f0b87 100644 --- a/src/IO/ReadBufferFromBlobStorage.cpp +++ b/src/IO/ReadBufferFromBlobStorage.cpp @@ -68,7 +68,7 @@ bool ReadBufferFromBlobStorage::nextImpl() data_capacity = internal_buffer.size(); } - size_t to_read_bytes = std::min(total_size - offset, data_capacity); + size_t to_read_bytes = std::min(static_cast(total_size - offset), data_capacity); size_t bytes_read = 0; size_t sleep_time_with_backoff_milliseconds = 100; diff --git a/src/IO/WriteBufferFromBlobStorage.cpp b/src/IO/WriteBufferFromBlobStorage.cpp index 119a72a6e01..2e63c8c4565 100644 --- a/src/IO/WriteBufferFromBlobStorage.cpp +++ b/src/IO/WriteBufferFromBlobStorage.cpp @@ -6,6 +6,7 @@ #include #include +#include namespace DB @@ -42,7 +43,7 @@ void WriteBufferFromBlobStorage::nextImpl() { auto part_len = std::min(len - read, max_single_part_upload_size); - auto block_id = getRandomName(64); + auto block_id = getRandomASCIIString(64); block_ids.push_back(block_id); Azure::Core::IO::MemoryBodyStream tmp_buffer(reinterpret_cast(buffer_begin + read), part_len); diff --git a/src/Parsers/ASTTableOverrides.h b/src/Parsers/ASTTableOverrides.h index d2cab9f27c4..7a84ed25121 100644 --- a/src/Parsers/ASTTableOverrides.h +++ b/src/Parsers/ASTTableOverrides.h @@ -40,7 +40,7 @@ public: String getID(char) const override { return "TableOverrideList"; } ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - void setTableOverride(const String & name, ASTPtr override); + void setTableOverride(const String & name, const ASTPtr ast); void removeTableOverride(const String & name); ASTPtr tryGetTableOverride(const String & name) const; bool hasOverride(const String & name) const; From d394f0e753ef679d9fc2ad8f4d331e5f4a4bf801 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 14 Dec 2021 17:19:18 +0300 Subject: [PATCH 158/358] Allpy some more optimizations to NO_QUERY ast. --- src/Interpreters/TreeRewriter.cpp | 2 +- .../integration/test_storage_rabbitmq/test.py | 22 +++++++++++++++++-- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index d864bb54b2e..639d38d36f1 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1117,7 +1117,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.rewrite_subqueries = PredicateExpressionsOptimizer(getContext(), tables_with_columns, settings).optimize(*select_query); /// Only apply AST optimization for initial queries. - if (getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) + if (getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) TreeOptimizer::apply(query, result, tables_with_columns, getContext()); /// array_join_alias_to_name, array_join_result_to_source. diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 66ec97ac027..5342473aefa 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -284,6 +284,12 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): ORDER BY key; CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.rabbitmq; + + CREATE TABLE test.view2 (key UInt64, value UInt64) + ENGINE = MergeTree() + ORDER BY key; + CREATE MATERIALIZED VIEW test.consumer2 TO test.view2 AS + SELECT * FROM test.rabbitmq group by (key, value); ''') credentials = pika.PlainCredentials('root', 'clickhouse') @@ -297,14 +303,26 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): for message in messages: channel.basic_publish(exchange='mv', routing_key='', body=message) - while True: + time_limit_sec = 60 + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: result = instance.query('SELECT * FROM test.view ORDER BY key') if (rabbitmq_check_result(result)): break - connection.close() rabbitmq_check_result(result, True) + deadline = time.monotonic() + time_limit_sec + + while time.monotonic() < deadline: + result = instance.query('SELECT * FROM test.view2 ORDER BY key') + if (rabbitmq_check_result(result)): + break + + rabbitmq_check_result(result, True) + connection.close() + def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): instance.query(''' From 84d6715e3d9d450d170b09b01544f2be2cc59f0e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 14 Dec 2021 18:28:55 +0300 Subject: [PATCH 159/358] fix race in skipping index of type hypothesis --- ...ergeTreeIndexHypothesisMergedCondition.cpp | 25 +++++++++++++------ .../MergeTreeIndexHypothesisMergedCondition.h | 5 +++- ...02150_index_hypothesis_race_long.reference | 1 + .../02150_index_hypothesis_race_long.sh | 23 +++++++++++++++++ 4 files changed, 45 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02150_index_hypothesis_race_long.reference create mode 100755 tests/queries/0_stateless/02150_index_hypothesis_race_long.sh diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp index 6900ae1e69b..84195eb71b2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp @@ -146,10 +146,15 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree values.push_back(granule->met); } - if (const auto it = answer_cache.find(values); it != std::end(answer_cache)) - return it->second; + const ComparisonGraph * graph = nullptr; - const auto & graph = getGraph(values); + { + std::lock_guard lock(cache_mutex); + if (const auto it = answer_cache.find(values); it != std::end(answer_cache)) + return it->second; + + graph = getGraph(values); + } bool always_false = false; expression_cnf->iterateGroups( @@ -166,7 +171,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree if (func && func->arguments->children.size() == 2) { const auto expected = ComparisonGraph::atomToCompareResult(atom); - if (graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1])) + if (graph->isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1])) { /// If graph failed use matching. /// We don't need to check constraints. @@ -177,6 +182,8 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree always_false = true; }); + std::lock_guard lock(cache_mutex); + answer_cache[values] = !always_false; return !always_false; } @@ -195,11 +202,13 @@ std::unique_ptr MergeTreeIndexhypothesisMergedCondition::buildG return std::make_unique(active_atomic_formulas); } -const ComparisonGraph & MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector & values) const +const ComparisonGraph * MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector & values) const { - if (!graph_cache.contains(values)) - graph_cache[values] = buildGraph(values); - return *graph_cache.at(values); + auto [it, inserted] = graph_cache.try_emplace(values); + if (inserted) + it->second = buildGraph(values); + + return it->second.get(); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h index 530e14e15cc..9ebcbe9d7dc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h @@ -21,11 +21,14 @@ public: private: void addConstraints(const ConstraintsDescription & constraints_description); std::unique_ptr buildGraph(const std::vector & values) const; - const ComparisonGraph & getGraph(const std::vector & values) const; + const ComparisonGraph * getGraph(const std::vector & values) const; ASTPtr expression_ast; std::unique_ptr expression_cnf; + /// Part analysis can be done in parallel. + /// So, we have shared answer and graph cache. + mutable std::mutex cache_mutex; mutable std::unordered_map, std::unique_ptr> graph_cache; mutable std::unordered_map, bool> answer_cache; diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.reference b/tests/queries/0_stateless/02150_index_hypothesis_race_long.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh new file mode 100755 index 00000000000..da2dcd055ea --- /dev/null +++ b/tests/queries/0_stateless/02150_index_hypothesis_race_long.sh @@ -0,0 +1,23 @@ +#!/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 t_index_hypothesis" + +$CLICKHOUSE_CLIENT -q "CREATE TABLE t_index_hypothesis (a UInt32, b UInt32, INDEX t a != b TYPE hypothesis GRANULARITY 1) ENGINE = MergeTree ORDER BY a" + +$CLICKHOUSE_CLIENT -q "INSERT INTO t_index_hypothesis SELECT number, number + 1 FROM numbers(10000000)" + +for _ in {0..30}; do + output=`$CLICKHOUSE_CLIENT -q "SELECT count() FROM t_index_hypothesis WHERE a = b"` + if [[ $output != "0" ]]; then + echo "output: $output, expected: 0" + exit 1 + fi +done + +echo OK + +$CLICKHOUSE_CLIENT -q "DROP TABLE t_index_hypothesis" From c4bc72672ef3b18d57209d351cc6e4bc503cafe3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 14 Dec 2021 18:46:21 +0300 Subject: [PATCH 160/358] SQLUserDefinedFunctions support subqueries --- .../InterpreterCreateFunctionQuery.cpp | 12 ++++--- ...l_user_defined_function_subquery.reference | 6 ++++ ...148_sql_user_defined_function_subquery.sql | 35 +++++++++++++++++++ 3 files changed, 49 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02148_sql_user_defined_function_subquery.reference create mode 100644 tests/queries/0_stateless/02148_sql_user_defined_function_subquery.sql diff --git a/src/Interpreters/InterpreterCreateFunctionQuery.cpp b/src/Interpreters/InterpreterCreateFunctionQuery.cpp index 1c43da07628..2f345f8b237 100644 --- a/src/Interpreters/InterpreterCreateFunctionQuery.cpp +++ b/src/Interpreters/InterpreterCreateFunctionQuery.cpp @@ -57,9 +57,14 @@ BlockIO InterpreterCreateFunctionQuery::execute() void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function, const String & name) { - const auto * args_tuple = function->as()->arguments->children.at(0)->as(); + auto & lambda_function = function->as(); + auto & lambda_function_expression_list = lambda_function.arguments->children; + + const auto & tuple_function_arguments = lambda_function_expression_list.at(0)->as(); + std::unordered_set arguments; - for (const auto & argument : args_tuple->arguments->children) + + for (const auto & argument : tuple_function_arguments.arguments->children) { const auto & argument_name = argument->as()->name(); auto [_, inserted] = arguments.insert(argument_name); @@ -67,7 +72,7 @@ void InterpreterCreateFunctionQuery::validateFunction(ASTPtr function, const Str throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Identifier {} already used as function parameter", argument_name); } - ASTPtr function_body = function->as()->children.at(0)->children.at(1); + ASTPtr function_body = lambda_function_expression_list.at(1); validateFunctionRecursiveness(function_body, name); } @@ -82,5 +87,4 @@ void InterpreterCreateFunctionQuery::validateFunctionRecursiveness(ASTPtr node, validateFunctionRecursiveness(child, function_to_create); } } - } diff --git a/tests/queries/0_stateless/02148_sql_user_defined_function_subquery.reference b/tests/queries/0_stateless/02148_sql_user_defined_function_subquery.reference new file mode 100644 index 00000000000..8851ce8322c --- /dev/null +++ b/tests/queries/0_stateless/02148_sql_user_defined_function_subquery.reference @@ -0,0 +1,6 @@ +1 +2 +2 +4 +(0,'Value') +Value diff --git a/tests/queries/0_stateless/02148_sql_user_defined_function_subquery.sql b/tests/queries/0_stateless/02148_sql_user_defined_function_subquery.sql new file mode 100644 index 00000000000..cc62d1ac495 --- /dev/null +++ b/tests/queries/0_stateless/02148_sql_user_defined_function_subquery.sql @@ -0,0 +1,35 @@ +-- Tags: no-parallel + +DROP FUNCTION IF EXISTS 02148_test_function; +CREATE FUNCTION 02148_test_function AS () -> (SELECT 1); + +SELECT 02148_test_function(); + +CREATE OR REPLACE FUNCTION 02148_test_function AS () -> (SELECT 2); + +SELECT 02148_test_function(); + +DROP FUNCTION 02148_test_function; + +CREATE FUNCTION 02148_test_function AS (x) -> (SELECT x + 1); +SELECT 02148_test_function(1); + +DROP FUNCTION IF EXISTS 02148_test_function_nested; +CREATE FUNCTION 02148_test_function_nested AS (x) -> 02148_test_function(x + 2); +SELECT 02148_test_function_nested(1); + +DROP FUNCTION 02148_test_function; +DROP FUNCTION 02148_test_function_nested; + +DROP TABLE IF EXISTS 02148_test_table; +CREATE TABLE 02148_test_table (id UInt64, value String) ENGINE=TinyLog; +INSERT INTO 02148_test_table VALUES (0, 'Value'); + +CREATE FUNCTION 02148_test_function AS () -> (SELECT * FROM 02148_test_table LIMIT 1); +SELECT 02148_test_function(); + +CREATE OR REPLACE FUNCTION 02148_test_function AS () -> (SELECT value FROM 02148_test_table LIMIT 1); +SELECT 02148_test_function(); + +DROP FUNCTION 02148_test_function; +DROP TABLE 02148_test_table; From 884801e1bd54c784700fa805690129e94b5eecff Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Dec 2021 19:08:08 +0300 Subject: [PATCH 161/358] Fixing --- src/Formats/ProtobufSerializer.cpp | 6 ++++-- .../MySQL/InterpretersMySQLDDLQuery.cpp | 6 +++--- src/Interpreters/executeQuery.cpp | 2 +- .../Impl/CustomSeparatedRowInputFormat.cpp | 19 ++++++++++++++++--- .../Impl/CustomSeparatedRowInputFormat.h | 2 +- 5 files changed, 25 insertions(+), 10 deletions(-) diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 3414edecf5b..5232b76b7fe 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -2307,10 +2307,9 @@ namespace if (parent_field_descriptor) out << " field " << quoteString(parent_field_descriptor->full_name()) << " (" << parent_field_descriptor->type_name() << ")"; - for (size_t i = 0; i != field_infos.size(); ++i) + for (const auto & field_info : field_infos) { out << "\n"; - const auto & field_info = field_infos[i]; writeIndent(out, indent + 1) << "Columns #"; for (size_t j = 0; j != field_info.column_indices.size(); ++j) { @@ -3017,8 +3016,11 @@ namespace if (nested_message_serializer) { std::vector column_names_used; + column_names_used.reserve(used_column_indices_in_nested.size()); + for (size_t i : used_column_indices_in_nested) column_names_used.emplace_back(nested_column_names[i]); + auto field_serializer = std::make_unique( std::move(column_names_used), field_descriptor, std::move(nested_message_serializer), get_root_desc_function); transformColumnIndices(used_column_indices_in_nested, nested_column_indices); diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index e599cd97e13..8851a5eb6ec 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -440,7 +440,7 @@ static ASTPtr tryGetTableOverride(const String & mapped_database, const String & if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database)) { auto create_query = database_ptr->getCreateDatabaseQuery(); - if (auto create_database_query = create_query->as()) + if (auto * create_database_query = create_query->as()) { if (create_database_query->table_overrides) { @@ -537,8 +537,8 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( if (auto table_override = tryGetTableOverride(mapped_to_database, create_query.table)) { - auto override = table_override->as(); - override->applyToCreateTableQuery(rewritten_query.get()); + auto * override_ast = table_override->as(); + override_ast->applyToCreateTableQuery(rewritten_query.get()); } return ASTs{rewritten_query}; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index b69bbcc6332..e177fd8e6b3 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -635,7 +635,7 @@ static std::tuple executeQueryImpl( std::unique_ptr span; if (context->query_trace_context.trace_id != UUID()) { - auto raw_interpreter_ptr = interpreter.get(); + auto * raw_interpreter_ptr = interpreter.get(); std::string class_name(abi::__cxa_demangle(typeid(*raw_interpreter_ptr).name(), nullptr, nullptr, nullptr)); span = std::make_unique(class_name + "::execute()"); } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 8cd9d154ae4..34b3bb2bd07 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -24,14 +24,27 @@ static FormatSettings updateFormatSettings(const FormatSettings & settings) CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( const Block & header_, - ReadBuffer & in_, + ReadBuffer & in_buf_, const Params & params_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(header_, buf, params_, with_names_, with_types_, updateFormatSettings(format_settings_)) - , buf(in_) + : CustomSeparatedRowInputFormat( + header_, std::make_unique(in_buf_), params_, with_names_, with_types_, ignore_spaces_, format_settings_) +{ +} + +CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( + const Block & header_, + std::unique_ptr buf_, + const Params & params_, + bool with_names_, + bool with_types_, + bool ignore_spaces_, + const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes(header_, *buf_, params_, with_names_, with_types_, updateFormatSettings(format_settings_)) + , buf(std::move(buf_)) , ignore_spaces(ignore_spaces_) , escaping_rule(format_settings_.custom.escaping_rule) { diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index 00ee28e50cc..8850b538650 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -23,7 +23,7 @@ public: private: CustomSeparatedRowInputFormat( const Block & header_, - std::unique_ptr in_, + std::unique_ptr in_buf_, const Params & params_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_); using EscapingRule = FormatSettings::EscapingRule; From c46c91250714d02422141a3361a7483c3b5ba3d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Dec 2021 19:16:22 +0300 Subject: [PATCH 162/358] Followup --- tests/ci/build_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 99c1ab56b04..9540a30f82d 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -165,7 +165,7 @@ if __name__ == "__main__": log_url = 'https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20') else: build_urls.append('https://s3.amazonaws.com/clickhouse-builds/' + url.replace('+', '%2B').replace(' ', '%20')) - create_json_artifact(temp_path, build_name, log_url, build_urls, build_config, 0, True) + create_json_artifact(temp_path, build_name, log_url, build_urls, build_config, 0, len(build_urls) > 0) sys.exit(0) image_name = get_image_name(build_config) From e7a7378cfc2d185db6cd71009a25af5c00dcd77b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 14 Dec 2021 19:26:09 +0300 Subject: [PATCH 163/358] Disable iconv for libxml --- contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h index 92d3414fdac..52f62214324 100644 --- a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h +++ b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h @@ -268,7 +268,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether iconv support is available */ -#if 1 +#if 0 #define LIBXML_ICONV_ENABLED #endif From 47f549166d5c1b7932067d145cf80a2b469c3901 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 14 Dec 2021 16:53:47 +0300 Subject: [PATCH 164/358] Table Override for PostgreSQL --- src/Core/PostgreSQL/insertPostgreSQLValue.cpp | 2 +- src/Databases/DatabaseFactory.cpp | 2 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 18 +-- src/Parsers/ASTTableOverrides.cpp | 18 +++ src/Parsers/ASTTableOverrides.h | 2 + .../PostgreSQLReplicationHandler.cpp | 8 +- .../StorageMaterializedPostgreSQL.cpp | 148 ++++++++++++------ .../StorageMaterializedPostgreSQL.h | 8 +- .../test.py | 31 +++- 9 files changed, 165 insertions(+), 72 deletions(-) diff --git a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp index 1c3230ec826..f4d47049554 100644 --- a/src/Core/PostgreSQL/insertPostgreSQLValue.cpp +++ b/src/Core/PostgreSQL/insertPostgreSQLValue.cpp @@ -85,7 +85,7 @@ void insertPostgreSQLValue( assert_cast(column).insertData(value.data(), value.size()); break; case ExternalResultDescription::ValueType::vtUUID: - assert_cast(column).insert(parse(value.data(), value.size())); + assert_cast(column).insertValue(parse(value.data(), value.size())); break; case ExternalResultDescription::ValueType::vtDate: assert_cast(column).insertValue(UInt16{LocalDate{std::string(value)}.getDayNum()}); diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index a4c8f3cdb77..5ea44e0f94c 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -117,7 +117,7 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String static const std::unordered_set engines_with_arguments{"MySQL", "MaterializeMySQL", "MaterializedMySQL", "Lazy", "Replicated", "PostgreSQL", "MaterializedPostgreSQL", "SQLite"}; - static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL"}; + static const std::unordered_set engines_with_table_overrides{"MaterializeMySQL", "MaterializedMySQL", "MaterializedPostgreSQL"}; bool engine_may_have_arguments = engines_with_arguments.contains(engine_name); if (engine_define->engine->arguments && !engine_may_have_arguments) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index e599cd97e13..a283bd001e2 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -435,22 +435,6 @@ void InterpreterCreateImpl::validate(const InterpreterCreateImpl::TQuery & creat } } -static ASTPtr tryGetTableOverride(const String & mapped_database, const String & table) -{ - if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database)) - { - auto create_query = database_ptr->getCreateDatabaseQuery(); - if (auto create_database_query = create_query->as()) - { - if (create_database_query->table_overrides) - { - return create_database_query->table_overrides->tryGetTableOverride(table); - } - } - } - return nullptr; -} - ASTs InterpreterCreateImpl::getRewrittenQueries( const TQuery & create_query, ContextPtr context, const String & mapped_to_database, const String & mysql_database) { @@ -535,7 +519,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( rewritten_query->set(rewritten_query->storage, storage); rewritten_query->set(rewritten_query->columns_list, columns); - if (auto table_override = tryGetTableOverride(mapped_to_database, create_query.table)) + if (auto table_override = ASTTableOverride::tryGetTableOverride(mapped_to_database, create_query.table)) { auto override = table_override->as(); override->applyToCreateTableQuery(rewritten_query.get()); diff --git a/src/Parsers/ASTTableOverrides.cpp b/src/Parsers/ASTTableOverrides.cpp index 0270c2cad82..ee351ab3eb3 100644 --- a/src/Parsers/ASTTableOverrides.cpp +++ b/src/Parsers/ASTTableOverrides.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include namespace DB { @@ -71,6 +73,22 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState settings.ostr << nl_or_nothing << ')'; } +ASTPtr ASTTableOverride::tryGetTableOverride(const String & mapped_database, const String & table) +{ + if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database)) + { + auto create_query = database_ptr->getCreateDatabaseQuery(); + if (auto create_database_query = create_query->as()) + { + if (create_database_query->table_overrides) + { + return create_database_query->table_overrides->tryGetTableOverride(table); + } + } + } + return nullptr; +} + void ASTTableOverride::applyToCreateTableQuery(ASTCreateQuery * create_query) const { if (columns) diff --git a/src/Parsers/ASTTableOverrides.h b/src/Parsers/ASTTableOverrides.h index d2cab9f27c4..edbc35553b3 100644 --- a/src/Parsers/ASTTableOverrides.h +++ b/src/Parsers/ASTTableOverrides.h @@ -27,6 +27,8 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void applyToCreateTableQuery(ASTCreateQuery * create_query) const; + + static ASTPtr tryGetTableOverride(const String & mapped_database, const String & table); }; /// List of table overrides, for example: diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 7cc71a63443..d706615b762 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,5 +1,6 @@ #include "PostgreSQLReplicationHandler.h" +#include #include #include #include @@ -279,7 +280,9 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ auto table_structure = std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, postgres_schema, true, true, true)); if (!table_structure) throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get PostgreSQL table structure"); - return storage->getCreateNestedTableQuery(std::move(table_structure)); + + auto table_override = ASTTableOverride::tryGetTableOverride(current_database_name, table_name); + return storage->getCreateNestedTableQuery(std::move(table_structure), table_override->as()); } @@ -297,7 +300,8 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & query_str = fmt::format("SELECT * FROM {}", quoted_name); LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); - materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); + auto table_override = ASTTableOverride::tryGetTableOverride(current_database_name, table_name); + materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name), table_override->as()); auto nested_storage = materialized_storage->getNested(); auto insert = std::make_shared(); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 591e10a88b9..a680792acad 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -181,18 +182,18 @@ StorageID StorageMaterializedPostgreSQL::getNestedStorageID() const } -void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure) +void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override) { if (tryGetNested()) return; - const auto ast_create = getCreateNestedTableQuery(std::move(table_structure)); - auto table_id = getStorageID(); - auto tmp_nested_table_id = StorageID(table_id.database_name, getNestedTableName()); - LOG_DEBUG(log, "Creating clickhouse table for postgresql table {}", table_id.getNameForLogs()); - try { + const auto ast_create = getCreateNestedTableQuery(std::move(table_structure), table_override); + auto table_id = getStorageID(); + auto tmp_nested_table_id = StorageID(table_id.database_name, getNestedTableName()); + LOG_DEBUG(log, "Creating clickhouse table for postgresql table {}", table_id.getNameForLogs()); + InterpreterCreateQuery interpreter(ast_create, nested_context); interpreter.execute(); @@ -200,10 +201,10 @@ void StorageMaterializedPostgreSQL::createNestedIfNeeded(PostgreSQLTableStructur /// Save storage_id with correct uuid. nested_table_id = nested_storage->getStorageID(); } - catch (Exception & e) + catch (...) { - e.addMessage("while creating nested table: {}", tmp_nested_table_id.getNameForLogs()); tryLogCurrentException(__PRETTY_FUNCTION__); + throw; } } @@ -362,12 +363,31 @@ ASTPtr StorageMaterializedPostgreSQL::getColumnDeclaration(const DataTypePtr & d } +std::shared_ptr StorageMaterializedPostgreSQL::getColumnsExpressionList(const NamesAndTypesList & columns) const +{ + auto columns_expression_list = std::make_shared(); + for (const auto & [name, type] : columns) + { + const auto & column_declaration = std::make_shared(); + + column_declaration->name = name; + column_declaration->type = getColumnDeclaration(type); + + columns_expression_list->children.emplace_back(column_declaration); + } + return columns_expression_list; +} + + /// For single storage MaterializedPostgreSQL get columns and primary key columns from storage definition. /// For database engine MaterializedPostgreSQL get columns and primary key columns by fetching from PostgreSQL, also using the same /// transaction with snapshot, which is used for initial tables dump. -ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure) +ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery( + PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override) { auto create_table_query = std::make_shared(); + if (table_override) + table_override->applyToCreateTableQuery(create_table_query.get()); auto table_id = getStorageID(); create_table_query->setTable(getNestedTableName()); @@ -375,40 +395,86 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(PostgreSQLTableS if (is_materialized_postgresql_database) create_table_query->uuid = table_id.uuid; + auto storage = std::make_shared(); + storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared("_version"))); + auto columns_declare_list = std::make_shared(); - auto columns_expression_list = std::make_shared(); auto order_by_expression = std::make_shared(); auto metadata_snapshot = getInMemoryMetadataPtr(); - const auto & columns = metadata_snapshot->getColumns(); + + ConstraintsDescription constraints; NamesAndTypesList ordinary_columns_and_types; - if (!is_materialized_postgresql_database) + if (is_materialized_postgresql_database) { - ordinary_columns_and_types = columns.getOrdinary(); - } - else - { - if (!table_structure) + if (!table_structure && !table_override) { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No table structure returned for table {}.{}", table_id.database_name, table_id.table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No table structure returned for table {}.{}", + table_id.database_name, table_id.table_name); } - if (!table_structure->columns) + if (!table_structure->columns && (!table_override || !table_override->columns)) { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "No columns returned for table {}.{}", table_id.database_name, table_id.table_name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "No columns returned for table {}.{}", + table_id.database_name, table_id.table_name); } - ordinary_columns_and_types = *table_structure->columns; + bool has_order_by_override = table_override && table_override->storage && table_override->storage->order_by; + if (has_order_by_override && !table_structure->replica_identity_columns) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Having PRIMARY KEY OVERRIDE is allowed only if there is " + "replica identity index for PostgreSQL table. (table {}.{})", + table_id.database_name, table_id.table_name); + } if (!table_structure->primary_key_columns && !table_structure->replica_identity_columns) { throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Table {}.{} has no primary key and no replica identity index", table_id.database_name, table_id.table_name); + "Table {}.{} has no primary key and no replica identity index", + table_id.database_name, table_id.table_name); } + if (table_override && table_override->columns) + { + table_override->applyToCreateTableQuery(create_table_query.get()); + if (table_override->columns) + { + auto children = table_override->columns->children; + const auto & columns = children[0]->as(); + if (columns) + { + for (const auto & child : columns->children) + { + const auto * column_declaration = child->as(); + auto type = DataTypeFactory::instance().get(column_declaration->type); + ordinary_columns_and_types.emplace_back(NameAndTypePair(column_declaration->name, type)); + } + } + + columns_declare_list->set(columns_declare_list->columns, children[0]); + } + else + { + ordinary_columns_and_types = *table_structure->columns; + columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types)); + } + + auto columns = table_override->columns; + if (columns && columns->constraints) + constraints = ConstraintsDescription(columns->constraints->children); + } + else + { + ordinary_columns_and_types = *table_structure->columns; + columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types)); + } + + if (ordinary_columns_and_types.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Table {}.{} has no columns", table_id.database_name, table_id.table_name); + NamesAndTypesList merging_columns; if (table_structure->primary_key_columns) merging_columns = *table_structure->primary_key_columns; @@ -417,39 +483,28 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(PostgreSQLTableS order_by_expression->name = "tuple"; order_by_expression->arguments = std::make_shared(); - for (const auto & column : merging_columns) order_by_expression->arguments->children.emplace_back(std::make_shared(column.name)); - } - for (const auto & [name, type] : ordinary_columns_and_types) + storage->set(storage->order_by, order_by_expression); + } + else { - const auto & column_declaration = std::make_shared(); + ordinary_columns_and_types = metadata_snapshot->getColumns().getOrdinary(); + columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types)); - column_declaration->name = name; - column_declaration->type = getColumnDeclaration(type); + auto primary_key_ast = metadata_snapshot->getPrimaryKeyAST(); + if (!primary_key_ast) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Storage MaterializedPostgreSQL must have primary key"); + storage->set(storage->order_by, primary_key_ast); - columns_expression_list->children.emplace_back(column_declaration); + constraints = metadata_snapshot->getConstraints(); } - columns_declare_list->set(columns_declare_list->columns, columns_expression_list); - columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_sign", "Int8", 1)); columns_declare_list->columns->children.emplace_back(getMaterializedColumnsDeclaration("_version", "UInt64", 1)); - create_table_query->set(create_table_query->columns_list, columns_declare_list); - /// Not nullptr for single storage (because throws exception if not specified), nullptr otherwise. - auto primary_key_ast = getInMemoryMetadataPtr()->getPrimaryKeyAST(); - - auto storage = std::make_shared(); - storage->set(storage->engine, makeASTFunction("ReplacingMergeTree", std::make_shared("_version"))); - - if (primary_key_ast) - storage->set(storage->order_by, primary_key_ast); - else - storage->set(storage->order_by, order_by_expression); - create_table_query->set(create_table_query->storage, storage); /// Add columns _sign and _version, so that they can be accessed from nested ReplacingMergeTree table if needed. @@ -458,8 +513,7 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery(PostgreSQLTableS StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription(ordinary_columns_and_types)); - storage_metadata.setConstraints(metadata_snapshot->getConstraints()); - + storage_metadata.setConstraints(constraints); setInMemoryMetadata(storage_metadata); return create_table_query; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index 10724fb9bf0..9e11f314738 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -99,7 +99,11 @@ public: /// only once - when nested table is successfully created and is never changed afterwards. bool hasNested() { return has_nested.load(); } - void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure); + void createNestedIfNeeded(PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override); + + ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure, const ASTTableOverride * table_override); + + std::shared_ptr getColumnsExpressionList(const NamesAndTypesList & columns) const; StoragePtr getNested() const; @@ -120,8 +124,6 @@ public: bool supportsFinal() const override { return true; } - ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure); - protected: StorageMaterializedPostgreSQL( const StorageID & table_id_, diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 99f2facbaf6..3bf494aa957 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -34,6 +34,10 @@ postgres_table_template_4 = """ CREATE TABLE IF NOT EXISTS "{}"."{}" ( key Integer NOT NULL, value Integer, PRIMARY KEY(key)) """ +postgres_table_template_5 = """ + CREATE TABLE IF NOT EXISTS "{}" ( + key Integer NOT NULL, value UUID, PRIMARY KEY(key)) + """ def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name='postgres_database', replication=False): if database == True: @@ -93,7 +97,7 @@ def drop_clickhouse_postgres_db(name='postgres_database'): def create_materialized_db(ip, port, materialized_database='test_database', postgres_database='postgres_database', - settings=[]): + settings=[], table_overrides=''): instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") create_query = f"CREATE DATABASE {materialized_database} ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', 'postgres', 'mysecretpassword')" if len(settings) > 0: @@ -102,6 +106,7 @@ def create_materialized_db(ip, port, if i != 0: create_query += ', ' create_query += settings[i] + create_query += table_overrides instance.query(create_query) assert materialized_database in instance.query('SHOW DATABASES') @@ -560,6 +565,30 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): drop_materialized_db() +def test_table_override(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) + cursor = conn.cursor() + table_name = 'table_override' + materialized_database = 'test_database' + create_postgres_table(cursor, table_name, template=postgres_table_template_5); + instance.query(f"create table {table_name}(key Int32, value UUID) engine = PostgreSQL (postgres1, table={table_name})") + instance.query(f"insert into {table_name} select number, generateUUIDv4() from numbers(10)") + table_overrides = f" TABLE OVERRIDE {table_name} (COLUMNS (key Int32, value UUID))" + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, + settings=[f"materialized_postgresql_tables_list = '{table_name}'"], table_overrides=table_overrides) + assert_nested_table_is_created(table_name, materialized_database) + result = instance.query(f"show create table {materialized_database}.{table_name}") + print(result) + expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` UUID,\\n `_sign` Int8() MATERIALIZED 1,\\n `_version` UInt64() MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nORDER BY tuple(key)" + assert(result.strip() == expected) + time.sleep(5) + result = instance.query(f"select * from {materialized_database}.{table_name} order by key") + expected = instance.query(f"select * from {table_name} order by key") + assert(result == expected) + drop_materialized_db() + drop_postgres_table(cursor, table_name) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 97301a19b9e0a49ef4a84e8479bf03cfb6ec8d24 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 8 Dec 2021 18:51:53 +0100 Subject: [PATCH 165/358] Unify init scripts in one --- tests/ci/worker/init_builder.sh | 34 ---------------- tests/ci/worker/init_func_tester.sh | 34 ---------------- tests/ci/worker/init_fuzzer_unit_tester.sh | 34 ---------------- tests/ci/worker/init_runner.sh | 47 ++++++++++++++++++++++ tests/ci/worker/init_stress_tester.sh | 34 ---------------- tests/ci/worker/init_style_checker.sh | 20 --------- 6 files changed, 47 insertions(+), 156 deletions(-) delete mode 100644 tests/ci/worker/init_builder.sh delete mode 100644 tests/ci/worker/init_func_tester.sh delete mode 100644 tests/ci/worker/init_fuzzer_unit_tester.sh create mode 100644 tests/ci/worker/init_runner.sh delete mode 100644 tests/ci/worker/init_stress_tester.sh delete mode 100644 tests/ci/worker/init_style_checker.sh diff --git a/tests/ci/worker/init_builder.sh b/tests/ci/worker/init_builder.sh deleted file mode 100644 index 8fd00c1db0a..00000000000 --- a/tests/ci/worker/init_builder.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash -set -uo pipefail - -echo "Running init script" -export DEBIAN_FRONTEND=noninteractive -export RUNNER_HOME=/home/ubuntu/actions-runner - -export RUNNER_URL="https://github.com/ClickHouse" -# Funny fact, but metadata service has fixed IP -export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` - -while true; do - runner_pid=`pgrep run.sh` - echo "Got runner pid $runner_pid" - - cd $RUNNER_HOME - if [ -z "$runner_pid" ]; then - echo "Receiving token" - RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` - - echo "Will try to remove runner" - sudo -u ubuntu ./config.sh remove --token $RUNNER_TOKEN ||: - - echo "Going to configure runner" - sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,builder' --work _work - - echo "Run" - sudo -u ubuntu ./run.sh & - sleep 15 - else - echo "Runner is working with pid $runner_pid, nothing to do" - sleep 10 - fi -done diff --git a/tests/ci/worker/init_func_tester.sh b/tests/ci/worker/init_func_tester.sh deleted file mode 100644 index d3ee3cb3d7f..00000000000 --- a/tests/ci/worker/init_func_tester.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash -set -uo pipefail - -echo "Running init script" -export DEBIAN_FRONTEND=noninteractive -export RUNNER_HOME=/home/ubuntu/actions-runner - -export RUNNER_URL="https://github.com/ClickHouse" -# Funny fact, but metadata service has fixed IP -export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` - -while true; do - runner_pid=`pgrep run.sh` - echo "Got runner pid $runner_pid" - - cd $RUNNER_HOME - if [ -z "$runner_pid" ]; then - echo "Receiving token" - RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` - - echo "Will try to remove runner" - sudo -u ubuntu ./config.sh remove --token $RUNNER_TOKEN ||: - - echo "Going to configure runner" - sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,func-tester' --work _work - - echo "Run" - sudo -u ubuntu ./run.sh & - sleep 15 - else - echo "Runner is working with pid $runner_pid, nothing to do" - sleep 10 - fi -done diff --git a/tests/ci/worker/init_fuzzer_unit_tester.sh b/tests/ci/worker/init_fuzzer_unit_tester.sh deleted file mode 100644 index 2fbedba9e40..00000000000 --- a/tests/ci/worker/init_fuzzer_unit_tester.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash -set -uo pipefail - -echo "Running init script" -export DEBIAN_FRONTEND=noninteractive -export RUNNER_HOME=/home/ubuntu/actions-runner - -export RUNNER_URL="https://github.com/ClickHouse" -# Funny fact, but metadata service has fixed IP -export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` - -while true; do - runner_pid=`pgrep run.sh` - echo "Got runner pid $runner_pid" - - cd $RUNNER_HOME - if [ -z "$runner_pid" ]; then - echo "Receiving token" - RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` - - echo "Will try to remove runner" - sudo -u ubuntu ./config.sh remove --token $RUNNER_TOKEN ||: - - echo "Going to configure runner" - sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,fuzzer-unit-tester' --work _work - - echo "Run" - sudo -u ubuntu ./run.sh & - sleep 15 - else - echo "Runner is working with pid $runner_pid, nothing to do" - sleep 10 - fi -done diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh new file mode 100644 index 00000000000..6838d925500 --- /dev/null +++ b/tests/ci/worker/init_runner.sh @@ -0,0 +1,47 @@ +#!/usr/bin/env bash +set -uo pipefail + +#################################### +# IMPORTANT! # +# EC2 instance should have # +# `github:runner-type` tag # +# set accordingly to a runner role # +#################################### + +echo "Running init script" +export DEBIAN_FRONTEND=noninteractive +export RUNNER_HOME=/home/ubuntu/actions-runner + +export RUNNER_URL="https://github.com/ClickHouse" +# Funny fact, but metadata service has fixed IP +INSTANCE_ID=$(curl -s http://169.254.169.254/latest/meta-data/instance-id) +export INSTANCE_ID + +# combine labels +RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" | jq '.Tags[] | select(."Key" == "github:runner-type") | .Value' -r) +LABELS="self-hosted,Linux,$(uname -m),$RUNNER_TYPE" +export LABELS + +while true; do + runner_pid=$(pgrep run.sh) + echo "Got runner pid $runner_pid" + + cd $RUNNER_HOME || exit 1 + if [ -z "$runner_pid" ]; then + echo "Receiving token" + RUNNER_TOKEN=$(/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value) + + echo "Will try to remove runner" + sudo -u ubuntu ./config.sh remove --token "$RUNNER_TOKEN" ||: + + echo "Going to configure runner" + sudo -u ubuntu ./config.sh --url $RUNNER_URL --token "$RUNNER_TOKEN" --name "$INSTANCE_ID" --runnergroup Default --labels "$LABELS" --work _work + + echo "Run" + sudo -u ubuntu ./run.sh & + sleep 15 + else + echo "Runner is working with pid $runner_pid, nothing to do" + sleep 10 + fi +done diff --git a/tests/ci/worker/init_stress_tester.sh b/tests/ci/worker/init_stress_tester.sh deleted file mode 100644 index 234f035e1ea..00000000000 --- a/tests/ci/worker/init_stress_tester.sh +++ /dev/null @@ -1,34 +0,0 @@ -#!/usr/bin/env bash -set -uo pipefail - -echo "Running init script" -export DEBIAN_FRONTEND=noninteractive -export RUNNER_HOME=/home/ubuntu/actions-runner - -export RUNNER_URL="https://github.com/ClickHouse" -# Funny fact, but metadata service has fixed IP -export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` - -while true; do - runner_pid=`pgrep run.sh` - echo "Got runner pid $runner_pid" - - cd $RUNNER_HOME - if [ -z "$runner_pid" ]; then - echo "Receiving token" - RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` - - echo "Will try to remove runner" - sudo -u ubuntu ./config.sh remove --token $RUNNER_TOKEN ||: - - echo "Going to configure runner" - sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,stress-tester' --work _work - - echo "Run" - sudo -u ubuntu ./run.sh & - sleep 15 - else - echo "Runner is working with pid $runner_pid, nothing to do" - sleep 10 - fi -done diff --git a/tests/ci/worker/init_style_checker.sh b/tests/ci/worker/init_style_checker.sh deleted file mode 100644 index 77cf66b5262..00000000000 --- a/tests/ci/worker/init_style_checker.sh +++ /dev/null @@ -1,20 +0,0 @@ -#!/usr/bin/bash -set -euo pipefail - -echo "Running init script" -export DEBIAN_FRONTEND=noninteractive -export RUNNER_HOME=/home/ubuntu/actions-runner - -echo "Receiving token" -export RUNNER_TOKEN=`/usr/local/bin/aws ssm get-parameter --name github_runner_registration_token --with-decryption --output text --query Parameter.Value` -export RUNNER_URL="https://github.com/ClickHouse" -# Funny fact, but metadata service has fixed IP -export INSTANCE_ID=`curl -s http://169.254.169.254/latest/meta-data/instance-id` - -cd $RUNNER_HOME - -echo "Going to configure runner" -sudo -u ubuntu ./config.sh --url $RUNNER_URL --token $RUNNER_TOKEN --name $INSTANCE_ID --runnergroup Default --labels 'self-hosted,Linux,X64,style-checker' --work _work - -echo "Run" -sudo -u ubuntu ./run.sh From 56a3f4a0000c1d7eed3d7545c6da02bebe99b850 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 14 Dec 2021 22:15:14 +0300 Subject: [PATCH 166/358] Cleanup code --- src/Interpreters/Aggregator.h | 47 ++++++++++++------- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 7 --- src/Interpreters/TreeOptimizer.cpp | 3 -- .../Transforms/AggregatingTransform.cpp | 1 - .../Transforms/GroupingSetsTransform.h | 4 +- src/QueryPipeline/Pipe.cpp | 12 +---- 7 files changed, 34 insertions(+), 42 deletions(-) diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index d64a181f3ca..3bfa16ed2aa 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -943,30 +943,41 @@ public: Params( const Block & src_header_, const ColumnNumbers & keys_, - const ColumnNumbersList & keys_vector_, const AggregateDescriptions & aggregates_, - bool overflow_row_, size_t max_rows_to_group_by_, OverflowMode group_by_overflow_mode_, - size_t group_by_two_level_threshold_, size_t group_by_two_level_threshold_bytes_, + const ColumnNumbersList & keys_vector_, + const AggregateDescriptions & aggregates_, + bool overflow_row_, + size_t max_rows_to_group_by_, + OverflowMode group_by_overflow_mode_, + size_t group_by_two_level_threshold_, + size_t group_by_two_level_threshold_bytes_, size_t max_bytes_before_external_group_by_, bool empty_result_for_aggregation_by_empty_set_, - VolumePtr tmp_volume_, size_t max_threads_, + VolumePtr tmp_volume_, + size_t max_threads_, size_t min_free_disk_space_, bool compile_aggregate_expressions_, size_t min_count_to_compile_aggregate_expression_, const Block & intermediate_header_ = {}) - : src_header(src_header_), - intermediate_header(intermediate_header_), - keys(keys_), - keys_vector(keys_vector_), aggregates(aggregates_), keys_size(keys.size()), aggregates_size(aggregates.size()), - overflow_row(overflow_row_), max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_), - group_by_two_level_threshold(group_by_two_level_threshold_), group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_), - max_bytes_before_external_group_by(max_bytes_before_external_group_by_), - empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_), - tmp_volume(tmp_volume_), max_threads(max_threads_), - min_free_disk_space(min_free_disk_space_), - compile_aggregate_expressions(compile_aggregate_expressions_), - min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_) - { - } + : src_header(src_header_) + , intermediate_header(intermediate_header_) + , keys(keys_) + , keys_vector(keys_vector_) + , aggregates(aggregates_) + , keys_size(keys.size()) + , aggregates_size(aggregates.size()) + , overflow_row(overflow_row_) + , max_rows_to_group_by(max_rows_to_group_by_) + , group_by_overflow_mode(group_by_overflow_mode_) + , group_by_two_level_threshold(group_by_two_level_threshold_) + , group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_) + , max_bytes_before_external_group_by(max_bytes_before_external_group_by_) + , empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_) + , tmp_volume(tmp_volume_) + , max_threads(max_threads_) + , min_free_disk_space(min_free_disk_space_) + , compile_aggregate_expressions(compile_aggregate_expressions_) + , min_count_to_compile_aggregate_expression(min_count_to_compile_aggregate_expression_) + {} /// Only parameters that matter during merge. Params(const Block & intermediate_header_, diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 72a85bbfa0f..afe98796d95 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1202,7 +1202,7 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain { for (const auto & ast : asts) { - for(const auto & ast_element : ast->children) + for (const auto & ast_element : ast->children) { step.addRequiredOutput(ast_element->getColumnName()); getRootActions(ast_element, only_types, step.actions()); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 90df6a47495..52783a105a7 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -50,13 +50,6 @@ #include #include #include -#include -#include -#include -#include -#include -#include -#include #include #include #include diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index e221b6da601..f9b018dd90f 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -84,9 +84,6 @@ void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context) { const FunctionFactory & function_factory = FunctionFactory::instance(); -// if (select_query->group_by_with_grouping_sets) -// return; - if (!select_query->groupBy()) return; diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 47f5f80857e..8357a997960 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -501,7 +501,6 @@ void AggregatingTransform::work() Processors AggregatingTransform::expandPipeline() { - LOG_DEBUG(log, "in AggregatingTransform::expandPipeline"); auto & out = processors.back()->getOutputs().front(); inputs.emplace_back(out.getHeader(), this); connect(out, inputs.back()); diff --git a/src/Processors/Transforms/GroupingSetsTransform.h b/src/Processors/Transforms/GroupingSetsTransform.h index f9c688b2cc8..e9b20867ef4 100644 --- a/src/Processors/Transforms/GroupingSetsTransform.h +++ b/src/Processors/Transforms/GroupingSetsTransform.h @@ -17,8 +17,8 @@ protected: private: AggregatingTransformParamsPtr params; - ColumnNumbers keys; - ColumnNumbersList keys_vector; + const ColumnNumbers & keys; + const ColumnNumbersList & keys_vector; Chunks consumed_chunks; Chunk grouping_sets_chunk; diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index dd4a2cb6c77..6aa10b87328 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -463,18 +463,10 @@ void Pipe::addParallelTransforms(Processors transforms) std::to_string(transforms.size()) + " transforms were passed, " "but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR); - size_t next_output = 0; for (size_t i = 0; i < inputs.size(); ++i) - { - connect(*output_ports[next_output], *inputs[i]); - ++next_output; - } + connect(*output_ports[i], *inputs[i]); - output_ports.clear(); - output_ports.reserve(outputs.size()); - - for (auto * output : outputs) - output_ports.emplace_back(std::move(output)); + output_ports = std::move(outputs); /// do not check output formats because they are different in case of parallel aggregations LOG_DEBUG(log, "addParallelTransforms do not check format"); From e8a96b53b1e359d2501f7f795752fab4149197ab Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 14 Dec 2021 22:44:54 +0300 Subject: [PATCH 167/358] Remove unused code --- src/QueryPipeline/Pipe.cpp | 49 ---------------------- src/QueryPipeline/Pipe.h | 1 - src/QueryPipeline/QueryPipelineBuilder.cpp | 6 --- src/QueryPipeline/QueryPipelineBuilder.h | 2 - 4 files changed, 58 deletions(-) diff --git a/src/QueryPipeline/Pipe.cpp b/src/QueryPipeline/Pipe.cpp index 6aa10b87328..25c5f7c0781 100644 --- a/src/QueryPipeline/Pipe.cpp +++ b/src/QueryPipeline/Pipe.cpp @@ -432,55 +432,6 @@ void Pipe::addTransform(ProcessorPtr transform) addTransform(std::move(transform), static_cast(nullptr), static_cast(nullptr)); } -void Pipe::addParallelTransforms(Processors transforms) -{ - LOG_DEBUG(log, "Begin addParallelTransforms, have {} transforms", transforms.size()); - - if (output_ports.empty()) - throw Exception("Cannot add parallel transforms to empty Pipe.", ErrorCodes::LOGICAL_ERROR); - - std::vector inputs; - std::vector outputs; - for (const auto & transform : transforms) - { - auto & current_transform_inputs = transform->getInputs(); - if (current_transform_inputs.size() != 1) - throw Exception("Each parallel transform should have one input port", ErrorCodes::LOGICAL_ERROR); - - inputs.push_back(&(current_transform_inputs.front())); - - auto & current_transform_outputs = transform->getOutputs(); - if (current_transform_outputs.size() != 1) - throw Exception("Each parallel transform should have one output port", ErrorCodes::LOGICAL_ERROR); - - outputs.push_back(&(current_transform_outputs.front())); - LOG_DEBUG(log, "addParallelTransforms, added inputs and outputs for processor {}", transform->getName()); - LOG_DEBUG(log, "output structure: {}", transform->getOutputs().front().getHeader().dumpStructure()); - } - - if (inputs.size() != output_ports.size()) - throw Exception("Cannot add parallel transforms to Pipes because " + - std::to_string(transforms.size()) + " transforms were passed, " - "but " + std::to_string(output_ports.size()) + " expected", ErrorCodes::LOGICAL_ERROR); - - for (size_t i = 0; i < inputs.size(); ++i) - connect(*output_ports[i], *inputs[i]); - - output_ports = std::move(outputs); - - /// do not check output formats because they are different in case of parallel aggregations - LOG_DEBUG(log, "addParallelTransforms do not check format"); - - if (collected_processors) - collected_processors->insert(collected_processors->end(), transforms.begin(), transforms.end()); - - for (auto & transform_ptr : transforms) - processors.emplace_back(std::move(transform_ptr)); - - /// Should not change streams number, so maybe not need max_parallel_streams update - max_parallel_streams = std::max(max_parallel_streams, output_ports.size()); -} - void Pipe::addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes) { if (output_ports.empty()) diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 21e8fbfe039..01a5d8bb961 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -64,7 +64,6 @@ public: /// Output ports should have same headers. /// If totals or extremes are not empty, transform shouldn't change header. void addTransform(ProcessorPtr transform); - void addParallelTransforms(Processors transform); void addTransform(ProcessorPtr transform, OutputPort * totals, OutputPort * extremes); void addTransform(ProcessorPtr transform, InputPort * totals, InputPort * extremes); diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 1377db34daf..40c64046560 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -155,12 +155,6 @@ void QueryPipelineBuilder::transform(const Transformer & transformer) pipe.transform(transformer); } -void QueryPipelineBuilder::addParallelTransforms(Processors transforms) -{ - checkInitializedAndNotCompleted(); - pipe.addParallelTransforms(transforms); -} - void QueryPipelineBuilder::setSinks(const Pipe::ProcessorGetterWithStreamKind & getter) { checkInitializedAndNotCompleted(); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index d920f8a7e81..9e198f45e98 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -67,8 +67,6 @@ public: /// Transform pipeline in general way. void transform(const Transformer & transformer); - /// Add transforms and connect it to outputs streams - void addParallelTransforms(Processors transform); /// Add TotalsHavingTransform. Resize pipeline to single input. Adds totals port. void addTotalsHavingTransform(ProcessorPtr transform); /// Add transform which calculates extremes. This transform adds extremes port and doesn't change inputs number. From 2e8b75ade4c0a0857cfb108b3189b30e5383ff69 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 10:45:27 +0300 Subject: [PATCH 168/358] Fix configs --- src/Parsers/tests/gtest_Parser.cpp | 6 +++--- tests/ci/build_report_check.py | 2 +- tests/ci/ci_config.py | 5 +++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 0511ce0333f..193d8cc6127 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -83,15 +83,15 @@ TEST_P(TableOverrideTest, applyOverrides) ASSERT_NE(nullptr, database); ASTPtr table_ast; ASSERT_NO_THROW(table_ast = parseQuery(parser, table_query, 0, 0)); - auto table = table_ast->as(); + auto * table = table_ast->as(); ASSERT_NE(nullptr, table); auto table_name = table->table->as()->name(); if (database->table_overrides) { auto override_ast = database->table_overrides->tryGetTableOverride(table_name); ASSERT_NE(nullptr, override_ast); - auto override = override_ast->as(); - ASSERT_NE(nullptr, override); + auto override_table_ast = override_ast->as(); + ASSERT_NE(nullptr, override_table_ast); override->applyToCreateTableQuery(table); } EXPECT_EQ(expected_query, serializeAST(*table)); diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 3d97a973017..c703b8511e3 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -25,7 +25,7 @@ class BuildResult(): self.with_coverage = with_coverage def group_by_artifacts(build_urls): - groups = {'deb': [], 'binary': [], 'tgz': [], 'rpm': [], 'preformance': []} + groups = {'deb': [], 'binary': [], 'tgz': [], 'rpm': [], 'performance': []} for url in build_urls: if url.endswith('performance.tgz'): groups['performance'].append(url) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index dac0699ad4e..9297b25cbb7 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -179,9 +179,10 @@ CI_CONFIG = { "binary_tidy", "binary_splitted", "binary_darwin", - "binary_arrach64", + "binary_aarch64", "binary_freebsd", - "binary_darwin_aarch64" + "binary_darwin_aarch64", + "binary_ppc64le", ], }, "tests_config": { From 2e813fe3cd6715e8aa93913179a399a9be5aff3c Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 15 Dec 2021 08:18:43 +0000 Subject: [PATCH 169/358] Update defaults --- src/Core/Settings.h | 4 ++-- src/Dictionaries/MySQLDictionarySource.cpp | 4 ++-- src/Storages/MySQL/MySQLSettings.h | 4 ++-- src/TableFunctions/TableFunctionMySQL.cpp | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 964ba8ffc13..9c090fdc1da 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -499,8 +499,8 @@ class IColumn; \ M(UInt64, external_storage_max_read_rows, 0, "Limit maximum number of rows when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ M(UInt64, external_storage_max_read_bytes, 0, "Limit maximum number of bytes when table with external engine should flush history data. Now supported only for MySQL table engine, database engine, dictionary and MaterializedMySQL. If equal to 0, this setting is disabled", 0) \ - M(UInt64, external_storage_connect_timeout, 60, "Connect timeout. Now supported only for MySQL", 0) \ - M(UInt64, external_storage_rw_timeout, 1800, "Read/write timeout. Now supported only for MySQL", 0) \ + M(UInt64, external_storage_connect_timeout_sec, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connect timeout in seconds. Now supported only for MySQL", 0) \ + M(UInt64, external_storage_rw_timeout_sec, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Read/write timeout in seconds. Now supported only for MySQL", 0) \ \ M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index fd090f99136..5f9a70caaa9 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -56,8 +56,8 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; MySQLSettings mysql_settings; const auto & settings = global_context->getSettingsRef(); - mysql_settings.connect_timeout = settings.external_storage_connect_timeout; - mysql_settings.read_write_timeout = settings.external_storage_rw_timeout; + mysql_settings.connect_timeout = settings.external_storage_connect_timeout_sec; + mysql_settings.read_write_timeout = settings.external_storage_rw_timeout_sec; pool = std::make_shared(createMySQLPoolWithFailover(configuration, mysql_settings)); } else diff --git a/src/Storages/MySQL/MySQLSettings.h b/src/Storages/MySQL/MySQLSettings.h index 896141ae673..aa2c2703d6b 100644 --- a/src/Storages/MySQL/MySQLSettings.h +++ b/src/Storages/MySQL/MySQLSettings.h @@ -19,8 +19,8 @@ class ASTStorage; M(UInt64, connection_max_tries, 3, "Number of retries for pool with failover", 0) \ M(UInt64, connection_wait_timeout, 5, "Timeout (in seconds) for waiting for free connection (in case of there is already connection_pool_size active connections), 0 - do not wait.", 0) \ M(Bool, connection_auto_close, true, "Auto-close connection after query execution, i.e. disable connection reuse.", 0) \ - M(UInt64, connect_timeout, 60, "Connect timeout", 0) \ - M(UInt64, read_write_timeout, 1800, "Read/write timeout", 0) \ + M(UInt64, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connect timeout (in seconds)", 0) \ + M(UInt64, read_write_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Read/write timeout (in seconds)", 0) \ DECLARE_SETTINGS_TRAITS(MySQLSettingsTraits, LIST_OF_MYSQL_SETTINGS) diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index 86abe352a7d..e959fa754c9 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -40,8 +40,8 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr configuration = StorageMySQL::getConfiguration(args_func.arguments->children, context); MySQLSettings mysql_settings; const auto & settings = context->getSettingsRef(); - mysql_settings.connect_timeout = settings.external_storage_connect_timeout; - mysql_settings.read_write_timeout = settings.external_storage_rw_timeout; + mysql_settings.connect_timeout = settings.external_storage_connect_timeout_sec; + mysql_settings.read_write_timeout = settings.external_storage_rw_timeout_sec; pool.emplace(createMySQLPoolWithFailover(*configuration, mysql_settings)); } From d1dcd0adbfa415afbe1fdd80f7964321955e1e0f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 11:44:18 +0300 Subject: [PATCH 170/358] Followup --- src/Parsers/tests/gtest_Parser.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 193d8cc6127..20a581120dc 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -92,7 +92,7 @@ TEST_P(TableOverrideTest, applyOverrides) ASSERT_NE(nullptr, override_ast); auto override_table_ast = override_ast->as(); ASSERT_NE(nullptr, override_table_ast); - override->applyToCreateTableQuery(table); + override_table_ast->applyToCreateTableQuery(table); } EXPECT_EQ(expected_query, serializeAST(*table)); } From 9c9f4a3e6c0c1f4dc51fda70decaa62820fe13a1 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 15 Dec 2021 16:55:18 +0800 Subject: [PATCH 171/358] build fix --- contrib/azure-cmake/CMakeLists.txt | 7 +++++-- src/CMakeLists.txt | 14 ++++++++++++++ src/Disks/BlobStorage/BlobStorageAuth.h | 18 ++++++++++++++++++ src/Disks/BlobStorage/DiskBlobStorage.h | 16 ++++++++++++++++ src/Disks/IO/ReadBufferFromRemoteFSGather.h | 19 +++++++++++++++++++ src/IO/ReadBufferFromBlobStorage.h | 18 ++++++++++++++++++ src/IO/WriteBufferFromBlobStorage.h | 17 +++++++++++++++++ 7 files changed, 107 insertions(+), 2 deletions(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 7f9476e37b7..ed428364598 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -46,14 +46,17 @@ include("${AZURE_DIR}/cmake-modules/AzureTransportAdapters.cmake") add_library(azure_sdk ${AZURE_SDK_UNIFIED_SRC}) if (COMPILER_CLANG) - target_compile_options(azure_sdk PUBLIC + target_compile_options(azure_sdk PRIVATE -Wno-deprecated-copy-dtor -Wno-extra-semi -Wno-suggest-destructor-override -Wno-inconsistent-missing-destructor-override -Wno-error=unknown-warning-option - -Wno-reserved-identifier ) + + if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 13) + target_compile_options(azure_sdk PRIVATE -Wno-reserved-identifier) + endif() endif() # Originally, on Windows azure-core is built with bcrypt and crypt32 by default diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index bca9bd9d280..428f817ac49 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -385,6 +385,20 @@ dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${PDQSORT_INCLUDE_DIR}) target_include_directories(clickhouse_common_io BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR}) +macro (add_check_flag_definitions flag modules) + if (${flag}) + if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) + target_compile_definitions(clickhouse_common_io PRIVATE ${flag}) + target_compile_definitions(dbms PRIVATE ${flag}) + else () + foreach (module ${modules}) + target_compile_definitions(${module} PRIVATE ${flag}) + endforeach () + endif () + endif () +endmacro () +add_check_flag_definitions(HAS_SUGGEST_DESTRUCTOR_OVERRIDE "clickhouse_common_io;clickhouse_disks") + if (ZSTD_LIBRARY) dbms_target_link_libraries(PRIVATE ${ZSTD_LIBRARY}) target_link_libraries (clickhouse_common_io PUBLIC ${ZSTD_LIBRARY}) diff --git a/src/Disks/BlobStorage/BlobStorageAuth.h b/src/Disks/BlobStorage/BlobStorageAuth.h index 16a088fc960..04abc2fd643 100644 --- a/src/Disks/BlobStorage/BlobStorageAuth.h +++ b/src/Disks/BlobStorage/BlobStorageAuth.h @@ -7,8 +7,26 @@ #if USE_AZURE_BLOB_STORAGE #include + +#if defined(__clang__) +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" +# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" +# pragma clang diagnostic ignored "-Wextra-semi" +# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE +# pragma clang diagnostic ignored "-Wsuggest-destructor-override" +# endif +# ifdef HAS_RESERVED_IDENTIFIER +# pragma clang diagnostic ignored "-Wreserved-identifier" +# endif +#endif + #include +#if defined(__clang__) +# pragma clang diagnostic pop +#endif + namespace DB { diff --git a/src/Disks/BlobStorage/DiskBlobStorage.h b/src/Disks/BlobStorage/DiskBlobStorage.h index 16ba216df96..d124a133eb2 100644 --- a/src/Disks/BlobStorage/DiskBlobStorage.h +++ b/src/Disks/BlobStorage/DiskBlobStorage.h @@ -11,9 +11,25 @@ #include #include +#if defined(__clang__) +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" +# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" +# pragma clang diagnostic ignored "-Wextra-semi" +# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE +# pragma clang diagnostic ignored "-Wsuggest-destructor-override" +# endif +# ifdef HAS_RESERVED_IDENTIFIER +# pragma clang diagnostic ignored "-Wreserved-identifier" +# endif +#endif + #include #include +#if defined(__clang__) +# pragma clang diagnostic pop +#endif namespace DB { diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 045ab43850d..6bef9488d05 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -6,7 +6,26 @@ #include #if USE_AZURE_BLOB_STORAGE + +#if defined(__clang__) +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" +# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" +# pragma clang diagnostic ignored "-Wextra-semi" +# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE +# pragma clang diagnostic ignored "-Wsuggest-destructor-override" +# endif +# ifdef HAS_RESERVED_IDENTIFIER +# pragma clang diagnostic ignored "-Wreserved-identifier" +# endif +#endif + #include + +#if defined(__clang__) +# pragma clang diagnostic pop +#endif + #endif namespace Aws diff --git a/src/IO/ReadBufferFromBlobStorage.h b/src/IO/ReadBufferFromBlobStorage.h index cd66e897e25..029b53641eb 100644 --- a/src/IO/ReadBufferFromBlobStorage.h +++ b/src/IO/ReadBufferFromBlobStorage.h @@ -9,8 +9,26 @@ #include #include #include + +#if defined(__clang__) +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" +# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" +# pragma clang diagnostic ignored "-Wextra-semi" +# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE +# pragma clang diagnostic ignored "-Wsuggest-destructor-override" +# endif +# ifdef HAS_RESERVED_IDENTIFIER +# pragma clang diagnostic ignored "-Wreserved-identifier" +# endif +#endif + #include +#if defined(__clang__) +# pragma clang diagnostic pop +#endif + namespace DB { diff --git a/src/IO/WriteBufferFromBlobStorage.h b/src/IO/WriteBufferFromBlobStorage.h index 5f8eaba3c0c..4bb995f907d 100644 --- a/src/IO/WriteBufferFromBlobStorage.h +++ b/src/IO/WriteBufferFromBlobStorage.h @@ -10,9 +10,26 @@ #include #include + +#if defined(__clang__) +# pragma clang diagnostic push +# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" +# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" +# pragma clang diagnostic ignored "-Wextra-semi" +# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE +# pragma clang diagnostic ignored "-Wsuggest-destructor-override" +# endif +# ifdef HAS_RESERVED_IDENTIFIER +# pragma clang diagnostic ignored "-Wreserved-identifier" +# endif +#endif + #include #include +#if defined(__clang__) +# pragma clang diagnostic pop +#endif namespace DB { From b5472d6d99a32f2e75f5ee862b83d0d1fa64f936 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 15 Dec 2021 12:42:07 +0300 Subject: [PATCH 172/358] Update LocalServer.cpp --- programs/local/LocalServer.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 33615080df4..8da8641bcdc 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -727,7 +727,6 @@ void LocalServer::printHelpMessage([[maybe_unused]] const OptionsDescription & o void LocalServer::addOptions(OptionsDescription & options_description) { options_description.main_description->add_options() - ("database,d", po::value(), "database") ("table,N", po::value(), "name of the initial table") /// If structure argument is omitted then initial query is not generated From c3324822681e4e15b486a4b627a81e36c16533de Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 12:48:56 +0300 Subject: [PATCH 173/358] Trying another stress servers --- .github/workflows/backport_branches.yml | 6 +++++- .github/workflows/main.yml | 6 +++++- .github/workflows/master.yml | 6 +++++- .github/workflows/release_branches.yml | 6 +++++- 4 files changed, 20 insertions(+), 4 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 859756f07af..7f9e147fa65 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -301,7 +301,11 @@ jobs: ############################################################################################## StressTestTsan: needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] + # func testers have 16 cores + 128 GB memory + # while stress testers have 36 cores + 72 memory + # It would be better to have something like 32 + 128, + # but such servers almost unavailable as spot instances. + runs-on: [self-hosted, func-tester] steps: - name: Download json reports uses: actions/download-artifact@v2 diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index be8a14dd38a..cb219e03320 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -1537,7 +1537,11 @@ jobs: sudo rm -fr $TEMP_PATH StressTestTsan: needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] + # func testers have 16 cores + 128 GB memory + # while stress testers have 36 cores + 72 memory + # It would be better to have something like 32 + 128, + # but such servers almost unavailable as spot instances. + runs-on: [self-hosted, func-tester] steps: - name: Download json reports uses: actions/download-artifact@v2 diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index a21a5a8c195..bcad1b68a51 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1417,7 +1417,11 @@ jobs: sudo rm -fr $TEMP_PATH StressTestTsan: needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] + # func testers have 16 cores + 128 GB memory + # while stress testers have 36 cores + 72 memory + # It would be better to have something like 32 + 128, + # but such servers almost unavailable as spot instances. + runs-on: [self-hosted, func-tester] steps: - name: Download json reports uses: actions/download-artifact@v2 diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 4489585541b..ea19ebc9be7 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -700,7 +700,11 @@ jobs: sudo rm -fr $TEMP_PATH StressTestTsan: needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] + # func testers have 16 cores + 128 GB memory + # while stress testers have 36 cores + 72 memory + # It would be better to have something like 32 + 128, + # but such servers almost unavailable as spot instances. + runs-on: [self-hosted, func-tester] steps: - name: Download json reports uses: actions/download-artifact@v2 From 5f02cfa595768d8fd6befe59fb0397ea6b46457f Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 15 Dec 2021 12:51:48 +0300 Subject: [PATCH 174/358] Update 01293_show_settings.reference --- tests/queries/0_stateless/01293_show_settings.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01293_show_settings.reference b/tests/queries/0_stateless/01293_show_settings.reference index 367a0670acf..75e0ec5ffec 100644 --- a/tests/queries/0_stateless/01293_show_settings.reference +++ b/tests/queries/0_stateless/01293_show_settings.reference @@ -2,7 +2,7 @@ send_timeout Seconds 300 connect_timeout Seconds 10 connect_timeout_with_failover_ms Milliseconds 2000 connect_timeout_with_failover_secure_ms Milliseconds 3000 -external_storage_connect_timeout UInt64 60 +external_storage_connect_timeout_sec UInt64 10 max_memory_usage UInt64 10000000000 max_untracked_memory UInt64 1048576 memory_profiler_step UInt64 1048576 From 31600cdcf9e77f8ff61f9f47849eb0c67c492ace Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 13:02:17 +0300 Subject: [PATCH 175/358] Clang-tidy mocking me --- src/Parsers/tests/gtest_Parser.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 20a581120dc..e7abcee6b43 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -90,7 +90,7 @@ TEST_P(TableOverrideTest, applyOverrides) { auto override_ast = database->table_overrides->tryGetTableOverride(table_name); ASSERT_NE(nullptr, override_ast); - auto override_table_ast = override_ast->as(); + auto * override_table_ast = override_ast->as(); ASSERT_NE(nullptr, override_table_ast); override_table_ast->applyToCreateTableQuery(table); } From 546a03281f1c7911621c232d38ba1180ffaceb64 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Dec 2021 11:10:47 +0100 Subject: [PATCH 176/358] Rework image building: - Make it arch-agnostic - Add necessary packages - Update runner version - Get public keys of ClickHouse/core members --- tests/ci/worker/ubuntu_style_check.sh | 49 ++++++++++++++++++++++----- 1 file changed, 41 insertions(+), 8 deletions(-) diff --git a/tests/ci/worker/ubuntu_style_check.sh b/tests/ci/worker/ubuntu_style_check.sh index bf5c6057bed..585c8610772 100644 --- a/tests/ci/worker/ubuntu_style_check.sh +++ b/tests/ci/worker/ubuntu_style_check.sh @@ -1,25 +1,47 @@ #!/usr/bin/env bash -set -euo pipefail +set -xeuo pipefail echo "Running prepare script" export DEBIAN_FRONTEND=noninteractive -export RUNNER_VERSION=2.283.1 +export RUNNER_VERSION=2.285.1 export RUNNER_HOME=/home/ubuntu/actions-runner +deb_arch() { + case $(uname -m) in + x86_64 ) + echo amd64;; + aarch64 ) + echo arm64;; + esac +} + +runner_arch() { + case $(uname -m) in + x86_64 ) + echo x64;; + aarch64 ) + echo arm64;; + esac +} + apt-get update apt-get install --yes --no-install-recommends \ apt-transport-https \ + build-essential \ ca-certificates \ curl \ gnupg \ + jq \ lsb-release \ + pigz \ + python3-dev \ python3-pip \ unzip curl -fsSL https://download.docker.com/linux/ubuntu/gpg | gpg --dearmor -o /usr/share/keyrings/docker-archive-keyring.gpg -echo "deb [arch=amd64 signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | tee /etc/apt/sources.list.d/docker.list > /dev/null +echo "deb [arch=$(deb_arch) signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | tee /etc/apt/sources.list.d/docker.list > /dev/null apt-get update @@ -37,21 +59,32 @@ EOT systemctl restart docker -pip install boto3 pygithub requests urllib3 unidiff +pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory mkdir -p $RUNNER_HOME && cd $RUNNER_HOME -curl -O -L https://github.com/actions/runner/releases/download/v$RUNNER_VERSION/actions-runner-linux-x64-$RUNNER_VERSION.tar.gz +RUNNER_ARCHIVE="actions-runner-linux-$(runner_arch)-$RUNNER_VERSION.tar.gz" -tar xzf ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz -rm -f ./actions-runner-linux-x64-$RUNNER_VERSION.tar.gz +curl -O -L "https://github.com/actions/runner/releases/download/v$RUNNER_VERSION/$RUNNER_ARCHIVE" + +tar xzf "./$RUNNER_ARCHIVE" +rm -f "./$RUNNER_ARCHIVE" ./bin/installdependencies.sh chown -R ubuntu:ubuntu $RUNNER_HOME cd /home/ubuntu -curl "https://awscli.amazonaws.com/awscli-exe-linux-x86_64.zip" -o "awscliv2.zip" +curl "https://awscli.amazonaws.com/awscli-exe-linux-$(uname -m).zip" -o "awscliv2.zip" unzip awscliv2.zip ./aws/install rm -rf /home/ubuntu/awscliv2.zip /home/ubuntu/aws + +# SSH keys of core team +mkdir -p /home/ubuntu/.ssh + +# ~/.ssh/authorized_keys is cleaned out, so we use deprecated but working ~/.ssh/authorized_keys2 +aws lambda invoke --region us-east-1 --function-name team-keys-lambda /tmp/core.keys +jq < /tmp/core.keys -r '.body' > /home/ubuntu/.ssh/authorized_keys2 +chown ubuntu: /home/ubuntu/.ssh -R +chmod 0700 /home/ubuntu/.ssh From bd5931e5afc80f62bc63d391a5fae96229eef1c1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 15 Dec 2021 13:21:21 +0300 Subject: [PATCH 177/358] improve gdb script --- docker/test/fuzzer/run-fuzzer.sh | 9 +++++++++ docker/test/stress/run.sh | 9 +++++++++ 2 files changed, 18 insertions(+) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 5d44e542269..8b9be4077ae 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -175,6 +175,15 @@ info signals continue backtrace full info locals +info registers +disassemble /s +up +info locals +disassemble /s +up +info locals +disassemble /s +p \"done\" detach quit " > script.gdb diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 138da284d5a..2efb62689ff 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -148,6 +148,15 @@ info signals continue backtrace full info locals +info registers +disassemble /s +up +info locals +disassemble /s +up +info locals +disassemble /s +p \"done\" detach quit " > script.gdb From 07bfbc26d65346acfcde1488fab386e9e1c83ddf Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 15 Dec 2021 15:08:08 +0300 Subject: [PATCH 178/358] fix restarting with unavailable mysql --- src/Databases/DatabaseFactory.cpp | 3 ++- src/Databases/MySQL/DatabaseMySQL.cpp | 17 +++++++++++-- src/Databases/MySQL/DatabaseMySQL.h | 3 ++- .../test_mysql_database_engine/test.py | 25 ++++++++++++++++++- 4 files changed, 43 insertions(+), 5 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index a4c8f3cdb77..d0c5544607d 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -204,7 +204,8 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String mysql_database_settings->loadFromQuery(*engine_define); /// higher priority return std::make_shared( - context, database_name, metadata_path, engine_define, configuration.database, std::move(mysql_database_settings), std::move(mysql_pool)); + context, database_name, metadata_path, engine_define, configuration.database, + std::move(mysql_database_settings), std::move(mysql_pool), create.attach); } MySQLClient client(configuration.host, configuration.port, configuration.username, configuration.password); diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index f62e06aff8d..e4963d6755b 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -53,7 +53,8 @@ DatabaseMySQL::DatabaseMySQL( const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, std::unique_ptr settings_, - mysqlxx::PoolWithFailover && pool) + mysqlxx::PoolWithFailover && pool, + bool attach) : IDatabase(database_name_) , WithContext(context_->getGlobalContext()) , metadata_path(metadata_path_) @@ -62,7 +63,19 @@ DatabaseMySQL::DatabaseMySQL( , database_settings(std::move(settings_)) , mysql_pool(std::move(pool)) { - empty(); /// test database is works fine. + try + { + /// Test database is working fine and fetch tables. + empty(); + } + catch (...) + { + if (attach) + tryLogCurrentException("DatabaseMySQL"); + else + throw; + } + thread = ThreadFromGlobalPool{&DatabaseMySQL::cleanOutdatedTables, this}; } diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index e57ac442db1..1ee090ecd52 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -45,7 +45,8 @@ public: const ASTStorage * database_engine_define, const String & database_name_in_mysql, std::unique_ptr settings_, - mysqlxx::PoolWithFailover && pool); + mysqlxx::PoolWithFailover && pool, + bool attach); String getEngineName() const override { return "MySQL"; } diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 49206ab1abe..efa5a38b08a 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -6,9 +6,10 @@ import pymysql.cursors import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) -clickhouse_node = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/named_collections.xml'], with_mysql=True) +clickhouse_node = cluster.add_instance('node1', main_configs=['configs/remote_servers.xml', 'configs/named_collections.xml'], with_mysql=True, stay_alive=True) @pytest.fixture(scope="module") @@ -27,6 +28,7 @@ class MySQLNodeInstance: self.hostname = hostname self.password = password self.mysql_connection = None # lazy init + self.ip_address = hostname def query(self, execution_query): if self.mysql_connection is None: @@ -424,3 +426,24 @@ def test_predefined_connection_configuration(started_cluster): clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL(mysql1, port=3306)") assert clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`").rstrip() == '100' + + +def test_restart_server(started_cluster): + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: + mysql_node.query("DROP DATABASE IF EXISTS test_restart") + clickhouse_node.query("DROP DATABASE IF EXISTS test_restart") + clickhouse_node.query_and_get_error("CREATE DATABASE test_restart ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") + assert 'test_restart' not in clickhouse_node.query('SHOW DATABASES') + + mysql_node.query("CREATE DATABASE test_restart DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE `test_restart`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + clickhouse_node.query("CREATE DATABASE test_restart ENGINE = MySQL('mysql57:3306', 'test_restart', 'root', 'clickhouse')") + + assert 'test_restart' in clickhouse_node.query('SHOW DATABASES') + assert 'test_table' in clickhouse_node.query('SHOW TABLES FROM test_restart') + + with PartitionManager() as pm: + pm.partition_instances(clickhouse_node, mysql_node, action='REJECT --reject-with tcp-reset') + clickhouse_node.restart_clickhouse() + clickhouse_node.query_and_get_error('SHOW TABLES FROM test_restart') + assert 'test_table' in clickhouse_node.query('SHOW TABLES FROM test_restart') From ac3cb8c12b4f75fbd277a76b6450a7322d4c515d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 15 Dec 2021 15:55:28 +0300 Subject: [PATCH 179/358] CacheDictionary dictionary source access race fix --- src/Dictionaries/CacheDictionary.cpp | 4 +-- src/Dictionaries/CacheDictionary.h | 6 ++--- src/Dictionaries/CassandraDictionarySource.h | 2 +- .../ClickHouseDictionarySource.cpp | 1 - src/Dictionaries/ClickHouseDictionarySource.h | 2 +- src/Dictionaries/DirectDictionary.h | 2 +- .../ExecutableDictionarySource.cpp | 2 +- .../ExecutablePoolDictionarySource.cpp | 2 +- src/Dictionaries/ExternalQueryBuilder.cpp | 15 ++++++++--- src/Dictionaries/FileDictionarySource.h | 2 +- src/Dictionaries/FlatDictionary.h | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Dictionaries/HashedArrayDictionary.h | 2 +- src/Dictionaries/HashedDictionary.h | 2 +- src/Dictionaries/IDictionary.h | 25 ++++++++++--------- src/Dictionaries/IDictionarySource.h | 3 +-- src/Dictionaries/IPAddressDictionary.h | 2 +- src/Dictionaries/LibraryDictionarySource.cpp | 2 +- src/Dictionaries/MongoDBDictionarySource.h | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 2 +- src/Dictionaries/PolygonDictionary.h | 2 +- .../PostgreSQLDictionarySource.cpp | 2 +- src/Dictionaries/RangeHashedDictionary.h | 2 +- src/Dictionaries/RedisDictionarySource.h | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Dictionaries/writeParenthesisedString.cpp | 12 --------- src/Dictionaries/writeParenthesisedString.h | 11 -------- src/Functions/FunctionsExternalDictionaries.h | 10 +++++--- 28 files changed, 56 insertions(+), 69 deletions(-) delete mode 100644 src/Dictionaries/writeParenthesisedString.cpp delete mode 100644 src/Dictionaries/writeParenthesisedString.h diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 723457fba5b..5b9d5e37b47 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -110,12 +110,12 @@ std::exception_ptr CacheDictionary::getLastException() cons } template -const IDictionarySource * CacheDictionary::getSource() const +DictionarySourcePtr CacheDictionary::getSource() const { /// Mutex required here because of the getSourceAndUpdateIfNeeded() function /// which is used from another thread. std::lock_guard lock(source_mutex); - return source_ptr.get(); + return source_ptr; } template diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index d2c2ed9a212..de04e7e098b 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -104,7 +104,7 @@ public: allow_read_expired_keys); } - const IDictionarySource * getSource() const override; + DictionarySourcePtr getSource() const override; const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } @@ -172,7 +172,7 @@ private: /// MultiVersion is not used here because it works with constant pointers. /// For some reason almost all methods in IDictionarySource interface are /// not constant. - SharedDictionarySourcePtr getSourceAndUpdateIfNeeded() const + DictionarySourcePtr getSourceAndUpdateIfNeeded() const { std::lock_guard lock(source_mutex); if (error_count) @@ -190,7 +190,7 @@ private: /// Dictionary source should be used with mutex mutable std::mutex source_mutex; - mutable SharedDictionarySourcePtr source_ptr; + mutable DictionarySourcePtr source_ptr; CacheDictionaryStoragePtr cache_storage_ptr; mutable CacheDictionaryUpdateQueue update_queue; diff --git a/src/Dictionaries/CassandraDictionarySource.h b/src/Dictionaries/CassandraDictionarySource.h index 35419d3ea7d..76ad2316366 100644 --- a/src/Dictionaries/CassandraDictionarySource.h +++ b/src/Dictionaries/CassandraDictionarySource.h @@ -61,7 +61,7 @@ public: DictionarySourcePtr clone() const override { - return std::make_unique(dict_struct, configuration, sample_block); + return std::make_shared(dict_struct, configuration, sample_block); } Pipe loadIds(const std::vector & ids) override; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 1ddcdd96454..6abd5f317e2 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -17,7 +17,6 @@ #include "DictionaryStructure.h" #include "ExternalQueryBuilder.h" #include "readInvalidateQuery.h" -#include "writeParenthesisedString.h" #include "DictionaryFactory.h" #include "DictionarySourceHelpers.h" diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index be09fa415fd..cdcc0ee824f 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -60,7 +60,7 @@ public: bool hasUpdateField() const override; - DictionarySourcePtr clone() const override { return std::make_unique(*this); } + DictionarySourcePtr clone() const override { return std::make_shared(*this); } std::string toString() const override; diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index edf4c8d1d9a..4bf24e6ae98 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -58,7 +58,7 @@ public: return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone()); } - const IDictionarySource * getSource() const override { return source_ptr.get(); } + DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index c09993c2a84..5816b942d36 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -158,7 +158,7 @@ bool ExecutableDictionarySource::hasUpdateField() const DictionarySourcePtr ExecutableDictionarySource::clone() const { - return std::make_unique(*this); + return std::make_shared(*this); } std::string ExecutableDictionarySource::toString() const diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index dce2ce94b93..01be40412b6 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -152,7 +152,7 @@ bool ExecutablePoolDictionarySource::hasUpdateField() const DictionarySourcePtr ExecutablePoolDictionarySource::clone() const { - return std::make_unique(*this); + return std::make_shared(*this); } std::string ExecutablePoolDictionarySource::toString() const diff --git a/src/Dictionaries/ExternalQueryBuilder.cpp b/src/Dictionaries/ExternalQueryBuilder.cpp index f513c7b2f61..1701f08fd67 100644 --- a/src/Dictionaries/ExternalQueryBuilder.cpp +++ b/src/Dictionaries/ExternalQueryBuilder.cpp @@ -1,14 +1,23 @@ #include "ExternalQueryBuilder.h" + +#include + #include #include #include -#include -#include "DictionaryStructure.h" -#include "writeParenthesisedString.h" +#include namespace DB { + +static inline void writeParenthesisedString(const String & s, WriteBuffer & buf) +{ + writeChar('(', buf); + writeString(s, buf); + writeChar(')', buf); +} + namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index c8e37986b2f..8fe2d87d8b9 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -51,7 +51,7 @@ public: ///Not supported for FileDictionarySource bool hasUpdateField() const override { return false; } - DictionarySourcePtr clone() const override { return std::make_unique(*this); } + DictionarySourcePtr clone() const override { return std::make_shared(*this); } std::string toString() const override; diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index 5c3a1d634d8..308cd72d55b 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -61,7 +61,7 @@ public: return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, configuration, update_field_loaded_block); } - const IDictionarySource * getSource() const override { return source_ptr.get(); } + DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index aba6b40f206..308570644d1 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -207,7 +207,7 @@ bool HTTPDictionarySource::hasUpdateField() const DictionarySourcePtr HTTPDictionarySource::clone() const { - return std::make_unique(*this); + return std::make_shared(*this); } std::string HTTPDictionarySource::toString() const diff --git a/src/Dictionaries/HashedArrayDictionary.h b/src/Dictionaries/HashedArrayDictionary.h index ca5d7cb1bf6..0d07c43477a 100644 --- a/src/Dictionaries/HashedArrayDictionary.h +++ b/src/Dictionaries/HashedArrayDictionary.h @@ -71,7 +71,7 @@ public: return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); } - const IDictionarySource * getSource() const override { return source_ptr.get(); } + DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryLifetime & getLifetime() const override { return configuration.lifetime; } diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 16be4e4c73e..6f63c5ec546 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -78,7 +78,7 @@ public: return std::make_shared>(getDictionaryID(), dict_struct, source_ptr->clone(), configuration, update_field_loaded_block); } - const IDictionarySource * getSource() const override { return source_ptr.get(); } + DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryLifetime & getLifetime() const override { return configuration.lifetime; } diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 66e35c8fa12..b1923306003 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -1,16 +1,16 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include - #include #include +#include +#include +#include +#include +#include +#include +#include + namespace DB { @@ -19,7 +19,7 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -struct IDictionary; +class IDictionary; using DictionaryPtr = std::unique_ptr; /** DictionaryKeyType provides IDictionary client information about @@ -47,8 +47,9 @@ enum class DictionarySpecialKeyType /** * Base class for Dictionaries implementation. */ -struct IDictionary : public IExternalLoadable +class IDictionary : public IExternalLoadable { +public: explicit IDictionary(const StorageID & dictionary_id_) : dictionary_id(dictionary_id_) , full_name(dictionary_id.getInternalDictionaryName()) @@ -99,7 +100,7 @@ struct IDictionary : public IExternalLoadable virtual double getLoadFactor() const = 0; - virtual const IDictionarySource * getSource() const = 0; + virtual DictionarySourcePtr getSource() const = 0; virtual const DictionaryStructure & getStructure() const = 0; @@ -200,7 +201,7 @@ struct IDictionary : public IExternalLoadable bool isModified() const override { - const auto * source = getSource(); + const auto source = getSource(); return source && source->isModified(); } diff --git a/src/Dictionaries/IDictionarySource.h b/src/Dictionaries/IDictionarySource.h index 5071b69d2bf..128595b815f 100644 --- a/src/Dictionaries/IDictionarySource.h +++ b/src/Dictionaries/IDictionarySource.h @@ -10,8 +10,7 @@ namespace DB { class IDictionarySource; -using DictionarySourcePtr = std::unique_ptr; -using SharedDictionarySourcePtr = std::shared_ptr; +using DictionarySourcePtr = std::shared_ptr; /** Data-provider interface for external dictionaries, * abstracts out the data source (file, MySQL, ClickHouse, external program, network request et cetera) diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index ed0d8692d21..9f604b5aeb8 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -56,7 +56,7 @@ public: return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); } - const IDictionarySource * getSource() const override { return source_ptr.get(); } + DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 42683fb884c..b79ee9be59a 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -129,7 +129,7 @@ Pipe LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::v DictionarySourcePtr LibraryDictionarySource::clone() const { - return std::make_unique(*this); + return std::make_shared(*this); } diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index 3625deca9c6..85531f89902 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -65,7 +65,7 @@ public: ///Not yet supported bool hasUpdateField() const override { return false; } - DictionarySourcePtr clone() const override { return std::make_unique(*this); } + DictionarySourcePtr clone() const override { return std::make_shared(*this); } std::string toString() const override; diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index f6de6ca0cc1..18b4c512f2f 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -225,7 +225,7 @@ bool MySQLDictionarySource::hasUpdateField() const DictionarySourcePtr MySQLDictionarySource::clone() const { - return std::make_unique(*this); + return std::make_shared(*this); } std::string MySQLDictionarySource::toString() const diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index 346160c342f..762c136b8e0 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -87,7 +87,7 @@ public: double getLoadFactor() const override { return 1.0; } - const IDictionarySource * getSource() const override { return source_ptr.get(); } + DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryStructure & getStructure() const override { return dict_struct; } diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index c9fb8b86b77..0ac84b35048 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -161,7 +161,7 @@ bool PostgreSQLDictionarySource::supportsSelectiveLoad() const DictionarySourcePtr PostgreSQLDictionarySource::clone() const { - return std::make_unique(*this); + return std::make_shared(*this); } diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 1605e2bab81..fca72d5d7cc 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -67,7 +67,7 @@ public: return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, update_field_loaded_block); } - const IDictionarySource * getSource() const override { return source_ptr.get(); } + DictionarySourcePtr getSource() const override { return source_ptr; } const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index 053094e2303..eff97dede0c 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -76,7 +76,7 @@ namespace ErrorCodes bool hasUpdateField() const override { return false; } - DictionarySourcePtr clone() const override { return std::make_unique(*this); } + DictionarySourcePtr clone() const override { return std::make_shared(*this); } std::string toString() const override; diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index f827c0cd8d0..89cf18d530b 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -162,7 +162,7 @@ bool XDBCDictionarySource::hasUpdateField() const DictionarySourcePtr XDBCDictionarySource::clone() const { - return std::make_unique(*this); + return std::make_shared(*this); } diff --git a/src/Dictionaries/writeParenthesisedString.cpp b/src/Dictionaries/writeParenthesisedString.cpp deleted file mode 100644 index 5e237aa1e6c..00000000000 --- a/src/Dictionaries/writeParenthesisedString.cpp +++ /dev/null @@ -1,12 +0,0 @@ -#include "writeParenthesisedString.h" - -namespace DB -{ -void writeParenthesisedString(const String & s, WriteBuffer & buf) -{ - writeChar('(', buf); - writeString(s, buf); - writeChar(')', buf); -} - -} diff --git a/src/Dictionaries/writeParenthesisedString.h b/src/Dictionaries/writeParenthesisedString.h deleted file mode 100644 index ec61e944d38..00000000000 --- a/src/Dictionaries/writeParenthesisedString.h +++ /dev/null @@ -1,11 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ -void writeParenthesisedString(const String & s, WriteBuffer & buf); - - -} diff --git a/src/Functions/FunctionsExternalDictionaries.h b/src/Functions/FunctionsExternalDictionaries.h index c52d54f30aa..71597f2b433 100644 --- a/src/Functions/FunctionsExternalDictionaries.h +++ b/src/Functions/FunctionsExternalDictionaries.h @@ -68,11 +68,12 @@ public: std::shared_ptr getDictionary(const String & dictionary_name) { - auto dict = getContext()->getExternalDictionariesLoader().getDictionary(dictionary_name, getContext()); + auto current_context = getContext(); + auto dict = current_context->getExternalDictionariesLoader().getDictionary(dictionary_name, current_context); if (!access_checked) { - getContext()->checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName()); + current_context->checkAccess(AccessType::dictGet, dict->getDatabaseOrNoDatabaseTag(), dict->getDictionaryID().getTableName()); access_checked = true; } @@ -106,8 +107,9 @@ public: if (!attr_name_col) throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument of function dictGet must be a constant string"); - const auto dictionary_name = dict_name_col->getValue(); - const auto attribute_name = attr_name_col->getValue(); + const auto & dictionary_name = dict_name_col->getValue(); + const auto & attribute_name = attr_name_col->getValue(); + return getDictionary(dictionary_name)->isInjective(attribute_name); } From b9c13485415d945d896ceb64b3f69b8fda98adfa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 9 Dec 2021 19:09:46 +0100 Subject: [PATCH 180/358] Set global environment for jobs --- .github/workflows/backport.yml | 15 +- .github/workflows/backport_branches.yml | 174 +-- .github/workflows/docs_check.yml | 11 +- .github/workflows/main.yml | 1326 +++++++++++++---------- .github/workflows/master.yml | 1249 ++++++++++++--------- .github/workflows/release.yml | 15 +- .github/workflows/release_branches.yml | 476 ++++---- 7 files changed, 1909 insertions(+), 1357 deletions(-) diff --git a/.github/workflows/backport.yml b/.github/workflows/backport.yml index c83d3f6d5bd..284af965714 100644 --- a/.github/workflows/backport.yml +++ b/.github/workflows/backport.yml @@ -8,18 +8,21 @@ jobs: CherryPick: runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/cherry_pick + ROBOT_CLICKHOUSE_SSH_KEY=${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} + REPO_OWNER=ClickHouse + REPO_NAME=ClickHouse + REPO_TEAM=core + EOF - name: Check out repository code uses: actions/checkout@v2 with: token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} fetch-depth: 0 - name: Cherry pick - env: - TEMP_PATH: ${{runner.temp}}/cherry_pick - ROBOT_CLICKHOUSE_SSH_KEY: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} - REPO_OWNER: "ClickHouse" - REPO_NAME: "ClickHouse" - REPO_TEAM: "core" run: | sudo pip install GitPython cd $GITHUB_WORKSPACE/tests/ci diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 7f9e147fa65..5062cd36a73 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -22,17 +22,20 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck - env: - TEMP_PATH: ${{runner.temp}}/compatibility_check - REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -51,24 +54,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_release + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_release' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -78,7 +84,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -89,24 +95,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_asan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_asan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -116,7 +125,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -127,24 +136,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_tsan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_tsan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -154,7 +166,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -165,24 +177,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_debug + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_debug' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -192,7 +207,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -210,17 +225,20 @@ jobs: - BuilderDebDebug runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse build check (actions) + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder - env: - TEMP_PATH: ${{runner.temp}}/report_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'ClickHouse build check (actions)' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -239,19 +257,22 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -271,19 +292,22 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -307,18 +331,21 @@ jobs: # but such servers almost unavailable as spot instances. runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (thread, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (thread, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -338,18 +365,21 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (release, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 9e47f96320d..eab1b4b44fe 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -39,17 +39,20 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/docs_check + REPO_COPY=${{runner.temp}}/docs_check/ClickHouse + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/docs_check + path: ${{ env.TEMP_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Docs Check - env: - TEMP_PATH: ${{runner.temp}}/docs_check - REPO_COPY: ${{runner.temp}}/docs_check/ClickHouse run: | cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index cb219e03320..97be3cc1960 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -42,16 +42,19 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{ runner.temp }}/style_check + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/style_check + path: ${{ env.TEMP_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Style Check - env: - TEMP_PATH: ${{ runner.temp }}/style_check run: | cd $GITHUB_WORKSPACE/tests/ci python3 style_check.py @@ -65,13 +68,16 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/fasttest + REPO_COPY=${{runner.temp}}/fasttest/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Fast Test - env: - TEMP_PATH: ${{runner.temp}}/fasttest - REPO_COPY: ${{runner.temp}}/fasttest/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -87,14 +93,17 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/pvs_check + REPO_COPY=${{runner.temp}}/pvs_check/ClickHouse + EOF - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' - name: PVS Check - env: - TEMP_PATH: ${{runner.temp}}/pvs_check - REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -110,17 +119,20 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck - env: - TEMP_PATH: ${{runner.temp}}/compatibility_check - REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -136,17 +148,20 @@ jobs: needs: [BuilderDebSplitted] runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/split_build_check + REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Split build check - env: - TEMP_PATH: ${{runner.temp}}/split_build_check - REPO_COPY: ${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -165,24 +180,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_release + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_release' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -192,7 +210,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -203,24 +221,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=performance + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'performance' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -230,7 +251,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -241,24 +262,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_release + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_release' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -268,7 +292,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -279,24 +303,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_asan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_asan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -306,7 +333,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -317,24 +344,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_ubsan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_ubsan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -344,7 +374,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -355,24 +385,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_tsan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_tsan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -382,7 +415,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -393,24 +426,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_msan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_msan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -420,7 +456,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -431,24 +467,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_debug + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_debug' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -458,7 +497,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -472,24 +511,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_splitted + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_splitted' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -499,7 +541,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -510,24 +552,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_tidy + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_tidy' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -537,7 +582,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -548,24 +593,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_darwin + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_darwin' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -575,7 +623,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -586,24 +634,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_aarch64 + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_aarch64' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -613,7 +664,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -624,24 +675,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_freebsd + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_freebsd' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -651,7 +705,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -662,24 +716,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_darwin_aarch64 + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_darwin_aarch64' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -689,7 +746,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -700,24 +757,27 @@ jobs: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_ppc64le + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_ppc64le' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -727,7 +787,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -749,17 +809,20 @@ jobs: runs-on: [self-hosted, style-checker] if: always() steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse build check (actions) + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder - env: - TEMP_PATH: ${{runner.temp}}/report_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'ClickHouse build check (actions)' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -783,17 +846,20 @@ jobs: runs-on: [self-hosted, style-checker] if: always() steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse special build check (actions) + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder - env: - TEMP_PATH: ${{runner.temp}}/report_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'ClickHouse special build check (actions)' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -812,19 +878,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, actions) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -841,21 +910,24 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_database_replicated - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, DatabaseReplicated, actions)' - REPO_COPY: ${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -872,21 +944,24 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_database_replicated - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, DatabaseReplicated, actions)' - REPO_COPY: ${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -903,19 +978,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_wide_parts + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, wide parts enabled, actions) + REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_wide_parts - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, wide parts enabled, actions)' - REPO_COPY: ${{runner.temp}}/stateless_wide_parts/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -932,21 +1010,24 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -963,21 +1044,24 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -994,21 +1078,24 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1025,21 +1112,24 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1056,21 +1146,24 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1087,19 +1180,22 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1116,21 +1212,24 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1147,21 +1246,24 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1178,21 +1280,24 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1209,21 +1314,24 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1240,21 +1348,24 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1271,21 +1382,24 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1302,19 +1416,22 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_flaky_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests flaky check (address, actions) + REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_flaky_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests flaky check (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_flaky_asan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1334,19 +1451,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release, actions) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1363,19 +1483,22 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (address, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1392,19 +1515,22 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1421,19 +1547,22 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1450,19 +1579,22 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1479,19 +1611,22 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1511,18 +1646,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (address, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (address, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1543,18 +1681,21 @@ jobs: # but such servers almost unavailable as spot instances. runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (thread, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (thread, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1571,18 +1712,21 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (memory, actions) + REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (memory, actions)' - REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1599,18 +1743,21 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_undefined + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (undefined, actions) + REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_undefined - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (undefined, actions)' - REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1627,18 +1774,21 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (debug, actions) + REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (debug, actions)' - REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1658,18 +1808,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (ASan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (ASan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_asan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1686,18 +1839,21 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (TSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (TSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_tsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1714,18 +1870,21 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (UBSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (UBSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1742,18 +1901,21 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (MSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (MSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_msan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1770,18 +1932,21 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (debug, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (debug, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_debug/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1801,20 +1966,23 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1831,20 +1999,23 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1861,20 +2032,23 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1891,20 +2065,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1921,20 +2098,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1951,20 +2131,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1981,20 +2164,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 3 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2011,20 +2197,23 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (release, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2041,20 +2230,23 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (release, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2071,18 +2263,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests flaky check (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan_flaky_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests flaky check (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2102,18 +2297,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (asan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2130,18 +2328,21 @@ jobs: needs: [BuilderBinRelease] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (release-clang, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (release-clang, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2158,18 +2359,21 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (tsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (tsan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_tsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2186,18 +2390,21 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (msan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (msan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_msan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2214,18 +2421,21 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2245,20 +2455,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2275,20 +2488,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2305,20 +2521,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2335,20 +2554,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 3 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index bcad1b68a51..0249784a8da 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -22,16 +22,19 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{ runner.temp }}/style_check + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/style_check + path: ${{ env.TEMP_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Style Check - env: - TEMP_PATH: ${{ runner.temp }}/style_check run: | cd $GITHUB_WORKSPACE/tests/ci python3 style_check.py @@ -45,17 +48,20 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck - env: - TEMP_PATH: ${{runner.temp}}/compatibility_check - REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -72,17 +78,20 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/split_build_check + REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Split build check - env: - TEMP_PATH: ${{runner.temp}}/split_build_check - REPO_COPY: ${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -101,24 +110,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_release + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_release' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -128,7 +140,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -139,24 +151,27 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=performance + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'performance' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -166,7 +181,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -178,24 +193,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_release + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_release' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -205,7 +223,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -216,24 +234,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_asan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_asan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -243,7 +264,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -254,24 +275,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_ubsan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_ubsan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -281,7 +305,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -292,24 +316,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_tsan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_tsan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -319,7 +346,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -330,24 +357,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_msan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_msan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -357,7 +387,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -368,24 +398,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_debug + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_debug' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -395,7 +428,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -410,24 +443,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_splitted + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_splitted' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -437,7 +473,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -449,24 +485,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_tidy + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_tidy' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -476,7 +515,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -488,24 +527,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_darwin + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_darwin' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -515,7 +557,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -527,24 +569,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_aarch64 + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_aarch64' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -554,7 +599,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -566,24 +611,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_freebsd + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_freebsd' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -593,7 +641,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -605,24 +653,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_darwin_aarch64 + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_darwin_aarch64' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -632,7 +683,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -644,24 +695,27 @@ jobs: if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=binary_ppc64le + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'binary_ppc64le' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -671,7 +725,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -692,17 +746,20 @@ jobs: - BuilderDebDebug runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse build check (actions) + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder - env: - TEMP_PATH: ${{runner.temp}}/report_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'ClickHouse build check (actions)' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -725,17 +782,20 @@ jobs: - BuilderBinPPC64 runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse special build check (actions) + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder - env: - TEMP_PATH: ${{runner.temp}}/report_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'ClickHouse special build check (actions)' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -754,19 +814,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, actions) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -783,19 +846,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release_database_ordinary + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseOrdinary, actions) + REPO_COPY=${{runner.temp}}/stateless_release_database_ordinary/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_release_database_ordinary - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, DatabaseOrdinary, actions)' - REPO_COPY: ${{runner.temp}}/stateless_release_database_ordinary/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -812,21 +878,24 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -843,21 +912,24 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -874,21 +946,24 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -905,21 +980,24 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -936,21 +1014,24 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -967,19 +1048,22 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -996,21 +1080,24 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1027,21 +1114,24 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1058,21 +1148,24 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1089,21 +1182,24 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1120,21 +1216,24 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1151,21 +1250,24 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1185,19 +1287,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release, actions) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1214,19 +1319,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release_database_ordinary + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release, DatabaseOrdinary, actions) + REPO_COPY=${{runner.temp}}/stateful_release_database_ordinary/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_release_database_ordinary - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (release, DatabaseOrdinary, actions)' - REPO_COPY: ${{runner.temp}}/stateful_release_database_ordinary/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1243,19 +1351,22 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (address, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1272,19 +1383,22 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1301,19 +1415,22 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1330,19 +1447,22 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1359,19 +1479,22 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1391,18 +1514,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (address, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (address, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1423,18 +1549,21 @@ jobs: # but such servers almost unavailable as spot instances. runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (thread, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (thread, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1451,18 +1580,21 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (memory, actions) + REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (memory, actions)' - REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1479,18 +1611,21 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_undefined + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (undefined, actions) + REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_undefined - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (undefined, actions)' - REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1507,18 +1642,21 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (debug, actions) + REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (debug, actions)' - REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1538,20 +1676,23 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1568,20 +1709,23 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1598,20 +1742,23 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1628,20 +1775,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1658,20 +1808,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1688,20 +1841,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1718,20 +1874,23 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM: 3 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1748,20 +1907,23 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (release, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1778,20 +1940,23 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (release, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 2 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1811,18 +1976,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (ASan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (ASan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_asan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1839,18 +2007,21 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (TSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (TSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_tsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1867,18 +2038,21 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (UBSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (UBSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1895,18 +2069,21 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (MSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (MSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_msan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1923,18 +2100,21 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (debug, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (debug, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_debug/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1954,18 +2134,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (asan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -1982,18 +2165,21 @@ jobs: needs: [BuilderBinRelease] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (release-clang, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (release-clang, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2010,18 +2196,21 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (tsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (tsan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_tsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2038,18 +2227,21 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (msan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (msan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_msan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2066,18 +2258,21 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, fuzzer-unit-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2097,20 +2292,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2127,20 +2325,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2157,20 +2358,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -2187,20 +2391,23 @@ jobs: needs: [BuilderPerformance] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison - env: - TEMP_PATH: ${{runner.temp}}/performance_comparison - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Performance Comparison (actions)' - REPO_COPY: ${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM: 3 - RUN_BY_HASH_TOTAL: 4 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index dd576b04c8c..1ba25a75017 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -30,19 +30,22 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/docs_release + REPO_COPY=${{runner.temp}}/docs_release/ClickHouse + CLOUDFLARE_TOKEN=${{secrets.CLOUDFLARE}} + ROBOT_CLICKHOUSE_SSH_KEY=${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{runner.temp}}/docs_release + path: ${{ env.TEMP_PATH }} - name: Docs Release - env: - TEMP_PATH: ${{runner.temp}}/docs_release - REPO_COPY: ${{runner.temp}}/docs_release/ClickHouse - CLOUDFLARE_TOKEN: ${{secrets.CLOUDFLARE}} - ROBOT_CLICKHOUSE_SSH_KEY: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} run: | cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index ea19ebc9be7..cd7088e116c 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -25,17 +25,20 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck - env: - TEMP_PATH: ${{runner.temp}}/compatibility_check - REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -54,24 +57,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_release + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_release' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -81,7 +87,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -92,24 +98,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_asan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_asan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -119,7 +128,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -130,24 +139,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_ubsan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_ubsan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -157,7 +169,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -168,24 +180,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_tsan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_tsan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -195,7 +210,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -206,24 +221,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_msan + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_msan' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -233,7 +251,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -244,24 +262,27 @@ jobs: needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_debug + EOF - name: Download changed images uses: actions/download-artifact@v2 with: name: changed_images - path: ${{ runner.temp }}/images_path + path: ${{ env.IMAGES_PATH }} - name: Check out repository code uses: actions/checkout@v2 with: submodules: 'recursive' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NAME: 'package_debug' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -271,7 +292,7 @@ jobs: uses: actions/upload-artifact@v2 with: name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_NAME }}.json - name: Cleanup if: always() run: | @@ -291,17 +312,20 @@ jobs: - BuilderDebDebug runs-on: [self-hosted, style-checker] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse build check (actions) + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder - env: - TEMP_PATH: ${{runner.temp}}/report_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'ClickHouse build check (actions)' run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -320,19 +344,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -349,19 +376,22 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -378,19 +408,22 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -407,19 +440,22 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -436,19 +472,22 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -465,19 +504,22 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -497,19 +539,22 @@ jobs: needs: [BuilderDebRelease] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -526,19 +571,22 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (address, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -555,19 +603,22 @@ jobs: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -584,19 +635,22 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -613,19 +667,22 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -642,19 +699,22 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -674,18 +734,21 @@ jobs: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (address, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (address, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -706,18 +769,21 @@ jobs: # but such servers almost unavailable as spot instances. runs-on: [self-hosted, func-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (thread, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (thread, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -734,18 +800,21 @@ jobs: needs: [BuilderDebMsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (memory, actions) + REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (memory, actions)' - REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -762,18 +831,21 @@ jobs: needs: [BuilderDebUBsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_undefined + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (undefined, actions) + REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_undefined - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (undefined, actions)' - REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -790,18 +862,21 @@ jobs: needs: [BuilderDebDebug] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (debug, actions) + REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (debug, actions)' - REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -821,18 +896,21 @@ jobs: needs: [BuilderDebAsan, FunctionalStatelessTestAsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -849,18 +927,21 @@ jobs: needs: [BuilderDebTsan, FunctionalStatelessTestTsan] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH @@ -877,18 +958,21 @@ jobs: needs: [BuilderDebRelease, FunctionalStatelessTestRelease] runs-on: [self-hosted, stress-tester] steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + EOF - name: Download json reports uses: actions/download-artifact@v2 with: - path: ${{runner.temp}}/reports_dir + path: ${{ env.REPORTS_PATH }} - name: Check out repository code uses: actions/checkout@v2 - name: Integration test - env: - TEMP_PATH: ${{runner.temp}}/integration_tests_release - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Integration tests (release, actions)' - REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse run: | sudo rm -fr $TEMP_PATH mkdir -p $TEMP_PATH From ac8bbdc7be54b7f9fca0906580faf521be3808f2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 10 Dec 2021 13:16:19 +0100 Subject: [PATCH 181/358] Clean ccache on builders --- .github/workflows/backport_branches.yml | 8 +++---- .github/workflows/main.yml | 30 ++++++++++++------------- .github/workflows/master.yml | 28 +++++++++++------------ .github/workflows/release_branches.yml | 12 +++++----- 4 files changed, 39 insertions(+), 39 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 5062cd36a73..68ee66fb014 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -90,7 +90,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebAsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -131,7 +131,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebTsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -172,7 +172,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebDebug: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -213,7 +213,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH ############################################################################################ ##################################### BUILD REPORTER ####################################### ############################################################################################ diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 97be3cc1960..ef9a2372537 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -88,7 +88,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH PVSCheck: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, func-tester] @@ -257,7 +257,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinRelease: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -298,7 +298,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebAsan: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -339,7 +339,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebUBsan: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -380,7 +380,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebTsan: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -421,7 +421,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebMsan: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -462,7 +462,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebDebug: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -503,7 +503,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH ########################################################################################## ##################################### SPECIAL BUILDS ##################################### ########################################################################################## @@ -547,7 +547,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinTidy: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -588,7 +588,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinDarwin: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -629,7 +629,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinAarch64: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -670,7 +670,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinFreeBSD: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -711,7 +711,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinDarwinAarch64: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -752,7 +752,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinPPC64: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -793,7 +793,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH ############################################################################################ ##################################### BUILD REPORTER ####################################### ############################################################################################ diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 0249784a8da..754249ccaa8 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -187,7 +187,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinRelease: needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -229,7 +229,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebAsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -270,7 +270,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebUBsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -311,7 +311,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebTsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -352,7 +352,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebMsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -393,7 +393,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebDebug: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -434,7 +434,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH ########################################################################################## ##################################### SPECIAL BUILDS ##################################### ########################################################################################## @@ -479,7 +479,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinTidy: needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -521,7 +521,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinDarwin: needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -563,7 +563,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinAarch64: needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -605,7 +605,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinFreeBSD: needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -647,7 +647,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinDarwinAarch64: needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -689,7 +689,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderBinPPC64: needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -731,7 +731,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH ############################################################################################ ##################################### BUILD REPORTER ####################################### ############################################################################################ diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index cd7088e116c..7d20d8edbe2 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -93,7 +93,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebAsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -134,7 +134,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebUBsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -175,7 +175,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebTsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -216,7 +216,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebMsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -257,7 +257,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebDebug: needs: [DockerHubPush] runs-on: [self-hosted, builder] @@ -298,7 +298,7 @@ jobs: run: | docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH + sudo rm -fr $TEMP_PATH $CACHES_PATH ############################################################################################ ##################################### BUILD REPORTER ####################################### ############################################################################################ From 5430f1a4134f095ad2181e0850a8389e64dd77e9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 16:26:46 +0300 Subject: [PATCH 182/358] Update pr_info.py --- tests/ci/pr_info.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index b1775f0fc6c..c47518aea35 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -71,7 +71,7 @@ class PRInfo: self.user_orgs = set(org['id'] for org in response_json) self.diff_url = github_event['pull_request']['diff_url'] - elif 'commits' in github_event: + elif 'commits' in github_event: # push (commit) event self.sha = github_event['after'] pull_request = get_pr_for_commit(self.sha, github_event['ref']) repo_prefix = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}" From 8d6bea91e8684864c8ae32f0ab5cb43ad6e94a19 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 16:35:53 +0300 Subject: [PATCH 183/358] Update pr_info.py --- tests/ci/pr_info.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index c47518aea35..b1775f0fc6c 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -71,7 +71,7 @@ class PRInfo: self.user_orgs = set(org['id'] for org in response_json) self.diff_url = github_event['pull_request']['diff_url'] - elif 'commits' in github_event: # push (commit) event + elif 'commits' in github_event: self.sha = github_event['after'] pull_request = get_pr_for_commit(self.sha, github_event['ref']) repo_prefix = f"{GITHUB_SERVER_URL}/{GITHUB_REPOSITORY}" From 9f0a2b348b46145b7be82614e7cc614dd33e1b42 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 17:50:51 +0300 Subject: [PATCH 184/358] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index d10351f312c..bb69d48e0ee 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit d10351f312c1ae1ca3fdda433693dfbef3acfece +Subproject commit bb69d48e0ee35c87a0f19e509a09a914f71f0cff From 2147658432591ef0fd5e1d139f8a7b217beb475f Mon Sep 17 00:00:00 2001 From: Dmitrii Mokhnatkin Date: Fri, 10 Dec 2021 15:44:30 +0300 Subject: [PATCH 185/358] Proper handler for apache arrow column duplication --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index dea753b14e3..fee5c77f9ec 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -59,11 +59,12 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; + extern const int DUPLICATE_COLUMN; + extern const int THERE_IS_NO_COLUMN; + extern const int UNKNOWN_EXCEPTION; extern const int UNKNOWN_TYPE; extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE; - extern const int THERE_IS_NO_COLUMN; - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_EXCEPTION; } @@ -519,9 +520,11 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) { NameToColumnPtr name_to_column_ptr; - for (const auto& column_name : table->ColumnNames()) + for (const auto & column_name : table->ColumnNames()) { std::shared_ptr arrow_column = table->GetColumnByName(column_name); + if (!arrow_column) + throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Column '{}' is duplicated", column_name); name_to_column_ptr[column_name] = arrow_column; } From 2c46ede8bbb483c6c9c63b51f824c787f91fc176 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 15 Dec 2021 16:52:15 +0300 Subject: [PATCH 186/358] Add test arrow_duplicate_columns --- .../02147_arrow_duplicate_columns.reference | 1 + .../02147_arrow_duplicate_columns.sh | 31 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/02147_arrow_duplicate_columns.reference create mode 100755 tests/queries/0_stateless/02147_arrow_duplicate_columns.sh diff --git a/tests/queries/0_stateless/02147_arrow_duplicate_columns.reference b/tests/queries/0_stateless/02147_arrow_duplicate_columns.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02147_arrow_duplicate_columns.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh b/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh new file mode 100755 index 00000000000..11c1522d10b --- /dev/null +++ b/tests/queries/0_stateless/02147_arrow_duplicate_columns.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +set -e + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +# Reproduce GZDATA: +# +# ```python +# import pyarrow as pa +# data = [pa.array([1]), pa.array([2]), pa.array([3])] +# batch = pa.record_batch(data, names=['x', 'y', 'x']) +# with pa.ipc.new_file('data.arrow', batch.schema) as writer: +# writer.write_batch(batch) +# ``` +# +# ```bash +# cat data.arrow | gzip | base64 +# ``` + +GZDATA="H4sIAHTzuWEAA9VTuw3CMBB9+RCsyIULhFIwAC0SJQWZACkNi1CAxCCMwCCMQMEIKdkgPJ8PJbIIEiVPujuf73yfp6Rumt1+BXTEA4CDRwmLAhMYnogkpw96hjpXDWSUA2Wt/pU1mJz6GjO9k+eUI+UicSRbqvuX3BPlNsh1zDCcZypTOJ0xvF186GOYZ5ht9NrX8Pu12svDYq4bWqmKLEdFU+GNkmcr23oOzspNgh4FxmEiO3bvoriL4jJa1Bc/+OmghkcXeJU+lmwUwoALHHDbDfUSgVNfo9V3T7U9Pz3++bswDNbyD7wAxr434AoDAAA=" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS t1" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE t1 ( x Int64, y Int64, z Int64 ) ENGINE = Memory" + +echo ${GZDATA} | base64 --decode | gunzip | ${CLICKHOUSE_CLIENT} -q "INSERT INTO t1 FORMAT Arrow" 2>&1 | grep -qF "DUPLICATE_COLUMN" && echo 'OK' || echo 'FAIL' ||: + +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t1" From 6799ce2162c9ea3d8243b57581d154c7edfe808f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 18:46:40 +0300 Subject: [PATCH 187/358] Add retries to curl --- docker/test/performance-comparison/entrypoint.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 047c64406c3..6d4f1222432 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -55,7 +55,7 @@ function find_reference_sha ) for path in "${urls_to_try[@]}" do - if curl --fail --head "$path" + if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$path" then found="$path" break @@ -76,7 +76,7 @@ chmod 777 workspace output cd workspace # Download the package for the version we are going to test. -if curl --fail --head "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" +if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" then right_path="$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" fi From 6f2543b9439668ef00ac111ab97ba762b2faa2b1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 18:50:11 +0300 Subject: [PATCH 188/358] Better fix --- docker/test/performance-comparison/entrypoint.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 6d4f1222432..0434400d50a 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -55,7 +55,7 @@ function find_reference_sha ) for path in "${urls_to_try[@]}" do - if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$path" + if curl --fail --retry-all-errors --retry 5 --retry-delay 1 --retry-max-time 15 --head "$path" then found="$path" break @@ -76,7 +76,7 @@ chmod 777 workspace output cd workspace # Download the package for the version we are going to test. -if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" +if curl --fail --retry-all-errors --retry 5 --retry-delay 1 --retry-max-time 15 --head "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" then right_path="$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" fi From 43fa39cec183a724a3e4611dd161df64354163f0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 15 Dec 2021 12:16:49 -0400 Subject: [PATCH 189/358] Update group-by.md --- docs/ru/sql-reference/statements/select/group-by.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/ru/sql-reference/statements/select/group-by.md b/docs/ru/sql-reference/statements/select/group-by.md index 8bc1b765ad3..27a9d67cded 100644 --- a/docs/ru/sql-reference/statements/select/group-by.md +++ b/docs/ru/sql-reference/statements/select/group-by.md @@ -203,6 +203,9 @@ SELECT year, month, day, count(*) FROM t GROUP BY year, month, day WITH CUBE; - Ð’ `Pretty*` форматах, Ñтрока выводитÑÑ Ð² виде отдельной таблицы поÑле оÑновного результата. - Ð’ других форматах она не доÑтупна. +!!! note "Примечание" + totals выводитÑÑ Ñ‚Ð¾Ð»ÑŒÐºÐ¾ в результатах запроÑов `SELECT`, и не вывоводитÑÑ Ð² `INSERT INTO ... SELECT`. + При иÑпользовании Ñекции [HAVING](having.md) поведение `WITH TOTALS` контролируетÑÑ Ð½Ð°Ñтройкой `totals_mode`. ### ÐаÑтройка обработки итогов {#configuring-totals-processing} From 41fc080c041374976147246717c7c663cceaa48d Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 15 Dec 2021 12:18:50 -0400 Subject: [PATCH 190/358] Update group-by.md --- docs/en/sql-reference/statements/select/group-by.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/statements/select/group-by.md b/docs/en/sql-reference/statements/select/group-by.md index 26dd51d806d..969a39ce51f 100644 --- a/docs/en/sql-reference/statements/select/group-by.md +++ b/docs/en/sql-reference/statements/select/group-by.md @@ -206,6 +206,9 @@ This extra row is only produced in `JSON*`, `TabSeparated*`, and `Pretty*` forma - In `Pretty*` formats, the row is output as a separate table after the main result. - In the other formats it is not available. +!!! note "Note" + totals is output in the results of `SELECT` queries, and is not output in `INSERT INTO ... SELECT`. + `WITH TOTALS` can be run in different ways when [HAVING](../../../sql-reference/statements/select/having.md) is present. The behavior depends on the `totals_mode` setting. ### Configuring Totals Processing {#configuring-totals-processing} From f367634271cb715ece3577e2a4377b35b5855ed4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 15 Dec 2021 17:24:58 +0100 Subject: [PATCH 191/358] Do not use scalar cache when processing MV blocks --- .../Transforms/buildPushingToViewsChain.cpp | 2 ++ .../02139_MV_with_scalar_subquery.reference | 4 ++++ .../02139_MV_with_scalar_subquery.sql | 24 +++++++++++++++++++ 3 files changed, 30 insertions(+) create mode 100644 tests/queries/0_stateless/02139_MV_with_scalar_subquery.reference create mode 100644 tests/queries/0_stateless/02139_MV_with_scalar_subquery.sql diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index 503f34593c7..ff5296029c7 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -397,6 +397,8 @@ static void process(Block & block, ViewRuntimeData & view, const ViewsData & vie /// but it will contain single block (that is INSERT-ed into main table). /// InterpreterSelectQuery will do processing of alias columns. auto local_context = Context::createCopy(context); + /// Remove the query context to avoid caching and reusing the result of scalar subqueries between runs with different blocks + local_context->setQueryContext(nullptr); local_context->addViewSource(StorageValues::create( views_data.source_storage_id, views_data.source_metadata_snapshot->getColumns(), diff --git a/tests/queries/0_stateless/02139_MV_with_scalar_subquery.reference b/tests/queries/0_stateless/02139_MV_with_scalar_subquery.reference new file mode 100644 index 00000000000..51cfca81ddb --- /dev/null +++ b/tests/queries/0_stateless/02139_MV_with_scalar_subquery.reference @@ -0,0 +1,4 @@ +2000 +2 +1500 0 1499 1500 0 1499 +500 1500 1999 500 1500 1999 diff --git a/tests/queries/0_stateless/02139_MV_with_scalar_subquery.sql b/tests/queries/0_stateless/02139_MV_with_scalar_subquery.sql new file mode 100644 index 00000000000..f0285bbec3d --- /dev/null +++ b/tests/queries/0_stateless/02139_MV_with_scalar_subquery.sql @@ -0,0 +1,24 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/9587#issuecomment-944431385 + +CREATE TABLE source (a Int32) ENGINE=MergeTree() ORDER BY tuple(); +CREATE TABLE source_null AS source ENGINE=Null; +CREATE TABLE dest_a (count UInt32, min Int32, max Int32, count_subquery Int32, min_subquery Int32, max_subquery Int32) ENGINE=MergeTree() ORDER BY tuple(); + +CREATE MATERIALIZED VIEW mv_null TO source_null AS SELECT * FROM source; +CREATE MATERIALIZED VIEW mv_a to dest_a AS +SELECT + count() AS count, + min(a) AS min, + max(a) AS max, + (SELECT count() FROM source_null) AS count_subquery, + (SELECT min(a) FROM source_null) AS min_subquery, + (SELECT max(a) FROM source_null) AS max_subquery +FROM source_null +GROUP BY count_subquery, min_subquery, max_subquery; + + +INSERT INTO source SELECT number FROM numbers(2000) SETTINGS min_insert_block_size_rows=1500, max_insert_block_size=1500; + +SELECT count() FROM source; +SELECT count() FROM dest_a; +SELECT * from dest_a ORDER BY count DESC; From c61277fbac7581a4d9fd58a6f81e4e65e71de1d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 15 Dec 2021 17:45:07 +0100 Subject: [PATCH 192/358] Move the logic to the saving part instead This way we can reuse scalars from other queries (non MVs) and leave the door open for future improvements --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +++- src/Processors/Transforms/buildPushingToViewsChain.cpp | 2 -- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index c8f48f2ed1f..451d4fda262 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -386,7 +386,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( query.setFinal(); /// Save scalar sub queries's results in the query context - if (!options.only_analyze && context->hasQueryContext()) + /// But discard them if the Storage has been modified + /// In an ideal situation we would only discard the scalars affected by the storage change + if (!options.only_analyze && context->hasQueryContext() && !context->getViewSource()) for (const auto & it : syntax_analyzer_result->getScalars()) context->getQueryContext()->addScalar(it.first, it.second); diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index ff5296029c7..503f34593c7 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -397,8 +397,6 @@ static void process(Block & block, ViewRuntimeData & view, const ViewsData & vie /// but it will contain single block (that is INSERT-ed into main table). /// InterpreterSelectQuery will do processing of alias columns. auto local_context = Context::createCopy(context); - /// Remove the query context to avoid caching and reusing the result of scalar subqueries between runs with different blocks - local_context->setQueryContext(nullptr); local_context->addViewSource(StorageValues::create( views_data.source_storage_id, views_data.source_metadata_snapshot->getColumns(), From 10cbdcd34f465824eb8f43de174cf02ad95509e2 Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Thu, 16 Dec 2021 01:15:23 +0800 Subject: [PATCH 193/358] fix docs anchor --- website/css/main.css | 2 +- website/src/scss/_variables.scss | 2 ++ website/src/scss/components/_navbar.scss | 33 ++++++++++++++++++++++-- website/templates/docs/content.html | 2 +- 4 files changed, 35 insertions(+), 4 deletions(-) diff --git a/website/css/main.css b/website/css/main.css index 0e36340c25c..56230529a11 100644 --- a/website/css/main.css +++ b/website/css/main.css @@ -1 +1 @@ -@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-group{position:relative}form .form-group.is-select:before{border-left:6px solid transparent;border-right:6px solid transparent;border-top:8px solid #6c757d;content:"";display:block;position:absolute;right:33px;top:calc(50% - 4px);z-index:10}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none;padding-right:24px;white-space:pre-wrap}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.severity-table th{background:#f1f6f9;font-size:.875rem;padding:8px 16px}.severity-table td{border-top:1px solid #d6dbdf;padding:16px}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file +@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-group{position:relative}form .form-group.is-select:before{border-left:6px solid transparent;border-right:6px solid transparent;border-top:8px solid #6c757d;content:"";display:block;position:absolute;right:33px;top:calc(50% - 4px);z-index:10}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none;padding-right:24px;white-space:pre-wrap}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}@media screen and (min-width:616px){.navbar.py-1+div .anchor-fixer :target{scroll-margin-top:62px}}@media screen and (min-width:616px){.navbar.py-2+div .anchor-fixer :target{scroll-margin-top:78px}}@media screen and (min-width:616px){.navbar.py-3+div .anchor-fixer :target{scroll-margin-top:94px}}@media screen and (min-width:616px){.navbar.py-4+div .anchor-fixer :target{scroll-margin-top:110px}}@media screen and (min-width:616px){.navbar.py-5+div .anchor-fixer :target{scroll-margin-top:126px}}@media screen and (min-width:616px){.navbar.py-6+div .anchor-fixer :target{scroll-margin-top:142px}}@media screen and (min-width:616px){.navbar.py-7+div .anchor-fixer :target{scroll-margin-top:158px}}@media screen and (min-width:616px){.navbar.py-8+div .anchor-fixer :target{scroll-margin-top:174px}}@media screen and (max-width:615.98px){.navbar+div .anchor-fixer :target{scroll-margin-top:73px}}@media screen and (max-width:399.98px){.navbar+div .anchor-fixer :target{scroll-margin-top:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.severity-table th{background:#f1f6f9;font-size:.875rem;padding:8px 16px}.severity-table td{border-top:1px solid #d6dbdf;padding:16px}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file diff --git a/website/src/scss/_variables.scss b/website/src/scss/_variables.scss index d511c757055..55b06ac8409 100644 --- a/website/src/scss/_variables.scss +++ b/website/src/scss/_variables.scss @@ -289,6 +289,8 @@ $nav-tabs-link-active-border-color: $gray-700; $navbar-padding-y: 24px; $navbar-padding-x: 0; +$navbar-nav-height: 46px; +$navbar-height-xl: 80px; // Cards diff --git a/website/src/scss/components/_navbar.scss b/website/src/scss/components/_navbar.scss index 53a834d2ed7..ca6bc52630b 100644 --- a/website/src/scss/components/_navbar.scss +++ b/website/src/scss/components/_navbar.scss @@ -52,7 +52,7 @@ &-nav { align-items: center; - height: 46px; + height: $navbar-nav-height; } .nav-item:not(:last-child) { @@ -131,6 +131,35 @@ } @media screen and (max-width: 399.98px) { - height: 80px; + height: $navbar-height-xl; + } +} + + +.navbar { + @for $i from 1 through 8 { + &.py-#{$i} { + + div { + .anchor-fixer { + :target { + @media screen and (min-width: 616px) { + scroll-margin-top: $navbar-nav-height + $spacer * $i * 2; + } + } + } + } + } + } + + div { + .anchor-fixer { + :target { + @media screen and (max-width: 615.98px) { + scroll-margin-top: 73px; + } + @media screen and (max-width: 399.98px) { + scroll-margin-top: $navbar-height-xl; + } + } + } } } diff --git a/website/templates/docs/content.html b/website/templates/docs/content.html index 3f4db728e99..c2835dd1f39 100644 --- a/website/templates/docs/content.html +++ b/website/templates/docs/content.html @@ -1,4 +1,4 @@ -

+
{% if not single_page %} {% set ancestors = page.ancestors|reverse|list %} From 1b021369410221de7140b6897b13e4c1a8382beb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 15 Dec 2021 21:19:29 +0300 Subject: [PATCH 194/358] fix skipping of some mutations --- src/Storages/MergeTree/MergeTreeData.cpp | 21 ++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.h | 1 + src/Storages/StorageMergeTree.cpp | 13 ++++++++--- src/Storages/StorageReplicatedMergeTree.cpp | 22 ------------------- src/Storages/StorageReplicatedMergeTree.h | 1 - ...02004_invalid_partition_mutation_stuck.sql | 2 +- 6 files changed, 33 insertions(+), 27 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 9e600286214..1b7be8ca98d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3743,6 +3743,27 @@ std::unordered_set MergeTreeData::getPartitionIDsFromQuery(const ASTs & return partition_ids; } +std::set MergeTreeData::getPartitionIdsAffectedByCommands( + const MutationCommands & commands, ContextPtr query_context) const +{ + std::set affected_partition_ids; + + for (const auto & command : commands) + { + if (!command.partition) + { + affected_partition_ids.clear(); + break; + } + + affected_partition_ids.insert( + getPartitionIDFromQuery(command.partition, query_context) + ); + } + + return affected_partition_ids; +} + MergeTreeData::DataPartsVector MergeTreeData::getDataPartsVector( const DataPartStates & affordable_states, DataPartStateVector * out_states, bool require_projection_parts) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 5a3dda785a0..380c2f4f4c5 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -688,6 +688,7 @@ public: /// For ATTACH/DETACH/DROP PARTITION. String getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr context) const; std::unordered_set getPartitionIDsFromQuery(const ASTs & asts, ContextPtr context) const; + std::set getPartitionIdsAffectedByCommands(const MutationCommands & commands, ContextPtr query_context) const; /// Extracts MergeTreeData of other *MergeTree* storage /// and checks that their structure suitable for ALTER TABLE ATTACH PARTITION FROM diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f83ab0c3f9e..03ac27d0e46 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -508,6 +508,9 @@ void StorageMergeTree::waitForMutation(Int64 version, const String & file_name) void StorageMergeTree::mutate(const MutationCommands & commands, ContextPtr query_context) { + /// Validate partition IDs (if any) before starting mutation + getPartitionIdsAffectedByCommands(commands, query_context); + String mutation_file_name; Int64 version = startMutation(commands, mutation_file_name); @@ -898,6 +901,7 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( auto commands = MutationCommands::create(); size_t current_ast_elements = 0; + auto last_mutation_to_apply = mutations_end_it; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { size_t commands_size = 0; @@ -934,7 +938,8 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( MergeTreeMutationEntry & entry = it->second; entry.latest_fail_time = time(nullptr); entry.latest_fail_reason = getCurrentExceptionMessage(false); - continue; + /// NOTE we should not skip mutations, because exception may be retryable (e.g. MEMORY_LIMIT_EXCEEDED) + break; } } @@ -943,8 +948,10 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( current_ast_elements += commands_size; commands->insert(commands->end(), it->second.commands.begin(), it->second.commands.end()); + last_mutation_to_apply = it; } + assert(commands->empty() == (last_mutation_to_apply == mutations_end_it)); if (!commands->empty()) { bool is_partition_affected = false; @@ -969,13 +976,13 @@ std::shared_ptr StorageMergeTree::selectPartsToMutate( /// Shall not create a new part, but will do that later if mutation with higher version appear. /// This is needed in order to not produce excessive mutations of non-related parts. auto block_range = std::make_pair(part->info.min_block, part->info.max_block); - updated_version_by_block_range[block_range] = current_mutations_by_version.rbegin()->first; + updated_version_by_block_range[block_range] = last_mutation_to_apply->first; were_some_mutations_for_some_parts_skipped = true; continue; } auto new_part_info = part->info; - new_part_info.mutation = current_mutations_by_version.rbegin()->first; + new_part_info.mutation = last_mutation_to_apply->first; future_part->parts.push_back(part); future_part->part_info = new_part_info; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ca877d8a72d..60535ed5211 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4532,28 +4532,6 @@ bool StorageReplicatedMergeTree::executeMetadataAlter(const StorageReplicatedMer } -std::set StorageReplicatedMergeTree::getPartitionIdsAffectedByCommands( - const MutationCommands & commands, ContextPtr query_context) const -{ - std::set affected_partition_ids; - - for (const auto & command : commands) - { - if (!command.partition) - { - affected_partition_ids.clear(); - break; - } - - affected_partition_ids.insert( - getPartitionIDFromQuery(command.partition, query_context) - ); - } - - return affected_partition_ids; -} - - PartitionBlockNumbersHolder StorageReplicatedMergeTree::allocateBlockNumbersInAffectedPartitions( const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const { diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index bcd364df30e..6861d89f070 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -717,7 +717,6 @@ private: std::unique_ptr getDefaultSettings() const override; - std::set getPartitionIdsAffectedByCommands(const MutationCommands & commands, ContextPtr query_context) const; PartitionBlockNumbersHolder allocateBlockNumbersInAffectedPartitions( const MutationCommands & commands, ContextPtr query_context, const zkutil::ZooKeeperPtr & zookeeper) const; diff --git a/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql b/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql index 481a5565095..71c8b9af652 100644 --- a/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql +++ b/tests/queries/0_stateless/02004_invalid_partition_mutation_stuck.sql @@ -28,6 +28,6 @@ PARTITION BY p ORDER BY t SETTINGS number_of_free_entries_in_pool_to_execute_mutation=0; INSERT INTO data VALUES (1, now()); -ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID 'NO_SUCH_PART'; -- { serverError 341 } +ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID 'NO_SUCH_PART'; -- { serverError 248 } ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID '1'; ALTER TABLE data MATERIALIZE INDEX idx IN PARTITION ID '2'; From 2c87e30d6de2288f85f50c44556bee93081cedde Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 15 Dec 2021 18:31:55 +0000 Subject: [PATCH 195/358] Done --- cmake/find/blob_storage.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/find/blob_storage.cmake b/cmake/find/blob_storage.cmake index 913d1d49bcf..4df25abb1ab 100644 --- a/cmake/find/blob_storage.cmake +++ b/cmake/find/blob_storage.cmake @@ -1,6 +1,6 @@ option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY "Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)" - ON) + ${ENABLE_LIBRARIES}) if (USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) set(USE_AZURE_BLOB_STORAGE 1) From c9b03eaf64f460bf0ccc5e9dc0036e4ccc5d45f2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 15 Dec 2021 22:28:21 +0300 Subject: [PATCH 196/358] Update entrypoint.sh --- docker/test/performance-comparison/entrypoint.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 0434400d50a..6d4f1222432 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -55,7 +55,7 @@ function find_reference_sha ) for path in "${urls_to_try[@]}" do - if curl --fail --retry-all-errors --retry 5 --retry-delay 1 --retry-max-time 15 --head "$path" + if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$path" then found="$path" break @@ -76,7 +76,7 @@ chmod 777 workspace output cd workspace # Download the package for the version we are going to test. -if curl --fail --retry-all-errors --retry 5 --retry-delay 1 --retry-max-time 15 --head "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" +if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" then right_path="$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" fi From 5c9dc4d79f13f87f61bf552840a880a54e97afd3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 16 Dec 2021 00:37:14 +0300 Subject: [PATCH 197/358] Ping CI From 7583c8007e2d0822572e07206d05da4ed538f430 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 16 Dec 2021 00:38:46 +0300 Subject: [PATCH 198/358] Some review fixes --- src/Common/getTableOverride.cpp | 27 +++++++++++++++++++ src/Common/getTableOverride.h | 8 ++++++ .../MySQL/InterpretersMySQLDDLQuery.cpp | 3 ++- src/Parsers/ASTTableOverrides.cpp | 18 ------------- src/Parsers/ASTTableOverrides.h | 2 -- .../PostgreSQLReplicationHandler.cpp | 11 ++++---- .../StorageMaterializedPostgreSQL.cpp | 13 ++++++--- .../test.py | 4 +-- 8 files changed, 55 insertions(+), 31 deletions(-) create mode 100644 src/Common/getTableOverride.cpp create mode 100644 src/Common/getTableOverride.h diff --git a/src/Common/getTableOverride.cpp b/src/Common/getTableOverride.cpp new file mode 100644 index 00000000000..fedcad85b3f --- /dev/null +++ b/src/Common/getTableOverride.cpp @@ -0,0 +1,27 @@ +#include "getTableOverride.h" + +#include +#include +#include +#include + +namespace DB +{ + +ASTPtr tryGetTableOverride(const String & mapped_database, const String & table) +{ + if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database)) + { + auto create_query = database_ptr->getCreateDatabaseQuery(); + if (auto create_database_query = create_query->as()) + { + if (create_database_query->table_overrides) + { + return create_database_query->table_overrides->tryGetTableOverride(table); + } + } + } + return nullptr; +} + +} diff --git a/src/Common/getTableOverride.h b/src/Common/getTableOverride.h new file mode 100644 index 00000000000..1a0a15e6fe2 --- /dev/null +++ b/src/Common/getTableOverride.h @@ -0,0 +1,8 @@ +#pragma once +#include +#include + +namespace DB +{ +ASTPtr tryGetTableOverride(const String & mapped_database, const String & table); +} diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 798283c6f50..e5c4a85dc27 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -519,7 +520,7 @@ ASTs InterpreterCreateImpl::getRewrittenQueries( rewritten_query->set(rewritten_query->storage, storage); rewritten_query->set(rewritten_query->columns_list, columns); - if (auto table_override = ASTTableOverride::tryGetTableOverride(mapped_to_database, create_query.table)) + if (auto table_override = tryGetTableOverride(mapped_to_database, create_query.table)) { auto * override_ast = table_override->as(); override_ast->applyToCreateTableQuery(rewritten_query.get()); diff --git a/src/Parsers/ASTTableOverrides.cpp b/src/Parsers/ASTTableOverrides.cpp index ee351ab3eb3..0270c2cad82 100644 --- a/src/Parsers/ASTTableOverrides.cpp +++ b/src/Parsers/ASTTableOverrides.cpp @@ -6,8 +6,6 @@ #include #include #include -#include -#include namespace DB { @@ -73,22 +71,6 @@ void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState settings.ostr << nl_or_nothing << ')'; } -ASTPtr ASTTableOverride::tryGetTableOverride(const String & mapped_database, const String & table) -{ - if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database)) - { - auto create_query = database_ptr->getCreateDatabaseQuery(); - if (auto create_database_query = create_query->as()) - { - if (create_database_query->table_overrides) - { - return create_database_query->table_overrides->tryGetTableOverride(table); - } - } - } - return nullptr; -} - void ASTTableOverride::applyToCreateTableQuery(ASTCreateQuery * create_query) const { if (columns) diff --git a/src/Parsers/ASTTableOverrides.h b/src/Parsers/ASTTableOverrides.h index a34b3619b2e..7a84ed25121 100644 --- a/src/Parsers/ASTTableOverrides.h +++ b/src/Parsers/ASTTableOverrides.h @@ -27,8 +27,6 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void applyToCreateTableQuery(ASTCreateQuery * create_query) const; - - static ASTPtr tryGetTableOverride(const String & mapped_database, const String & table); }; /// List of table overrides, for example: diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index d706615b762..dcd64314028 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,5 +1,7 @@ #include "PostgreSQLReplicationHandler.h" +#include +#include #include #include #include @@ -8,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -281,8 +282,8 @@ ASTPtr PostgreSQLReplicationHandler::getCreateNestedTableQuery(StorageMaterializ if (!table_structure) throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get PostgreSQL table structure"); - auto table_override = ASTTableOverride::tryGetTableOverride(current_database_name, table_name); - return storage->getCreateNestedTableQuery(std::move(table_structure), table_override->as()); + auto table_override = tryGetTableOverride(current_database_name, table_name); + return storage->getCreateNestedTableQuery(std::move(table_structure), table_override ? table_override->as() : nullptr); } @@ -300,8 +301,8 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & query_str = fmt::format("SELECT * FROM {}", quoted_name); LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); - auto table_override = ASTTableOverride::tryGetTableOverride(current_database_name, table_name); - materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name), table_override->as()); + auto table_override = tryGetTableOverride(current_database_name, table_name); + materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name), table_override ? table_override->as() : nullptr); auto nested_storage = materialized_storage->getNested(); auto insert = std::make_shared(); diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index a680792acad..ae9ade32ef0 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -2,29 +2,37 @@ #if USE_LIBPQXX #include + #include #include #include + #include +#include + #include #include #include #include #include + #include #include + #include #include #include #include -#include + #include #include #include + #include #include #include -#include + +#include namespace DB @@ -438,7 +446,6 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery( if (table_override && table_override->columns) { - table_override->applyToCreateTableQuery(create_table_query.get()); if (table_override->columns) { auto children = table_override->columns->children; diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 3bf494aa957..c8b63d8e667 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -582,9 +582,9 @@ def test_table_override(started_cluster): expected = "CREATE TABLE test_database.table_override\\n(\\n `key` Int32,\\n `value` UUID,\\n `_sign` Int8() MATERIALIZED 1,\\n `_version` UInt64() MATERIALIZED 1\\n)\\nENGINE = ReplacingMergeTree(_version)\\nORDER BY tuple(key)" assert(result.strip() == expected) time.sleep(5) - result = instance.query(f"select * from {materialized_database}.{table_name} order by key") + query = f"select * from {materialized_database}.{table_name} order by key" expected = instance.query(f"select * from {table_name} order by key") - assert(result == expected) + assert_eq_with_retry(instance, query, expected) drop_materialized_db() drop_postgres_table(cursor, table_name) From 6d82644094b2b0c0102881a6213d89b5084ff155 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 02:02:36 +0300 Subject: [PATCH 199/358] Update DatabaseMySQL.cpp --- src/Databases/MySQL/DatabaseMySQL.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index e4963d6755b..cc6d808a564 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -65,7 +65,7 @@ DatabaseMySQL::DatabaseMySQL( { try { - /// Test database is working fine and fetch tables. + /// Test that the database is working fine; it will also fetch tables. empty(); } catch (...) From 3506b12ccbf995945892e27701221577b99f5bb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Dec 2021 02:22:05 +0300 Subject: [PATCH 200/358] Remove recursive submodules --- .github/workflows/backport_branches.yml | 8 +++---- .github/workflows/main.yml | 32 ++++++++++++------------- .github/workflows/master.yml | 30 +++++++++++------------ .github/workflows/release_branches.yml | 12 +++++----- 4 files changed, 41 insertions(+), 41 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 68ee66fb014..5bcb0fa87b6 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -72,7 +72,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -113,7 +113,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -154,7 +154,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -195,7 +195,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index ef9a2372537..c27a19ce9b5 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -102,7 +102,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' - name: PVS Check run: | sudo rm -fr $TEMP_PATH @@ -198,7 +198,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -239,7 +239,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -280,7 +280,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -321,7 +321,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -362,7 +362,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -403,7 +403,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -444,7 +444,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -485,7 +485,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -529,7 +529,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -570,7 +570,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -611,7 +611,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -652,7 +652,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -693,7 +693,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -734,7 +734,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -775,7 +775,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 754249ccaa8..24ada19b80b 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -128,7 +128,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -169,7 +169,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -211,7 +211,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -252,7 +252,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -293,7 +293,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -334,7 +334,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -375,7 +375,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -416,7 +416,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -461,7 +461,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -503,7 +503,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -545,7 +545,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -587,7 +587,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -629,7 +629,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -671,7 +671,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -713,7 +713,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 7d20d8edbe2..dec0c4566b7 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -75,7 +75,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -116,7 +116,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -157,7 +157,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -198,7 +198,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -239,7 +239,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | @@ -280,7 +280,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | From 666c1973180ff591d255816d6189c82549f2c365 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 15 Dec 2021 20:40:42 -0400 Subject: [PATCH 201/358] Update CHANGELOG.md --- CHANGELOG.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 277f12431e3..9027cfb117a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -71,8 +71,8 @@ * Fix the issue that `LowCardinality` of `Int256` cannot be created. [#31832](https://github.com/ClickHouse/ClickHouse/pull/31832) ([alexey-milovidov](https://github.com/alexey-milovidov)). * Recreate `system.*_log` tables in case of different engine/partition_by. [#31824](https://github.com/ClickHouse/ClickHouse/pull/31824) ([Azat Khuzhin](https://github.com/azat)). * `MaterializedMySQL`: Fix issue with table named 'table'. [#31781](https://github.com/ClickHouse/ClickHouse/pull/31781) ([HÃ¥vard KvÃ¥len](https://github.com/havardk)). -* ClickHouse dictionary source: support named collections. Closes [#31705](https://github.com/ClickHouse/ClickHouse/issues/31705). [#31749](https://github.com/ClickHouse/ClickHouse/pull/31749) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Allow to use named collections configuration for Kafka and RabbitMQ engines (the same way as for other integration table engines). [#31691](https://github.com/ClickHouse/ClickHouse/pull/31691) ([Kseniia Sumarokova](https://github.com/kssenii)). +* ClickHouse dictionary source: support predefined connections. Closes [#31705](https://github.com/ClickHouse/ClickHouse/issues/31705). [#31749](https://github.com/ClickHouse/ClickHouse/pull/31749) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow to use predefined connections configuration for Kafka and RabbitMQ engines (the same way as for other integration table engines). [#31691](https://github.com/ClickHouse/ClickHouse/pull/31691) ([Kseniia Sumarokova](https://github.com/kssenii)). * Always re-render prompt while navigating history in clickhouse-client. This will improve usability of manipulating very long queries that don't fit on screen. [#31675](https://github.com/ClickHouse/ClickHouse/pull/31675) ([alexey-milovidov](https://github.com/alexey-milovidov)) (author: Amos Bird). * Add key bindings for navigating through history (instead of lines/history). [#31641](https://github.com/ClickHouse/ClickHouse/pull/31641) ([Azat Khuzhin](https://github.com/azat)). * Improve the `max_execution_time` checks. Fixed some cases when timeout checks do not happen and query could run too long. [#31636](https://github.com/ClickHouse/ClickHouse/pull/31636) ([Raúl Marín](https://github.com/Algunenano)). From bc4a2ca07305f817dbe72903fc68501d92ef8369 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Thu, 16 Dec 2021 09:25:38 +0800 Subject: [PATCH 202/358] Update KeeperStorage.cpp modify potential overflow sub. --- src/Coordination/KeeperStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index 93e4aecc1be..a770451a733 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -614,7 +614,7 @@ struct KeeperStorageSetRequestProcessor final : public KeeperStorageRequestProce value.stat.mzxid = zxid; value.stat.mtime = std::chrono::system_clock::now().time_since_epoch() / std::chrono::milliseconds(1); value.stat.dataLength = request.data.length(); - value.size_bytes += request.data.size() - value.data.size(); + value.size_bytes = value.size_bytes + request.data.size() - value.data.size(); value.data = request.data; }); From 8d2dfa82a6eaab86172941e6b980987c86799996 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Dec 2021 06:16:58 +0300 Subject: [PATCH 203/358] 21.12 blog post --- .../en/2021/clickhouse-v21.12-released.md | 263 ++++++++++++++++-- 1 file changed, 238 insertions(+), 25 deletions(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 555229ff843..d2303286c9e 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -1,60 +1,273 @@ --- -title: 'ClickHouse v21.12 Released' +title: 'What's New in ClickHouse 21.12' image: 'https://blog-images.clickhouse.com/en/2021/clickhouse-v21-12/featured.jpg' -date: '2021-12-13' -author: '[Rich Raposa](https://github.com/rfraposa), [Alexey Milovidov](https://github.com/alexey-milovidov)' +date: '2021-12-16' +author: '[Alexey Milovidov](https://github.com/alexey-milovidov)', '[Christoph Wurm](https://github.com/cwurm)' tags: ['company', 'community'] --- -We're continuing our monthly release cadence and blog updates at[ ClickHouse, Inc](https://clickhouse.com/blog/en/2021/clickhouse-inc/). Let's highlight some of these new exciting new capabilities in 21.12: +We're continuing our monthly release cadence. The 21.12 Christmas release includes 2460 new commits from 125 contributors, including 42 new contributors: -## Bool data type +Alex Cao, Amr Alaa, Andrey Torsunov, Constantine Peresypkin, Dmitriy Dorofeev, Egor O'Sten, Elykov Alexandr, Evgeny, Frank Chen, LB, Natasha Murashkina, Peignon Melvyn, Rich Raposa, Roman Chyrva, Roman, SuperDJY, Thom O'Connor, Timur Magomedov, Tom Risse, Tomáš Hromada, cfcz48, cgp, cms, cmsxbc, congbaoyangrou, dongyifeng, frank chen, freedomDR, jus1096, khamadiev, laurieliyang, leosunli, liyang830, loneylee, michael1589, msaf1980, p0ny, qieqieplus, spume, sunlisheng, yandd, zhanghuajie. -ClickHouse now natively supports a `bool` data type. Supported input values are “trueâ€/â€falseâ€, “TRUEâ€/â€FALSE†and “1â€/â€0â€. By default, the text representation for CSV/TSV output is “true†and “false†but can be adjusted to anything else using the settings `bool_true_representation` and `bool_false_representation` (for example, “yes†and “noâ€). +If you are wondering, this list is generated by the following command: + +``` +clickhouse-local --query " + SELECT arrayStringConcat(groupArray(s), ', ') + FROM file('contributors-21.12.txt', LineAsString, 's String') + WHERE s NOT IN ( + SELECT * + FROM file('contributors-21.11.txt', LineAsString, 's String')) + FORMAT TSVRaw" +``` + +And to list the contributors, you can always run the +``` +SELECT * FROM system.contributors +``` +query on your production server. + +Let's highlight some of the new capabilities in 21.12: + + +## ClickHouse Keeper is Feature Complete + +In 21.12 `clickhouse-keeper` started to support ["four letter commands"](https://zookeeper.apache.org/doc/r3.4.8/zookeeperAdmin.html#sc_zkCommands) for status and monitoring. This feature is contributed by **JackyWoo** and reviewed by **Alexander Sapin** (the author of ClickHouse Keeper). + +It was the only missing feature to implement. In this release, clickhouse-keeper is still considered in pre-production stage, but many companies already started to evaluate and use it as a replacement of ZooKeeper. You can also start using clickhouse-keeper in your testing environments and we will appreciate your feedback. + +ClickHouse Keeper development started in Sep 2020, more than a year ago. It was a long road, and most of the efforts were to ensure correctness and stability in unusual and exceptional scenarios. It is covered by [Jepsen](https://jepsen.io/) tests (including ZooKeeper tests and [new introduced tests](https://github.com/ClickHouse/ClickHouse/tree/master/tests/jepsen.clickhouse-keeper)), continuous randomized stress testing with ClickHouse functional and integration tests. It is started to be tested in Yandex Cloud and among our best friends. If you're pretending to be our best friend, you can also do it. **How does this help you?** -Native boolean data types exist today in other databases that are often integrated with ClickHouse, such as PostgreSQL. The `bool` data type in ClickHouse will make it more compatible with existing code and ease migration from other databases. +ClickHouse Keeper is a drop-in replacement for ZooKeeper. It implements ZooKeeper wire protocol and data model, but does it better. + +In contrast to ZooKeeper, there are no issues with zxid overflow or packet sizes. It has better memory usage and it does not require JVM tuning (because it does not use JVM). Logs and snapshots are compressed (about 10x typical) and checksummed. It can run as a separate process or directly inside clickhouse-server. You can use it with ClickHouse or with your Kafkas and Hadoops as well. + +[More info](http://presentations.clickhouse.tech/meetup54/keeper.pdf). -## Partitions for File, URL, and HDFS storage +## Partitions For INSERT INTO File, URL And HDFS Storages -When using the table engines file, url, and hdfs ClickHouse now supports partitions. When creating a table you can specify the partition key using the `PARTITION BY` clause e.g. `CREATE TABLE hits_files (<columns>) ENGINE=File(TabSeparated) PARTITION BY toYYYYMM(EventDate)`. +When using the table engines `File`, `URL`, and `HDFS` ClickHouse now supports partitions. When creating a table you can specify the partition key using the `PARTITION BY` clause e.g. `CREATE TABLE hits_files (...) ENGINE = File(TabSeparated) PARTITION BY toYYYYMM(EventDate)`. -Similarly, when exporting data from ClickHouse using the file, url, and hdfs table functions you can now specify that the data is to be partitioned into multiple files using a `PARTITION BY` clause. For example, `INSERT INTO TABLE FUNCTION file(’<path>/hits_{_partition_id}’, ‘TSV’, ‘<table_format>’) PARTITION BY toYYYYMM(EventDate) VALUES <values>’ will create as many files as there are unique month in the dataset. +Similarly, when exporting data from ClickHouse using the `file`, `url`, and `hdfs` table functions you can now specify that the data is to be partitioned into multiple files using a `PARTITION BY` clause. For example, `INSERT INTO TABLE FUNCTION file('path/hits_{_partition_id}', 'TSV', '<table_format>') PARTITION BY toYYYYMM(EventDate) VALUES <values>` will create as many files as there are unique months in the dataset. -The S3 table function has supported partitioned writes since ClickHouse 21.10. +The `s3` table function has supported partitioned writes since ClickHouse 21.10. **How does this help you?** -This feature makes using tables backed by files on local file systems and remote file systems accessed over HTTP and HDFS more convenient and flexible. Especially for large datasets you might not want all data to be in one big file. This feature allows you to split one table into smaller files any way you want e.g. by month. +If data is split into multiple files, then `SELECT` query will be automatically parallelized. Example: + +``` +SELECT user_id, count() FROM s3( + 'https://s3.us-east-2.amazonaws.com/.../*.csv.zstd', + '...', '...', + CSV, + 'user_id UInt64, ...') +``` + +You can even parallelize data processing across distributed compute cluster if you use `s3Cluster` table function: + +``` +SELECT user_id, count() FROM s3Cluster( + my_cluster, + 'https://s3.us-east-2.amazonaws.com/.../*.csv.zstd', + '...', + '...', CSV, + 'user_id UInt64, ...') +``` + +It can also be used for integrations with external data processing tools that consumes data from `s3`. -## Table constraints +## FROM INFILE in clickhouse-client now supports glob patterns and parallel reading -When creating tables, you can now (optionally) specify constraints. A constraint tells ClickHouse that a column has a specific relationship to another column in the same table. For example, a string column might store the prefix of another column. Then, when a select query is trying to calculate the prefix on the original column, ClickHouse will rewrite the query to use the prefix column. +Just write: -For now, constraints are implemented as assumptions, that is, ClickHouse does not validate that they are correct and will not reject new data that violates them. +``` +INSERT INTO my_table FROM INFILE '*.csv.gz' FORMAT CSV +``` -This feature is disabled by default. To turn it on, enable `optimize_using_constraints`, `optimize_substitute_columns` and/or `optimize_append_index`. +Glob patterns support `*`, `?` and `{n..m}` with `{1..10}` or (aligned) `{01..10}` forms. +This query will be automatically parallelized, it will also automatically detect compression format from file extension and decompress transparently. **How does this help you?** -Especially in large ClickHouse deployments with many complex tables it can be hard for users to always be up to date on the best way to query a given dataset. Constraints can help optimize queries without having to change the query structure itself. They can also make it easier to make changes to tables. For example, let’s say you have a table containing web requests and it includes a URL column that contains the full URL of each request. Many times, users will want to know the top level domain (.com, .co.uk, etc.), something ClickHouse provides the topLevelDomain function to calculate. If you discover that many people are using this function you might decide to create a new materialized column that pre-calculates the top level domain for each record. Rather than tell all your users to change their queries you can use a table constraint to tell ClickHouse that each time a user tries to call the topLevelDomain function the request should be rewritten to use the new materialized column. +Now you don't have to recall how to write parallel for loop in your command line shell. clickhouse-client will do everything for you, it works intuitively and fast. -## Read large remote files in chunks +## Support for INTERVAL operator inside WITH FILL modifier for ORDER BY clause -When reading large files in Parquet, ORC, and Arrow format using the s3, url, and hdfs table functions, ClickHouse will now automatically choose whether to read the entire file at once or read parts of it incrementally. This is now enabled by default and the setting `remote_read_min_bytes_for_seek` controls when to switch from reading it all to reading in chunks. The default is 1MiB. +What's the... WITH FILL modifier in ORDER BY clause? Just look at the example. + +``` +:) SELECT EventDate, count() FROM test.hits WHERE CounterID = 2841673 GROUP BY EventDate ORDER BY EventDate + +┌──EventDate─┬─count()─┠+│ 2014-03-17 │ 3 │ +│ 2014-03-19 │ 6 │ +│ 2014-03-21 │ 7 │ +│ 2014-03-22 │ 6 │ +└────────────┴─────────┘ +``` + +We have the report with Mar 17th, 19th, 21th, 22th. But Mar 18th and 20th are missing, because there is no data for these dates. +And this is how it works in all SQL databases. + +But ClickHouse also has quite unique and neat `WITH FILL` modifier for `ORDER BY clause`. + +You just write: +``` +SELECT EventDate, count() FROM test.hits WHERE CounterID = 2841673 GROUP BY EventDate +ORDER BY EventDate WITH FILL STEP 1 + +┌──EventDate─┬─count()─┠+│ 2014-03-17 │ 3 │ +│ 2014-03-18 │ 0 │ +│ 2014-03-19 │ 6 │ +│ 2014-03-20 │ 0 │ +│ 2014-03-21 │ 7 │ +│ 2014-03-22 │ 6 │ +└────────────┴─────────┘ +``` + +And missing data is automatically filled. + +You can also add `FROM` and `TO`: + +``` +ORDER BY date WITH FILL FROM '2014-03-01'::Date TO '2014-03-31'::Date STEP 1; +``` + +And it will automatically fill missing rows in the report. + +The STEP can be arbitrary number. But what to do if you want fill missing dates for report by months? You cannot just write STEP 30 or STEP 31 because months contain different number of days... + +Since ClickHouse version 21.12 you can do it like this: + +``` +ORDER BY EventDate WITH FILL STEP INTERVAL 1 MONTH +``` + +`INTERVAL` is the standard SQL operator, you can use SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER and YEAR. + +**How does this help you?** + +It allows to avoid preprocessing step for your reports. + + +## Add Support For "Identifier" Table and Database Query Parameters + +ClickHouse has support for parameterized queries. + +``` +SELECT uniq(user_id) FROM table WHERE website = {name:String} +``` + +It allows to safely substitute parameters without the risk of SQL injections: + +``` +curl https://clickhouse-server:8443/?param_name=upyachka -d 'SELECT uniq(user_id) FROM table WHERE website = {name:String}' +``` + +You can even create customized API handlers for clickhouse-server based on prepared queries. + +Since version 21.12 we introduce support for using parameters for tables and databases in your queries. This is implemented with `Identifier` table parameter: + +``` +SELECT uniq(user_id) FROM {tbl:Identifier} +``` + +**How does this help you?** + +Let ClickHouse do the heavy-lifting and keep your scripts safe and secure. + + +## Bool Data Type + +This feature is experimental in version 21.12. + +ClickHouse now natively supports a `Bool` data type. It allows to represent values as "true"/"false" during data import and export in text formats. It can also be adjusted to anything else using the settings `bool_true_representation` and `bool_false_representation` (for example, "yes" and "no"). + +**How does this help you?** + +Native boolean data types exist today in other databases that are often integrated with ClickHouse, such as PostgreSQL. The `Bool` data type in ClickHouse will make it more compatible with existing code and ease migration from other databases. + +Also it simplifies data ingestion from various text sources. + + +## Query Optimizations With Table Constraints + +This feature is [contributed](https://github.com/ClickHouse/ClickHouse/pull/18787) by **Nikita Vasilev**. Nikita is one of the most notable ClickHouse contributors. He started in 2019 by introducing data skipping indices into ClickHouse, then continued in 2020 with SSD-optimized key-value dictionaries and now contributed with the new advancements in the query optimizer. This feature is reviewed by **Anton Popov**. + +So, what optimizations? ClickHouse already allows to specify constraints for tables: + +``` +CREATE TABLE +( + URL String, + Domain String, + CONSTRAINT validate CHECK isValidUTF8(URL) AND length(URL) BETWEEN 10 AND 10000, + CONSTRAINT my_constraint CHECK Domain = domainWithoutWWW(URL) +) ... +``` + +Constraints are checked on INSERT. In this example we validate the URL and check that Domain column actually contains the domain of URL. + +Since version 21.12 constraints can also automatically optimize your queries! For example, if you write: + +``` +SELECT count() FROM hits WHERE domainWithoutWWW(URL) = 'ghe.clickhouse.tech' +``` + +The query can be automatically rewritten to: + +``` +SELECT count() FROM hits WHERE Domain = 'ghe.clickhouse.tech' +``` + +because `Domain` column is smaller, more compressable, will be faster to read and it does not require calculation of domain. +The only thing you need is to enable the `optimize_using_constraints` and `optimize_substitute_columns` settings. + +As a bonus, new type of constraints are introduced: `ASSUME`. + +``` +CONSTRAINT my_constraint ASSUME Domain = domainWithoutWWW(URL) +``` + +This type of constraint will not check anything on INSERT, but still use the assumption to optimize the queries. + +It can also do logical inference, simplify the conditions and remove the conditions that are proved to be satisfied by constraints. +It is controlled by `convert_query_to_cnf` setting. You can also enable `optimize_append_index` setting. With this setting ClickHouse will derive more consitions on the table primary key. + +The idea is so powerful that we cannot resist adding one more feature: *indices for hypothesis*. + +``` +INDEX my_index (a < b) TYPE hypothesis GRANULARITY 1 +``` + +The expression is checked and the result (true/false) is written as an index for query optimization. + +**How does this help you?** + +Especially in large ClickHouse deployments with many complex tables it can be hard for users to always be up to date on the best way to query a given dataset. Constraints can help optimize queries without having to change the query structure itself. They can also make it easier to make changes to tables. For example, let's say you have a table containing web requests and it includes a URL column that contains the full URL of each request. Many times, users will want to know the top level domain (.com, .co.uk, etc.), something ClickHouse provides the topLevelDomain function to calculate. If you discover that many people are using this function you might decide to create a new materialized column that pre-calculates the top level domain for each record. Rather than tell all your users to change their queries you can use a table constraint to tell ClickHouse that each time a user tries to call the topLevelDomain function the request should be rewritten to use the new materialized column. + + +## Read Large Remote Files In Chunks + +ClickHouse combines fast query engine and efficient data storage. It also allows to integrate external data sources for data import and export or even to process external datasets on the fly without the need for data import or preprocessing. + +When reading large files in `Parquet`, `ORC`, and `Arrow` format using the `s3`, `url`, and `hdfs` table functions, ClickHouse will now automatically choose whether to read the entire file at once or read parts of it incrementally. This is now enabled by default and the setting `remote_read_min_bytes_for_seek` controls when to switch from reading it all to reading in chunks. The default is 1MiB. + +`Parquet`, `ORC`, and `Arrow` are column-oriented formats (quite similar to ClickHouse Native format) and now we can read only requested columns even if they are being read from remote HTTP server with the `url` table function (range requests will be performed). **How does this help our ClickHouse Users?** -In previous versions, when reading files from remote locations with the s3, url, and hdfs table functions, ClickHouse would always read the entire file into memory. This works well when the files are small but will cause excessive memory usage or not work at all when the files are large. With this change, ClickHouse will read large files in chunks to keep memory usage in check and is now able to read even very large files. +In previous versions, when reading files in Arrow-based formats from remote locations with the `s3`, `url`, and `hdfs` table functions, ClickHouse would always read the entire file into memory. This works well when the files are small but will cause excessive memory usage or not work at all when the files are large. With this change, ClickHouse will read large files in chunks to keep memory usage in check and is now able to read even very large files. -ClickHouse Keeper is 100% feature complete. More updates to come in the coming weeks around where and how you can test and provide feedback for us! -Release 21.12 +## ... And Many More -Release Date: 2021-12-13 - -Release Notes: [21.12](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) +Read the [full changelog](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) for 21.12 "Christmas" release for the full list of the gifts from [ClickHouse Team](https://clickhouse.com/careers/). From e06788a29d6fcd0496bab0df8809809e01ed56db Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:21:38 +0300 Subject: [PATCH 204/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index d2303286c9e..6c0852770b0 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -200,7 +200,7 @@ Also it simplifies data ingestion from various text sources. ## Query Optimizations With Table Constraints -This feature is [contributed](https://github.com/ClickHouse/ClickHouse/pull/18787) by **Nikita Vasilev**. Nikita is one of the most notable ClickHouse contributors. He started in 2019 by introducing data skipping indices into ClickHouse, then continued in 2020 with SSD-optimized key-value dictionaries and now contributed with the new advancements in the query optimizer. This feature is reviewed by **Anton Popov**. +This feature is [contributed](https://github.com/ClickHouse/ClickHouse/pull/18787) by **Nikita Vasilev**. Nikita is one of the most notable ClickHouse contributors. He started in 2019 by introducing data skipping indices into ClickHouse, then continued in 2020 with SSD-optimized key-value dictionaries and now contributed the new advancements in the query optimizer. This feature is reviewed by **Anton Popov**. So, what optimizations? ClickHouse already allows to specify constraints for tables: From 74c505ebd335894fdee5c65b93e9a510adf58494 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:30:31 +0300 Subject: [PATCH 205/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 6c0852770b0..5bcbbadaa39 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -52,7 +52,7 @@ In contrast to ZooKeeper, there are no issues with zxid overflow or packet sizes When using the table engines `File`, `URL`, and `HDFS` ClickHouse now supports partitions. When creating a table you can specify the partition key using the `PARTITION BY` clause e.g. `CREATE TABLE hits_files (...) ENGINE = File(TabSeparated) PARTITION BY toYYYYMM(EventDate)`. -Similarly, when exporting data from ClickHouse using the `file`, `url`, and `hdfs` table functions you can now specify that the data is to be partitioned into multiple files using a `PARTITION BY` clause. For example, `INSERT INTO TABLE FUNCTION file('path/hits_{_partition_id}', 'TSV', '<table_format>') PARTITION BY toYYYYMM(EventDate) VALUES <values>` will create as many files as there are unique months in the dataset. +Similarly, when exporting data from ClickHouse using the `file`, `url`, and `hdfs` table functions you can now specify that the data is to be partitioned into multiple files using a `PARTITION BY` clause. For example, `INSERT INTO TABLE FUNCTION file('path/hits_{_partition_id}', 'TSV', 'columns...') PARTITION BY toYYYYMM(EventDate) VALUES ...` will create as many files as there are unique months in the dataset. The `s3` table function has supported partitioned writes since ClickHouse 21.10. From 2903a4867f8e0bffc527bf5787f88ee3169a7e9e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:34:44 +0300 Subject: [PATCH 206/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 5bcbbadaa39..009d77ee3a4 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -151,7 +151,7 @@ Since ClickHouse version 21.12 you can do it like this: ORDER BY EventDate WITH FILL STEP INTERVAL 1 MONTH ``` -`INTERVAL` is the standard SQL operator, you can use SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER and YEAR. +`INTERVAL` is a standard SQL operator, you can use SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER and YEAR. **How does this help you?** From 74a03810313b18b8e8fc3f45f5afed560ab82905 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:35:09 +0300 Subject: [PATCH 207/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 009d77ee3a4..fc6da428243 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -155,7 +155,7 @@ ORDER BY EventDate WITH FILL STEP INTERVAL 1 MONTH **How does this help you?** -It allows to avoid preprocessing step for your reports. +It allows to avoid postprocessing step for your reports. ## Add Support For "Identifier" Table and Database Query Parameters From 21cd8c24979f824447b4ab9e00b7de9f8d686f78 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:38:20 +0300 Subject: [PATCH 208/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index fc6da428243..1a2b48a71ff 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -228,10 +228,10 @@ The query can be automatically rewritten to: SELECT count() FROM hits WHERE Domain = 'ghe.clickhouse.tech' ``` -because `Domain` column is smaller, more compressable, will be faster to read and it does not require calculation of domain. +because `Domain` column is smaller, more compressable, will be faster to read and it does not require calculation of the domain from URL. The only thing you need is to enable the `optimize_using_constraints` and `optimize_substitute_columns` settings. -As a bonus, new type of constraints are introduced: `ASSUME`. +As a bonus, new type of constraints is introduced: `ASSUME`. ``` CONSTRAINT my_constraint ASSUME Domain = domainWithoutWWW(URL) From 708afa3dfca76ceca22789dcf79c9def3c596b97 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:40:34 +0300 Subject: [PATCH 209/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 1a2b48a71ff..9a56594bd51 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -261,7 +261,7 @@ ClickHouse combines fast query engine and efficient data storage. It also allows When reading large files in `Parquet`, `ORC`, and `Arrow` format using the `s3`, `url`, and `hdfs` table functions, ClickHouse will now automatically choose whether to read the entire file at once or read parts of it incrementally. This is now enabled by default and the setting `remote_read_min_bytes_for_seek` controls when to switch from reading it all to reading in chunks. The default is 1MiB. -`Parquet`, `ORC`, and `Arrow` are column-oriented formats (quite similar to ClickHouse Native format) and now we can read only requested columns even if they are being read from remote HTTP server with the `url` table function (range requests will be performed). +`Parquet`, `ORC`, and `Arrow` are column-oriented formats (quite similar to ClickHouse Native format) and now we can read only requested columns even if they are being read from remote HTTP server with the `url` table function (range requests will be performed to skip unneeded data). **How does this help our ClickHouse Users?** From 883647532b9a7372fa533ae9abcad7ef69c5a148 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:41:22 +0300 Subject: [PATCH 210/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 9a56594bd51..2a07235e1ba 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -252,7 +252,11 @@ The expression is checked and the result (true/false) is written as an index for **How does this help you?** -Especially in large ClickHouse deployments with many complex tables it can be hard for users to always be up to date on the best way to query a given dataset. Constraints can help optimize queries without having to change the query structure itself. They can also make it easier to make changes to tables. For example, let's say you have a table containing web requests and it includes a URL column that contains the full URL of each request. Many times, users will want to know the top level domain (.com, .co.uk, etc.), something ClickHouse provides the topLevelDomain function to calculate. If you discover that many people are using this function you might decide to create a new materialized column that pre-calculates the top level domain for each record. Rather than tell all your users to change their queries you can use a table constraint to tell ClickHouse that each time a user tries to call the topLevelDomain function the request should be rewritten to use the new materialized column. +Especially in large ClickHouse deployments with many complex tables it can be hard for users to always be up to date on the best way to query a given dataset. Constraints can help optimize queries without having to change the query structure itself. They can also make it easier to make changes to tables. + +For example, let's say you have a table containing web requests and it includes a URL column that contains the full URL of each request. Many times, users will want to know the top level domain (.com, .co.uk, etc.), something ClickHouse provides the `topLevelDomain` function to calculate. If you discover that many people are using this function you might decide to create a new materialized column that pre-calculates the top level domain for each record. + +Rather than tell all your users to change their queries you can use a table constraint to tell ClickHouse that each time a user tries to call the `topLevelDomain` function the request should be rewritten to use the new materialized column. ## Read Large Remote Files In Chunks From 12d24cd1529f8f9ee681d5809936391ad2fcf07b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:43:07 +0300 Subject: [PATCH 211/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 2a07235e1ba..3e0a9935e02 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -267,6 +267,8 @@ When reading large files in `Parquet`, `ORC`, and `Arrow` format using the `s3`, `Parquet`, `ORC`, and `Arrow` are column-oriented formats (quite similar to ClickHouse Native format) and now we can read only requested columns even if they are being read from remote HTTP server with the `url` table function (range requests will be performed to skip unneeded data). +This feature is implemented by **Kseniia Sumarokova**. + **How does this help our ClickHouse Users?** In previous versions, when reading files in Arrow-based formats from remote locations with the `s3`, `url`, and `hdfs` table functions, ClickHouse would always read the entire file into memory. This works well when the files are small but will cause excessive memory usage or not work at all when the files are large. With this change, ClickHouse will read large files in chunks to keep memory usage in check and is now able to read even very large files. From 9113020dcd25d456d61af4352359e958e9f2f3e4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:44:01 +0300 Subject: [PATCH 212/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 3e0a9935e02..99adb710bef 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -93,6 +93,8 @@ INSERT INTO my_table FROM INFILE '*.csv.gz' FORMAT CSV Glob patterns support `*`, `?` and `{n..m}` with `{1..10}` or (aligned) `{01..10}` forms. This query will be automatically parallelized, it will also automatically detect compression format from file extension and decompress transparently. +This improvement is done by **Arthur Filatenkov**. + **How does this help you?** Now you don't have to recall how to write parallel for loop in your command line shell. clickhouse-client will do everything for you, it works intuitively and fast. From e9e733f52bce4da614555fe325e8757dc1db77d9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:44:15 +0300 Subject: [PATCH 213/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 99adb710bef..b1dce8d3e93 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -8,7 +8,7 @@ tags: ['company', 'community'] We're continuing our monthly release cadence. The 21.12 Christmas release includes 2460 new commits from 125 contributors, including 42 new contributors: -Alex Cao, Amr Alaa, Andrey Torsunov, Constantine Peresypkin, Dmitriy Dorofeev, Egor O'Sten, Elykov Alexandr, Evgeny, Frank Chen, LB, Natasha Murashkina, Peignon Melvyn, Rich Raposa, Roman Chyrva, Roman, SuperDJY, Thom O'Connor, Timur Magomedov, Tom Risse, Tomáš Hromada, cfcz48, cgp, cms, cmsxbc, congbaoyangrou, dongyifeng, frank chen, freedomDR, jus1096, khamadiev, laurieliyang, leosunli, liyang830, loneylee, michael1589, msaf1980, p0ny, qieqieplus, spume, sunlisheng, yandd, zhanghuajie. +> Alex Cao, Amr Alaa, Andrey Torsunov, Constantine Peresypkin, Dmitriy Dorofeev, Egor O'Sten, Elykov Alexandr, Evgeny, Frank Chen, LB, Natasha Murashkina, Peignon Melvyn, Rich Raposa, Roman Chyrva, Roman, SuperDJY, Thom O'Connor, Timur Magomedov, Tom Risse, Tomáš Hromada, cfcz48, cgp, cms, cmsxbc, congbaoyangrou, dongyifeng, frank chen, freedomDR, jus1096, khamadiev, laurieliyang, leosunli, liyang830, loneylee, michael1589, msaf1980, p0ny, qieqieplus, spume, sunlisheng, yandd, zhanghuajie. If you are wondering, this list is generated by the following command: From 1888bd672160bb6ee060994e95c807e89d00fd1d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 06:50:18 +0300 Subject: [PATCH 214/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index b1dce8d3e93..393ed5bb0a2 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -155,6 +155,8 @@ ORDER BY EventDate WITH FILL STEP INTERVAL 1 MONTH `INTERVAL` is a standard SQL operator, you can use SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER and YEAR. +This is implemented by **Anton Popov** who is the author of "WITH FILL" feature. + **How does this help you?** It allows to avoid postprocessing step for your reports. @@ -182,6 +184,8 @@ Since version 21.12 we introduce support for using parameters for tables and dat SELECT uniq(user_id) FROM {tbl:Identifier} ``` +Identifier parameters also work for CREATE, DROP and all DDL queries. This is implemented by **Nikolai Degterinskiy**. + **How does this help you?** Let ClickHouse do the heavy-lifting and keep your scripts safe and secure. @@ -189,7 +193,7 @@ Let ClickHouse do the heavy-lifting and keep your scripts safe and secure. ## Bool Data Type -This feature is experimental in version 21.12. +This feature is experimental in version 21.12. It is implemented by **Kevin Wan (MaxWk)** on top of initial work by **hczhcz** and reviewed by **Pavel Kruglov**. ClickHouse now natively supports a `Bool` data type. It allows to represent values as "true"/"false" during data import and export in text formats. It can also be adjusted to anything else using the settings `bool_true_representation` and `bool_false_representation` (for example, "yes" and "no"). From 744f856a913df8a3017d7fa7d78db559931da277 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Dec 2021 07:32:48 +0300 Subject: [PATCH 215/358] Fix YAML in MarkDown --- website/blog/en/2021/clickhouse-v21.12-released.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 393ed5bb0a2..01450280fa0 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -1,8 +1,8 @@ --- -title: 'What's New in ClickHouse 21.12' +title: 'What''s New in ClickHouse 21.12' image: 'https://blog-images.clickhouse.com/en/2021/clickhouse-v21-12/featured.jpg' date: '2021-12-16' -author: '[Alexey Milovidov](https://github.com/alexey-milovidov)', '[Christoph Wurm](https://github.com/cwurm)' +author: '[Alexey Milovidov](https://github.com/alexey-milovidov), [Christoph Wurm](https://github.com/cwurm)' tags: ['company', 'community'] --- @@ -258,7 +258,7 @@ The expression is checked and the result (true/false) is written as an index for **How does this help you?** -Especially in large ClickHouse deployments with many complex tables it can be hard for users to always be up to date on the best way to query a given dataset. Constraints can help optimize queries without having to change the query structure itself. They can also make it easier to make changes to tables. +Especially in large ClickHouse deployments with many complex tables it can be hard for users to always be up to date on the best way to query a given dataset. Constraints can help optimize queries without having to change the query structure itself. They can also make it easier to make changes to tables. For example, let's say you have a table containing web requests and it includes a URL column that contains the full URL of each request. Many times, users will want to know the top level domain (.com, .co.uk, etc.), something ClickHouse provides the `topLevelDomain` function to calculate. If you discover that many people are using this function you might decide to create a new materialized column that pre-calculates the top level domain for each record. From d37c0d0c50a93627bc845befe9fcda16d619b5ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Dec 2021 07:36:44 +0300 Subject: [PATCH 216/358] Add the link to the main page --- website/templates/index/hero.html | 20 +++++++------------- 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/website/templates/index/hero.html b/website/templates/index/hero.html index 83853dc1345..cfd21273092 100644 --- a/website/templates/index/hero.html +++ b/website/templates/index/hero.html @@ -1,23 +1,17 @@
- +

- ClickHouse v21.11 Released + ClickHouse v21.12 Released

{{ _('ClickHouse® is an open-source, high performance columnar OLAP database management system for real-time analytics using SQL.') }}

- -

- Read the Blog Post + Read the Blog Post

@@ -28,15 +22,15 @@
- +

ClickHouse Announces $250 Million in Funding

- -

Raising the Company’s Valuation to $2B

+ +

Raising the Company’s Valuation to $2B

From e4cab544abf860c7a315831e963c761785b47067 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Dec 2021 07:46:23 +0300 Subject: [PATCH 217/358] Fix fast test --- .../0_stateless/02113_base64encode_trailing_bytes.sql | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql index 11582242a13..0de1859dddf 100644 --- a/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql +++ b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql @@ -1,5 +1,9 @@ +-- Tags: no-fasttest SET log_queries=1; +DROP TABLE IF EXISTS tabl_1; +DROP TABLE IF EXISTS tabl_2; + CREATE TABLE tabl_1 (key String) ENGINE MergeTree ORDER BY key; CREATE TABLE tabl_2 (key String) ENGINE MergeTree ORDER BY key; SELECT * FROM tabl_1 SETTINGS log_comment = 'ad15a651'; @@ -8,6 +12,9 @@ SYSTEM FLUSH LOGS; SELECT base64Decode(base64Encode(normalizeQuery(query))) FROM system.query_log - WHERE type='QueryFinish' AND log_comment = 'ad15a651' + WHERE type = 'QueryFinish' AND log_comment = 'ad15a651' GROUP BY normalizeQuery(query) ORDER BY normalizeQuery(query); + +DROP TABLE tabl_1; +DROP TABLE tabl_2; From c148718e73600c599b18eb821cf961b12f7d8922 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 16 Dec 2021 07:47:50 +0300 Subject: [PATCH 218/358] Fix fast test --- tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql index 0de1859dddf..120055b12e2 100644 --- a/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql +++ b/tests/queries/0_stateless/02113_base64encode_trailing_bytes.sql @@ -12,7 +12,7 @@ SYSTEM FLUSH LOGS; SELECT base64Decode(base64Encode(normalizeQuery(query))) FROM system.query_log - WHERE type = 'QueryFinish' AND log_comment = 'ad15a651' + WHERE type = 'QueryFinish' AND log_comment = 'ad15a651' AND current_database = currentDatabase() GROUP BY normalizeQuery(query) ORDER BY normalizeQuery(query); From ba2c9405daa1d5ca7ee371caf95b1470ae714757 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 16 Dec 2021 07:56:43 +0300 Subject: [PATCH 219/358] Fix test_storage_s3 flakiness --- tests/integration/test_storage_s3/test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index a4ba7a95dc7..bfeda84fa21 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -814,6 +814,7 @@ def test_seekable_formats(started_cluster): result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) + instance.query("SYSTEM FLUSH LOGS") result = instance.query(f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM s3') AND memory_usage > 0 ORDER BY event_time desc") print(result[:3]) assert(int(result[:3]) < 200) @@ -837,6 +838,7 @@ def test_seekable_formats_url(started_cluster): result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) + instance.query("SYSTEM FLUSH LOGS") result = instance.query(f"SELECT formatReadableSize(memory_usage) FROM system.query_log WHERE startsWith(query, 'SELECT count() FROM url') AND memory_usage > 0 ORDER BY event_time desc") print(result[:3]) assert(int(result[:3]) < 200) From 947ef81f531bc76d99b3e29cda9749854b225fbc Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 16 Dec 2021 09:14:09 +0300 Subject: [PATCH 220/358] Update ReadWriteBufferFromHTTP.h --- src/IO/ReadWriteBufferFromHTTP.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadWriteBufferFromHTTP.h b/src/IO/ReadWriteBufferFromHTTP.h index df1d443d5a0..11c4e99c353 100644 --- a/src/IO/ReadWriteBufferFromHTTP.h +++ b/src/IO/ReadWriteBufferFromHTTP.h @@ -427,7 +427,7 @@ namespace detail LOG_ERROR(log, "HTTP request to `{}` failed at try {}/{} with bytes read: {}/{}. " "Error: {}. (Current backoff wait is {}/{} ms)", - uri.toString(), i, settings.http_max_tries, + uri.toString(), i + 1, settings.http_max_tries, getOffset(), read_range.end ? toString(*read_range.end) : "unknown", e.displayText(), milliseconds_to_wait, settings.http_retry_max_backoff_ms); From 7ff727048cfe6cfa5c9ff256391a9b837686a7cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 10:42:00 +0300 Subject: [PATCH 221/358] Revert docs release --- .github/workflows/release.yml | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 1ba25a75017..5f4b36e7203 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -30,14 +30,6 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, func-tester] steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/docs_release - REPO_COPY=${{runner.temp}}/docs_release/ClickHouse - CLOUDFLARE_TOKEN=${{secrets.CLOUDFLARE}} - ROBOT_CLICKHOUSE_SSH_KEY=${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} - EOF - name: Check out repository code uses: actions/checkout@v2 - name: Download changed images @@ -46,6 +38,11 @@ jobs: name: changed_images path: ${{ env.TEMP_PATH }} - name: Docs Release + env: + TEMP_PATH: ${{runner.temp}}/docs_release + REPO_COPY: ${{runner.temp}}/docs_release/ClickHouse + CLOUDFLARE_TOKEN: ${{secrets.CLOUDFLARE}} + ROBOT_CLICKHOUSE_SSH_KEY: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} run: | cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci From 6c45c7eff313a1f76dba9f51baef86a4cb840c11 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 10:50:46 +0300 Subject: [PATCH 222/358] Followup --- .github/workflows/release.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 5f4b36e7203..20d2828b20e 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -44,6 +44,8 @@ jobs: CLOUDFLARE_TOKEN: ${{secrets.CLOUDFLARE}} ROBOT_CLICKHOUSE_SSH_KEY: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH cp -r $GITHUB_WORKSPACE $TEMP_PATH cd $REPO_COPY/tests/ci python3 docs_release.py From bfc06ab41aed724604da2196902963e622739a0f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 10:52:16 +0300 Subject: [PATCH 223/358] Run docs release manually --- .github/workflows/release.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 20d2828b20e..e9f028ac8d0 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -11,6 +11,7 @@ on: # yamllint disable-line rule:truthy - 'website/**' - 'benchmark/**' - 'docker/**' + workflow_dispatch: jobs: DockerHubPush: runs-on: [self-hosted, style-checker] From 39288ad7f6ceff1f071ee7232c969bbebee28a5b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 10:55:42 +0300 Subject: [PATCH 224/358] Trigger also --- .github/workflows/release.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index e9f028ac8d0..5d6f9c4a38e 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -11,6 +11,7 @@ on: # yamllint disable-line rule:truthy - 'website/**' - 'benchmark/**' - 'docker/**' + - '.github/**' workflow_dispatch: jobs: DockerHubPush: From 38f2e4c8ac95564e1143af980b546f5d7c52e454 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 10:59:42 +0300 Subject: [PATCH 225/358] Prevent docs release of being too smart --- tests/ci/docs_release.py | 7 ------- 1 file changed, 7 deletions(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 04922e8c5ab..384f1d3030b 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -25,13 +25,6 @@ if __name__ == "__main__": pr_info = PRInfo(get_event(), need_changed_files=True) gh = Github(get_best_robot_token()) - if not pr_info.has_changes_in_documentation(): - logging.info ("No changes in documentation") - commit = get_commit(gh, pr_info.sha) - commit.create_status(context=NAME, description="No changes in docs", state="success") - sys.exit(0) - - logging.info("Has changes in docs") if not os.path.exists(temp_path): os.makedirs(temp_path) From cf0241a2d5d4c04e08f57b0b7fbb16e75ab14b4b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 10:59:59 +0300 Subject: [PATCH 226/358] Prevent docs release of being too smart --- tests/ci/docs_release.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 384f1d3030b..4ea2eae5130 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -2,7 +2,6 @@ import logging import subprocess import os -import sys from github import Github From 25e8c5fc31995875685d3ea8830d5f92b672cb03 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 11:02:01 +0300 Subject: [PATCH 227/358] Add dispatch event to docs release --- .github/workflows/release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 5d6f9c4a38e..dae66fd3bb3 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -12,7 +12,7 @@ on: # yamllint disable-line rule:truthy - 'benchmark/**' - 'docker/**' - '.github/**' - workflow_dispatch: + workflow_dispatch: jobs: DockerHubPush: runs-on: [self-hosted, style-checker] From b64ad512fef0308c3076b1b4363f6f191a4e31fc Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 11:22:57 +0300 Subject: [PATCH 228/358] Remove repository before checkout --- .github/workflows/backport_branches.yml | 36 ++++ .github/workflows/docs_check.yml | 9 + .github/workflows/main.yml | 231 ++++++++++++++++++++++++ .github/workflows/master.yml | 213 ++++++++++++++++++++++ .github/workflows/release.yml | 6 + .github/workflows/release_branches.yml | 90 +++++++++ 6 files changed, 585 insertions(+) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 5bcb0fa87b6..f3217b8ebc6 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -7,6 +7,9 @@ jobs: DockerHubPush: runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -29,6 +32,9 @@ jobs: REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse REPORTS_PATH=${{runner.temp}}/reports_dir EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -69,6 +75,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -110,6 +119,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -151,6 +163,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -192,6 +207,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -236,6 +254,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -270,6 +291,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -305,6 +329,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -343,6 +370,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -377,6 +407,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -403,6 +436,9 @@ jobs: - CompatibilityCheck runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index eab1b4b44fe..5f860e52ec1 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -14,6 +14,9 @@ jobs: CheckLabels: runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Labels check @@ -24,6 +27,9 @@ jobs: needs: CheckLabels runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -50,6 +56,9 @@ jobs: with: name: changed_images path: ${{ env.TEMP_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Docs Check diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index c27a19ce9b5..cfa3ef130a9 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -17,6 +17,9 @@ jobs: CheckLabels: runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Labels check @@ -27,6 +30,9 @@ jobs: needs: CheckLabels runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -52,6 +58,9 @@ jobs: with: name: changed_images path: ${{ env.TEMP_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Style Check @@ -75,6 +84,9 @@ jobs: REPO_COPY=${{runner.temp}}/fasttest/ClickHouse CACHES_PATH=${{runner.temp}}/../ccaches EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fast Test @@ -99,6 +111,9 @@ jobs: TEMP_PATH=${{runner.temp}}/pvs_check REPO_COPY=${{runner.temp}}/pvs_check/ClickHouse EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -126,6 +141,9 @@ jobs: REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse REPORTS_PATH=${{runner.temp}}/reports_dir EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -155,6 +173,9 @@ jobs: REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse REPORTS_PATH=${{runner.temp}}/reports_dir EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -195,6 +216,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -236,6 +260,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -277,6 +304,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -318,6 +348,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -359,6 +392,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -400,6 +436,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -441,6 +480,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -482,6 +524,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -526,6 +571,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -567,6 +615,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -608,6 +659,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -649,6 +703,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -690,6 +747,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -731,6 +791,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -772,6 +835,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -820,6 +886,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -857,6 +926,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -891,6 +963,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -925,6 +1000,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -959,6 +1037,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -991,6 +1072,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1025,6 +1109,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1059,6 +1146,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1093,6 +1183,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1127,6 +1220,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1161,6 +1257,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1193,6 +1292,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1227,6 +1329,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1261,6 +1366,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1295,6 +1403,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1329,6 +1440,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1363,6 +1477,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1397,6 +1514,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1429,6 +1549,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1464,6 +1587,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1496,6 +1622,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1528,6 +1657,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1560,6 +1692,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1592,6 +1727,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1624,6 +1762,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1658,6 +1799,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1693,6 +1837,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1724,6 +1871,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1755,6 +1905,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1786,6 +1939,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1820,6 +1976,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -1851,6 +2010,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -1882,6 +2044,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -1913,6 +2078,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -1944,6 +2112,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -1980,6 +2151,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2013,6 +2187,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2046,6 +2223,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2079,6 +2259,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2112,6 +2295,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2145,6 +2331,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2178,6 +2367,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2211,6 +2403,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2244,6 +2439,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2275,6 +2473,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2309,6 +2510,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2340,6 +2544,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2371,6 +2578,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2402,6 +2612,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2433,6 +2646,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2469,6 +2685,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2502,6 +2721,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2535,6 +2757,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2568,6 +2793,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2646,6 +2874,9 @@ jobs: - IntegrationTestsFlakyCheck runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 24ada19b80b..e8541d6d338 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -7,6 +7,9 @@ jobs: DockerHubPush: runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -32,6 +35,9 @@ jobs: with: name: changed_images path: ${{ env.TEMP_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Style Check @@ -55,6 +61,9 @@ jobs: REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse REPORTS_PATH=${{runner.temp}}/reports_dir EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -85,6 +94,9 @@ jobs: REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse REPORTS_PATH=${{runner.temp}}/reports_dir EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -125,6 +137,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -166,6 +181,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -208,6 +226,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -249,6 +270,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -290,6 +314,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -331,6 +358,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -372,6 +402,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -413,6 +446,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -458,6 +494,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -500,6 +539,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -542,6 +584,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -584,6 +629,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -626,6 +674,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -668,6 +719,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -710,6 +764,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -757,6 +814,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -793,6 +853,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -827,6 +890,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -859,6 +925,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -893,6 +962,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -927,6 +999,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -961,6 +1036,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -995,6 +1073,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1029,6 +1110,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1061,6 +1145,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1095,6 +1182,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1129,6 +1219,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1163,6 +1256,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1197,6 +1293,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1231,6 +1330,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1265,6 +1367,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1300,6 +1405,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1332,6 +1440,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1364,6 +1475,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1396,6 +1510,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1428,6 +1545,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1460,6 +1580,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1492,6 +1615,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1526,6 +1652,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1561,6 +1690,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1592,6 +1724,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1623,6 +1758,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1654,6 +1792,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1690,6 +1831,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1723,6 +1867,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1756,6 +1903,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1789,6 +1939,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1822,6 +1975,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1855,6 +2011,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1888,6 +2047,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1921,6 +2083,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1954,6 +2119,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1988,6 +2156,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2019,6 +2190,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2050,6 +2224,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2081,6 +2258,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2112,6 +2292,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2146,6 +2329,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2177,6 +2363,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2208,6 +2397,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2239,6 +2431,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2270,6 +2465,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2306,6 +2504,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2339,6 +2540,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2372,6 +2576,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2405,6 +2612,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2477,6 +2687,9 @@ jobs: - SplitBuildSmokeTest runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index dae66fd3bb3..ad20266f292 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -17,6 +17,9 @@ jobs: DockerHubPush: runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -32,6 +35,9 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, func-tester] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download changed images diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index dec0c4566b7..0825a2f96cc 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -10,6 +10,9 @@ jobs: DockerHubPush: runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -32,6 +35,9 @@ jobs: REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse REPORTS_PATH=${{runner.temp}}/reports_dir EOF + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -72,6 +78,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -113,6 +122,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -154,6 +166,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -195,6 +210,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -236,6 +254,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -277,6 +298,9 @@ jobs: with: name: changed_images path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -323,6 +347,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -357,6 +384,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -389,6 +419,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -421,6 +454,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -453,6 +489,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -485,6 +524,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -517,6 +559,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -552,6 +597,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -584,6 +632,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -616,6 +667,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -648,6 +702,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -680,6 +737,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -712,6 +772,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -746,6 +809,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -781,6 +847,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -812,6 +881,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -843,6 +915,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -874,6 +949,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -908,6 +986,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -939,6 +1020,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -970,6 +1054,9 @@ jobs: uses: actions/download-artifact@v2 with: path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1012,6 +1099,9 @@ jobs: - CompatibilityCheck runs-on: [self-hosted, style-checker] steps: + - name: Clear repository + run: | + sudo rm -rf $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label From 9b653e2eeabaca83a7fe540594f6c67e4653b550 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 12:00:23 +0300 Subject: [PATCH 229/358] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 65c28feca4d..b0d9672a9f5 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -54,6 +54,7 @@ Internal coordination settings are located in `..` section and contain servers description. From 58b3bdf10771d4bab3401a7a188da6fd264189ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 12:03:17 +0300 Subject: [PATCH 230/358] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 186 ++++++++++++++++++++++++ 1 file changed, 186 insertions(+) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index b0d9672a9f5..0061970cb25 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -102,6 +102,192 @@ Examples of configuration for quorum with three nodes can be found in [integrati ``` +## Four Letter Word Commands + +ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. + +The 4lw commands has a white list configuration `four_letter_word_white_list` which has default value "conf,cons,crst,envi,ruok,srst,srvr,stat,wchc,wchs,dirs,mntr,isro". + +You can issue the commands to ClickHouse Keeper via telnet or nc, at the client port. +``` +echo mntr | nc localhost 9181 +``` + +Bellow is the detailed 4lw commands: + +- `ruok`: Tests if server is running in a non-error state. The server will respond with imok if it is running. Otherwise it will not respond at all. A response of "imok" does not necessarily indicate that the server has joined the quorum, just that the server process is active and bound to the specified client port. Use "stat" for details on state wrt quorum and client connection information. + +``` +imok +``` + +- `mntr`: Outputs a list of variables that could be used for monitoring the health of the cluster. + +``` +zk_version v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +zk_avg_latency 0 +zk_max_latency 0 +zk_min_latency 0 +zk_packets_received 68 +zk_packets_sent 68 +zk_num_alive_connections 1 +zk_outstanding_requests 0 +zk_server_state leader +zk_znode_count 4 +zk_watch_count 1 +zk_ephemerals_count 0 +zk_approximate_data_size 723 +zk_open_file_descriptor_count 310 +zk_max_file_descriptor_count 10240 +zk_followers 0 +zk_synced_followers 0 +``` + +- `srvr`: Lists full details for the server. + +``` +ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +Latency min/avg/max: 0/0/0 +Received: 2 +Sent : 2 +Connections: 1 +Outstanding: 0 +Zxid: 34 +Mode: leader +Node count: 4 +``` + +- `stat`: Lists brief details for the server and connected clients. + +``` +ClickHouse Keeper version: v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +Clients: + 192.168.1.1:52852(recved=0,sent=0) + 192.168.1.1:52042(recved=24,sent=48) +Latency min/avg/max: 0/0/0 +Received: 4 +Sent : 4 +Connections: 1 +Outstanding: 0 +Zxid: 36 +Mode: leader +Node count: 4 +``` + +- `srst`: Reset server statistics. The command will affect the result of `srvr`, `mntr` and `stat`. + +``` +Server stats reset. +``` + +- `conf`: Print details about serving configuration. + +``` +server_id=1 +tcp_port=2181 +four_letter_word_white_list=* +log_storage_path=./coordination/logs +snapshot_storage_path=./coordination/snapshots +max_requests_batch_size=100 +session_timeout_ms=30000 +operation_timeout_ms=10000 +dead_session_check_period_ms=500 +heart_beat_interval_ms=500 +election_timeout_lower_bound_ms=1000 +election_timeout_upper_bound_ms=2000 +reserved_log_items=1000000000000000 +snapshot_distance=10000 +auto_forwarding=true +shutdown_timeout=5000 +startup_timeout=240000 +raft_logs_level=information +snapshots_to_keep=3 +rotate_log_storage_interval=100000 +stale_log_gap=10000 +fresh_log_gap=200 +max_requests_batch_size=100 +quorum_reads=false +force_sync=false +compress_logs=true +compress_snapshots_with_zstd_format=true +configuration_change_tries_count=20 +``` + +- `cons`: List full connection/session details for all clients connected to this server. Includes information on numbers of packets received/sent, session id, operation latencies, last operation performed, etc... + +``` + 192.168.1.1:52163(recved=0,sent=0,sid=0xffffffffffffffff,lop=NA,est=1636454787393,to=30000,lzxid=0xffffffffffffffff,lresp=0,llat=0,minlat=0,avglat=0,maxlat=0) + 192.168.1.1:52042(recved=9,sent=18,sid=0x0000000000000001,lop=List,est=1636454739887,to=30000,lcxid=0x0000000000000005,lzxid=0x0000000000000005,lresp=1636454739892,llat=0,minlat=0,avglat=0,maxlat=0) +``` + +- `crst`: Reset connection/session statistics for all connections. + +``` +Connection stats reset. +``` + +- `envi`: Print details about serving environment + +``` +Environment: +clickhouse.keeper.version=v21.11.1.1-prestable-7a4a0b0edef0ad6e0aa662cd3b90c3f4acf796e7 +host.name=ZBMAC-C02D4054M.local +os.name=Darwin +os.arch=x86_64 +os.version=19.6.0 +cpu.count=12 +user.name=root +user.home=/Users/JackyWoo/ +user.dir=/Users/JackyWoo/project/jd/clickhouse/cmake-build-debug/programs/ +user.tmp=/var/folders/b4/smbq5mfj7578f2jzwn602tt40000gn/T/ +``` + + +- `dirs`: Shows the total size of snapshot and log files in bytes + +``` +snapshot_dir_size: 0 +log_dir_size: 3875 +``` + +- `isro`: Tests if server is running in read-only mode. The server will respond with "ro" if in read-only mode or "rw" if not in read-only mode. + +``` +rw +``` + +- `wchs`: Lists brief information on watches for the server. + +``` +1 connections watching 1 paths +Total watches:1 +``` + +- `wchc`: Lists detailed information on watches for the server, by session. This outputs a list of sessions (connections) with associated watches (paths). Note, depending on the number of watches this operation may be expensive (ie impact server performance), use it carefully. + +``` +0x0000000000000001 + /clickhouse/task_queue/ddl +``` + +- `wchp`: Lists detailed information on watches for the server, by path. This outputs a list of paths (znodes) with associated sessions. Note, depending on the number of watches this operation may be expensive (i. e. impact server performance), use it carefully. + +``` +/clickhouse/task_queue/ddl + 0x0000000000000001 +``` + +- `dump`: Lists the outstanding sessions and ephemeral nodes. This only works on the leader. + +``` +Sessions dump (2): +0x0000000000000001 +0x0000000000000002 +Sessions with Ephemerals (1): +0x0000000000000001 + /clickhouse/task_queue/ddl +``` + ## How to run ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with: From e65e21ada3e00a5f37b9d2f2fb74cb2782813f33 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 12:04:10 +0300 Subject: [PATCH 231/358] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 0061970cb25..2b5c4c837f9 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -102,6 +102,14 @@ Examples of configuration for quorum with three nodes can be found in [integrati ``` +## How to run + +ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with: + +```bash +clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon +``` + ## Four Letter Word Commands ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. @@ -288,14 +296,6 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -## How to run - -ClickHouse Keeper is bundled into the ClickHouse server package, just add configuration of `` and start ClickHouse server as always. If you want to run standalone ClickHouse Keeper you can start it in a similar way with: - -```bash -clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon -``` - ## [experimental] Migration from ZooKeeper Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: From 2a909725c9bf7002d46889cfbc41e563687fe655 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 12:04:40 +0300 Subject: [PATCH 232/358] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 2b5c4c837f9..6738f77cff9 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -313,6 +313,3 @@ clickhouse-keeper-converter --zookeeper-logs-dir /var/lib/zookeeper/version-2 -- 4. Copy snapshot to ClickHouse server nodes with a configured `keeper` or start ClickHouse Keeper instead of ZooKeeper. The snapshot must persist on all nodes, otherwise, empty nodes can be faster and one of them can become a leader. [Original article](https://clickhouse.com/docs/en/operations/clickhouse-keeper/) - - - From 0292ddcdb3365e7e88eb1af176d21ceb6ec249d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 12:19:01 +0300 Subject: [PATCH 233/358] Trying to debug --- .github/workflows/main.yml | 155 +++++++++++++++++++------------------ 1 file changed, 78 insertions(+), 77 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index cfa3ef130a9..66e4813a47b 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -19,7 +19,8 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + ls -la $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Labels check @@ -32,7 +33,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -60,7 +61,7 @@ jobs: path: ${{ env.TEMP_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Style Check @@ -86,7 +87,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fast Test @@ -113,7 +114,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -143,7 +144,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -175,7 +176,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -218,7 +219,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -262,7 +263,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -306,7 +307,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -350,7 +351,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -394,7 +395,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -438,7 +439,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -482,7 +483,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -526,7 +527,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -573,7 +574,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -617,7 +618,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -661,7 +662,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -705,7 +706,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -749,7 +750,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -793,7 +794,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -837,7 +838,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -888,7 +889,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -928,7 +929,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -965,7 +966,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1002,7 +1003,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1039,7 +1040,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1074,7 +1075,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1111,7 +1112,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1148,7 +1149,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1185,7 +1186,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1222,7 +1223,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1259,7 +1260,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1294,7 +1295,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1331,7 +1332,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1368,7 +1369,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1405,7 +1406,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1442,7 +1443,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1479,7 +1480,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1516,7 +1517,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1551,7 +1552,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1589,7 +1590,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1624,7 +1625,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1659,7 +1660,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1694,7 +1695,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1729,7 +1730,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1764,7 +1765,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1801,7 +1802,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1839,7 +1840,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1873,7 +1874,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1907,7 +1908,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1941,7 +1942,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1978,7 +1979,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2012,7 +2013,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2046,7 +2047,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2080,7 +2081,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2114,7 +2115,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2153,7 +2154,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2189,7 +2190,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2225,7 +2226,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2261,7 +2262,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2297,7 +2298,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2333,7 +2334,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2369,7 +2370,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2405,7 +2406,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2441,7 +2442,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2475,7 +2476,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2512,7 +2513,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2546,7 +2547,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2580,7 +2581,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2614,7 +2615,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2648,7 +2649,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2687,7 +2688,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2723,7 +2724,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2759,7 +2760,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2795,7 +2796,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2876,7 +2877,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label From 9256c75ea5027923c2a5f539dec6f8ddb1cc0090 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 12:20:20 +0300 Subject: [PATCH 234/358] Followup --- .github/workflows/backport_branches.yml | 24 ++-- .github/workflows/main.yml | 1 - .github/workflows/master.yml | 142 ++++++++++++------------ .github/workflows/release.yml | 4 +- .github/workflows/release_branches.yml | 60 +++++----- 5 files changed, 115 insertions(+), 116 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index f3217b8ebc6..53d7a4aece8 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -9,7 +9,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -34,7 +34,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -77,7 +77,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -121,7 +121,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -165,7 +165,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -209,7 +209,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -256,7 +256,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -293,7 +293,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -331,7 +331,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -372,7 +372,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -409,7 +409,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -438,7 +438,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 66e4813a47b..acd365aea9a 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -19,7 +19,6 @@ jobs: steps: - name: Clear repository run: | - ls -la $GITHUB_WORKSPACE sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e8541d6d338..4a756c5e0e9 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -9,7 +9,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -37,7 +37,7 @@ jobs: path: ${{ env.TEMP_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Style Check @@ -63,7 +63,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -96,7 +96,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -139,7 +139,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -183,7 +183,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -228,7 +228,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -272,7 +272,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -316,7 +316,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -360,7 +360,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -404,7 +404,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -448,7 +448,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -496,7 +496,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -541,7 +541,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -586,7 +586,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -631,7 +631,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -676,7 +676,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -721,7 +721,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -766,7 +766,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -816,7 +816,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -855,7 +855,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -892,7 +892,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -927,7 +927,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -964,7 +964,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1001,7 +1001,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1038,7 +1038,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1075,7 +1075,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1112,7 +1112,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1147,7 +1147,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1184,7 +1184,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1221,7 +1221,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1258,7 +1258,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1295,7 +1295,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1332,7 +1332,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1369,7 +1369,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1407,7 +1407,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1442,7 +1442,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1477,7 +1477,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1512,7 +1512,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1547,7 +1547,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1582,7 +1582,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1617,7 +1617,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -1654,7 +1654,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1692,7 +1692,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1726,7 +1726,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1760,7 +1760,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1794,7 +1794,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -1833,7 +1833,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1869,7 +1869,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1905,7 +1905,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1941,7 +1941,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1977,7 +1977,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2013,7 +2013,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2049,7 +2049,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2085,7 +2085,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2121,7 +2121,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -2158,7 +2158,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2192,7 +2192,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2226,7 +2226,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2260,7 +2260,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2294,7 +2294,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Fuzzer @@ -2331,7 +2331,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2365,7 +2365,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2399,7 +2399,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2433,7 +2433,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2467,7 +2467,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Unit test @@ -2506,7 +2506,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2542,7 +2542,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2578,7 +2578,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2614,7 +2614,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Performance Comparison @@ -2689,7 +2689,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index ad20266f292..977dbf07dbe 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -19,7 +19,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -37,7 +37,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download changed images diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 0825a2f96cc..1fb4545fa7f 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -12,7 +12,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -37,7 +37,7 @@ jobs: EOF - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports @@ -80,7 +80,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -124,7 +124,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -168,7 +168,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -212,7 +212,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -256,7 +256,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -300,7 +300,7 @@ jobs: path: ${{ env.IMAGES_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 with: @@ -349,7 +349,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Report Builder @@ -386,7 +386,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -421,7 +421,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -456,7 +456,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -491,7 +491,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -526,7 +526,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -561,7 +561,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -599,7 +599,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -634,7 +634,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -669,7 +669,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -704,7 +704,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -739,7 +739,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -774,7 +774,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Functional test @@ -811,7 +811,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -849,7 +849,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -883,7 +883,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -917,7 +917,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -951,7 +951,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Stress test @@ -988,7 +988,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1022,7 +1022,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1056,7 +1056,7 @@ jobs: path: ${{ env.REPORTS_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Integration test @@ -1101,7 +1101,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Finish label From 858d9f0f5fab1efb0c7602677f80d3de6b005d34 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 16 Dec 2021 12:29:20 +0300 Subject: [PATCH 235/358] StorageLiveView fix function style --- src/Storages/LiveView/StorageLiveView.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/LiveView/StorageLiveView.h b/src/Storages/LiveView/StorageLiveView.h index 115897b65c0..17e2f50e7ec 100644 --- a/src/Storages/LiveView/StorageLiveView.h +++ b/src/Storages/LiveView/StorageLiveView.h @@ -232,8 +232,8 @@ private: const StorageID & table_id_, ContextPtr context_, const ASTCreateQuery & query, - const ColumnsDescription & columns - , const String & comment); + const ColumnsDescription & columns, + const String & comment); }; } From dba6f87671e952fa8a5ee0f5b41082d6918a69c9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 12:29:35 +0300 Subject: [PATCH 236/358] Fix release branches filter --- .github/workflows/release_branches.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index dec0c4566b7..7bc6ebe97ff 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -2,10 +2,10 @@ name: ReleaseCI on: # yamllint disable-line rule:truthy push: branches: - - '21.**' - - '22.**' - - '23.**' - - '24.**' + - '21.[1-9][1-9]' + - '22.[1-9][1-9]' + - '23.[1-9][1-9]' + - '24.[1-9][1-9]' jobs: DockerHubPush: runs-on: [self-hosted, style-checker] From ccf6210eb1b25d6afe134e09a3312c442c088d13 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 3 Dec 2021 15:30:14 +0100 Subject: [PATCH 237/358] Add docker container for AWS lambda --- tests/ci/team_keys_lambda/Dockerfile | 13 ++++ tests/ci/team_keys_lambda/app.py | 70 ++++++++++++++++++++++ tests/ci/team_keys_lambda/requirements.txt | 1 + 3 files changed, 84 insertions(+) create mode 100644 tests/ci/team_keys_lambda/Dockerfile create mode 100644 tests/ci/team_keys_lambda/app.py create mode 100644 tests/ci/team_keys_lambda/requirements.txt diff --git a/tests/ci/team_keys_lambda/Dockerfile b/tests/ci/team_keys_lambda/Dockerfile new file mode 100644 index 00000000000..0d50224c51d --- /dev/null +++ b/tests/ci/team_keys_lambda/Dockerfile @@ -0,0 +1,13 @@ +FROM public.ecr.aws/lambda/python:3.9 + +# Install the function's dependencies using file requirements.txt +# from your project folder. + +COPY requirements.txt . +RUN pip3 install -r requirements.txt --target "${LAMBDA_TASK_ROOT}" + +# Copy function code +COPY app.py ${LAMBDA_TASK_ROOT} + +# Set the CMD to your handler (could also be done as a parameter override outside of the Dockerfile) +CMD [ "app.handler" ] diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py new file mode 100644 index 00000000000..ad34d949d36 --- /dev/null +++ b/tests/ci/team_keys_lambda/app.py @@ -0,0 +1,70 @@ +#!/usr/bin/env python3 + +import requests +import argparse +import json + + +def get_org_team_members(token: str, org: str, team_slug: str) -> tuple: + headers = { + "Authorization": f"token {token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.get( + f"https://api.github.com/orgs/{org}/teams/{team_slug}/members", headers=headers + ) + response.raise_for_status() + data = response.json() + return tuple(m["login"] for m in data) + + +def get_members_keys(members: tuple) -> str: + keys = "" + for m in members: + response = requests.get( + f"https://github.com/{m}.keys", + ) + response.raise_for_status() + keys += f"# {m}\n{response.text}" + return keys + + +def get_token_from_aws() -> str: + import boto3 + + secret_name = "clickhouse_robot_token" + session = boto3.session.Session() + client = session.client( + service_name="secretsmanager", + ) + get_secret_value_response = client.get_secret_value(SecretId=secret_name) + data = json.loads(get_secret_value_response["SecretString"]) + return data["clickhouse_robot_token"] + + +def main(token: str, org: str, team_slug: str) -> str: + members = get_org_team_members(token, org, team_slug) + keys = get_members_keys(members) + + return keys + + +def handler(event, context): + token = get_token_from_aws() + return main(token, "ClickHouse", "core") + + +if __name__ == "__main__": + parser = argparse.ArgumentParser( + description="Get the public SSH keys for members of given org and team" + ) + parser.add_argument("--token", required=True, help="Github PAT") + parser.add_argument( + "--organization", help="GitHub organization name", default="ClickHouse" + ) + parser.add_argument("--team", help="GitHub team name", default="core") + + args = parser.parse_args() + keys = main(args.token, args.organization, args.team) + + print(f"Just shoing off the keys:\n{keys}") diff --git a/tests/ci/team_keys_lambda/requirements.txt b/tests/ci/team_keys_lambda/requirements.txt new file mode 100644 index 00000000000..f2293605cf1 --- /dev/null +++ b/tests/ci/team_keys_lambda/requirements.txt @@ -0,0 +1 @@ +requests From 31a5afe855bb6859dee5910bc36b07f4af6001f1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 6 Dec 2021 12:21:42 +0100 Subject: [PATCH 238/358] Use threads to speed-up parallel requests --- tests/ci/team_keys_lambda/app.py | 54 +++++++++++++++++++++++++++----- 1 file changed, 46 insertions(+), 8 deletions(-) diff --git a/tests/ci/team_keys_lambda/app.py b/tests/ci/team_keys_lambda/app.py index ad34d949d36..8a82e8dd353 100644 --- a/tests/ci/team_keys_lambda/app.py +++ b/tests/ci/team_keys_lambda/app.py @@ -4,6 +4,9 @@ import requests import argparse import json +from threading import Thread +from queue import Queue + def get_org_team_members(token: str, org: str, team_slug: str) -> tuple: headers = { @@ -19,14 +22,42 @@ def get_org_team_members(token: str, org: str, team_slug: str) -> tuple: def get_members_keys(members: tuple) -> str: - keys = "" + class Worker(Thread): + def __init__(self, request_queue): + Thread.__init__(self) + self.queue = request_queue + self.results = [] + + def run(self): + while True: + m = self.queue.get() + if m == "": + break + response = requests.get(f"https://github.com/{m}.keys") + self.results.append(f"# {m}\n{response.text}") + self.queue.task_done() + + q = Queue() + workers = [] for m in members: - response = requests.get( - f"https://github.com/{m}.keys", - ) - response.raise_for_status() - keys += f"# {m}\n{response.text}" - return keys + q.put(m) + # Create workers and add to the queue + worker = Worker(q) + worker.start() + workers.append(worker) + + # Workers keep working till they receive an empty string + for _ in workers: + q.put("") + + # Join workers to wait till they finished + for worker in workers: + worker.join() + + responses = [] + for worker in workers: + responses.extend(worker.results) + return "".join(responses) def get_token_from_aws() -> str: @@ -51,7 +82,14 @@ def main(token: str, org: str, team_slug: str) -> str: def handler(event, context): token = get_token_from_aws() - return main(token, "ClickHouse", "core") + result = { + "statusCode": 200, + "headers": { + "Content-Type": "text/html", + }, + "body": main(token, "ClickHouse", "core"), + } + return result if __name__ == "__main__": From c3e640bfaf279486e23a1c25d8b0f46032710f33 Mon Sep 17 00:00:00 2001 From: kreuzerkrieg Date: Thu, 16 Dec 2021 12:53:02 +0200 Subject: [PATCH 239/358] stop trying to add `-j` to the variable, it simply doesnt work --- debian/rules | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/debian/rules b/debian/rules index 4562d24bec4..62081704e92 100755 --- a/debian/rules +++ b/debian/rules @@ -77,10 +77,6 @@ else THREADS_COUNT = 1 endif -ifneq ($(THREADS_COUNT),) - THREADS_COUNT:=-j$(THREADS_COUNT) -endif - %: dh $@ $(DH_FLAGS) --buildsystem=cmake @@ -89,11 +85,11 @@ override_dh_auto_configure: override_dh_auto_build: # Fix for ninja. Do not add -O. - $(MAKE) $(THREADS_COUNT) -C $(BUILDDIR) $(MAKE_TARGET) + $(MAKE) -j$(THREADS_COUNT) -C $(BUILDDIR) $(MAKE_TARGET) override_dh_auto_test: ifeq (,$(filter nocheck,$(DEB_BUILD_OPTIONS))) - cd $(BUILDDIR) && ctest $(THREADS_COUNT) -V + cd $(BUILDDIR) && ctest -j$(THREADS_COUNT) -V endif override_dh_clean: @@ -120,7 +116,7 @@ override_dh_install: dh_install --list-missing --sourcedir=$(DESTDIR) override_dh_auto_install: - env DESTDIR=$(DESTDIR) $(MAKE) $(THREADS_COUNT) -C $(BUILDDIR) install + env DESTDIR=$(DESTDIR) $(MAKE) -j$(THREADS_COUNT) -C $(BUILDDIR) install override_dh_shlibdeps: true # We depend only on libc and dh_shlibdeps gives us wrong (too strict) dependency. From c174289233417b9b83c60d7d849bfae85f043fb2 Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Thu, 16 Dec 2021 11:09:05 +0000 Subject: [PATCH 240/358] Fixes to 21.12 release blog post. --- .../en/2021/clickhouse-v21.12-released.md | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index 01450280fa0..fde943d3641 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -41,9 +41,9 @@ ClickHouse Keeper development started in Sep 2020, more than a year ago. It was **How does this help you?** -ClickHouse Keeper is a drop-in replacement for ZooKeeper. It implements ZooKeeper wire protocol and data model, but does it better. +ClickHouse Keeper is a drop-in replacement for ZooKeeper. It implements the ZooKeeper wire protocol and data model, but does it better. -In contrast to ZooKeeper, there are no issues with zxid overflow or packet sizes. It has better memory usage and it does not require JVM tuning (because it does not use JVM). Logs and snapshots are compressed (about 10x typical) and checksummed. It can run as a separate process or directly inside clickhouse-server. You can use it with ClickHouse or with your Kafkas and Hadoops as well. +In contrast to ZooKeeper, there are no issues with zxid overflow or packet sizes. It has better memory usage and it does not require JVM tuning (because it does not use the JVM). Logs and snapshots are compressed (by about 10x typically) and checksummed. It can run as a separate process or directly inside clickhouse-server. You can use it with ClickHouse or with your Kafka and Hadoop as well. [More info](http://presentations.clickhouse.tech/meetup54/keeper.pdf). @@ -54,11 +54,11 @@ When using the table engines `File`, `URL`, and `HDFS` ClickHouse now supports p Similarly, when exporting data from ClickHouse using the `file`, `url`, and `hdfs` table functions you can now specify that the data is to be partitioned into multiple files using a `PARTITION BY` clause. For example, `INSERT INTO TABLE FUNCTION file('path/hits_{_partition_id}', 'TSV', 'columns...') PARTITION BY toYYYYMM(EventDate) VALUES ...` will create as many files as there are unique months in the dataset. -The `s3` table function has supported partitioned writes since ClickHouse 21.10. +The `s3` table function has already supported partitioned writes since ClickHouse 21.10. **How does this help you?** -If data is split into multiple files, then `SELECT` query will be automatically parallelized. Example: +If data is split into multiple files, `SELECT` queries will be automatically parallelized. For example: ``` SELECT user_id, count() FROM s3( @@ -68,7 +68,7 @@ SELECT user_id, count() FROM s3( 'user_id UInt64, ...') ``` -You can even parallelize data processing across distributed compute cluster if you use `s3Cluster` table function: +You can even parallelize data processing across a distributed compute cluster if you use the `s3Cluster` table function: ``` SELECT user_id, count() FROM s3Cluster( @@ -79,7 +79,7 @@ SELECT user_id, count() FROM s3Cluster( 'user_id UInt64, ...') ``` -It can also be used for integrations with external data processing tools that consumes data from `s3`. +It can also be used for integration with external data processing tools that consume data from `s3`. ## FROM INFILE in clickhouse-client now supports glob patterns and parallel reading @@ -91,18 +91,18 @@ INSERT INTO my_table FROM INFILE '*.csv.gz' FORMAT CSV ``` Glob patterns support `*`, `?` and `{n..m}` with `{1..10}` or (aligned) `{01..10}` forms. -This query will be automatically parallelized, it will also automatically detect compression format from file extension and decompress transparently. +This query will be automatically parallelized and it will also automatically detect the compression format from the file extension and decompress transparently. This improvement is done by **Arthur Filatenkov**. **How does this help you?** -Now you don't have to recall how to write parallel for loop in your command line shell. clickhouse-client will do everything for you, it works intuitively and fast. +Now you don't have to recall how to write a parallel for loop in your command line shell. clickhouse-client will do everything for you, it works intuitively and fast. ## Support for INTERVAL operator inside WITH FILL modifier for ORDER BY clause -What's the... WITH FILL modifier in ORDER BY clause? Just look at the example. +What's the... `WITH FILL` modifier in the `ORDER BY` clause? Take a look at the example: ``` :) SELECT EventDate, count() FROM test.hits WHERE CounterID = 2841673 GROUP BY EventDate ORDER BY EventDate @@ -115,10 +115,10 @@ What's the... WITH FILL modifier in ORDER BY clause? Just look at the example. └────────────┴─────────┘ ``` -We have the report with Mar 17th, 19th, 21th, 22th. But Mar 18th and 20th are missing, because there is no data for these dates. +We have the report with Mar 17th, 19th, 21st, and 22nd. But Mar 18th and 20th are missing, because there is no data for these dates. And this is how it works in all SQL databases. -But ClickHouse also has quite unique and neat `WITH FILL` modifier for `ORDER BY clause`. +But ClickHouse also has a quite unique and neat `WITH FILL` modifier for the `ORDER BY` clause. You just write: ``` @@ -140,12 +140,12 @@ And missing data is automatically filled. You can also add `FROM` and `TO`: ``` -ORDER BY date WITH FILL FROM '2014-03-01'::Date TO '2014-03-31'::Date STEP 1; +ORDER BY EventDate WITH FILL FROM '2014-03-01'::Date TO '2014-03-31'::Date STEP 1; ``` And it will automatically fill missing rows in the report. -The STEP can be arbitrary number. But what to do if you want fill missing dates for report by months? You cannot just write STEP 30 or STEP 31 because months contain different number of days... +The `STEP` can be an arbitrary number. But what can you do if you want to fill missing dates for a report by months? You cannot just write `STEP 30` or `STEP 31` because different months contain different number of days... Since ClickHouse version 21.12 you can do it like this: @@ -155,16 +155,16 @@ ORDER BY EventDate WITH FILL STEP INTERVAL 1 MONTH `INTERVAL` is a standard SQL operator, you can use SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER and YEAR. -This is implemented by **Anton Popov** who is the author of "WITH FILL" feature. +This is implemented by **Anton Popov** who is the author of the "WITH FILL" feature. **How does this help you?** -It allows to avoid postprocessing step for your reports. +It allows you to avoid a postprocessing step for your reports. ## Add Support For "Identifier" Table and Database Query Parameters -ClickHouse has support for parameterized queries. +ClickHouse has support for parameterized queries. For example: ``` SELECT uniq(user_id) FROM table WHERE website = {name:String} @@ -178,7 +178,7 @@ curl https://clickhouse-server:8443/?param_name=upyachka -d 'SELECT uniq(user_id You can even create customized API handlers for clickhouse-server based on prepared queries. -Since version 21.12 we introduce support for using parameters for tables and databases in your queries. This is implemented with `Identifier` table parameter: +In version 21.12 we introduce support for using parameters for tables and databases in your queries. This is implemented with the `Identifier` table parameter: ``` SELECT uniq(user_id) FROM {tbl:Identifier} @@ -188,7 +188,7 @@ Identifier parameters also work for CREATE, DROP and all DDL queries. This is im **How does this help you?** -Let ClickHouse do the heavy-lifting and keep your scripts safe and secure. +Let ClickHouse do the heavy lifting and keep your scripts safe and secure. ## Bool Data Type @@ -220,7 +220,7 @@ CREATE TABLE ) ... ``` -Constraints are checked on INSERT. In this example we validate the URL and check that Domain column actually contains the domain of URL. +Constraints are checked on `INSERT`. In this example we validate the URL and check that the `Domain` column actually contains the domain of the URL. Since version 21.12 constraints can also automatically optimize your queries! For example, if you write: @@ -234,19 +234,19 @@ The query can be automatically rewritten to: SELECT count() FROM hits WHERE Domain = 'ghe.clickhouse.tech' ``` -because `Domain` column is smaller, more compressable, will be faster to read and it does not require calculation of the domain from URL. -The only thing you need is to enable the `optimize_using_constraints` and `optimize_substitute_columns` settings. +Because the `Domain` column is smaller and more compressable it will be faster to read and does not require calculation of the domain from the URL. +The only thing you need to do is to enable the `optimize_using_constraints` and `optimize_substitute_columns` settings. -As a bonus, new type of constraints is introduced: `ASSUME`. +As a bonus, we introduced a new type of constraint: `ASSUME`. ``` CONSTRAINT my_constraint ASSUME Domain = domainWithoutWWW(URL) ``` -This type of constraint will not check anything on INSERT, but still use the assumption to optimize the queries. +This type of constraint will not check anything on `INSERT` but still use the assumption to optimize the queries. It can also do logical inference, simplify the conditions and remove the conditions that are proved to be satisfied by constraints. -It is controlled by `convert_query_to_cnf` setting. You can also enable `optimize_append_index` setting. With this setting ClickHouse will derive more consitions on the table primary key. +It is controlled by the `convert_query_to_cnf` setting. You can also enable `optimize_append_index`. With this setting ClickHouse will derive more conditions on the table primary key. The idea is so powerful that we cannot resist adding one more feature: *indices for hypothesis*. @@ -267,11 +267,11 @@ Rather than tell all your users to change their queries you can use a table cons ## Read Large Remote Files In Chunks -ClickHouse combines fast query engine and efficient data storage. It also allows to integrate external data sources for data import and export or even to process external datasets on the fly without the need for data import or preprocessing. +ClickHouse combines a fast query engine and efficient data storage. It also allows to integrate external data sources for data import and export or even to process external datasets on the fly without the need for data import or preprocessing. When reading large files in `Parquet`, `ORC`, and `Arrow` format using the `s3`, `url`, and `hdfs` table functions, ClickHouse will now automatically choose whether to read the entire file at once or read parts of it incrementally. This is now enabled by default and the setting `remote_read_min_bytes_for_seek` controls when to switch from reading it all to reading in chunks. The default is 1MiB. -`Parquet`, `ORC`, and `Arrow` are column-oriented formats (quite similar to ClickHouse Native format) and now we can read only requested columns even if they are being read from remote HTTP server with the `url` table function (range requests will be performed to skip unneeded data). +`Parquet`, `ORC`, and `Arrow` are column-oriented formats (quite similar to the ClickHouse Native format) and now we can read only requested columns even if they are being read from a remote HTTP server with the `url` table function (range requests will be performed to skip unneeded data). This feature is implemented by **Kseniia Sumarokova**. @@ -282,4 +282,4 @@ In previous versions, when reading files in Arrow-based formats from remote loca ## ... And Many More -Read the [full changelog](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) for 21.12 "Christmas" release for the full list of the gifts from [ClickHouse Team](https://clickhouse.com/careers/). +Read the [full changelog](https://github.com/ClickHouse/ClickHouse/blob/master/CHANGELOG.md) for the 21.12 "Christmas" release for the full list of gifts from the [ClickHouse Team](https://clickhouse.com/company/). From c338bee718dea86e5b126f259c4dcbad91990cff Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 14:37:12 +0300 Subject: [PATCH 241/358] Revert "Fix build issue related to azure blob storage" --- contrib/azure-cmake/CMakeLists.txt | 7 ++----- src/CMakeLists.txt | 14 -------------- src/Disks/BlobStorage/BlobStorageAuth.h | 18 ------------------ src/Disks/BlobStorage/DiskBlobStorage.h | 16 ---------------- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 19 ------------------- src/IO/ReadBufferFromBlobStorage.h | 18 ------------------ src/IO/WriteBufferFromBlobStorage.h | 17 ----------------- 7 files changed, 2 insertions(+), 107 deletions(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index ed428364598..7f9476e37b7 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -46,17 +46,14 @@ include("${AZURE_DIR}/cmake-modules/AzureTransportAdapters.cmake") add_library(azure_sdk ${AZURE_SDK_UNIFIED_SRC}) if (COMPILER_CLANG) - target_compile_options(azure_sdk PRIVATE + target_compile_options(azure_sdk PUBLIC -Wno-deprecated-copy-dtor -Wno-extra-semi -Wno-suggest-destructor-override -Wno-inconsistent-missing-destructor-override -Wno-error=unknown-warning-option + -Wno-reserved-identifier ) - - if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 13) - target_compile_options(azure_sdk PRIVATE -Wno-reserved-identifier) - endif() endif() # Originally, on Windows azure-core is built with bcrypt and crypt32 by default diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 428f817ac49..bca9bd9d280 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -385,20 +385,6 @@ dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${PDQSORT_INCLUDE_DIR}) target_include_directories(clickhouse_common_io BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR}) dbms_target_include_directories(SYSTEM BEFORE PUBLIC ${MINISELECT_INCLUDE_DIR}) -macro (add_check_flag_definitions flag modules) - if (${flag}) - if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) - target_compile_definitions(clickhouse_common_io PRIVATE ${flag}) - target_compile_definitions(dbms PRIVATE ${flag}) - else () - foreach (module ${modules}) - target_compile_definitions(${module} PRIVATE ${flag}) - endforeach () - endif () - endif () -endmacro () -add_check_flag_definitions(HAS_SUGGEST_DESTRUCTOR_OVERRIDE "clickhouse_common_io;clickhouse_disks") - if (ZSTD_LIBRARY) dbms_target_link_libraries(PRIVATE ${ZSTD_LIBRARY}) target_link_libraries (clickhouse_common_io PUBLIC ${ZSTD_LIBRARY}) diff --git a/src/Disks/BlobStorage/BlobStorageAuth.h b/src/Disks/BlobStorage/BlobStorageAuth.h index 04abc2fd643..16a088fc960 100644 --- a/src/Disks/BlobStorage/BlobStorageAuth.h +++ b/src/Disks/BlobStorage/BlobStorageAuth.h @@ -7,26 +7,8 @@ #if USE_AZURE_BLOB_STORAGE #include - -#if defined(__clang__) -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" -# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" -# pragma clang diagnostic ignored "-Wextra-semi" -# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE -# pragma clang diagnostic ignored "-Wsuggest-destructor-override" -# endif -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif -#endif - #include -#if defined(__clang__) -# pragma clang diagnostic pop -#endif - namespace DB { diff --git a/src/Disks/BlobStorage/DiskBlobStorage.h b/src/Disks/BlobStorage/DiskBlobStorage.h index d124a133eb2..16ba216df96 100644 --- a/src/Disks/BlobStorage/DiskBlobStorage.h +++ b/src/Disks/BlobStorage/DiskBlobStorage.h @@ -11,25 +11,9 @@ #include #include -#if defined(__clang__) -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" -# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" -# pragma clang diagnostic ignored "-Wextra-semi" -# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE -# pragma clang diagnostic ignored "-Wsuggest-destructor-override" -# endif -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif -#endif - #include #include -#if defined(__clang__) -# pragma clang diagnostic pop -#endif namespace DB { diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 6bef9488d05..045ab43850d 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -6,26 +6,7 @@ #include #if USE_AZURE_BLOB_STORAGE - -#if defined(__clang__) -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" -# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" -# pragma clang diagnostic ignored "-Wextra-semi" -# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE -# pragma clang diagnostic ignored "-Wsuggest-destructor-override" -# endif -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif -#endif - #include - -#if defined(__clang__) -# pragma clang diagnostic pop -#endif - #endif namespace Aws diff --git a/src/IO/ReadBufferFromBlobStorage.h b/src/IO/ReadBufferFromBlobStorage.h index 029b53641eb..cd66e897e25 100644 --- a/src/IO/ReadBufferFromBlobStorage.h +++ b/src/IO/ReadBufferFromBlobStorage.h @@ -9,26 +9,8 @@ #include #include #include - -#if defined(__clang__) -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" -# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" -# pragma clang diagnostic ignored "-Wextra-semi" -# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE -# pragma clang diagnostic ignored "-Wsuggest-destructor-override" -# endif -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif -#endif - #include -#if defined(__clang__) -# pragma clang diagnostic pop -#endif - namespace DB { diff --git a/src/IO/WriteBufferFromBlobStorage.h b/src/IO/WriteBufferFromBlobStorage.h index 4bb995f907d..5f8eaba3c0c 100644 --- a/src/IO/WriteBufferFromBlobStorage.h +++ b/src/IO/WriteBufferFromBlobStorage.h @@ -10,26 +10,9 @@ #include #include - -#if defined(__clang__) -# pragma clang diagnostic push -# pragma clang diagnostic ignored "-Winconsistent-missing-destructor-override" -# pragma clang diagnostic ignored "-Wdeprecated-copy-dtor" -# pragma clang diagnostic ignored "-Wextra-semi" -# ifdef HAS_SUGGEST_DESTRUCTOR_OVERRIDE -# pragma clang diagnostic ignored "-Wsuggest-destructor-override" -# endif -# ifdef HAS_RESERVED_IDENTIFIER -# pragma clang diagnostic ignored "-Wreserved-identifier" -# endif -#endif - #include #include -#if defined(__clang__) -# pragma clang diagnostic pop -#endif namespace DB { From 3ae654168c9fe53ea0233cea945ea2290bccb6dc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 14 Dec 2021 18:47:45 +0100 Subject: [PATCH 242/358] Apply black formatter --- tests/ci/ci_config.py | 40 ++++++++++++++++++++-------------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 9297b25cbb7..6cca75f0834 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -11,7 +11,7 @@ CI_CONFIG = { "splitted": "unsplitted", "alien_pkgs": True, "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "performance": { "compiler": "clang-13", @@ -21,7 +21,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_gcc": { "compiler": "gcc-11", @@ -31,7 +31,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "package_asan": { "compiler": "clang-13", @@ -41,7 +41,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "package_ubsan": { "compiler": "clang-13", @@ -51,7 +51,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "package_tsan": { "compiler": "clang-13", @@ -61,7 +61,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "package_msan": { "compiler": "clang-13", @@ -71,7 +71,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "package_debug": { "compiler": "clang-13", @@ -81,7 +81,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_release": { "compiler": "clang-13", @@ -91,7 +91,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_tidy": { "compiler": "clang-13", @@ -101,7 +101,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "enable", - "with_coverage": False + "with_coverage": False, }, "binary_splitted": { "compiler": "clang-13", @@ -111,7 +111,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "splitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_darwin": { "compiler": "clang-13-darwin", @@ -121,7 +121,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_aarch64": { "compiler": "clang-13-aarch64", @@ -131,7 +131,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_freebsd": { "compiler": "clang-13-freebsd", @@ -141,7 +141,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_darwin_aarch64": { "compiler": "clang-13-darwin-aarch64", @@ -151,7 +151,7 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False + "with_coverage": False, }, "binary_ppc64le": { "compiler": "clang-13-ppc64le", @@ -161,8 +161,8 @@ CI_CONFIG = { "bundled": "bundled", "splitted": "unsplitted", "tidy": "disable", - "with_coverage": False - } + "with_coverage": False, + }, }, "builds_report_config": { "ClickHouse build check (actions)": [ @@ -173,7 +173,7 @@ CI_CONFIG = { "package_tsan", "package_msan", "package_debug", - "binary_release" + "binary_release", ], "ClickHouse special build check (actions)": [ "binary_tidy", @@ -320,6 +320,6 @@ CI_CONFIG = { }, "Performance Comparison (actions)": { "required_build": "performance", - } - } + }, + }, } From eb5a26891f8a94824089cff7b51a6b2b00bf238c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 8 Dec 2021 22:52:00 +0100 Subject: [PATCH 243/358] Add aarch64 deb package --- .github/workflows/main.yml | 42 ++++++++++++++++++++++++++++++++++ debian/rules | 4 ++++ docker/packager/deb/Dockerfile | 10 ++++++-- docker/packager/packager | 2 ++ release | 2 +- tests/ci/ci_config.py | 11 +++++++++ 6 files changed, 68 insertions(+), 3 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index acd365aea9a..590c59cf477 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -329,6 +329,47 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH $CACHES_PATH + BuilderDebAarch64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + CHECK_NAME=ClickHouse build check (actions) + BUILD_NAME=package_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NAME + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH $CACHES_PATH BuilderDebAsan: needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] @@ -867,6 +908,7 @@ jobs: needs: - BuilderDebRelease - BuilderBinRelease + - BuilderDebAarch64 - BuilderDebAsan - BuilderDebTsan - BuilderDebUBsan diff --git a/debian/rules b/debian/rules index 4562d24bec4..00eedad7829 100755 --- a/debian/rules +++ b/debian/rules @@ -45,6 +45,10 @@ ifdef DEB_CXX ifeq ($(DEB_BUILD_GNU_TYPE),$(DEB_HOST_GNU_TYPE)) CC := $(DEB_CC) CXX := $(DEB_CXX) +else ifeq (clang,$(findstring clang,$(DEB_CXX))) +# If we crosscompile with clang, it knows what to do + CC := $(DEB_CC) + CXX := $(DEB_CXX) else CC := $(DEB_HOST_GNU_TYPE)-$(DEB_CC) CXX := $(DEB_HOST_GNU_TYPE)-$(DEB_CXX) diff --git a/docker/packager/deb/Dockerfile b/docker/packager/deb/Dockerfile index 873edfe4afc..89c34846efa 100644 --- a/docker/packager/deb/Dockerfile +++ b/docker/packager/deb/Dockerfile @@ -64,8 +64,14 @@ RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ && apt-get install gcc-11 g++-11 --yes -# This symlink required by gcc to find lld compiler -RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld +# These symlinks are required: +# /usr/bin/ld.lld: by gcc to find lld compiler +# /usr/bin/aarch64-linux-gnu-obj*: for debug symbols stripping +RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld \ + && ln -sf /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-objcopy /usr/bin/aarch64-linux-gnu-strip \ + && ln -sf /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-objcopy /usr/bin/aarch64-linux-gnu-objcopy \ + && ln -sf /usr/lib/llvm-${LLVM_VERSION}/bin/llvm-objdump /usr/bin/aarch64-linux-gnu-objdump + COPY build.sh / diff --git a/docker/packager/packager b/docker/packager/packager index 9cce12be949..cd2427dec86 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -90,6 +90,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ elif is_cross_arm: cc = compiler[:-len(ARM_SUFFIX)] cmake_flags.append("-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-aarch64.cmake") + result.append("DEB_ARCH_FLAG=-aarm64") elif is_cross_freebsd: cc = compiler[:-len(FREEBSD_SUFFIX)] cmake_flags.append("-DCMAKE_TOOLCHAIN_FILE=/build/cmake/freebsd/toolchain-x86_64.cmake") @@ -98,6 +99,7 @@ def parse_env_variables(build_type, compiler, sanitizer, package_type, image_typ cmake_flags.append("-DCMAKE_TOOLCHAIN_FILE=/build/cmake/linux/toolchain-ppc64le.cmake") else: cc = compiler + result.append("DEB_ARCH_FLAG=-aamd64") cxx = cc.replace('gcc', 'g++').replace('clang', 'clang++') diff --git a/release b/release index 6e6970d7b00..3eb5591fe2c 100755 --- a/release +++ b/release @@ -87,7 +87,7 @@ if [ -z "$NO_BUILD" ] ; then # Build (only binary packages). debuild --preserve-env -e PATH \ -e DEB_CC=$DEB_CC -e DEB_CXX=$DEB_CXX -e CMAKE_FLAGS="$CMAKE_FLAGS" \ - -b ${DEBUILD_NOSIGN_OPTIONS} ${DEBUILD_NODEPS_OPTIONS} + -b ${DEBUILD_NOSIGN_OPTIONS} ${DEBUILD_NODEPS_OPTIONS} ${DEB_ARCH_FLAG} fi if [ -n "$MAKE_RPM" ]; then diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 6cca75f0834..aa9f259ad53 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -33,6 +33,16 @@ CI_CONFIG = { "tidy": "disable", "with_coverage": False, }, + "package_aarch64": { + "compiler": "clang-13-aarch64", + "build_type": "", + "sanitizer": "", + "package_type": "deb", + "bundled": "bundled", + "splitted": "unsplitted", + "tidy": "disable", + "with_coverage": False, + }, "package_asan": { "compiler": "clang-13", "build_type": "", @@ -168,6 +178,7 @@ CI_CONFIG = { "ClickHouse build check (actions)": [ "package_release", "performance", + "package_aarch64", "package_asan", "package_ubsan", "package_tsan", From 91083bea214bf2947f961e03f3c12b10ccfbd0fa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 14 Dec 2021 15:19:47 +0100 Subject: [PATCH 244/358] Fix building of packager containers --- docker/packager/packager | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/packager/packager b/docker/packager/packager index cd2427dec86..c042db2251d 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -29,7 +29,13 @@ def pull_image(image_name): return False def build_image(image_name, filepath): - subprocess.check_call("docker build --network=host -t {} -f {} .".format(image_name, filepath), shell=True) + context = os.path.dirname(filepath) + subprocess.check_call( + "docker build --network=host -t {} -f {} {}".format( + image_name, filepath, context + ), + shell=True, + ) def run_docker_image_with_env(image_name, output, env_variables, ch_root, ccache_dir, docker_image_version): env_part = " -e ".join(env_variables) From d2d1f5aba7745458216c61fb7a74efca1221118f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 14 Dec 2021 15:22:31 +0100 Subject: [PATCH 245/358] Optimize binary packager container --- docker/packager/binary/Dockerfile | 31 +++++++++++++------------------ 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 28e84d359b3..8f886ea357d 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -24,40 +24,34 @@ RUN apt-get update \ && apt-key add /tmp/llvm-snapshot.gpg.key \ && export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \ && echo "deb [trusted=yes] https://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \ - /etc/apt/sources.list + /etc/apt/sources.list \ + && apt-get clean # initial packages -RUN apt-get update \ - && apt-get install \ - bash \ - fakeroot \ - ccache \ - curl \ - software-properties-common \ - --yes --no-install-recommends - RUN apt-get update \ && apt-get install \ bash \ build-essential \ ccache \ + clang-${LLVM_VERSION} \ + clang-tidy-${LLVM_VERSION} \ cmake \ curl \ + fakeroot \ gdb \ git \ gperf \ - clang-${LLVM_VERSION} \ - clang-tidy-${LLVM_VERSION} \ lld-${LLVM_VERSION} \ llvm-${LLVM_VERSION} \ llvm-${LLVM_VERSION}-dev \ - libicu-dev \ moreutils \ ninja-build \ pigz \ rename \ + software-properties-common \ tzdata \ - --yes --no-install-recommends + --yes --no-install-recommends \ + && apt-get clean # This symlink required by gcc to find lld compiler RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld @@ -66,7 +60,7 @@ ENV CC=clang-${LLVM_VERSION} ENV CXX=clang++-${LLVM_VERSION} # libtapi is required to support .tbh format from recent MacOS SDKs -RUN git clone https://github.com/tpoechtrager/apple-libtapi.git \ +RUN git clone --depth 1 https://github.com/tpoechtrager/apple-libtapi.git \ && cd apple-libtapi \ && INSTALLPREFIX=/cctools ./build.sh \ && ./install.sh \ @@ -74,7 +68,7 @@ RUN git clone https://github.com/tpoechtrager/apple-libtapi.git \ && rm -rf apple-libtapi # Build and install tools for cross-linking to Darwin (x86-64) -RUN git clone https://github.com/tpoechtrager/cctools-port.git \ +RUN git clone --depth 1 https://github.com/tpoechtrager/cctools-port.git \ && cd cctools-port/cctools \ && ./configure --prefix=/cctools --with-libtapi=/cctools \ --target=x86_64-apple-darwin \ @@ -83,7 +77,7 @@ RUN git clone https://github.com/tpoechtrager/cctools-port.git \ && rm -rf cctools-port # Build and install tools for cross-linking to Darwin (aarch64) -RUN git clone https://github.com/tpoechtrager/cctools-port.git \ +RUN git clone --depth 1 https://github.com/tpoechtrager/cctools-port.git \ && cd cctools-port/cctools \ && ./configure --prefix=/cctools --with-libtapi=/cctools \ --target=aarch64-apple-darwin \ @@ -97,7 +91,8 @@ RUN wget -nv https://github.com/phracker/MacOSX-SDKs/releases/download/11.3/MacO # NOTE: Seems like gcc-11 is too new for ubuntu20 repository RUN add-apt-repository ppa:ubuntu-toolchain-r/test --yes \ && apt-get update \ - && apt-get install gcc-11 g++-11 --yes + && apt-get install gcc-11 g++-11 --yes \ + && apt-get clean COPY build.sh / From 0c55e27066ac0936104eb2fb86b5ce10e8874ef2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Dec 2021 20:47:08 +0100 Subject: [PATCH 246/358] Unbuffer python outputs to get realtime logs --- .github/workflows/anchore-analysis.yml | 4 ++++ .github/workflows/backport.yml | 5 +++++ .github/workflows/backport_branches.yml | 5 +++++ .github/workflows/cancel.yml | 5 +++++ .github/workflows/docs_check.yml | 5 +++++ .github/workflows/main.yml | 5 +++++ .github/workflows/master.yml | 5 +++++ .github/workflows/release.yml | 5 +++++ .github/workflows/release_branches.yml | 5 +++++ 9 files changed, 44 insertions(+) diff --git a/.github/workflows/anchore-analysis.yml b/.github/workflows/anchore-analysis.yml index 1005c8f6c38..9f3f944c696 100644 --- a/.github/workflows/anchore-analysis.yml +++ b/.github/workflows/anchore-analysis.yml @@ -8,6 +8,10 @@ name: Docker Container Scan (clickhouse-server) +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + "on": pull_request: paths: diff --git a/.github/workflows/backport.yml b/.github/workflows/backport.yml index 284af965714..163c91d05fc 100644 --- a/.github/workflows/backport.yml +++ b/.github/workflows/backport.yml @@ -1,4 +1,9 @@ name: CherryPick + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + concurrency: group: cherry-pick on: # yamllint disable-line rule:truthy diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 53d7a4aece8..a9b3931fbaf 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -1,4 +1,9 @@ name: BackportPR + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + on: # yamllint disable-line rule:truthy push: branches: diff --git a/.github/workflows/cancel.yml b/.github/workflows/cancel.yml index dccd27c4319..857cbf2c495 100644 --- a/.github/workflows/cancel.yml +++ b/.github/workflows/cancel.yml @@ -1,4 +1,9 @@ name: Cancel + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + on: # yamllint disable-line rule:truthy workflow_run: workflows: ["CIGithubActions", "ReleaseCI", "DocsCheck", "BackportPR"] diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 5f860e52ec1..8e75e517657 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -1,4 +1,9 @@ name: DocsCheck + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + on: # yamllint disable-line rule:truthy pull_request: types: diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 590c59cf477..0e62c6e4601 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -1,4 +1,9 @@ name: CIGithubActions + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + on: # yamllint disable-line rule:truthy pull_request: types: diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 4a756c5e0e9..19adf028084 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1,4 +1,9 @@ name: MasterCI + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + on: # yamllint disable-line rule:truthy push: branches: diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 977dbf07dbe..49ee424816f 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -1,4 +1,9 @@ name: DocsReleaseChecks + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + concurrency: group: master-release cancel-in-progress: true diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index c9f553fbda4..bad444d9961 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -1,4 +1,9 @@ name: ReleaseCI + +env: + # Force the stdout and stderr streams to be unbuffered + PYTHONUNBUFFERED: 1 + on: # yamllint disable-line rule:truthy push: branches: From 9a6e8dc61505b6bf9b27c61fc56e4fa257653aa6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 15 Dec 2021 20:51:41 +0100 Subject: [PATCH 247/358] Build rpm and tgz for aarch64 too --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index aa9f259ad53..d5f8757ffdf 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -40,6 +40,7 @@ CI_CONFIG = { "package_type": "deb", "bundled": "bundled", "splitted": "unsplitted", + "alien_pkgs": True, "tidy": "disable", "with_coverage": False, }, From 2945eac8c9af0f4cec3c2b2e86fc67d5999f4306 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Dec 2021 15:05:42 +0300 Subject: [PATCH 248/358] fix build --- src/DataTypes/tests/gtest_split_name.cpp | 38 ++++++++++++------------ 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/src/DataTypes/tests/gtest_split_name.cpp b/src/DataTypes/tests/gtest_split_name.cpp index 2cc2f0a8be0..09e5c11f09b 100644 --- a/src/DataTypes/tests/gtest_split_name.cpp +++ b/src/DataTypes/tests/gtest_split_name.cpp @@ -6,27 +6,27 @@ using namespace DB; TEST(SplitName, forward) { - ASSERT_EQ(Nested::splitName("abc"), (std::pair{"abc", ""})); - ASSERT_EQ(Nested::splitName("a.b"), (std::pair{"a", "b"})); - ASSERT_EQ(Nested::splitName("a.b.c"), (std::pair{"a", "b.c"})); - ASSERT_EQ(Nested::splitName("a.1"), (std::pair{"a", "1"})); - ASSERT_EQ(Nested::splitName("a.1.b"), (std::pair{"a", "1.b"})); - ASSERT_EQ(Nested::splitName("1.a"), (std::pair{"1", "a"})); - ASSERT_EQ(Nested::splitName("a.b1.b2"), (std::pair{"a", "b1.b2"})); - ASSERT_EQ(Nested::splitName("a.b1.2a.3a"), (std::pair{"a", "b1.2a.3a"})); - ASSERT_EQ(Nested::splitName(".."), (std::pair{"..", ""})); + ASSERT_EQ(Nested::splitName(String("abc")), (std::pair{"abc", ""})); + ASSERT_EQ(Nested::splitName(String("a.b")), (std::pair{"a", "b"})); + ASSERT_EQ(Nested::splitName(String("a.b.c")), (std::pair{"a", "b.c"})); + ASSERT_EQ(Nested::splitName(String("a.1")), (std::pair{"a", "1"})); + ASSERT_EQ(Nested::splitName(String("a.1.b")), (std::pair{"a", "1.b"})); + ASSERT_EQ(Nested::splitName(String("1.a")), (std::pair{"1", "a"})); + ASSERT_EQ(Nested::splitName(String("a.b1.b2")), (std::pair{"a", "b1.b2"})); + ASSERT_EQ(Nested::splitName(String("a.b1.2a.3a")), (std::pair{"a", "b1.2a.3a"})); + ASSERT_EQ(Nested::splitName(String("..")), (std::pair{"..", ""})); } TEST(SplitName, reverse) { - ASSERT_EQ(Nested::splitName("abc", true), (std::pair{"abc", ""})); - ASSERT_EQ(Nested::splitName("a.b", true), (std::pair{"a", "b"})); - ASSERT_EQ(Nested::splitName("a.b.c", true), (std::pair{"a.b", "c"})); - ASSERT_EQ(Nested::splitName("a.1", true), (std::pair{"a", "1"})); - ASSERT_EQ(Nested::splitName("a.1a.b", true), (std::pair{"a.1a", "b"})); - ASSERT_EQ(Nested::splitName("1a.b", true), (std::pair{"1a", "b"})); - ASSERT_EQ(Nested::splitName("a.b1.b2", true), (std::pair{"a.b1", "b2"})); - ASSERT_EQ(Nested::splitName("a.b1.2a.3a", true), (std::pair{"a.b1.2a", "3a"})); - ASSERT_EQ(Nested::splitName("a.b1.b2.b3", true), (std::pair{"a.b1.b2", "b3"})); - ASSERT_EQ(Nested::splitName("..", true), (std::pair{"..", ""})); + ASSERT_EQ(Nested::splitName(String("abc", true)), (std::pair{"abc", ""})); + ASSERT_EQ(Nested::splitName(String("a.b", true)), (std::pair{"a", "b"})); + ASSERT_EQ(Nested::splitName(String("a.b.c", true)), (std::pair{"a.b", "c"})); + ASSERT_EQ(Nested::splitName(String("a.1", true)), (std::pair{"a", "1"})); + ASSERT_EQ(Nested::splitName(String("a.1a.b", true)), (std::pair{"a.1a", "b"})); + ASSERT_EQ(Nested::splitName(String("1a.b", true)), (std::pair{"1a", "b"})); + ASSERT_EQ(Nested::splitName(String("a.b1.b2", true)), (std::pair{"a.b1", "b2"})); + ASSERT_EQ(Nested::splitName(String("a.b1.2a.3a", true)), (std::pair{"a.b1.2a", "3a"})); + ASSERT_EQ(Nested::splitName(String("a.b1.b2.b3", true)), (std::pair{"a.b1.b2", "b3"})); + ASSERT_EQ(Nested::splitName(String("..", true)), (std::pair{"..", ""})); } From ad342178736b6f950fdc82cf158ee8229bcadcec Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 16 Dec 2021 17:42:49 +0800 Subject: [PATCH 249/358] Add hints for clickhouse client options --- src/Client/ClientBase.cpp | 17 +++++++++++++++++ src/Client/ClientBase.h | 6 +++++- 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 58bc239f003..136b9bd6c5c 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1635,7 +1635,13 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, /// Check unrecognized options without positional options. auto unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::exclude_positional); if (!unrecognized_options.empty()) + { + auto hints = this->getHints(unrecognized_options[0]); + if (!hints.empty()) + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", unrecognized_options[0], toString(hints)); + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); + } /// Check positional options (options after ' -- ', ex: clickhouse-client -- ). unrecognized_options = po::collect_unrecognized(parsed.options, po::collect_unrecognized_mode::include_positional); @@ -1713,6 +1719,17 @@ void ClientBase::init(int argc, char ** argv) ; addOptions(options_description); + + auto getter = [](const auto & op) { + String op_long_name = op->long_name(); + return "--" + String(op_long_name); + }; + + const auto & main_options = options_description.main_description->options(); + const auto & external_options = options_description.external_description->options(); + std::transform(main_options.begin(), main_options.end(), std::back_inserter(cmd_options), getter); + std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter); + parseAndCheckOptions(options_description, options, common_arguments); po::notify(options); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 4c5d29b390b..1926df5afea 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -1,5 +1,6 @@ #pragma once +#include "Common/NamePrompter.h" #include #include #include @@ -37,7 +38,7 @@ void interruptSignalHandler(int signum); class InternalTextLogs; -class ClientBase : public Poco::Util::Application +class ClientBase : public Poco::Util::Application, public IHints<2, ClientBase> { public: @@ -48,6 +49,8 @@ public: void init(int argc, char ** argv); + std::vector getAllRegisteredNames() const override { return cmd_options; } + protected: void runInteractive(); void runNonInteractive(); @@ -145,6 +148,7 @@ protected: std::vector queries_files; /// If not empty, queries will be read from these files std::vector interleave_queries_files; /// If not empty, run queries from these files before processing every file from 'queries_files'. + std::vector cmd_options; bool stdin_is_a_tty = false; /// stdin is a terminal. bool stdout_is_a_tty = false; /// stdout is a terminal. From bf1098484e51d4a6f2768a7f0f53d9e4075f41f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Thu, 16 Dec 2021 20:31:06 +0800 Subject: [PATCH 250/358] modify code style --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 136b9bd6c5c..3fa39b13551 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1639,7 +1639,7 @@ void ClientBase::parseAndCheckOptions(OptionsDescription & options_description, auto hints = this->getHints(unrecognized_options[0]); if (!hints.empty()) throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'. Maybe you meant {}", unrecognized_options[0], toString(hints)); - + throw Exception(ErrorCodes::UNRECOGNIZED_ARGUMENTS, "Unrecognized option '{}'", unrecognized_options[0]); } From 0b1b4fe9addc00cdc3905d823e03a0eda2590aaa Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Thu, 16 Dec 2021 12:31:28 +0000 Subject: [PATCH 251/358] Fix list formatting in Distributed docs. --- docs/en/engines/table-engines/special/distributed.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/engines/table-engines/special/distributed.md b/docs/en/engines/table-engines/special/distributed.md index d169e137038..faa1026b919 100644 --- a/docs/en/engines/table-engines/special/distributed.md +++ b/docs/en/engines/table-engines/special/distributed.md @@ -39,10 +39,10 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] AS [db2.]name2 - `policy_name` - (optionally) policy name, it will be used to store temporary files for async send -See also: +**See Also** - - [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) setting - - [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) for the examples + - [insert_distributed_sync](../../../operations/settings/settings.md#insert_distributed_sync) setting + - [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) for the examples **Distributed Settings** From 0f3e2bb32f553b83a4bd8a9f445e32e66edb002d Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 16 Dec 2021 15:39:25 +0300 Subject: [PATCH 252/358] Fix removing constant columns --- src/Interpreters/ExpressionAnalyzer.cpp | 23 ++++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index afe98796d95..d1e799b0ec9 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -360,24 +360,29 @@ void ExpressionAnalyzer::analyzeAggregation(ActionsDAGPtr & temp_actions) for (ssize_t j = 0; j < ssize_t(group_elements_ast.size()); ++j) { + ssize_t group_size = group_elements_ast.size(); const auto & column_name = group_elements_ast[j]->getColumnName(); const auto * node = temp_actions->tryFindInIndex(column_name); if (!node) throw Exception("Unknown identifier (in GROUP BY): " + column_name, ErrorCodes::UNKNOWN_IDENTIFIER); - /// Constant expressions have non-null column pointer at this stage. - if (node->column && isColumnConst(*node->column)) + /// Only removes constant keys if it's an initiator or distributed_group_by_no_merge is enabled. + if (getContext()->getClientInfo().distributed_depth == 0 || settings.distributed_group_by_no_merge > 0) { - /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. - if (!aggregate_descriptions.empty() || size > 1) + /// Constant expressions have non-null column pointer at this stage. + if (node->column && isColumnConst(*node->column)) { - if (j + 1 < static_cast(size)) - group_asts[i] = std::move(group_asts.back()); + /// But don't remove last key column if no aggregate functions, otherwise aggregation will not work. + if (!aggregate_descriptions.empty() || group_size > 1) + { + if (j + 1 < static_cast(group_size)) + group_elements_ast[j] = std::move(group_elements_ast.back()); - group_asts.pop_back(); + group_elements_ast.pop_back(); - --j; - continue; + --j; + continue; + } } } From a0afe677777741099205c257aa44fac86978af67 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 16 Dec 2021 16:13:40 +0300 Subject: [PATCH 253/358] Add cwurm to TRUSTED_CONTIBUTORS --- tests/ci/run_check.py | 1 + tests/ci/workflow_approve_rerun_lambda/app.py | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 176a0f4d4c1..bcaefbce07a 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -34,6 +34,7 @@ TRUSTED_CONTRIBUTORS = {e.lower() for e in [ "bobrik", # Seasoned contributor, CloundFlare "BohuTANG", "codyrobert", # Flickerbox engineer + "cwurm", # Employee "damozhaeva", # DOCSUP "den-crane", "flickerbox-tom", # Flickerbox diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 0c2768b9ac3..0f601f7f52a 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -61,6 +61,7 @@ TRUSTED_CONTRIBUTORS = {e.lower() for e in [ "bharatnc", # Newbie, but already with many contributions. "bobrik", # Seasoned contributor, CloundFlare "BohuTANG", + "cwurm", # Employee "damozhaeva", # DOCSUP "den-crane", "gyuton", # DOCSUP From d593f94d3c8bf100b92d5f9fb7240fc99a16c29a Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Thu, 16 Dec 2021 14:27:38 +0100 Subject: [PATCH 254/358] Update some broken links The links were linking to the page itself. The information probably has been moved to the SQL reference page. --- .../table-engines/mergetree-family/custom-partitioning-key.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index 2711c76aeb6..5ac2105e9fd 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -66,9 +66,9 @@ WHERE table = 'visits' └───────────┴────────────────┴────────┘ ``` -The `partition` column contains the names of the partitions. There are two partitions in this example: `201901` and `201902`. You can use this column value to specify the partition name in [ALTER … PARTITION](#alter_manipulations-with-partitions) queries. +The `partition` column contains the names of the partitions. There are two partitions in this example: `201901` and `201902`. You can use this column value to specify the partition name in [ALTER … PARTITION](../../../sql-reference/statements/alter/partition.md) queries. -The `name` column contains the names of the partition data parts. You can use this column to specify the name of the part in the [ALTER ATTACH PART](#alter_attach-partition) query. +The `name` column contains the names of the partition data parts. You can use this column to specify the name of the part in the [ALTER ATTACH PART](../../../sql-reference/statements/alter/partition.md#alter_attach-partition) query. Let’s break down the name of the first part: `201901_1_3_1`: From 7e39fd14c249dbaf062ac3a6c27439fe943348c0 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 16 Dec 2021 16:44:55 +0300 Subject: [PATCH 255/358] fix test --- src/DataTypes/tests/gtest_split_name.cpp | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/DataTypes/tests/gtest_split_name.cpp b/src/DataTypes/tests/gtest_split_name.cpp index 09e5c11f09b..04ce4d5e108 100644 --- a/src/DataTypes/tests/gtest_split_name.cpp +++ b/src/DataTypes/tests/gtest_split_name.cpp @@ -19,14 +19,14 @@ TEST(SplitName, forward) TEST(SplitName, reverse) { - ASSERT_EQ(Nested::splitName(String("abc", true)), (std::pair{"abc", ""})); - ASSERT_EQ(Nested::splitName(String("a.b", true)), (std::pair{"a", "b"})); - ASSERT_EQ(Nested::splitName(String("a.b.c", true)), (std::pair{"a.b", "c"})); - ASSERT_EQ(Nested::splitName(String("a.1", true)), (std::pair{"a", "1"})); - ASSERT_EQ(Nested::splitName(String("a.1a.b", true)), (std::pair{"a.1a", "b"})); - ASSERT_EQ(Nested::splitName(String("1a.b", true)), (std::pair{"1a", "b"})); - ASSERT_EQ(Nested::splitName(String("a.b1.b2", true)), (std::pair{"a.b1", "b2"})); - ASSERT_EQ(Nested::splitName(String("a.b1.2a.3a", true)), (std::pair{"a.b1.2a", "3a"})); - ASSERT_EQ(Nested::splitName(String("a.b1.b2.b3", true)), (std::pair{"a.b1.b2", "b3"})); - ASSERT_EQ(Nested::splitName(String("..", true)), (std::pair{"..", ""})); + ASSERT_EQ(Nested::splitName(String("abc"), true), (std::pair{"abc", ""})); + ASSERT_EQ(Nested::splitName(String("a.b"), true), (std::pair{"a", "b"})); + ASSERT_EQ(Nested::splitName(String("a.b.c"), true), (std::pair{"a.b", "c"})); + ASSERT_EQ(Nested::splitName(String("a.1"), true), (std::pair{"a", "1"})); + ASSERT_EQ(Nested::splitName(String("a.1a.b"), true), (std::pair{"a.1a", "b"})); + ASSERT_EQ(Nested::splitName(String("1a.b"), true), (std::pair{"1a", "b"})); + ASSERT_EQ(Nested::splitName(String("a.b1.b2"), true), (std::pair{"a.b1", "b2"})); + ASSERT_EQ(Nested::splitName(String("a.b1.2a.3a"), true), (std::pair{"a.b1.2a", "3a"})); + ASSERT_EQ(Nested::splitName(String("a.b1.b2.b3"), true), (std::pair{"a.b1.b2", "b3"})); + ASSERT_EQ(Nested::splitName(String(".."), true), (std::pair{"..", ""})); } From a65dd4824c83a17263f4963f40e89b3ca4763a30 Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Thu, 16 Dec 2021 14:46:12 +0000 Subject: [PATCH 256/358] Docs: Fix formatting in HDFS engine --- .../table-engines/integrations/hdfs.md | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index 0fcf7a63dd8..82227215da2 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -5,8 +5,7 @@ toc_title: HDFS # HDFS {#table_engines-hdfs} -This engine provides integration with [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) ecosystem by allowing to manage data on [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html) via ClickHouse. This engine is similar -to the [File](../../../engines/table-engines/special/file.md#table_engines-file) and [URL](../../../engines/table-engines/special/url.md#table_engines-url) engines, but provides Hadoop-specific features. +This engine provides integration with the [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) ecosystem by allowing to manage data on [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html) via ClickHouse. This engine is similar to the [File](../../../engines/table-engines/special/file.md#table_engines-file) and [URL](../../../engines/table-engines/special/url.md#table_engines-url) engines, but provides Hadoop-specific features. ## Usage {#usage} @@ -14,12 +13,13 @@ to the [File](../../../engines/table-engines/special/file.md#table_engines-file) ENGINE = HDFS(URI, format) ``` -The `URI` parameter is the whole file URI in HDFS. -The `format` parameter specifies one of the available file formats. To perform +**Engine Parameters** + +- `URI` - whole file URI in HDFS. The path part of `URI` may contain globs. In this case the table would be readonly. +- `format` - specifies one of the available file formats. To perform `SELECT` queries, the format must be supported for input, and to perform `INSERT` queries – for output. The available formats are listed in the [Formats](../../../interfaces/formats.md#formats) section. -The path part of `URI` may contain globs. In this case the table would be readonly. **Example:** @@ -71,12 +71,12 @@ Constructions with `{}` are similar to the [remote](../../../sql-reference/table 1. Suppose we have several files in TSV format with the following URIs on HDFS: -- 'hdfs://hdfs1:9000/some_dir/some_file_1' -- 'hdfs://hdfs1:9000/some_dir/some_file_2' -- 'hdfs://hdfs1:9000/some_dir/some_file_3' -- 'hdfs://hdfs1:9000/another_dir/some_file_1' -- 'hdfs://hdfs1:9000/another_dir/some_file_2' -- 'hdfs://hdfs1:9000/another_dir/some_file_3' + - 'hdfs://hdfs1:9000/some_dir/some_file_1' + - 'hdfs://hdfs1:9000/some_dir/some_file_2' + - 'hdfs://hdfs1:9000/some_dir/some_file_3' + - 'hdfs://hdfs1:9000/another_dir/some_file_1' + - 'hdfs://hdfs1:9000/another_dir/some_file_2' + - 'hdfs://hdfs1:9000/another_dir/some_file_3' 1. There are several ways to make a table consisting of all six files: @@ -132,6 +132,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | **parameter** | **default value** | +| - | - | | rpc\_client\_connect\_tcpnodelay | true | | dfs\_client\_read\_shortcircuit | true | | output\_replace-datanode-on-failure | true | @@ -181,25 +182,26 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us #### ClickHouse extras {#clickhouse-extras} | **parameter** | **default value** | +| - | - | |hadoop\_kerberos\_keytab | "" | |hadoop\_kerberos\_principal | "" | |hadoop\_kerberos\_kinit\_command | kinit | |libhdfs3\_conf | "" | ### Limitations {#limitations} - * hadoop\_security\_kerberos\_ticket\_cache\_path and libhdfs3\_conf can be global only, not user specific + * `hadoop_security_kerberos_ticket_cache_path` and `libhdfs3_conf` can be global only, not user specific ## Kerberos support {#kerberos-support} -If hadoop\_security\_authentication parameter has value 'kerberos', ClickHouse authentifies via Kerberos facility. -Parameters [here](#clickhouse-extras) and hadoop\_security\_kerberos\_ticket\_cache\_path may be of help. +If the `hadoop_security_authentication` parameter has the value `kerberos`, ClickHouse authenticates via Kerberos. +Parameters are [here](#clickhouse-extras) and `hadoop_security_kerberos_ticket_cache_path` may be of help. Note that due to libhdfs3 limitations only old-fashioned approach is supported, -datanode communications are not secured by SASL (HADOOP\_SECURE\_DN\_USER is a reliable indicator of such -security approach). Use tests/integration/test\_storage\_kerberized\_hdfs/hdfs_configs/bootstrap.sh for reference. +datanode communications are not secured by SASL (`HADOOP_SECURE_DN_USER` is a reliable indicator of such +security approach). Use `tests/integration/test_storage_kerberized_hdfs/hdfs_configs/bootstrap.sh` for reference. -If hadoop\_kerberos\_keytab, hadoop\_kerberos\_principal or hadoop\_kerberos\_kinit\_command is specified, kinit will be invoked. hadoop\_kerberos\_keytab and hadoop\_kerberos\_principal are mandatory in this case. kinit tool and krb5 configuration files are required. +If `hadoop_kerberos_keytab`, `hadoop_kerberos_principal` or `hadoop_kerberos_kinit_command` is specified, `kinit` will be invoked. `hadoop_kerberos_keytab` and `hadoop_kerberos_principal` are mandatory in this case. `kinit` tool and krb5 configuration files are required. -## HDFS Namenode HA support{#namenode-ha} +## HDFS Namenode HA support {#namenode-ha} libhdfs3 support HDFS namenode HA. From e95cf9e5caf5905048831ad598c4a256e11062f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 16 Dec 2021 18:45:44 +0300 Subject: [PATCH 257/358] Followup --- .github/workflows/docs_check.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 5f860e52ec1..da2f98c9fb3 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -16,7 +16,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -rf $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Labels check @@ -29,7 +29,7 @@ jobs: steps: - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -rf $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Images check @@ -58,7 +58,7 @@ jobs: path: ${{ env.TEMP_PATH }} - name: Clear repository run: | - sudo rm -rf $GITHUB_WORKSPACE + sudo rm -rf $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE - name: Check out repository code uses: actions/checkout@v2 - name: Docs Check From 1b8ed6276aff49ead0c271551acf5f8a57c2b667 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 16 Dec 2021 19:00:59 +0300 Subject: [PATCH 258/358] Allow string for constant if consition optmization. --- .../OptimizeIfWithConstantConditionVisitor.cpp | 9 +++++++++ ...nstant_if_condition_and_not_existing_column.reference | 2 ++ ...125_constant_if_condition_and_not_existing_column.sql | 2 ++ 3 files changed, 13 insertions(+) diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 0440c52797c..8587eefaff7 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -25,6 +25,15 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v value = literal->value.get(); return true; } + else if (literal->value.getType() == Field::Types::String) + { + auto str = literal->value.get(); + if (str == "0" || str == "1") + { + value = str[0] - '0'; + return true; + } + } } /// cast of numeric constant in condition to UInt8 diff --git a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference index 67f2590a0c6..d971cea19e5 100644 --- a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference +++ b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference @@ -6,3 +6,5 @@ 42 42 42 +42 +42 diff --git a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql index ad3d417bc26..7fc2ae93776 100644 --- a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql +++ b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql @@ -11,4 +11,6 @@ select if(toUInt8(1), 42, y) from test; select if(toInt8(1), 42, y) from test; select if(toUInt8(toUInt8(0)), y, 42) from test; select if(cast(cast(0, 'UInt8'), 'UInt8'), y, 42) from test; +select if(cast('0', 'UInt8'), y, 42) from test; +select if(_CAST(0, 'UInt8'), y, 42) from test; drop table if exists t; From 3a8b2b15fc01a55434c7cbd6470683a2a520a826 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 16 Dec 2021 19:00:36 +0300 Subject: [PATCH 259/358] fix --- tests/integration/test_mysql_database_engine/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index efa5a38b08a..ff1c955d78b 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -432,7 +432,7 @@ def test_restart_server(started_cluster): with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("DROP DATABASE IF EXISTS test_restart") clickhouse_node.query("DROP DATABASE IF EXISTS test_restart") - clickhouse_node.query_and_get_error("CREATE DATABASE test_restart ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") + clickhouse_node.query_and_get_error("CREATE DATABASE test_restart ENGINE = MySQL('mysql57:3306', 'test_restart', 'root', 'clickhouse')") assert 'test_restart' not in clickhouse_node.query('SHOW DATABASES') mysql_node.query("CREATE DATABASE test_restart DEFAULT CHARACTER SET 'utf8'") From 9270b63b8dbc30ab33a4eda67e89f777e6a4b202 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 16 Dec 2021 19:31:11 +0300 Subject: [PATCH 260/358] Update clickhouse-v21.12-released.md --- website/blog/en/2021/clickhouse-v21.12-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.12-released.md b/website/blog/en/2021/clickhouse-v21.12-released.md index fde943d3641..d1e6cddbc35 100644 --- a/website/blog/en/2021/clickhouse-v21.12-released.md +++ b/website/blog/en/2021/clickhouse-v21.12-released.md @@ -43,7 +43,7 @@ ClickHouse Keeper development started in Sep 2020, more than a year ago. It was ClickHouse Keeper is a drop-in replacement for ZooKeeper. It implements the ZooKeeper wire protocol and data model, but does it better. -In contrast to ZooKeeper, there are no issues with zxid overflow or packet sizes. It has better memory usage and it does not require JVM tuning (because it does not use the JVM). Logs and snapshots are compressed (by about 10x typically) and checksummed. It can run as a separate process or directly inside clickhouse-server. You can use it with ClickHouse or with your Kafka and Hadoop as well. +In contrast to ZooKeeper, there are no issues with zxid overflow or packet sizes. It has better memory usage and it does not require JVM tuning (because it does not use the JVM). Logs and snapshots are compressed (by about 10x typically) and checksummed. It can run as a separate process or directly inside clickhouse-server. You can use it with ClickHouse or with your Kafkas and Hadoops as well. [More info](http://presentations.clickhouse.tech/meetup54/keeper.pdf). From a9fe2a076777b0c577689a835b3e304dbafce154 Mon Sep 17 00:00:00 2001 From: Niek <93536181+nikoloko@users.noreply.github.com> Date: Thu, 16 Dec 2021 17:37:53 +0100 Subject: [PATCH 261/358] Update formats.md --- docs/en/interfaces/formats.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 245a0c8fe89..f266d0e6354 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -204,7 +204,7 @@ When parsing with this format, tabs or linefeeds are not allowed in each field. This format is also available under the name `TSVRawWithNames`. -## TabSeparatedWithNamesAndTypes {#tabseparatedrawwithnamesandtypes} +## TabSeparatedRawWithNamesAndTypes {#tabseparatedrawwithnamesandtypes} Differs from `TabSeparatedWithNamesAndTypes` format in that the rows are written without escaping. When parsing with this format, tabs or linefeeds are not allowed in each field. From 49a9bc6b90601988c78dc1803d71d426c6bf7656 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 16 Dec 2021 20:29:04 +0300 Subject: [PATCH 262/358] Add debug logging --- src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 36fad32e430..2fb445be1b6 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -16,6 +17,9 @@ namespace ErrorCodes static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) { + const auto * log = &Poco::Logger::get("tryExtractConstValueFromCondition"); + LOG_TRACE(log, "{}", condition->dumpTree()); + /// numeric constant in condition if (const auto * literal = condition->as()) { From 8c11541722439d0765de0e89528e032881b3845c Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 16 Dec 2021 21:56:43 +0300 Subject: [PATCH 263/358] Ping CI From d9bb3aad2a3ef84ff126dbf89ca5fe1110864ba8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 16 Dec 2021 21:47:04 +0100 Subject: [PATCH 264/358] Use a proper syntax for multiline env in GITHUB_ENV --- .github/workflows/backport.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/backport.yml b/.github/workflows/backport.yml index 284af965714..4af3479d778 100644 --- a/.github/workflows/backport.yml +++ b/.github/workflows/backport.yml @@ -12,7 +12,9 @@ jobs: run: | cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/cherry_pick - ROBOT_CLICKHOUSE_SSH_KEY=${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} + ROBOT_CLICKHOUSE_SSH_KEY< Date: Wed, 15 Dec 2021 20:13:08 -0400 Subject: [PATCH 265/358] test for summap_nullable 0 --- tests/queries/0_stateless/01634_summap_nullable.reference | 2 ++ tests/queries/0_stateless/01634_summap_nullable.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/01634_summap_nullable.reference create mode 100644 tests/queries/0_stateless/01634_summap_nullable.sql diff --git a/tests/queries/0_stateless/01634_summap_nullable.reference b/tests/queries/0_stateless/01634_summap_nullable.reference new file mode 100644 index 00000000000..babed7df00d --- /dev/null +++ b/tests/queries/0_stateless/01634_summap_nullable.reference @@ -0,0 +1,2 @@ +(['a'],[1]) +(['a','b'],[1,0]) diff --git a/tests/queries/0_stateless/01634_summap_nullable.sql b/tests/queries/0_stateless/01634_summap_nullable.sql new file mode 100644 index 00000000000..226da645e9f --- /dev/null +++ b/tests/queries/0_stateless/01634_summap_nullable.sql @@ -0,0 +1,2 @@ +SELECT sumMap(['a', 'b'], [1, NULL]); +SELECT sumMap(['a', 'b'], [1, toNullable(0)]); From faaed87945dd3e7f7f82ffe605dd4403965b4bea Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 17 Dec 2021 01:24:21 +0300 Subject: [PATCH 266/358] warm up for test_kafka_json_as_string_no_kdc --- .../test_storage_kerberized_kafka/test.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index 5f5063a879f..b5f779cbd3d 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -97,6 +97,19 @@ def test_kafka_json_as_string(kafka_cluster): assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") def test_kafka_json_as_string_no_kdc(kafka_cluster): + instance.query(''' + CREATE TABLE test.kafka_no_kdc_warm_up (field String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kerberized_kafka1:19092', + kafka_topic_list = 'kafka_json_as_string_no_kdc_warm_up', + kafka_group_name = 'kafka_json_as_string_no_kdc_warm_up', + kafka_commit_on_select = 1, + kafka_format = 'JSONAsString', + kafka_flush_interval_ms=1000; + ''') + + instance.query('SELECT * FROM test.kafka_no_kdc_warm_up;') + kafka_produce(kafka_cluster, 'kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) kafka_cluster.pause_container('kafka_kerberos') From 40a456171685c89cc09453adef2f4da4d2e93b39 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Fri, 17 Dec 2021 09:31:57 +0800 Subject: [PATCH 267/358] Update tables.md --- docs/zh/operations/system-tables/tables.md | 122 ++++++++++++++++----- 1 file changed, 96 insertions(+), 26 deletions(-) diff --git a/docs/zh/operations/system-tables/tables.md b/docs/zh/operations/system-tables/tables.md index 6a719a92ca0..4441d8400f6 100644 --- a/docs/zh/operations/system-tables/tables.md +++ b/docs/zh/operations/system-tables/tables.md @@ -1,58 +1,128 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# 系统表 {#system-tables} -# 系统。表 {#system-tables} +包å«æœåŠ¡å™¨çŸ¥é“çš„æ¯ä¸ªè¡¨çš„元数æ®ã€‚ [分离的](https://clickhouse.com/docs/zh/sql-reference/statements/detach/)表ä¸åœ¨ `system.tables` 显示。 -包å«æœåŠ¡å™¨çŸ¥é“çš„æ¯ä¸ªè¡¨çš„元数æ®ã€‚ 分离的表ä¸æ˜¾ç¤ºåœ¨ `system.tables`。 +[临时表](https://clickhouse.com/docs/zh/sql-reference/statements/create/table/#temporary-tables)åªåœ¨åˆ›å»ºå®ƒä»¬çš„会è¯ä¸­çš„ `system.tables` 中æ‰å¯è§ã€‚它们的数æ®åº“字段显示为空,并且 `is_temporary` 标志显示为开å¯ã€‚ 此表包å«ä»¥ä¸‹åˆ—(列类型显示在括å·ä¸­): -- `database` (String) — 表所在的数æ®åº“表å。 +- `database` ([String](../../sql-reference/data-types/string.md)) — 表所在的数æ®åº“å。 -- `name` (String) — 表å。 +- `name` ([String](../../sql-reference/data-types/string.md)) — 表å。 -- `engine` (String) — 表引擎å (ä¸åŒ…å«å‚æ•°)。 +- `engine` ([String](../../sql-reference/data-types/string.md)) — 表引擎å (ä¸åŒ…å«å‚æ•°)。 -- `is_temporary` (UInt8)-指示表是å¦æ˜¯ä¸´æ—¶çš„标志。 +- `is_temporary` ([UInt8](../../sql-reference/data-types/int-uint.md)) - 指示表是å¦æ˜¯ä¸´æ—¶çš„标志。 -- `data_path` (String)-文件系统中表数æ®çš„路径。 +- `data_path` ([String](../../sql-reference/data-types/string.md)) - 表数æ®åœ¨æ–‡ä»¶ç³»ç»Ÿä¸­çš„路径。 -- `metadata_path` (String)-文件系统中表元数æ®çš„路径。 +- `metadata_path` ([String](../../sql-reference/data-types/string.md)) - 表元数æ®åœ¨æ–‡ä»¶ç³»ç»Ÿä¸­çš„路径。 -- `metadata_modification_time` (DateTime)-表元数æ®çš„最新修改时间。 +- `metadata_modification_time` ([DateTime](../../sql-reference/data-types/datetime.md)) - 表元数æ®çš„最新修改时间。 -- `dependencies_database` (数组(字符串))-æ•°æ®åº“ä¾èµ–关系。 +- `dependencies_database` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) - æ•°æ®åº“ä¾èµ–关系。 -- `dependencies_table` (数组(字符串))-表ä¾èµ–关系 ([MaterializedView](../../engines/table-engines/special/materializedview.md) 基于当å‰è¡¨çš„表)。 +- `dependencies_table` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)))- 表ä¾èµ–关系 (基于当å‰è¡¨çš„ [物化视图](../../engines/table-engines/special/materializedview.md) 表)。 -- `create_table_query` (String)-用于创建表的SQL语å¥ã€‚ +- `create_table_query` ([String](../../sql-reference/data-types/string.md)) - 用于创建表的 SQL 语å¥ã€‚ -- `engine_full` (String)-表引擎的å‚数。 +- `engine_full` ([String](../../sql-reference/data-types/string.md)) - 表引擎的å‚数。 -- `partition_key` (String)-表中指定的分区键表达å¼ã€‚ +- `as_select` ([String](../../sql-reference/data-types/string.md)) - 视图的 `SELECT` 语å¥ã€‚ -- `sorting_key` (String)-表中指定的排åºé”®è¡¨è¾¾å¼ã€‚ +- `partition_key` ([String](../../sql-reference/data-types/string.md)) - 表中指定的分区键表达å¼ã€‚ -- `primary_key` (String)-表中指定的主键表达å¼ã€‚ +- `sorting_key` ([String](../../sql-reference/data-types/string.md)) - 表中指定的排åºé”®è¡¨è¾¾å¼ã€‚ -- `sampling_key` (String)-表中指定的采样键表达å¼ã€‚ +- `primary_key` ([String](../../sql-reference/data-types/string.md)) - 表中指定的主键表达å¼ã€‚ -- `storage_policy` (字符串)-存储策略: +- `sampling_key` ([String](../../sql-reference/data-types/string.md)) - 表中指定的采样键表达å¼ã€‚ + +- `storage_policy` ([String](../../sql-reference/data-types/string.md)) - 存储策略: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - - [分布](../../engines/table-engines/special/distributed.md#distributed) + - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` (Nullable(UInt64))-总行数,如果å¯ä»¥å¿«é€Ÿç¡®å®šè¡¨ä¸­çš„确切行数,å¦åˆ™è¡Œæ•°ä¸º`Null`(包括底层 `Buffer` 表)。 +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - 总行数,如果无法快速确定表中的确切行数,则行数返回为 `NULL`(包括底层 `Buffer` 表)。 -- `total_bytes` (Nullable(UInt64))-总字节数,如果å¯ä»¥å¿«é€Ÿç¡®å®šå­˜å‚¨è¡¨çš„确切字节数,å¦åˆ™å­—节数为`Null` (å³**ä¸** 包括任何底层存储)。 +- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - 总字节数,如果无法快速确定存储表的确切字节数,则字节数返回为 `NULL` ( **ä¸** 包括任何底层存储)。 - 如果表将数æ®å­˜åœ¨ç£ç›˜ä¸Šï¼Œè¿”回实际使用的ç£ç›˜ç©ºé—´ï¼ˆåŽ‹ç¼©åŽï¼‰ã€‚ - 如果表在内存中存储数æ®ï¼Œè¿”回在内存中使用的近似字节数。 -- `lifetime_rows` (Nullbale(UInt64))-æœåŠ¡å¯åŠ¨åŽæ’入的总行数(åªé’ˆå¯¹`Buffer`表)。 +- `lifetime_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - æœåŠ¡å¯åŠ¨åŽæ’入的总行数(åªé’ˆå¯¹ `Buffer` 表)。 + + +- `lifetime_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - æœåŠ¡å¯åŠ¨åŽæ’入的总字节数(åªé’ˆå¯¹ `Buffer` 表)。 + + +- `comment` ([String](../../sql-reference/data-types/string.md)) - 表的注释. + +- `has_own_data` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 标志,表示表本身是å¦åœ¨ç£ç›˜ä¸Šå­˜å‚¨æ•°æ®ï¼Œæˆ–者访问其他æ¥æºã€‚ `system.tables` 表被用于 `SHOW TABLES` 的查询实现中。 +**示例** + +```sql +SELECT * FROM system.tables LIMIT 2 FORMAT Vertical; +``` + +```text +Row 1: +────── +database: base +name: t1 +uuid: 81b1c20a-b7c6-4116-a2ce-7583fb6b6736 +engine: MergeTree +is_temporary: 0 +data_paths: ['/var/lib/clickhouse/store/81b/81b1c20a-b7c6-4116-a2ce-7583fb6b6736/'] +metadata_path: /var/lib/clickhouse/store/461/461cf698-fd0b-406d-8c01-5d8fd5748a91/t1.sql +metadata_modification_time: 2021-01-25 19:14:32 +dependencies_database: [] +dependencies_table: [] +create_table_query: CREATE TABLE base.t1 (`n` UInt64) ENGINE = MergeTree ORDER BY n SETTINGS index_granularity = 8192 +engine_full: MergeTree ORDER BY n SETTINGS index_granularity = 8192 +as_select: SELECT database AS table_catalog +partition_key: +sorting_key: n +primary_key: n +sampling_key: +storage_policy: default +total_rows: 1 +total_bytes: 99 +lifetime_rows: á´ºáµá´¸á´¸ +lifetime_bytes: á´ºáµá´¸á´¸ +comment: +has_own_data: 0 + +Row 2: +────── +database: default +name: 53r93yleapyears +uuid: 00000000-0000-0000-0000-000000000000 +engine: MergeTree +is_temporary: 0 +data_paths: ['/var/lib/clickhouse/data/default/53r93yleapyears/'] +metadata_path: /var/lib/clickhouse/metadata/default/53r93yleapyears.sql +metadata_modification_time: 2020-09-23 09:05:36 +dependencies_database: [] +dependencies_table: [] +create_table_query: CREATE TABLE default.`53r93yleapyears` (`id` Int8, `febdays` Int8) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 8192 +engine_full: MergeTree ORDER BY id SETTINGS index_granularity = 8192 +as_select: SELECT name AS catalog_name +partition_key: +sorting_key: id +primary_key: id +sampling_key: +storage_policy: default +total_rows: 2 +total_bytes: 155 +lifetime_rows: á´ºáµá´¸á´¸ +lifetime_bytes: á´ºáµá´¸á´¸ +comment: +has_own_data: 0 +``` + + [原文](https://clickhouse.com/docs/zh/operations/system-tables/tables) From ff1a20e6c22e6d2db96631df8c31d51b67a4a05c Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Fri, 17 Dec 2021 09:45:04 +0800 Subject: [PATCH 268/358] Update merge_tree_settings.md --- .../system-tables/merge_tree_settings.md | 63 +++++++++++++++---- 1 file changed, 51 insertions(+), 12 deletions(-) diff --git a/docs/zh/operations/system-tables/merge_tree_settings.md b/docs/zh/operations/system-tables/merge_tree_settings.md index a6ad6f78f8e..5e3ae6b845c 100644 --- a/docs/zh/operations/system-tables/merge_tree_settings.md +++ b/docs/zh/operations/system-tables/merge_tree_settings.md @@ -1,16 +1,55 @@ ---- -machine_translated: true -machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 ---- +# merge_tree_settings 系统表 {#system-merge_tree_settings} -# 系统。merge_tree_settings {#system-merge_tree_settings} - -包å«æœ‰å…³ä»¥ä¸‹è®¾ç½®çš„ä¿¡æ¯ `MergeTree` æ¡Œå­ +åŒ…å« `MergeTree` 表的é…置信æ¯ã€‚ 列: -- `name` (String) — Setting name. -- `value` (String) — Setting value. -- `description` (String) — Setting description. -- `type` (String) — Setting type (implementation specific string value). -- `changed` (UInt8) — Whether the setting was explicitly defined in the config or explicitly changed. +- `name` (String) — é…ç½®å称. +- `value` (String) — é…置的值. +- `description` (String) — é…ç½®æè¿°. +- `type` (String) — é…置类型 (执行特定的字符串值). +- `changed` (UInt8) — 该设置是å¦åœ¨é…置中明确定义或明确改å˜. + + +**示例** +```sql +:) SELECT * FROM system.merge_tree_settings LIMIT 4 FORMAT Vertical; +``` + +```text +Row 1: +────── +name: index_granularity +value: 8192 +changed: 0 +description: How many rows correspond to one primary key value. +type: SettingUInt64 + +Row 2: +────── +name: min_bytes_for_wide_part +value: 0 +changed: 0 +description: Minimal uncompressed size in bytes to create part in wide format instead of compact +type: SettingUInt64 + +Row 3: +────── +name: min_rows_for_wide_part +value: 0 +changed: 0 +description: Minimal number of rows to create part in wide format instead of compact +type: SettingUInt64 + +Row 4: +────── +name: merge_max_block_size +value: 8192 +changed: 0 +description: How many rows in blocks should be formed for merge operations. +type: SettingUInt64 + +4 rows in set. Elapsed: 0.001 sec. +``` + +[原文](https://clickhouse.com/docs/zh/operations/system-tables/merge_tree_settings) From d0d00edc05f32ca6b7c0d05311e44169da962e96 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Fri, 17 Dec 2021 10:23:32 +0800 Subject: [PATCH 269/358] fix title --- docs/zh/operations/system-tables/tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/tables.md b/docs/zh/operations/system-tables/tables.md index 4441d8400f6..41276c2209a 100644 --- a/docs/zh/operations/system-tables/tables.md +++ b/docs/zh/operations/system-tables/tables.md @@ -1,4 +1,4 @@ -# 系统表 {#system-tables} +# system.tables {#system-tables} 包å«æœåŠ¡å™¨çŸ¥é“çš„æ¯ä¸ªè¡¨çš„元数æ®ã€‚ [分离的](https://clickhouse.com/docs/zh/sql-reference/statements/detach/)表ä¸åœ¨ `system.tables` 显示。 From 497f8995ac881041259a0d95f27709c5b9415fec Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Fri, 17 Dec 2021 10:25:08 +0800 Subject: [PATCH 270/358] fix title --- docs/zh/operations/system-tables/merge_tree_settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/merge_tree_settings.md b/docs/zh/operations/system-tables/merge_tree_settings.md index 5e3ae6b845c..1592aae10a9 100644 --- a/docs/zh/operations/system-tables/merge_tree_settings.md +++ b/docs/zh/operations/system-tables/merge_tree_settings.md @@ -1,4 +1,4 @@ -# merge_tree_settings 系统表 {#system-merge_tree_settings} +# system.merge_tree_settings {#system-merge_tree_settings} åŒ…å« `MergeTree` 表的é…置信æ¯ã€‚ From 9816dfb09d7d65d6e53ceee4cc2167959f6d8e19 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 17 Dec 2021 08:46:02 +0300 Subject: [PATCH 271/358] Update main.yml --- .github/workflows/main.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 0e62c6e4601..4cda4eac33e 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -356,7 +356,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 with: - submodules: 'recursive' + submodules: 'true' fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | From e929175dae5cdab3536826abeb45ec35827d5043 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 17 Dec 2021 08:55:46 +0300 Subject: [PATCH 272/358] Update run_check.py --- tests/ci/run_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index bcaefbce07a..72db667688a 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -66,6 +66,7 @@ TRUSTED_CONTRIBUTORS = {e.lower() for e in [ "vzakaznikov", "YiuRULE", "zlobober", # Developer of YT + "ilejn", # Arenadata, responsible for Kerberized Kafka ]} From e0077ac2642f0a6b6c71c9847fb390455b775fbb Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Fri, 17 Dec 2021 14:10:28 +0800 Subject: [PATCH 273/358] fix full-stop --- docs/zh/operations/system-tables/tables.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/operations/system-tables/tables.md b/docs/zh/operations/system-tables/tables.md index 41276c2209a..bb8110e1da3 100644 --- a/docs/zh/operations/system-tables/tables.md +++ b/docs/zh/operations/system-tables/tables.md @@ -56,7 +56,7 @@ - `lifetime_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - æœåŠ¡å¯åŠ¨åŽæ’入的总字节数(åªé’ˆå¯¹ `Buffer` 表)。 -- `comment` ([String](../../sql-reference/data-types/string.md)) - 表的注释. +- `comment` ([String](../../sql-reference/data-types/string.md)) - 表的注释。 - `has_own_data` ([UInt8](../../sql-reference/data-types/int-uint.md)) — 标志,表示表本身是å¦åœ¨ç£ç›˜ä¸Šå­˜å‚¨æ•°æ®ï¼Œæˆ–者访问其他æ¥æºã€‚ From 4f233c5fac3e35f7f47f1014ea2f1b908a9a2d86 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 16 Dec 2021 22:07:51 +0100 Subject: [PATCH 274/358] Fix multiline SSH_KEY in GITHUB_ENV file --- .github/workflows/backport.yml | 1 + .github/workflows/release.yml | 16 +++++++++++----- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/.github/workflows/backport.yml b/.github/workflows/backport.yml index 4af3479d778..003dc5b7cf5 100644 --- a/.github/workflows/backport.yml +++ b/.github/workflows/backport.yml @@ -9,6 +9,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Set envs + # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings run: | cat >> "$GITHUB_ENV" << 'EOF' TEMP_PATH=${{runner.temp}}/cherry_pick diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 977dbf07dbe..6179bdba8b6 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -35,6 +35,17 @@ jobs: needs: DockerHubPush runs-on: [self-hosted, func-tester] steps: + - name: Set envs + # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/docs_release + REPO_COPY=${{runner.temp}}/docs_release/ClickHouse + CLOUDFLARE_TOKEN=${{secrets.CLOUDFLARE}} + ROBOT_CLICKHOUSE_SSH_KEY< Date: Thu, 16 Dec 2021 14:51:43 +0300 Subject: [PATCH 275/358] Try fix flaky test: order for test_s3_zero_copy_replication --- docker/test/integration/runner/Dockerfile | 1 + tests/integration/test_s3_zero_copy_replication/test.py | 2 ++ 2 files changed, 3 insertions(+) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index a953a8a904a..16c0c283461 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -77,6 +77,7 @@ RUN python3 -m pip install \ psycopg2-binary==2.8.6 \ pymongo==3.11.0 \ pytest \ + pytest-order==1.0.0 \ pytest-timeout \ pytest-xdist \ pytest-repeat \ diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 5be98ab5188..2e38a5f451c 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -60,6 +60,8 @@ def wait_for_active_parts(node, num_expected_parts, table_name, timeout=30): assert num_parts == num_expected_parts +# Result of `get_large_objects_count` can be changed in other tests, so run this case at the beginning +@pytest.mark.order(0) @pytest.mark.parametrize( "policy", ["s3"] ) From ed426205305726a6eb5a177a55ab0e1cdb120df6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 Dec 2021 01:57:08 +0300 Subject: [PATCH 276/358] Rename BlobStorage to AzureBlobStorage --- src/CMakeLists.txt | 2 +- src/Common/ErrorCodes.cpp | 2 +- .../AzureBlobStorageAuth.cpp} | 22 ++++----- .../AzureBlobStorageAuth.h} | 2 +- .../DiskAzureBlobStorage.cpp} | 48 +++++++++---------- .../DiskAzureBlobStorage.h} | 18 ++++--- .../registerDiskAzureBlobStorage.cpp} | 32 ++++++------- src/Disks/DiskType.h | 6 +-- src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 6 +-- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 6 +-- .../IO/WriteIndirectBufferFromRemoteFS.cpp | 4 +- src/Disks/registerDisks.cpp | 4 +- ...cpp => ReadBufferFromAzureBlobStorage.cpp} | 12 ++--- ...age.h => ReadBufferFromAzureBlobStorage.h} | 6 +-- ...pp => WriteBufferFromAzureBlobStorage.cpp} | 10 ++-- ...ge.h => WriteBufferFromAzureBlobStorage.h} | 6 +-- .../__init__.py | 0 .../configs/config.d/storage_conf.xml | 2 +- .../test.py | 0 .../__init__.py | 0 .../config.d/bg_processing_pool_conf.xml | 0 .../configs/config.d/storage_conf.xml | 2 +- .../configs/config.xml | 0 .../test.py | 6 +-- 24 files changed, 97 insertions(+), 99 deletions(-) rename src/Disks/{BlobStorage/BlobStorageAuth.cpp => AzureBlobStorage/AzureBlobStorageAuth.cpp} (80%) rename src/Disks/{BlobStorage/BlobStorageAuth.h => AzureBlobStorage/AzureBlobStorageAuth.h} (76%) rename src/Disks/{BlobStorage/DiskBlobStorage.cpp => AzureBlobStorage/DiskAzureBlobStorage.cpp} (71%) rename src/Disks/{BlobStorage/DiskBlobStorage.h => AzureBlobStorage/DiskAzureBlobStorage.h} (84%) rename src/Disks/{BlobStorage/registerDiskBlobStorage.cpp => AzureBlobStorage/registerDiskAzureBlobStorage.cpp} (69%) rename src/IO/{ReadBufferFromBlobStorage.cpp => ReadBufferFromAzureBlobStorage.cpp} (93%) rename src/IO/{ReadBufferFromBlobStorage.h => ReadBufferFromAzureBlobStorage.h} (87%) rename src/IO/{WriteBufferFromBlobStorage.cpp => WriteBufferFromAzureBlobStorage.cpp} (83%) rename src/IO/{WriteBufferFromBlobStorage.h => WriteBufferFromAzureBlobStorage.h} (81%) rename tests/integration/{test_blob_storage_zero_copy_replication => test_azure_blob_storage_zero_copy_replication}/__init__.py (100%) rename tests/integration/{test_blob_storage_zero_copy_replication => test_azure_blob_storage_zero_copy_replication}/configs/config.d/storage_conf.xml (96%) rename tests/integration/{test_blob_storage_zero_copy_replication => test_azure_blob_storage_zero_copy_replication}/test.py (100%) rename tests/integration/{test_merge_tree_blob_storage => test_merge_tree_azure_blob_storage}/__init__.py (100%) rename tests/integration/{test_merge_tree_blob_storage => test_merge_tree_azure_blob_storage}/configs/config.d/bg_processing_pool_conf.xml (100%) rename tests/integration/{test_merge_tree_blob_storage => test_merge_tree_azure_blob_storage}/configs/config.d/storage_conf.xml (96%) rename tests/integration/{test_merge_tree_blob_storage => test_merge_tree_azure_blob_storage}/configs/config.xml (100%) rename tests/integration/{test_merge_tree_blob_storage => test_merge_tree_azure_blob_storage}/test.py (98%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index bca9bd9d280..7124961821e 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,7 +107,7 @@ if (USE_AWS_S3) endif() if (USE_AZURE_BLOB_STORAGE) - add_headers_and_sources(dbms Disks/BlobStorage) + add_headers_and_sources(dbms Disks/AzureBlobStorage) endif() if (USE_HDFS) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 9b0e2cdc494..70d85433513 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -470,7 +470,7 @@ M(497, ACCESS_DENIED) \ M(498, LIMIT_BY_WITH_TIES_IS_NOT_SUPPORTED) \ M(499, S3_ERROR) \ - M(500, BLOB_STORAGE_ERROR) \ + M(500, AZURE_BLOB_STORAGE_ERROR) \ M(501, CANNOT_CREATE_DATABASE) \ M(502, CANNOT_SIGQUEUE) \ M(503, AGGREGATE_FUNCTION_THROW) \ diff --git a/src/Disks/BlobStorage/BlobStorageAuth.cpp b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.cpp similarity index 80% rename from src/Disks/BlobStorage/BlobStorageAuth.cpp rename to src/Disks/AzureBlobStorage/AzureBlobStorageAuth.cpp index 2378ffe12a7..94553ba04e9 100644 --- a/src/Disks/BlobStorage/BlobStorageAuth.cpp +++ b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_AZURE_BLOB_STORAGE @@ -17,7 +17,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -struct BlobStorageEndpoint +struct AzureBlobStorageEndpoint { const String storage_account_url; const String container_name; @@ -41,18 +41,18 @@ void validateContainerName(const String & container_name) auto len = container_name.length(); if (len < 3 || len > 64) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Blob Storage container name is not valid, should have length between 3 and 64, but has length: {}", len); + "AzureBlob Storage container name is not valid, should have length between 3 and 64, but has length: {}", len); const auto * container_name_pattern_str = R"([a-z][a-z0-9-]+)"; static const RE2 container_name_pattern(container_name_pattern_str); if (!re2::RE2::FullMatch(container_name, container_name_pattern)) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Blob Storage container name is not valid, should follow the format: {}, got: {}", container_name_pattern_str, container_name); + "AzureBlob Storage container name is not valid, should follow the format: {}, got: {}", container_name_pattern_str, container_name); } -BlobStorageEndpoint processBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) +AzureBlobStorageEndpoint processAzureBlobStorageEndpoint(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { String storage_account_url = config.getString(config_prefix + ".storage_account_url"); validateStorageAccountUrl(storage_account_url); @@ -86,7 +86,7 @@ std::shared_ptr getClientWithConnectionString( template -std::shared_ptr getBlobStorageClientWithAuth( +std::shared_ptr getAzureBlobStorageClientWithAuth( const String & url, const String & container_name, const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { if (config.has(config_prefix + ".connection_string")) @@ -109,19 +109,19 @@ std::shared_ptr getBlobStorageClientWithAuth( } -std::shared_ptr getBlobContainerClient( +std::shared_ptr getAzureBlobContainerClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix) { - auto endpoint = processBlobStorageEndpoint(config, config_prefix); + auto endpoint = processAzureBlobStorageEndpoint(config, config_prefix); auto container_name = endpoint.container_name; auto final_url = endpoint.storage_account_url + (endpoint.storage_account_url.back() == '/' ? "" : "/") + container_name; if (endpoint.container_already_exists.value_or(false)) - return getBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); + return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); - auto blob_service_client = getBlobStorageClientWithAuth(endpoint.storage_account_url, container_name, config, config_prefix); + auto blob_service_client = getAzureBlobStorageClientWithAuth(endpoint.storage_account_url, container_name, config, config_prefix); if (!endpoint.container_already_exists.has_value()) { @@ -132,7 +132,7 @@ std::shared_ptr getBlobContainerClient( for (const auto & blob_container : blob_containers) { if (blob_container.Name == endpoint.container_name) - return getBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); + return getAzureBlobStorageClientWithAuth(final_url, container_name, config, config_prefix); } } diff --git a/src/Disks/BlobStorage/BlobStorageAuth.h b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h similarity index 76% rename from src/Disks/BlobStorage/BlobStorageAuth.h rename to src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h index 16a088fc960..1cef6105d41 100644 --- a/src/Disks/BlobStorage/BlobStorageAuth.h +++ b/src/Disks/AzureBlobStorage/AzureBlobStorageAuth.h @@ -12,7 +12,7 @@ namespace DB { -std::shared_ptr getBlobContainerClient( +std::shared_ptr getAzureBlobContainerClient( const Poco::Util::AbstractConfiguration & config, const String & config_prefix); } diff --git a/src/Disks/BlobStorage/DiskBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp similarity index 71% rename from src/Disks/BlobStorage/DiskBlobStorage.cpp rename to src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index f33e698a50d..7883bc0b537 100644 --- a/src/Disks/BlobStorage/DiskBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -1,4 +1,4 @@ -#include +#include #if USE_AZURE_BLOB_STORAGE @@ -15,11 +15,11 @@ namespace DB namespace ErrorCodes { - extern const int BLOB_STORAGE_ERROR; + extern const int AZURE_BLOB_STORAGE_ERROR; } -DiskBlobStorageSettings::DiskBlobStorageSettings( +DiskAzureBlobStorageSettings::DiskAzureBlobStorageSettings( UInt64 max_single_part_upload_size_, UInt64 min_bytes_for_seek_, int max_single_read_retries_, @@ -32,11 +32,11 @@ DiskBlobStorageSettings::DiskBlobStorageSettings( thread_pool_size(thread_pool_size_) {} -class BlobStoragePathKeeper : public RemoteFSPathKeeper +class AzureBlobStoragePathKeeper : public RemoteFSPathKeeper { public: /// RemoteFSPathKeeper constructed with a placeholder argument for chunk_limit, it is unused in this class - BlobStoragePathKeeper() : RemoteFSPathKeeper(1000) {} + AzureBlobStoragePathKeeper() : RemoteFSPathKeeper(1000) {} void addPath(const String & path) override { @@ -47,19 +47,19 @@ public: }; -DiskBlobStorage::DiskBlobStorage( +DiskAzureBlobStorage::DiskAzureBlobStorage( const String & name_, DiskPtr metadata_disk_, std::shared_ptr blob_container_client_, SettingsPtr settings_, GetDiskSettings settings_getter_) : - IDiskRemote(name_, "", metadata_disk_, "DiskBlobStorage", settings_->thread_pool_size), + IDiskRemote(name_, "", metadata_disk_, "DiskAzureBlobStorage", settings_->thread_pool_size), blob_container_client(blob_container_client_), current_settings(std::move(settings_)), settings_getter(settings_getter_) {} -std::unique_ptr DiskBlobStorage::readFile( +std::unique_ptr DiskAzureBlobStorage::readFile( const String & path, const ReadSettings & read_settings, std::optional /*estimated_size*/) const @@ -71,7 +71,7 @@ std::unique_ptr DiskBlobStorage::readFile( bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; - auto reader_impl = std::make_unique( + auto reader_impl = std::make_unique( path, blob_container_client, metadata, settings->max_single_read_retries, settings->max_single_download_retries, read_settings, threadpool_read); @@ -88,7 +88,7 @@ std::unique_ptr DiskBlobStorage::readFile( } -std::unique_ptr DiskBlobStorage::writeFile( +std::unique_ptr DiskAzureBlobStorage::writeFile( const String & path, size_t buf_size, WriteMode mode) @@ -96,38 +96,38 @@ std::unique_ptr DiskBlobStorage::writeFile( auto metadata = readOrCreateMetaForWriting(path, mode); auto blob_path = path + "_" + getRandomASCIIString(8); /// NOTE: path contains the tmp_* prefix in the blob name - LOG_TRACE(log, "{} to file by path: {}. Blob Storage path: {}", + LOG_TRACE(log, "{} to file by path: {}. AzureBlob Storage path: {}", mode == WriteMode::Rewrite ? "Write" : "Append", backQuote(metadata_disk->getPath() + path), blob_path); - auto buffer = std::make_unique( + auto buffer = std::make_unique( blob_container_client, blob_path, current_settings.get()->max_single_part_upload_size, buf_size); - return std::make_unique>(std::move(buffer), std::move(metadata), blob_path); + return std::make_unique>(std::move(buffer), std::move(metadata), blob_path); } -DiskType DiskBlobStorage::getType() const +DiskType DiskAzureBlobStorage::getType() const { - return DiskType::BlobStorage; + return DiskType::AzureBlobStorage; } -bool DiskBlobStorage::isRemote() const +bool DiskAzureBlobStorage::isRemote() const { return true; } -bool DiskBlobStorage::supportZeroCopyReplication() const +bool DiskAzureBlobStorage::supportZeroCopyReplication() const { return true; } -bool DiskBlobStorage::checkUniqueId(const String & id) const +bool DiskAzureBlobStorage::checkUniqueId(const String & id) const { Azure::Storage::Blobs::ListBlobsOptions blobs_list_options; blobs_list_options.Prefix = id; @@ -146,9 +146,9 @@ bool DiskBlobStorage::checkUniqueId(const String & id) const } -void DiskBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) +void DiskAzureBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) { - auto * paths_keeper = dynamic_cast(fs_paths_keeper.get()); + auto * paths_keeper = dynamic_cast(fs_paths_keeper.get()); if (paths_keeper) { @@ -158,7 +158,7 @@ void DiskBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) { auto delete_info = blob_container_client->DeleteBlob(path); if (!delete_info.Value.Deleted) - throw Exception(ErrorCodes::BLOB_STORAGE_ERROR, "Failed to delete file in Blob Storage: {}", path); + throw Exception(ErrorCodes::AZURE_BLOB_STORAGE_ERROR, "Failed to delete file in AzureBlob Storage: {}", path); } catch (const Azure::Storage::StorageException& e) { @@ -170,13 +170,13 @@ void DiskBlobStorage::removeFromRemoteFS(RemoteFSPathKeeperPtr fs_paths_keeper) } -RemoteFSPathKeeperPtr DiskBlobStorage::createFSPathKeeper() const +RemoteFSPathKeeperPtr DiskAzureBlobStorage::createFSPathKeeper() const { - return std::make_shared(); + return std::make_shared(); } -void DiskBlobStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) +void DiskAzureBlobStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, ContextPtr context, const String &, const DisksMap &) { auto new_settings = settings_getter(config, "storage_configuration.disks." + name, context); diff --git a/src/Disks/BlobStorage/DiskBlobStorage.h b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h similarity index 84% rename from src/Disks/BlobStorage/DiskBlobStorage.h rename to src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h index 16ba216df96..f90ede1add9 100644 --- a/src/Disks/BlobStorage/DiskBlobStorage.h +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.h @@ -1,14 +1,12 @@ #pragma once -#if !defined(ARCADIA_BUILD) #include -#endif #if USE_AZURE_BLOB_STORAGE #include -#include -#include +#include +#include #include #include @@ -18,9 +16,9 @@ namespace DB { -struct DiskBlobStorageSettings final +struct DiskAzureBlobStorageSettings final { - DiskBlobStorageSettings( + DiskAzureBlobStorageSettings( UInt64 max_single_part_upload_size_, UInt64 min_bytes_for_seek_, int max_single_read_retries, @@ -35,14 +33,14 @@ struct DiskBlobStorageSettings final }; -class DiskBlobStorage final : public IDiskRemote +class DiskAzureBlobStorage final : public IDiskRemote { public: - using SettingsPtr = std::unique_ptr; + using SettingsPtr = std::unique_ptr; using GetDiskSettings = std::function; - DiskBlobStorage( + DiskAzureBlobStorage( const String & name_, DiskPtr metadata_disk_, std::shared_ptr blob_container_client_, @@ -78,7 +76,7 @@ private: /// client used to access the files in the Blob Storage cloud std::shared_ptr blob_container_client; - MultiVersion current_settings; + MultiVersion current_settings; /// Gets disk settings from context. GetDiskSettings settings_getter; }; diff --git a/src/Disks/BlobStorage/registerDiskBlobStorage.cpp b/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp similarity index 69% rename from src/Disks/BlobStorage/registerDiskBlobStorage.cpp rename to src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp index 58b84a3f972..243452353d3 100644 --- a/src/Disks/BlobStorage/registerDiskBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/registerDiskAzureBlobStorage.cpp @@ -9,8 +9,8 @@ #include #include #include -#include -#include +#include +#include namespace DB @@ -62,9 +62,9 @@ void checkRemoveAccess(IDisk & disk) } -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/) +std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr /*context*/) { - return std::make_unique( + return std::make_unique( config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024), config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".max_single_read_retries", 3), @@ -74,7 +74,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractC } -void registerDiskBlobStorage(DiskFactory & factory) +void registerDiskAzureBlobStorage(DiskFactory & factory) { auto creator = []( const String & name, @@ -85,33 +85,33 @@ void registerDiskBlobStorage(DiskFactory & factory) { auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); - std::shared_ptr blob_storage_disk = std::make_shared( + std::shared_ptr azure_blob_storage_disk = std::make_shared( name, metadata_disk, - getBlobContainerClient(config, config_prefix), + getAzureBlobContainerClient(config, config_prefix), getSettings(config, config_prefix, context), getSettings ); if (!config.getBool(config_prefix + ".skip_access_check", false)) { - checkWriteAccess(*blob_storage_disk); - checkReadAccess(*blob_storage_disk); - checkReadWithOffset(*blob_storage_disk); - checkRemoveAccess(*blob_storage_disk); + checkWriteAccess(*azure_blob_storage_disk); + checkReadAccess(*azure_blob_storage_disk); + checkReadWithOffset(*azure_blob_storage_disk); + checkRemoveAccess(*azure_blob_storage_disk); } - blob_storage_disk->startup(); + azure_blob_storage_disk->startup(); if (config.getBool(config_prefix + ".cache_enabled", true)) { String cache_path = config.getString(config_prefix + ".cache_path", context->getPath() + "disks/" + name + "/cache/"); - blob_storage_disk = wrapWithCache(blob_storage_disk, "blob-storage-cache", cache_path, metadata_path); + azure_blob_storage_disk = wrapWithCache(azure_blob_storage_disk, "azure-blob-storage-cache", cache_path, metadata_path); } - return std::make_shared(blob_storage_disk); + return std::make_shared(azure_blob_storage_disk); }; - factory.registerDiskType("blob_storage", creator); + factory.registerDiskType("azure_blob_storage", creator); } } @@ -121,7 +121,7 @@ void registerDiskBlobStorage(DiskFactory & factory) namespace DB { -void registerDiskBlobStorage(DiskFactory &) {} +void registerDiskAzureBlobStorage(DiskFactory &) {} } diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index a1dd276c51f..435f427b05a 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -10,10 +10,10 @@ enum class DiskType Local, RAM, S3, - BlobStorage, HDFS, Encrypted, WebServer, + AzureBlobStorage, }; inline String toString(DiskType disk_type) @@ -26,14 +26,14 @@ inline String toString(DiskType disk_type) return "memory"; case DiskType::S3: return "s3"; - case DiskType::BlobStorage: - return "blob_storage"; case DiskType::HDFS: return "hdfs"; case DiskType::Encrypted: return "encrypted"; case DiskType::WebServer: return "web"; + case DiskType::AzureBlobStorage: + return "azure_blob_storage"; } __builtin_unreachable(); } diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 534258eaca6..be63223b313 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -9,7 +9,7 @@ #endif #if USE_AZURE_BLOB_STORAGE -#include +#include #endif #if USE_HDFS @@ -35,9 +35,9 @@ SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBuffer(const S #if USE_AZURE_BLOB_STORAGE -SeekableReadBufferPtr ReadBufferFromBlobStorageGather::createImplementationBuffer(const String & path, size_t read_until_position_) const +SeekableReadBufferPtr ReadBufferFromAzureBlobStorageGather::createImplementationBuffer(const String & path, size_t read_until_position_) const { - return std::make_unique(blob_container_client, path, max_single_read_retries, + return std::make_unique(blob_container_client, path, max_single_read_retries, max_single_download_retries, settings.remote_fs_buffer_size, threadpool_read, read_until_position_); } #endif diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index 045ab43850d..9dd5d79d883 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -102,11 +102,11 @@ private: #if USE_AZURE_BLOB_STORAGE -/// Reads data from Blob Storage using paths stored in metadata. -class ReadBufferFromBlobStorageGather final : public ReadBufferFromRemoteFSGather +/// Reads data from AzureBlob Storage using paths stored in metadata. +class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFSGather { public: - ReadBufferFromBlobStorageGather( + ReadBufferFromAzureBlobStorageGather( const String & path_, std::shared_ptr blob_container_client_, IDiskRemote::Metadata metadata_, diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp index 6e10aefc7a2..87453440693 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp @@ -1,7 +1,7 @@ #include "WriteIndirectBufferFromRemoteFS.h" #include -#include +#include #include #include @@ -60,7 +60,7 @@ class WriteIndirectBufferFromRemoteFS; #if USE_AZURE_BLOB_STORAGE template -class WriteIndirectBufferFromRemoteFS; +class WriteIndirectBufferFromRemoteFS; #endif #if USE_HDFS diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 44522f56130..88c3fdde1e0 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -15,7 +15,7 @@ void registerDiskS3(DiskFactory & factory); #endif #if USE_AZURE_BLOB_STORAGE -void registerDiskBlobStorage(DiskFactory & factory); +void registerDiskAzureBlobStorage(DiskFactory & factory); #endif #if USE_SSL @@ -41,7 +41,7 @@ void registerDisks() #endif #if USE_AZURE_BLOB_STORAGE - registerDiskBlobStorage(factory); + registerDiskAzureBlobStorage(factory); #endif #if USE_SSL diff --git a/src/IO/ReadBufferFromBlobStorage.cpp b/src/IO/ReadBufferFromAzureBlobStorage.cpp similarity index 93% rename from src/IO/ReadBufferFromBlobStorage.cpp rename to src/IO/ReadBufferFromAzureBlobStorage.cpp index ada462f0b87..d4762ecc044 100644 --- a/src/IO/ReadBufferFromBlobStorage.cpp +++ b/src/IO/ReadBufferFromAzureBlobStorage.cpp @@ -4,7 +4,7 @@ #if USE_AZURE_BLOB_STORAGE -#include +#include #include #include #include @@ -22,7 +22,7 @@ namespace ErrorCodes } -ReadBufferFromBlobStorage::ReadBufferFromBlobStorage( +ReadBufferFromAzureBlobStorage::ReadBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & path_, size_t max_single_read_retries_, @@ -48,7 +48,7 @@ ReadBufferFromBlobStorage::ReadBufferFromBlobStorage( } -bool ReadBufferFromBlobStorage::nextImpl() +bool ReadBufferFromAzureBlobStorage::nextImpl() { if (read_until_position) { @@ -102,7 +102,7 @@ bool ReadBufferFromBlobStorage::nextImpl() } -off_t ReadBufferFromBlobStorage::seek(off_t offset_, int whence) +off_t ReadBufferFromAzureBlobStorage::seek(off_t offset_, int whence) { if (initialized) throw Exception("Seek is allowed only before first read attempt from the buffer.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); @@ -119,13 +119,13 @@ off_t ReadBufferFromBlobStorage::seek(off_t offset_, int whence) } -off_t ReadBufferFromBlobStorage::getPosition() +off_t ReadBufferFromAzureBlobStorage::getPosition() { return offset - available(); } -void ReadBufferFromBlobStorage::initialize() +void ReadBufferFromAzureBlobStorage::initialize() { if (initialized) return; diff --git a/src/IO/ReadBufferFromBlobStorage.h b/src/IO/ReadBufferFromAzureBlobStorage.h similarity index 87% rename from src/IO/ReadBufferFromBlobStorage.h rename to src/IO/ReadBufferFromAzureBlobStorage.h index cd66e897e25..53749ad3199 100644 --- a/src/IO/ReadBufferFromBlobStorage.h +++ b/src/IO/ReadBufferFromAzureBlobStorage.h @@ -14,11 +14,11 @@ namespace DB { -class ReadBufferFromBlobStorage : public SeekableReadBuffer +class ReadBufferFromAzureBlobStorage : public SeekableReadBuffer { public: - explicit ReadBufferFromBlobStorage( + explicit ReadBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & path_, size_t max_single_read_retries_, @@ -55,7 +55,7 @@ private: char * data_ptr; size_t data_capacity; - Poco::Logger * log = &Poco::Logger::get("ReadBufferFromBlobStorage"); + Poco::Logger * log = &Poco::Logger::get("ReadBufferFromAzureBlobStorage"); }; } diff --git a/src/IO/WriteBufferFromBlobStorage.cpp b/src/IO/WriteBufferFromAzureBlobStorage.cpp similarity index 83% rename from src/IO/WriteBufferFromBlobStorage.cpp rename to src/IO/WriteBufferFromAzureBlobStorage.cpp index 2e63c8c4565..5fa312d973a 100644 --- a/src/IO/WriteBufferFromBlobStorage.cpp +++ b/src/IO/WriteBufferFromAzureBlobStorage.cpp @@ -4,7 +4,7 @@ #if USE_AZURE_BLOB_STORAGE -#include +#include #include #include @@ -12,7 +12,7 @@ namespace DB { -WriteBufferFromBlobStorage::WriteBufferFromBlobStorage( +WriteBufferFromAzureBlobStorage::WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, size_t max_single_part_upload_size_, @@ -23,13 +23,13 @@ WriteBufferFromBlobStorage::WriteBufferFromBlobStorage( blob_path(blob_path_) {} -WriteBufferFromBlobStorage::~WriteBufferFromBlobStorage() +WriteBufferFromAzureBlobStorage::~WriteBufferFromAzureBlobStorage() { finalize(); } -void WriteBufferFromBlobStorage::nextImpl() +void WriteBufferFromAzureBlobStorage::nextImpl() { if (!offset()) return; @@ -54,7 +54,7 @@ void WriteBufferFromBlobStorage::nextImpl() } -void WriteBufferFromBlobStorage::finalizeImpl() +void WriteBufferFromAzureBlobStorage::finalizeImpl() { next(); diff --git a/src/IO/WriteBufferFromBlobStorage.h b/src/IO/WriteBufferFromAzureBlobStorage.h similarity index 81% rename from src/IO/WriteBufferFromBlobStorage.h rename to src/IO/WriteBufferFromAzureBlobStorage.h index 5f8eaba3c0c..78e0c490b39 100644 --- a/src/IO/WriteBufferFromBlobStorage.h +++ b/src/IO/WriteBufferFromAzureBlobStorage.h @@ -17,17 +17,17 @@ namespace DB { -class WriteBufferFromBlobStorage : public BufferWithOwnMemory +class WriteBufferFromAzureBlobStorage : public BufferWithOwnMemory { public: - explicit WriteBufferFromBlobStorage( + explicit WriteBufferFromAzureBlobStorage( std::shared_ptr blob_container_client_, const String & blob_path_, size_t max_single_part_upload_size_, size_t buf_size_); - ~WriteBufferFromBlobStorage() override; + ~WriteBufferFromAzureBlobStorage() override; void nextImpl() override; diff --git a/tests/integration/test_blob_storage_zero_copy_replication/__init__.py b/tests/integration/test_azure_blob_storage_zero_copy_replication/__init__.py similarity index 100% rename from tests/integration/test_blob_storage_zero_copy_replication/__init__.py rename to tests/integration/test_azure_blob_storage_zero_copy_replication/__init__.py diff --git a/tests/integration/test_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml similarity index 96% rename from tests/integration/test_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml rename to tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml index 73eae98a80d..4235083f5ca 100644 --- a/tests/integration/test_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml +++ b/tests/integration/test_azure_blob_storage_zero_copy_replication/configs/config.d/storage_conf.xml @@ -2,7 +2,7 @@ - blob_storage + azure_blob_storage http://azurite1:10000/devstoreaccount1 cont false diff --git a/tests/integration/test_blob_storage_zero_copy_replication/test.py b/tests/integration/test_azure_blob_storage_zero_copy_replication/test.py similarity index 100% rename from tests/integration/test_blob_storage_zero_copy_replication/test.py rename to tests/integration/test_azure_blob_storage_zero_copy_replication/test.py diff --git a/tests/integration/test_merge_tree_blob_storage/__init__.py b/tests/integration/test_merge_tree_azure_blob_storage/__init__.py similarity index 100% rename from tests/integration/test_merge_tree_blob_storage/__init__.py rename to tests/integration/test_merge_tree_azure_blob_storage/__init__.py diff --git a/tests/integration/test_merge_tree_blob_storage/configs/config.d/bg_processing_pool_conf.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/bg_processing_pool_conf.xml similarity index 100% rename from tests/integration/test_merge_tree_blob_storage/configs/config.d/bg_processing_pool_conf.xml rename to tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/bg_processing_pool_conf.xml diff --git a/tests/integration/test_merge_tree_blob_storage/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml similarity index 96% rename from tests/integration/test_merge_tree_blob_storage/configs/config.d/storage_conf.xml rename to tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml index c4f381db941..09fa0d6c767 100644 --- a/tests/integration/test_merge_tree_blob_storage/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.d/storage_conf.xml @@ -2,7 +2,7 @@ - blob_storage + azure_blob_storage http://azurite1:10000/devstoreaccount1 cont false diff --git a/tests/integration/test_merge_tree_blob_storage/configs/config.xml b/tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml similarity index 100% rename from tests/integration/test_merge_tree_blob_storage/configs/config.xml rename to tests/integration/test_merge_tree_azure_blob_storage/configs/config.xml diff --git a/tests/integration/test_merge_tree_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py similarity index 98% rename from tests/integration/test_merge_tree_blob_storage/test.py rename to tests/integration/test_merge_tree_azure_blob_storage/test.py index 2a38ac128ac..a1f1eeb928d 100644 --- a/tests/integration/test_merge_tree_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -12,7 +12,7 @@ CONFIG_PATH = os.path.join(SCRIPT_DIR, './{}/node/configs/config.d/storage_conf. NODE_NAME = "node" TABLE_NAME = "blob_storage_table" -BLOB_STORAGE_DISK = "blob_storage_disk" +AZURE_BLOB_STORAGE_DISK = "blob_storage_disk" LOCAL_DISK = "hdd" CONTAINER_NAME = "cont" @@ -196,7 +196,7 @@ def test_move_partition_to_another_disk(cluster): node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{LOCAL_DISK}'") assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" - node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{BLOB_STORAGE_DISK}'") + node.query(f"ALTER TABLE {TABLE_NAME} MOVE PARTITION '2020-01-04' TO DISK '{AZURE_BLOB_STORAGE_DISK}'") assert node.query(f"SELECT count(*) FROM {TABLE_NAME} FORMAT Values") == "(8192)" @@ -327,7 +327,7 @@ def test_restart_during_load(cluster): def restart_disk(): for iii in range(0, 2): logging.info(f"Restarting disk, attempt {iii}") - node.query(f"SYSTEM RESTART DISK {BLOB_STORAGE_DISK}") + node.query(f"SYSTEM RESTART DISK {AZURE_BLOB_STORAGE_DISK}") logging.info(f"Disk restarted, attempt {iii}") time.sleep(0.5) From 47c907d41a752eecf5f35ebffe72d537007a4f3f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Dec 2021 13:20:48 +0300 Subject: [PATCH 277/358] Add dockerhub proxy --- docker/test/integration/runner/dockerd-entrypoint.sh | 4 ++-- docker/test/testflows/runner/dockerd-entrypoint.sh | 4 ++-- tests/ci/worker/ubuntu_style_check.sh | 4 +++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 5a021036b26..ad8a8e4eb84 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -8,8 +8,8 @@ echo '{ "ip-forward": true, "log-level": "debug", "storage-driver": "overlay2", - "insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"], - "registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"] + "insecure-registries" : ["dockerhub-proxy.dockerhub-proxy-zone:5000"], + "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"] }' | dd of=/etc/docker/daemon.json 2>/dev/null dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & diff --git a/docker/test/testflows/runner/dockerd-entrypoint.sh b/docker/test/testflows/runner/dockerd-entrypoint.sh index 8abbd9e1c8e..0e15396082a 100755 --- a/docker/test/testflows/runner/dockerd-entrypoint.sh +++ b/docker/test/testflows/runner/dockerd-entrypoint.sh @@ -5,8 +5,8 @@ echo "Configure to use Yandex dockerhub-proxy" mkdir -p /etc/docker/ cat > /etc/docker/daemon.json << EOF { - "insecure-registries" : ["dockerhub-proxy.sas.yp-c.yandex.net:5000"], - "registry-mirrors" : ["http://dockerhub-proxy.sas.yp-c.yandex.net:5000"] + "insecure-registries" : ["dockerhub-proxy.dockerhub-proxy-zone:5000"], + "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"] } EOF diff --git a/tests/ci/worker/ubuntu_style_check.sh b/tests/ci/worker/ubuntu_style_check.sh index 585c8610772..3fabbb1f8a4 100644 --- a/tests/ci/worker/ubuntu_style_check.sh +++ b/tests/ci/worker/ubuntu_style_check.sh @@ -53,7 +53,9 @@ usermod -aG docker ubuntu cat < /etc/docker/daemon.json { "ipv6": true, - "fixed-cidr-v6": "2001:db8:1::/64" + "fixed-cidr-v6": "2001:db8:1::/64", + "insecure-registries" : ["dockerhub-proxy.dockerhub-proxy-zone:5000"], + "registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"] } EOT From 7b56e97cf8a81cd27dc8a746fcaac7b30a129724 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Dec 2021 13:26:05 +0300 Subject: [PATCH 278/358] Revert "Add debug logging" This reverts commit 49a9bc6b90601988c78dc1803d71d426c6bf7656. --- src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 2fb445be1b6..36fad32e430 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -17,9 +16,6 @@ namespace ErrorCodes static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) { - const auto * log = &Poco::Logger::get("tryExtractConstValueFromCondition"); - LOG_TRACE(log, "{}", condition->dumpTree()); - /// numeric constant in condition if (const auto * literal = condition->as()) { From f02a40bbdcbfebf03f06994534da6e577947d566 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Dec 2021 13:48:42 +0300 Subject: [PATCH 279/358] Rename script and add template for dockerhub proxy --- tests/ci/worker/dockerhub_proxy_template.sh | 8 ++++++++ .../{ubuntu_style_check.sh => ubuntu_ami_for_ci.sh} | 0 2 files changed, 8 insertions(+) create mode 100644 tests/ci/worker/dockerhub_proxy_template.sh rename tests/ci/worker/{ubuntu_style_check.sh => ubuntu_ami_for_ci.sh} (100%) diff --git a/tests/ci/worker/dockerhub_proxy_template.sh b/tests/ci/worker/dockerhub_proxy_template.sh new file mode 100644 index 00000000000..5ee63a05125 --- /dev/null +++ b/tests/ci/worker/dockerhub_proxy_template.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash +set -xeuo pipefail + +mkdir /home/ubuntu/registrystorage + +sed -i 's/preserve_hostname: false/preserve_hostname: true/g' /etc/cloud/cloud.cfg + +docker run -d --network=host -p 5000:5000 -v /home/ubuntu/registrystorage:/var/lib/registry -e REGISTRY_HTTP_ADDR=0.0.0.0:5000 -e REGISTRY_STORAGE_DELETE_ENABLED=true -e REGISTRY_PROXY_REMOTEURL=https://registry-1.docker.io --restart=always --name registry registry:2 diff --git a/tests/ci/worker/ubuntu_style_check.sh b/tests/ci/worker/ubuntu_ami_for_ci.sh similarity index 100% rename from tests/ci/worker/ubuntu_style_check.sh rename to tests/ci/worker/ubuntu_ami_for_ci.sh From b1794a47c39dc466efe5ad1f8691ae4646531677 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 Dec 2021 14:03:37 +0300 Subject: [PATCH 280/358] StorageURL improve --- src/Storages/StorageURL.cpp | 87 ++++++++++++++++++++++++------------- 1 file changed, 56 insertions(+), 31 deletions(-) diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index fe05d168c31..5fa16a25900 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -95,11 +95,20 @@ namespace class StorageURLSource : public SourceWithProgress { + using URIParams = std::vector>; public: + struct URIInfo + { + using FailoverOptions = std::vector; + std::vector uri_list_to_read; + std::atomic next_uri_to_read = 0; + }; + using URIInfoPtr = std::shared_ptr; + StorageURLSource( - const std::vector & uri_options, + URIInfoPtr uri_info_, const std::string & http_method, std::function callback, const String & format, @@ -114,10 +123,12 @@ namespace const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers_ = {}, const URIParams & params = {}) : SourceWithProgress(sample_block), name(std::move(name_)) + , uri_info(uri_info_) { auto headers = getHeaders(headers_); + /// Lazy initialization. We should not perform requests in constructor, because we need to do it in query pipeline. - initialize = [=, this] + initialize = [=, this](const URIInfo::FailoverOptions & uri_options) { WriteBufferFromOwnString error_message; for (auto option = uri_options.begin(); option < uri_options.end(); ++option) @@ -135,10 +146,11 @@ namespace if (n != std::string::npos) { credentials.setUsername(user_info.substr(0, n)); - credentials.setPassword(user_info.substr(n+1)); + credentials.setPassword(user_info.substr(n + 1)); } } + /// Get first alive uri. read_buf = wrapReadBufferWithCompressionMethod( std::make_unique( request_uri, @@ -188,29 +200,34 @@ namespace Chunk generate() override { - if (initialize) + while (true) { - initialize(); - initialize = {}; + if (!reader) + { + auto current_uri_pos = uri_info->next_uri_to_read.fetch_add(1); + if (current_uri_pos >= uri_info->uri_list_to_read.size()) + return {}; + + auto current_uri = uri_info->uri_list_to_read[current_uri_pos]; + initialize(current_uri); + } + + Chunk chunk; + if (reader->pull(chunk)) + return chunk; + + pipeline->reset(); + reader.reset(); } - - if (!reader) - return {}; - - Chunk chunk; - if (reader->pull(chunk)) - return chunk; - - pipeline->reset(); - reader.reset(); - - return {}; } private: - std::function initialize; + using InitializeFunc = std::function; + InitializeFunc initialize; String name; + URIInfoPtr uri_info; + std::unique_ptr read_buf; std::unique_ptr pipeline; std::unique_ptr reader; @@ -332,7 +349,7 @@ Pipe IStorageURLBase::read( ContextPtr local_context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned /*num_streams*/) + unsigned num_streams) { auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size); bool with_globs = (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) @@ -341,18 +358,23 @@ Pipe IStorageURLBase::read( if (with_globs) { size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; - std::vector url_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); - std::vector uri_options; + auto uri_descriptions = parseRemoteDescription(uri, 0, uri.size(), ',', max_addresses); + + if (num_streams > uri_descriptions.size()) + num_streams = uri_descriptions.size(); + + /// For each uri (which acts like shard) check if it has failover options + auto uri_info = std::make_shared(); + for (const auto & description : uri_descriptions) + uri_info->uri_list_to_read.emplace_back(parseRemoteDescription(description, 0, description.size(), '|', max_addresses)); Pipes pipes; - for (const auto & url_description : url_descriptions) - { - /// For each uri (which acts like shard) check if it has failover options - uri_options = parseRemoteDescription(url_description, 0, url_description.size(), '|', max_addresses); - StoragePtr shard; + pipes.reserve(num_streams); + for (size_t i = 0; i < num_streams; ++i) + { pipes.emplace_back(std::make_shared( - uri_options, + uri_info, getReadMethod(), getReadPOSTDataCallback( column_names, metadata_snapshot, query_info, @@ -371,9 +393,10 @@ Pipe IStorageURLBase::read( } else { - std::vector uri_options{uri}; + auto uri_info = std::make_shared(); + uri_info->uri_list_to_read.emplace_back(std::vector{uri}); return Pipe(std::make_shared( - uri_options, + uri_info, getReadMethod(), getReadPOSTDataCallback( column_names, metadata_snapshot, query_info, @@ -402,8 +425,10 @@ Pipe StorageURLWithFailover::read( { auto params = getReadURIParams(column_names, metadata_snapshot, query_info, local_context, processed_stage, max_block_size); + auto uri_info = std::make_shared(); + uri_info->uri_list_to_read.emplace_back(uri_options); auto pipe = Pipe(std::make_shared( - uri_options, + uri_info, getReadMethod(), getReadPOSTDataCallback( column_names, metadata_snapshot, query_info, From d465ed05269a5482aff8e944a36795b88c4fcc16 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 17 Dec 2021 14:19:54 +0300 Subject: [PATCH 281/358] Update 01069_window_view_proc_tumble_watch.py --- .../0_stateless/01069_window_view_proc_tumble_watch.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py index adab2988e39..03068789ea0 100755 --- a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py +++ b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py @@ -37,11 +37,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('WATCH 01069_window_view_proc_tumble_watch.wv') client1.expect('Query id' + end_of_block) - client2.send("INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 1)") + client2.send("INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)") client2.expect("Ok.") client1.expect('1' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') - client2.send("INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 1)") + client2.send("INSERT INTO 01069_window_view_proc_tumble_watch.mt VALUES (1, now('US/Samoa') + 3)") client2.expect("Ok.") client1.expect('1' + end_of_block) client1.expect('Progress: 2.00 rows.*\)') From fcb0efcf7d87f695b6f2426f94ad9f5c0d9f4570 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 Dec 2021 14:34:49 +0300 Subject: [PATCH 282/358] Update test --- tests/integration/test_merge_tree_blob_storage/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_merge_tree_blob_storage/test.py b/tests/integration/test_merge_tree_blob_storage/test.py index 2a38ac128ac..cd36d953305 100644 --- a/tests/integration/test_merge_tree_blob_storage/test.py +++ b/tests/integration/test_merge_tree_blob_storage/test.py @@ -206,8 +206,8 @@ def test_table_manipulations(cluster): renamed_table = TABLE_NAME + "_renamed" - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") - node.query(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") + node.query_with_retry(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}") + node.query_with_retry(f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-04', 4096)}") node.query(f"RENAME TABLE {TABLE_NAME} TO {renamed_table}") assert node.query(f"SELECT count(*) FROM {renamed_table} FORMAT Values") == "(8192)" From 2821e14492b4c686d7239b6f4280a0128ed1398e Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 17 Dec 2021 15:01:57 +0300 Subject: [PATCH 283/358] typo fixed --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index c8745294c5b..830b8e149f6 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -630,7 +630,7 @@ class ClickhouseIntegrationTestsRunner: random.shuffle(items_to_run) for group, tests in items_to_run: - logging.info("Running test group %s countaining %s tests", group, len(tests)) + logging.info("Running test group %s containing %s tests", group, len(tests)) group_counters, group_test_times, log_paths = self.try_run_test_group(repo_path, group, tests, MAX_RETRY, NUM_WORKERS) total_tests = 0 for counter, value in group_counters.items(): From 20b88d9b4c081088c422f9c1d169a348f746a846 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Dec 2021 15:31:30 +0300 Subject: [PATCH 284/358] Always apply const if optimixation. --- .../OptimizeIfWithConstantConditionVisitor.cpp | 9 --------- src/Interpreters/TreeOptimizer.cpp | 2 -- src/Interpreters/TreeRewriter.cpp | 2 ++ ...nstant_if_condition_and_not_existing_column.reference | 2 -- ...125_constant_if_condition_and_not_existing_column.sql | 2 -- 5 files changed, 2 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp index 36fad32e430..802bf4e43ce 100644 --- a/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp +++ b/src/Interpreters/OptimizeIfWithConstantConditionVisitor.cpp @@ -25,15 +25,6 @@ static bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & v value = literal->value.get(); return true; } - else if (literal->value.getType() == Field::Types::String) - { - auto str = literal->value.get(); - if (str == "0" || str == "1") - { - value = str[0] - '0'; - return true; - } - } } /// cast of numeric constant in condition to UInt8 diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 86fd9ae5ddd..64b25ca9777 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -744,8 +744,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, && result.storage->supportsSubcolumns() && result.metadata_snapshot) optimizeFunctionsToSubcolumns(query, result.metadata_snapshot); - optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); - /// Move arithmetic operations out of aggregation functions if (settings.optimize_arithmetic_operations_in_aggregate_functions) optimizeAggregationFunctions(query); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 6b3a50d88e2..44b0c760d8d 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1120,6 +1120,8 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Push the predicate expression down to subqueries. The optimization should be applied to both initial and secondary queries. result.rewrite_subqueries = PredicateExpressionsOptimizer(getContext(), tables_with_columns, settings).optimize(*select_query); + TreeOptimizer::optimizeIf(query, result.aliases, settings.optimize_if_chain_to_multiif); + /// Only apply AST optimization for initial queries. if (getContext()->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY) TreeOptimizer::apply(query, result, tables_with_columns, getContext()); diff --git a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference index 916d6e017c2..a7903610a42 100644 --- a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference +++ b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.reference @@ -6,8 +6,6 @@ 42 42 42 -42 -42 SELECT x, concat(x, \'_\') diff --git a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql index d72047a96bc..4aad7ae3694 100644 --- a/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql +++ b/tests/queries/0_stateless/02125_constant_if_condition_and_not_existing_column.sql @@ -11,7 +11,5 @@ select if(toUInt8(1), 42, y) from test; select if(toInt8(1), 42, y) from test; select if(toUInt8(toUInt8(0)), y, 42) from test; select if(cast(cast(0, 'UInt8'), 'UInt8'), y, 42) from test; -select if(cast('0', 'UInt8'), y, 42) from test; -select if(_CAST(0, 'UInt8'), y, 42) from test; explain syntax select x, if((select hasColumnInTable(currentDatabase(), 'test', 'y')), y, x || '_') from test; drop table if exists t; From ce6987ad4dd677bf0eb96355bf2acc39b64d1d45 Mon Sep 17 00:00:00 2001 From: cmsxbc Date: Fri, 17 Dec 2021 20:59:50 +0800 Subject: [PATCH 285/358] fix wrong testcase in gtest_Parser --- src/Parsers/tests/gtest_Parser.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index bb1bd5645a3..e7abcee6b43 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -198,7 +198,7 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) LAYOUT(FLAT()) LIFETIME(MIN 0 MAX 1000) COMMENT 'Test dictionary with comment'; -sql", +)sql", R"sql(CREATE DICTIONARY `2024_dictionary_with_comment` ( `id` UInt64, From 24480ac4de6d104fc815fbd939ba4725c78f3cf4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Dec 2021 16:16:35 +0300 Subject: [PATCH 286/358] More agressive retries in perf tests --- docker/test/performance-comparison/entrypoint.sh | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 6d4f1222432..7f07bf41f77 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -11,6 +11,12 @@ if [[ $S3_URL == *"s3.amazonaws.com"* ]]; then COMMON_BUILD_PREFIX="" fi +function curl_with_retry +{ + for i in 1 2 3 4; do curl --fail --head "$1" && return 0 || sleep 0.5; done + return 1 +} + # Use the packaged repository to find the revision we will compare to. function find_reference_sha { @@ -55,7 +61,7 @@ function find_reference_sha ) for path in "${urls_to_try[@]}" do - if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$path" + if curl_with_retry "$path" then found="$path" break @@ -76,7 +82,7 @@ chmod 777 workspace output cd workspace # Download the package for the version we are going to test. -if curl --fail --retry 5 --retry-delay 1 --retry-max-time 15 --head "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" +if curl_with_retry "$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" then right_path="$S3_URL/$PR_TO_TEST/$SHA_TO_TEST$COMMON_BUILD_PREFIX/performance/performance.tgz" fi From 1931726e15dc6dcee603d687516da90cfcc248b1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Dec 2021 16:19:44 +0300 Subject: [PATCH 287/358] Followup --- docker/test/performance-comparison/entrypoint.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 7f07bf41f77..9bc7e604971 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -11,6 +11,8 @@ if [[ $S3_URL == *"s3.amazonaws.com"* ]]; then COMMON_BUILD_PREFIX="" fi +# Sometimes AWS responde with DNS error and it's impossible to retry it with +# current curl version options. function curl_with_retry { for i in 1 2 3 4; do curl --fail --head "$1" && return 0 || sleep 0.5; done From c2e8a73f145408800586a64d440d5e0a1e7c25c4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 17 Dec 2021 16:43:58 +0300 Subject: [PATCH 288/358] Fix style --- docker/test/performance-comparison/entrypoint.sh | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index 9bc7e604971..3d37a6c0e92 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -15,7 +15,13 @@ fi # current curl version options. function curl_with_retry { - for i in 1 2 3 4; do curl --fail --head "$1" && return 0 || sleep 0.5; done + for _ in 1 2 3 4; do + if curl --fail --head "$1";then + return 0 + else + sleep 0.5 + fi + done return 1 } From 5c487b3bdc57c18ee4680b96c1fbbaad6856358b Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Dec 2021 17:54:12 +0300 Subject: [PATCH 289/358] Fix types for topKWeightedState --- .../AggregateFunctionTopK.cpp | 22 +++++++++++-------- .../AggregateFunctionTopK.h | 4 ++-- .../0_stateless/02149_issue_32487.reference | 0 .../queries/0_stateless/02149_issue_32487.sql | 1 + 4 files changed, 16 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02149_issue_32487.reference create mode 100644 tests/queries/0_stateless/02149_issue_32487.sql diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index c3b80cae080..3a1689a4e64 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -16,9 +16,10 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -42,19 +43,22 @@ class AggregateFunctionTopKDateTime : public AggregateFunctionTopK -static IAggregateFunction * createWithExtraTypes(const DataTypePtr & argument_type, UInt64 threshold, UInt64 load_factor, const Array & params) +static IAggregateFunction * createWithExtraTypes(const DataTypes & argument_types, UInt64 threshold, UInt64 load_factor, const Array & params) { - WhichDataType which(argument_type); + if (argument_types.empty()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Got empty arguments list"); + + WhichDataType which(argument_types[0]); if (which.idx == TypeIndex::Date) - return new AggregateFunctionTopKDate(threshold, load_factor, {argument_type}, params); + return new AggregateFunctionTopKDate(threshold, load_factor, argument_types, params); if (which.idx == TypeIndex::DateTime) - return new AggregateFunctionTopKDateTime(threshold, load_factor, {argument_type}, params); + return new AggregateFunctionTopKDateTime(threshold, load_factor, argument_types, params); /// Check that we can use plain version of AggregateFunctionTopKGeneric - if (argument_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) - return new AggregateFunctionTopKGeneric(threshold, load_factor, argument_type, params); + if (argument_types[0]->isValueUnambiguouslyRepresentedInContiguousMemoryRegion()) + return new AggregateFunctionTopKGeneric(threshold, load_factor, argument_types, params); else - return new AggregateFunctionTopKGeneric(threshold, load_factor, argument_type, params); + return new AggregateFunctionTopKGeneric(threshold, load_factor, argument_types, params); } @@ -106,7 +110,7 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const *argument_types[0], threshold, load_factor, argument_types, params)); if (!res) - res = AggregateFunctionPtr(createWithExtraTypes(argument_types[0], threshold, load_factor, params)); + res = AggregateFunctionPtr(createWithExtraTypes(argument_types, threshold, load_factor, params)); if (!res) throw Exception("Illegal type " + argument_types[0]->getName() + diff --git a/src/AggregateFunctions/AggregateFunctionTopK.h b/src/AggregateFunctions/AggregateFunctionTopK.h index eb84288a1ae..98774254695 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/src/AggregateFunctions/AggregateFunctionTopK.h @@ -132,8 +132,8 @@ private: public: AggregateFunctionTopKGeneric( - UInt64 threshold_, UInt64 load_factor, const DataTypePtr & input_data_type_, const Array & params) - : IAggregateFunctionDataHelper>({input_data_type_}, params) + UInt64 threshold_, UInt64 load_factor, const DataTypes & argument_types_, const Array & params) + : IAggregateFunctionDataHelper>(argument_types_, params) , threshold(threshold_), reserved(load_factor * threshold), input_data_type(this->argument_types[0]) {} String getName() const override { return is_weighted ? "topKWeighted" : "topK"; } diff --git a/tests/queries/0_stateless/02149_issue_32487.reference b/tests/queries/0_stateless/02149_issue_32487.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02149_issue_32487.sql b/tests/queries/0_stateless/02149_issue_32487.sql new file mode 100644 index 00000000000..4e75c981774 --- /dev/null +++ b/tests/queries/0_stateless/02149_issue_32487.sql @@ -0,0 +1 @@ +SELECT topKWeightedState(2)(now(), 1) FORMAT Null; From 52af96627c86c2a00c007846a5cea88a6ce1b56b Mon Sep 17 00:00:00 2001 From: Dao Date: Fri, 17 Dec 2021 22:50:41 +0800 Subject: [PATCH 290/358] introduce new version and add test --- .../AggregateFunctionGroupBitmap.h | 31 ++++++- src/Core/ProtocolDefines.h | 2 +- .../__init__.py | 0 .../configs/clusters.xml | 32 ++++++++ .../test.py | 81 +++++++++++++++++++ 5 files changed, 141 insertions(+), 5 deletions(-) create mode 100644 tests/integration/test_groupBitmapAnd_on_distributed/__init__.py create mode 100644 tests/integration/test_groupBitmapAnd_on_distributed/configs/clusters.xml create mode 100644 tests/integration/test_groupBitmapAnd_on_distributed/test.py diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index f3322cf15c8..fbe48ae8e39 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -54,6 +54,8 @@ public: template class AggregateFunctionBitmapL2 final : public IAggregateFunctionDataHelper> { +private: + static constexpr auto STATE_VERSION_1_MIN_REVISION = 54454; public: AggregateFunctionBitmapL2(const DataTypePtr & type) : IAggregateFunctionDataHelper>({type}, {}) @@ -105,15 +107,36 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + bool isVersioned() const override { return true; } + + size_t getDefaultVersion() const override { return 1; } + + size_t getVersionFromRevision(size_t revision) const override { - DB::writeBoolText(this->data(place).init, buf); + if (revision >= STATE_VERSION_1_MIN_REVISION) + return 1; + else + return 0; + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override + { + if (!version) + version = getDefaultVersion(); + + if (*version >= 1) + DB::writeBoolText(this->data(place).init, buf); + this->data(place).rbs.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena *) const override { - DB::readBoolText(this->data(place).init, buf); + if (!version) + version = getDefaultVersion(); + + if (*version >= 1) + DB::readBoolText(this->data(place).init, buf); this->data(place).rbs.read(buf); } diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 36820788b91..720a581777b 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -52,6 +52,6 @@ /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54453 +#define DBMS_TCP_PROTOCOL_VERSION 54454 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/__init__.py b/tests/integration/test_groupBitmapAnd_on_distributed/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/configs/clusters.xml b/tests/integration/test_groupBitmapAnd_on_distributed/configs/clusters.xml new file mode 100644 index 00000000000..5ac57bed6a6 --- /dev/null +++ b/tests/integration/test_groupBitmapAnd_on_distributed/configs/clusters.xml @@ -0,0 +1,32 @@ + + + + + + node1 + 9000 + + + + + node2 + 9000 + + + + + + + node3 + 9000 + + + + + node4 + 9000 + + + + + diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py new file mode 100644 index 00000000000..5268774d1be --- /dev/null +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -0,0 +1,81 @@ +import pytest + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, exec_query_with_retry +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', main_configs=["configs/clusters.xml"]) +node2 = cluster.add_instance('node2', main_configs=["configs/clusters.xml"]) +node3 = cluster.add_instance('node3', main_configs=["configs/clusters.xml"]) +node4 = cluster.add_instance('node4', main_configs=["configs/clusters.xml"], image='yandex/clickhouse-server', tag='21.5') + +def insert_data(node, table_name): + node.query("""INSERT INTO {} + VALUES (bitmapBuild(cast([1,2,3,4,5,6,7,8,9,10] as Array(UInt32))));""".format(table_name)) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_groupBitmapAnd_on_distributed_table(start_cluster): + local_table_name = "bitmap_column_expr_test" + distributed_table_name = "bitmap_column_expr_test_dst" + cluster_name = "awsome_cluster" + + node1.query("""CREATE TABLE {} on cluster {} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = MergeTree() + ORDER BY tuple()""".format(local_table_name, cluster_name)) + + node1.query("""CREATE TABLE {0} on cluster {1} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = Distributed({1}, currentDatabase(), '{0}')""".format(distributed_table_name, cluster_name)) + insert_data(node1, local_table_name) + + expected = "10" + node1_result = node1.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() + node2_result = node1.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() + assert(node1_result == expected) + assert(node2_result == expected) + +def test_aggregate_function_versioning(start_cluster): + local_table_name = "bitmap_column_expr_versioning_test" + distributed_table_name = "bitmap_column_expr_versioning_test_dst" + cluster_name = "test_version_cluster" + + node3.query("""CREATE TABLE {} on cluster {} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = MergeTree() + ORDER BY tuple()""".format(local_table_name, cluster_name)) + + node3.query("""CREATE TABLE {0} on cluster {1} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = Distributed({1}, currentDatabase(), '{0}')""".format(distributed_table_name, cluster_name)) + + insert_data(node3, local_table_name) + insert_data(node4, local_table_name) + + expected = "10" + new_version_distributed_result = node3.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() + old_version_distributed_result = node4.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() + assert(new_version_distributed_result == expected) + assert(old_version_distributed_result == expected) + + result_from_old_to_new_version = node3.query("select groupBitmapAnd(z) FROM remote('node4', default.{})".format(local_table_name)).strip() + assert(result_from_old_to_new_version != expected) + + result_from_new_to_old_version = node4.query("select groupBitmapAnd(z) FROM remote('node3', default.{})".format(local_table_name)).strip() + assert(result_from_new_to_old_version != expected) From 1eb032ac8318688db1be67f79c422868e647cc70 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 17 Dec 2021 18:04:59 +0300 Subject: [PATCH 291/358] Style AggregateFunctionTopK.cpp --- .../AggregateFunctionTopK.cpp | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index 3a1689a4e64..1312cf701d1 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -7,11 +7,12 @@ #include -#define TOP_K_MAX_SIZE 0xFFFFFF +constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; namespace DB { + struct Settings; namespace ErrorCodes @@ -82,28 +83,26 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const if (!params.empty()) { if (params.size() > 2) - throw Exception("Aggregate function " + name + " requires two parameters or less.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Aggregate function '{}' requires two parameters or less", name); - UInt64 k = applyVisitor(FieldVisitorConvertToNumber(), params[0]); if (params.size() == 2) { load_factor = applyVisitor(FieldVisitorConvertToNumber(), params[1]); if (load_factor < 1) - throw Exception("Too small parameter 'load_factor' for aggregate function " + name + ". Minimum: 1", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Too small parameter 'load_factor' for aggregate function '{}' (got {}, minimum is 1)", name, load_factor); } - if (k > TOP_K_MAX_SIZE || load_factor > TOP_K_MAX_SIZE || k * load_factor > TOP_K_MAX_SIZE) - throw Exception("Too large parameter(s) for aggregate function " + name + ". Maximum: " + toString(TOP_K_MAX_SIZE), - ErrorCodes::ARGUMENT_OUT_OF_BOUND); + threshold = applyVisitor(FieldVisitorConvertToNumber(), params[0]); - if (k == 0) - throw Exception("Parameter 0 is illegal for aggregate function " + name, - ErrorCodes::ARGUMENT_OUT_OF_BOUND); + if (threshold > TOP_K_MAX_SIZE || load_factor > TOP_K_MAX_SIZE || threshold * load_factor > TOP_K_MAX_SIZE) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, + "Too large parameter(s) for aggregate function '{}' (maximum is {})", name, toString(TOP_K_MAX_SIZE)); - threshold = k; + if (threshold == 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Parameter 0 is illegal for aggregate function '{}'", name); } AggregateFunctionPtr res(createWithNumericType( @@ -113,9 +112,8 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const res = AggregateFunctionPtr(createWithExtraTypes(argument_types, threshold, load_factor, params)); if (!res) - throw Exception("Illegal type " + argument_types[0]->getName() + - " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument for aggregate function '{}'", argument_types[0]->getName(), name); return res; } From 2fb554c61f1e0af7fd8964794f6ddd771034b1e5 Mon Sep 17 00:00:00 2001 From: Dao Date: Fri, 17 Dec 2021 23:09:08 +0800 Subject: [PATCH 292/358] increase version to resolve conflict with master branch --- src/AggregateFunctions/AggregateFunctionGroupBitmap.h | 2 +- src/Core/ProtocolDefines.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index fbe48ae8e39..94d64d47b51 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -55,7 +55,7 @@ template class AggregateFunctionBitmapL2 final : public IAggregateFunctionDataHelper> { private: - static constexpr auto STATE_VERSION_1_MIN_REVISION = 54454; + static constexpr auto STATE_VERSION_1_MIN_REVISION = 54455; public: AggregateFunctionBitmapL2(const DataTypePtr & type) : IAggregateFunctionDataHelper>({type}, {}) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index c9a93f034a2..93f44b02ce3 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -53,6 +53,6 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54454 +#define DBMS_TCP_PROTOCOL_VERSION 54455 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 From efe1478ffe11a8142afea47b10e5cab14b750d91 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 17 Dec 2021 19:12:12 +0300 Subject: [PATCH 293/358] Update 01069_window_view_proc_tumble_watch.py --- .../queries/0_stateless/01069_window_view_proc_tumble_watch.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py index 03068789ea0..eb2b7835483 100755 --- a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py +++ b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py @@ -1,4 +1,6 @@ #!/usr/bin/env python3 +#Tags: no-parallel + import os import sys import signal From c8a92c046f95216e68ed3ba66b8f5c1eaedd8a1d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 17 Dec 2021 20:36:37 +0300 Subject: [PATCH 294/358] Another try --- src/Interpreters/SelectQueryOptions.h | 9 +++++++++ src/Interpreters/TreeRewriter.cpp | 2 +- src/Storages/ProjectionsDescription.cpp | 6 ++++-- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index 709ecdc239c..bc95a940c18 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -41,6 +41,9 @@ struct SelectQueryOptions /// It is needed because lazy normal projections require special planning in FetchColumns stage, such as adding WHERE transform. /// It is also used to avoid adding aggregating step when aggregate projection is chosen. bool is_projection_query = false; + /// This flag is needed for projection description. + /// Otherwise, keys for GROUP BY may be removed as constants. + bool ignore_ast_optimizations = false; bool ignore_alias = false; bool is_internal = false; bool is_subquery = false; // non-subquery can also have subquery_depth > 0, e.g. insert select @@ -120,6 +123,12 @@ struct SelectQueryOptions return *this; } + SelectQueryOptions & ignoreASTOptimizationsAlias(bool value = true) + { + ignore_ast_optimizations = value; + return *this; + } + SelectQueryOptions & setInternal(bool value = false) { is_internal = value; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 639d38d36f1..48ab2822c82 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1117,7 +1117,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.rewrite_subqueries = PredicateExpressionsOptimizer(getContext(), tables_with_columns, settings).optimize(*select_query); /// Only apply AST optimization for initial queries. - if (getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) + if (getContext()->getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY && !select_options.ignore_ast_optimizations) TreeOptimizer::apply(query, result, tables_with_columns, getContext()); /// array_join_alias_to_name, array_join_result_to_source. diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index f1a0372a07d..791583e2495 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -201,7 +201,7 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( select_expression_list->children.push_back(makeASTFunction("count")); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); - if (partition_columns) + if (partition_columns && !partition_columns->children.empty()) select_query->setExpression(ASTProjectionSelectQuery::Expression::GROUP_BY, partition_columns->clone()); result.definition_ast = select_query; @@ -211,7 +211,9 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( auto external_storage_holder = std::make_shared(query_context, columns, ConstraintsDescription{}); StoragePtr storage = external_storage_holder->getTable(); InterpreterSelectQuery select( - result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias()); + result.query_ast, query_context, storage, {}, + /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizationsAlias()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); From 492fa982768fa37bbba65588ee109d6985e44d28 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Dec 2021 22:09:05 +0300 Subject: [PATCH 295/358] Add replxx/jemalloc into fasttest build of clickhouse Both of the are pretty small, and so should not affect build time, but will have additional bonus, since jemalloc is the production allocator and replxx is just useful to use binary with completion (and later some test can be enabled for fasttest with completion) --- docker/test/fasttest/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index baa5945d347..01b6314217a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -173,6 +173,8 @@ function clone_submodules contrib/dragonbox contrib/fast_float contrib/NuRaft + contrib/jemalloc + contrib/replxx ) git submodule sync @@ -193,6 +195,8 @@ function run_cmake "-DENABLE_THINLTO=0" "-DUSE_UNWIND=1" "-DENABLE_NURAFT=1" + "-DENABLE_JEMALLOC=1" + "-DENABLE_REPLXX=1" ) # TODO remove this? we don't use ccache anyway. An option would be to download it From cb394c95d0f3fd59e7c3e34c9b740c77a759aec3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 17 Dec 2021 22:14:37 +0300 Subject: [PATCH 296/358] Enable tests that requires completion in fasttest --- tests/queries/0_stateless/01180_client_syntax_errors.expect | 1 - .../01293_client_interactive_vertical_multiline.expect | 1 - .../01293_client_interactive_vertical_singleline.expect | 1 - .../01300_client_save_history_when_terminated_long.expect | 2 +- .../01370_client_autocomplete_word_break_characters.expect | 1 - tests/queries/0_stateless/01520_client_print_query_id.expect | 1 - .../0_stateless/01565_reconnect_after_client_error.expect | 1 - .../0_stateless/01676_long_clickhouse_client_autocomplete.sh | 2 +- .../01755_client_highlight_multi_line_comment_regression.expect | 1 - .../01910_client_replxx_container_overflow_long.expect | 2 +- .../0_stateless/01933_client_replxx_convert_history.expect | 2 +- tests/queries/0_stateless/02003_memory_limit_in_client.expect | 2 +- tests/queries/0_stateless/02047_client_exception.expect | 1 - .../0_stateless/02049_clickhouse_local_merge_tree.expect | 1 - .../queries/0_stateless/02105_backslash_letter_commands.expect | 1 - .../02112_delayed_clickhouse_client_with_queries_file.expect | 2 +- tests/queries/0_stateless/02112_delayed_clickhouse_local.expect | 1 - .../02112_delayed_clickhouse_local_with_queries_file.expect | 2 +- tests/queries/0_stateless/02116_interactive_hello.expect | 1 - .../queries/0_stateless/02132_client_history_navigation.expect | 1 - 20 files changed, 7 insertions(+), 20 deletions(-) diff --git a/tests/queries/0_stateless/01180_client_syntax_errors.expect b/tests/queries/0_stateless/01180_client_syntax_errors.expect index e4b108fc9a9..c20982b2991 100755 --- a/tests/queries/0_stateless/01180_client_syntax_errors.expect +++ b/tests/queries/0_stateless/01180_client_syntax_errors.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect index 14ca38093bf..5e845754402 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_multiline.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect index 3abed6cae03..c68b153d5d3 100755 --- a/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect +++ b/tests/queries/0_stateless/01293_client_interactive_vertical_singleline.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect index 62a3df95abc..05d9d408228 100755 --- a/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect +++ b/tests/queries/0_stateless/01300_client_save_history_when_terminated_long.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: long, no-fasttest +# Tags: long log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect index fd2de93c39c..e0d01d905bb 100755 --- a/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect +++ b/tests/queries/0_stateless/01370_client_autocomplete_word_break_characters.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01520_client_print_query_id.expect b/tests/queries/0_stateless/01520_client_print_query_id.expect index 1989ab51aea..b0ff5d9d165 100755 --- a/tests/queries/0_stateless/01520_client_print_query_id.expect +++ b/tests/queries/0_stateless/01520_client_print_query_id.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect index 261e46c2111..712fe4ff64a 100755 --- a/tests/queries/0_stateless/01565_reconnect_after_client_error.expect +++ b/tests/queries/0_stateless/01565_reconnect_after_client_error.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest # This is a separate test, because we want to test the interactive mode. # https://github.com/ClickHouse/ClickHouse/issues/19353 diff --git a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh index fb2d97b6270..e029d90a686 100755 --- a/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh +++ b/tests/queries/0_stateless/01676_long_clickhouse_client_autocomplete.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest +# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect index 922a6914584..5543af4dd05 100755 --- a/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect +++ b/tests/queries/0_stateless/01755_client_highlight_multi_line_comment_regression.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect index 3003a0de42d..138727d296d 100755 --- a/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect +++ b/tests/queries/0_stateless/01910_client_replxx_container_overflow_long.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: long, no-fasttest +# Tags: long log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect index 664c3f06d20..59231161d91 100755 --- a/tests/queries/0_stateless/01933_client_replxx_convert_history.expect +++ b/tests/queries/0_stateless/01933_client_replxx_convert_history.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: no-parallel, no-fasttest +# Tags: no-parallel # Tag no-parallel: Uses non unique history file log_user 0 diff --git a/tests/queries/0_stateless/02003_memory_limit_in_client.expect b/tests/queries/0_stateless/02003_memory_limit_in_client.expect index 47ac4926537..19e3d4019ac 100755 --- a/tests/queries/0_stateless/02003_memory_limit_in_client.expect +++ b/tests/queries/0_stateless/02003_memory_limit_in_client.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: no-parallel, no-fasttest +# Tags: no-parallel # This is a test for system.warnings. Testing in interactive mode is necessary, # as we want to see certain warnings from client diff --git a/tests/queries/0_stateless/02047_client_exception.expect b/tests/queries/0_stateless/02047_client_exception.expect index 0025afa88eb..57a38c4f6aa 100755 --- a/tests/queries/0_stateless/02047_client_exception.expect +++ b/tests/queries/0_stateless/02047_client_exception.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 20 diff --git a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect index 17b98b077d5..89271805fb3 100755 --- a/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect +++ b/tests/queries/0_stateless/02049_clickhouse_local_merge_tree.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 20 diff --git a/tests/queries/0_stateless/02105_backslash_letter_commands.expect b/tests/queries/0_stateless/02105_backslash_letter_commands.expect index 9c6f3e10227..89d896fdedc 100755 --- a/tests/queries/0_stateless/02105_backslash_letter_commands.expect +++ b/tests/queries/0_stateless/02105_backslash_letter_commands.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 02 diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect index b676c221c65..73b12637906 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_client_with_queries_file.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: no-parallel, no-fasttest +# Tags: no-parallel log_user 0 set timeout 20 diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect index cd42388c099..fa146577234 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_local.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 20 diff --git a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect index f0aef1550c3..fbf79629f71 100755 --- a/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect +++ b/tests/queries/0_stateless/02112_delayed_clickhouse_local_with_queries_file.expect @@ -1,5 +1,5 @@ #!/usr/bin/expect -f -# Tags: no-parallel, no-fasttest +# Tags: no-parallel log_user 0 set timeout 20 diff --git a/tests/queries/0_stateless/02116_interactive_hello.expect b/tests/queries/0_stateless/02116_interactive_hello.expect index 1642ac91e42..49a167e5a6e 100755 --- a/tests/queries/0_stateless/02116_interactive_hello.expect +++ b/tests/queries/0_stateless/02116_interactive_hello.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 60 diff --git a/tests/queries/0_stateless/02132_client_history_navigation.expect b/tests/queries/0_stateless/02132_client_history_navigation.expect index 129a65e0a0a..cd83454c85e 100755 --- a/tests/queries/0_stateless/02132_client_history_navigation.expect +++ b/tests/queries/0_stateless/02132_client_history_navigation.expect @@ -1,5 +1,4 @@ #!/usr/bin/expect -f -# Tags: no-fasttest log_user 0 set timeout 3 From e6ebb55c4e5d14adc4a41d818e6f03407947fd6b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 17 Dec 2021 22:19:10 +0300 Subject: [PATCH 297/358] Move to Interpreters --- src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- src/{Common => Interpreters}/getTableOverride.cpp | 0 src/{Common => Interpreters}/getTableOverride.h | 0 src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 +- 4 files changed, 2 insertions(+), 2 deletions(-) rename src/{Common => Interpreters}/getTableOverride.cpp (100%) rename src/{Common => Interpreters}/getTableOverride.h (100%) diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index e5c4a85dc27..853d84a9695 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Common/getTableOverride.cpp b/src/Interpreters/getTableOverride.cpp similarity index 100% rename from src/Common/getTableOverride.cpp rename to src/Interpreters/getTableOverride.cpp diff --git a/src/Common/getTableOverride.h b/src/Interpreters/getTableOverride.h similarity index 100% rename from src/Common/getTableOverride.h rename to src/Interpreters/getTableOverride.h diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index dcd64314028..984a9cdd47a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,12 +1,12 @@ #include "PostgreSQLReplicationHandler.h" #include -#include #include #include #include #include #include +#include #include #include #include From 587b663e56033bc2bb4c2c4847f9bb7ed702391f Mon Sep 17 00:00:00 2001 From: Ilya Golshtein Date: Fri, 17 Dec 2021 22:49:40 +0300 Subject: [PATCH 298/358] a comment requested by reviewer --- tests/integration/test_storage_kerberized_kafka/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index b5f779cbd3d..567a9b7184d 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -97,6 +97,9 @@ def test_kafka_json_as_string(kafka_cluster): assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") def test_kafka_json_as_string_no_kdc(kafka_cluster): + # When the test is run alone (not preceded by any other kerberized kafka test), + # we need a ticket to + # assert instance.contains_in_log("Ticket expired") instance.query(''' CREATE TABLE test.kafka_no_kdc_warm_up (field String) ENGINE = Kafka From 8bbd006d42751bf579e853c62bc3a6400874f31c Mon Sep 17 00:00:00 2001 From: Dao Date: Sat, 18 Dec 2021 07:55:12 +0800 Subject: [PATCH 299/358] enable zookeeper for integration test --- .../test.py | 71 ++++++++++--------- 1 file changed, 36 insertions(+), 35 deletions(-) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index 5268774d1be..d237dde6717 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -1,13 +1,12 @@ import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry, exec_query_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', main_configs=["configs/clusters.xml"]) -node2 = cluster.add_instance('node2', main_configs=["configs/clusters.xml"]) -node3 = cluster.add_instance('node3', main_configs=["configs/clusters.xml"]) -node4 = cluster.add_instance('node4', main_configs=["configs/clusters.xml"], image='yandex/clickhouse-server', tag='21.5') +node1 = cluster.add_instance('node1', main_configs=["configs/clusters.xml"], with_zookeeper=True) +node2 = cluster.add_instance('node2', main_configs=["configs/clusters.xml"], with_zookeeper=True) +node3 = cluster.add_instance('node3', main_configs=["configs/clusters.xml"], with_zookeeper=True) +node4 = cluster.add_instance('node4', main_configs=["configs/clusters.xml"], image='yandex/clickhouse-server', tag='21.5', with_zookeeper=True) def insert_data(node, table_name): node.query("""INSERT INTO {} @@ -27,46 +26,48 @@ def test_groupBitmapAnd_on_distributed_table(start_cluster): distributed_table_name = "bitmap_column_expr_test_dst" cluster_name = "awsome_cluster" - node1.query("""CREATE TABLE {} on cluster {} - ( - z AggregateFunction(groupBitmap, UInt32) - ) - ENGINE = MergeTree() - ORDER BY tuple()""".format(local_table_name, cluster_name)) + for node in (node1, node2): + node.query("""CREATE TABLE {} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = MergeTree() + ORDER BY tuple()""".format(local_table_name)) - node1.query("""CREATE TABLE {0} on cluster {1} - ( - z AggregateFunction(groupBitmap, UInt32) - ) - ENGINE = Distributed({1}, currentDatabase(), '{0}')""".format(distributed_table_name, cluster_name)) + node.query("""CREATE TABLE {0} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = Distributed('{1}', 'default', '{0}')""".format(distributed_table_name, cluster_name)) insert_data(node1, local_table_name) expected = "10" - node1_result = node1.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() - node2_result = node1.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() - assert(node1_result == expected) - assert(node2_result == expected) + + for node in (node1, node2): + result = node.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() + assert(result == expected) def test_aggregate_function_versioning(start_cluster): local_table_name = "bitmap_column_expr_versioning_test" distributed_table_name = "bitmap_column_expr_versioning_test_dst" cluster_name = "test_version_cluster" - node3.query("""CREATE TABLE {} on cluster {} - ( - z AggregateFunction(groupBitmap, UInt32) - ) - ENGINE = MergeTree() - ORDER BY tuple()""".format(local_table_name, cluster_name)) + for node in (node3, node4): + node.query("""CREATE TABLE {} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = MergeTree() + ORDER BY tuple()""".format(local_table_name)) - node3.query("""CREATE TABLE {0} on cluster {1} - ( - z AggregateFunction(groupBitmap, UInt32) - ) - ENGINE = Distributed({1}, currentDatabase(), '{0}')""".format(distributed_table_name, cluster_name)) + node.query("""CREATE TABLE {0} + ( + z AggregateFunction(groupBitmap, UInt32) + ) + ENGINE = Distributed('{1}', 'default', '{0}')""".format(distributed_table_name, cluster_name)) - insert_data(node3, local_table_name) - insert_data(node4, local_table_name) + node.query("""INSERT INTO {} VALUES + (bitmapBuild(cast([1,2,3,4,5,6,7,8,9,10] as Array(UInt32))));""".format(local_table_name)) expected = "10" new_version_distributed_result = node3.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() @@ -74,8 +75,8 @@ def test_aggregate_function_versioning(start_cluster): assert(new_version_distributed_result == expected) assert(old_version_distributed_result == expected) - result_from_old_to_new_version = node3.query("select groupBitmapAnd(z) FROM remote('node4', default.{})".format(local_table_name)).strip() + result_from_old_to_new_version = node3.query("select groupBitmapAnd(z) FROM remote('node4', default.{})".format(local_table_name)).strip() assert(result_from_old_to_new_version != expected) - result_from_new_to_old_version = node4.query("select groupBitmapAnd(z) FROM remote('node3', default.{})".format(local_table_name)).strip() + result_from_new_to_old_version = node4.query("select groupBitmapAnd(z) FROM remote('node3', default.{})".format(local_table_name)).strip() assert(result_from_new_to_old_version != expected) From 21b8a119074b40fcb2f202b9a477811034414b88 Mon Sep 17 00:00:00 2001 From: Dao Date: Sat, 18 Dec 2021 12:27:31 +0800 Subject: [PATCH 300/358] fix intergration test --- .../test_groupBitmapAnd_on_distributed/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index d237dde6717..96b6420a59c 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -34,11 +34,11 @@ def test_groupBitmapAnd_on_distributed_table(start_cluster): ENGINE = MergeTree() ORDER BY tuple()""".format(local_table_name)) - node.query("""CREATE TABLE {0} + node.query("""CREATE TABLE {} ( z AggregateFunction(groupBitmap, UInt32) ) - ENGINE = Distributed('{1}', 'default', '{0}')""".format(distributed_table_name, cluster_name)) + ENGINE = Distributed('{}', 'default', '{}')""".format(distributed_table_name, cluster_name, local_table_name)) insert_data(node1, local_table_name) expected = "10" @@ -60,11 +60,11 @@ def test_aggregate_function_versioning(start_cluster): ENGINE = MergeTree() ORDER BY tuple()""".format(local_table_name)) - node.query("""CREATE TABLE {0} + node.query("""CREATE TABLE {} ( z AggregateFunction(groupBitmap, UInt32) ) - ENGINE = Distributed('{1}', 'default', '{0}')""".format(distributed_table_name, cluster_name)) + ENGINE = Distributed('{}', 'default', '{}')""".format(distributed_table_name, cluster_name, local_table_name)) node.query("""INSERT INTO {} VALUES (bitmapBuild(cast([1,2,3,4,5,6,7,8,9,10] as Array(UInt32))));""".format(local_table_name)) From 8571501cf753a4e713d1258ed9d95cc34b94d62a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 18 Dec 2021 07:49:35 +0300 Subject: [PATCH 301/358] Update AggregateFunctionTopK.cpp --- src/AggregateFunctions/AggregateFunctionTopK.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index 1312cf701d1..801f3d5e28d 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -7,7 +7,7 @@ #include -constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; +static inline constexpr UInt64 TOP_K_MAX_SIZE = 0xFFFFFF; namespace DB From 5500d0e439cfcfc09b56baf35ddbf5891ae808cf Mon Sep 17 00:00:00 2001 From: Dao Date: Sat, 18 Dec 2021 13:35:36 +0800 Subject: [PATCH 302/358] fix intergration test --- tests/integration/test_groupBitmapAnd_on_distributed/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index 96b6420a59c..e51a5c46595 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -24,7 +24,7 @@ def start_cluster(): def test_groupBitmapAnd_on_distributed_table(start_cluster): local_table_name = "bitmap_column_expr_test" distributed_table_name = "bitmap_column_expr_test_dst" - cluster_name = "awsome_cluster" + cluster_name = "awesome_cluster" for node in (node1, node2): node.query("""CREATE TABLE {} From e3f16cb73f273f22bb765555d5bb16ee76c85894 Mon Sep 17 00:00:00 2001 From: Dao Date: Sat, 18 Dec 2021 14:42:42 +0800 Subject: [PATCH 303/358] fix intergration test --- .../integration/test_groupBitmapAnd_on_distributed/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_groupBitmapAnd_on_distributed/test.py b/tests/integration/test_groupBitmapAnd_on_distributed/test.py index e51a5c46595..b0fb55b13ff 100644 --- a/tests/integration/test_groupBitmapAnd_on_distributed/test.py +++ b/tests/integration/test_groupBitmapAnd_on_distributed/test.py @@ -47,7 +47,7 @@ def test_groupBitmapAnd_on_distributed_table(start_cluster): result = node.query("select groupBitmapAnd(z) FROM {};".format(distributed_table_name)).strip() assert(result == expected) -def test_aggregate_function_versioning(start_cluster): +def test_groupBitmapAnd_function_versioning(start_cluster): local_table_name = "bitmap_column_expr_versioning_test" distributed_table_name = "bitmap_column_expr_versioning_test_dst" cluster_name = "test_version_cluster" @@ -76,7 +76,7 @@ def test_aggregate_function_versioning(start_cluster): assert(old_version_distributed_result == expected) result_from_old_to_new_version = node3.query("select groupBitmapAnd(z) FROM remote('node4', default.{})".format(local_table_name)).strip() - assert(result_from_old_to_new_version != expected) + assert(result_from_old_to_new_version == expected) result_from_new_to_old_version = node4.query("select groupBitmapAnd(z) FROM remote('node3', default.{})".format(local_table_name)).strip() - assert(result_from_new_to_old_version != expected) + assert(result_from_new_to_old_version == expected) From f93eb61946d7cd0a2521813eb0040762238d6a50 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 18 Dec 2021 13:19:53 +0300 Subject: [PATCH 304/358] tests/integration/cleanup_environment: fix obtaining PYTEST_CLEANUP_CONTAINERS --- tests/integration/conftest.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 9461f4a81c5..4b0a9a2835b 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -8,12 +8,12 @@ from helpers.network import _NetworkManager @pytest.fixture(autouse=True, scope="session") def cleanup_environment(): try: - if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS")) == 1: + if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS", 0)) == 1: logging.debug(f"Cleaning all iptables rules") _NetworkManager.clean_all_user_iptables_rules() result = run_and_check(['docker ps | wc -l'], shell=True) if int(result) > 1: - if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS")) != 1: + if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS", 0)) != 1: logging.warning(f"Docker containters({int(result)}) are running before tests run. They can be left from previous pytest run and cause test failures.\n"\ "You can set env PYTEST_CLEANUP_CONTAINERS=1 or use runner with --cleanup-containers argument to enable automatic containers cleanup.") else: From 59ac72cf90dd860307c000b5d9de978e0cb5e6ce Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sat, 18 Dec 2021 20:41:09 +0800 Subject: [PATCH 305/358] fix some words --- .../operations/system-tables/merge_tree_settings.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/zh/operations/system-tables/merge_tree_settings.md b/docs/zh/operations/system-tables/merge_tree_settings.md index 1592aae10a9..48d9a7dd9af 100644 --- a/docs/zh/operations/system-tables/merge_tree_settings.md +++ b/docs/zh/operations/system-tables/merge_tree_settings.md @@ -1,14 +1,14 @@ # system.merge_tree_settings {#system-merge_tree_settings} -åŒ…å« `MergeTree` 表的é…置信æ¯ã€‚ +åŒ…å« `MergeTree` 表的设置 (Setting) ä¿¡æ¯ã€‚ 列: -- `name` (String) — é…ç½®å称. -- `value` (String) — é…置的值. -- `description` (String) — é…ç½®æè¿°. -- `type` (String) — é…置类型 (执行特定的字符串值). -- `changed` (UInt8) — 该设置是å¦åœ¨é…置中明确定义或明确改å˜. +- `name` (String) — 设置å称。 +- `value` (String) — 设置的值。 +- `description` (String) — 设置æ述。 +- `type` (String) — 设置类型 (执行特定的字符串值)。 +- `changed` (UInt8) — 该设置是å¦åœ¨é…置中明确定义或是明确改å˜ã€‚ **示例** From 4e4ed6e32231bc22584abe4cbacba24a24427229 Mon Sep 17 00:00:00 2001 From: meoww-bot <14239840+meoww-bot@users.noreply.github.com> Date: Sat, 18 Dec 2021 20:52:34 +0800 Subject: [PATCH 306/358] fix links --- docs/zh/operations/system-tables/tables.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/zh/operations/system-tables/tables.md b/docs/zh/operations/system-tables/tables.md index bb8110e1da3..03ea9f93d26 100644 --- a/docs/zh/operations/system-tables/tables.md +++ b/docs/zh/operations/system-tables/tables.md @@ -1,10 +1,10 @@ # system.tables {#system-tables} -包å«æœåŠ¡å™¨çŸ¥é“çš„æ¯ä¸ªè¡¨çš„元数æ®ã€‚ [分离的](https://clickhouse.com/docs/zh/sql-reference/statements/detach/)表ä¸åœ¨ `system.tables` 显示。 +包å«æœåŠ¡å™¨çŸ¥é“çš„æ¯ä¸ªè¡¨çš„元数æ®ã€‚ [分离的](../../sql-reference/statements/detach.md)表ä¸åœ¨ `system.tables` 显示。 -[临时表](https://clickhouse.com/docs/zh/sql-reference/statements/create/table/#temporary-tables)åªåœ¨åˆ›å»ºå®ƒä»¬çš„会è¯ä¸­çš„ `system.tables` 中æ‰å¯è§ã€‚它们的数æ®åº“字段显示为空,并且 `is_temporary` 标志显示为开å¯ã€‚ +[临时表](../../sql-reference/statements/create/table.md#temporary-tables)åªåœ¨åˆ›å»ºå®ƒä»¬çš„会è¯ä¸­çš„ `system.tables` 中æ‰å¯è§ã€‚它们的数æ®åº“字段显示为空,并且 `is_temporary` 标志显示为开å¯ã€‚ -此表包å«ä»¥ä¸‹åˆ—(列类型显示在括å·ä¸­): +此表包å«ä»¥ä¸‹åˆ— (列类型显示在括å·ä¸­): - `database` ([String](../../sql-reference/data-types/string.md)) — 表所在的数æ®åº“å。 @@ -22,7 +22,7 @@ - `dependencies_database` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) - æ•°æ®åº“ä¾èµ–关系。 -- `dependencies_table` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)))- 表ä¾èµ–关系 (基于当å‰è¡¨çš„ [物化视图](../../engines/table-engines/special/materializedview.md) 表)。 +- `dependencies_table` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) - 表ä¾èµ–关系 (基于当å‰è¡¨çš„ [物化视图](../../engines/table-engines/special/materializedview.md) 表) 。 - `create_table_query` ([String](../../sql-reference/data-types/string.md)) - 用于创建表的 SQL 语å¥ã€‚ @@ -43,17 +43,17 @@ - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - 总行数,如果无法快速确定表中的确切行数,则行数返回为 `NULL`(包括底层 `Buffer` 表)。 +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - 总行数,如果无法快速确定表中的确切行数,则行数返回为 `NULL` (包括底层 `Buffer` 表) 。 -- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - 总字节数,如果无法快速确定存储表的确切字节数,则字节数返回为 `NULL` ( **ä¸** 包括任何底层存储)。 +- `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - 总字节数,如果无法快速确定存储表的确切字节数,则字节数返回为 `NULL` ( **ä¸** 包括任何底层存储) 。 - - 如果表将数æ®å­˜åœ¨ç£ç›˜ä¸Šï¼Œè¿”回实际使用的ç£ç›˜ç©ºé—´ï¼ˆåŽ‹ç¼©åŽï¼‰ã€‚ + - 如果表将数æ®å­˜åœ¨ç£ç›˜ä¸Šï¼Œè¿”回实际使用的ç£ç›˜ç©ºé—´ (压缩åŽ) 。 - 如果表在内存中存储数æ®ï¼Œè¿”回在内存中使用的近似字节数。 -- `lifetime_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - æœåŠ¡å¯åŠ¨åŽæ’入的总行数(åªé’ˆå¯¹ `Buffer` 表)。 +- `lifetime_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - æœåŠ¡å¯åŠ¨åŽæ’入的总行数(åªé’ˆå¯¹ `Buffer` 表) 。 -- `lifetime_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - æœåŠ¡å¯åŠ¨åŽæ’入的总字节数(åªé’ˆå¯¹ `Buffer` 表)。 +- `lifetime_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - æœåŠ¡å¯åŠ¨åŽæ’入的总字节数(åªé’ˆå¯¹ `Buffer` 表) 。 - `comment` ([String](../../sql-reference/data-types/string.md)) - 表的注释。 From aa52eff0ecf8d2201972d84d2726db2fcf3896c3 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 18 Dec 2021 15:48:23 -0400 Subject: [PATCH 307/358] fix typo --- src/QueryPipeline/RemoteQueryExecutor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index d5603fd2281..bffbb64dc04 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -227,7 +227,7 @@ private: void processMergeTreeReadTaskRequest(PartitionReadRequest request); - /// Cancell query and restart it with info about duplicated UUIDs + /// Cancel query and restart it with info about duplicated UUIDs /// only for `allow_experimental_query_deduplication`. std::variant restartQueryWithoutDuplicatedUUIDs(std::unique_ptr * read_context = nullptr); From 293191247fc02f92326516ea74e937b2e6e87802 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sat, 18 Dec 2021 15:50:16 -0400 Subject: [PATCH 308/358] Update RemoteQueryExecutor.h --- src/QueryPipeline/RemoteQueryExecutor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index bffbb64dc04..655bd5603de 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -227,7 +227,7 @@ private: void processMergeTreeReadTaskRequest(PartitionReadRequest request); - /// Cancel query and restart it with info about duplicated UUIDs + /// Cancel query and restart it with info about duplicate UUIDs /// only for `allow_experimental_query_deduplication`. std::variant restartQueryWithoutDuplicatedUUIDs(std::unique_ptr * read_context = nullptr); From b059154a810ce6a38167eccf578a152ddb28328a Mon Sep 17 00:00:00 2001 From: "N. Kolotov" Date: Sat, 18 Dec 2021 23:13:42 +0300 Subject: [PATCH 309/358] Fixed segfault when smart pointer is dereferenced after move. --- src/Columns/ColumnDecimal.cpp | 8 +++++--- src/Columns/ColumnFixedString.cpp | 6 +++--- src/Columns/ColumnString.cpp | 11 +++++++---- src/Columns/ColumnVector.cpp | 8 +++++--- 4 files changed, 20 insertions(+), 13 deletions(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 70e2b4a6d96..99085f0f976 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -331,7 +331,8 @@ void ColumnDecimal::gather(ColumnGathererStream & gatherer) template ColumnPtr ColumnDecimal::compress() const { - size_t source_size = data.size() * sizeof(T); + const size_t data_size = data.size(); + const size_t source_size = data_size * sizeof(T); /// Don't compress small blocks. if (source_size < 4096) /// A wild guess. @@ -342,8 +343,9 @@ ColumnPtr ColumnDecimal::compress() const if (!compressed) return ColumnCompressed::wrap(this->getPtr()); - return ColumnCompressed::create(data.size(), compressed->size(), - [compressed = std::move(compressed), column_size = data.size(), scale = this->scale] + const size_t compressed_size = compressed->size(); + return ColumnCompressed::create(data_size, compressed_size, + [compressed = std::move(compressed), column_size = data_size, scale = this->scale] { auto res = ColumnDecimal::create(column_size, scale); ColumnCompressed::decompressBuffer( diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index 1eb2d4d5b1f..f474888b65a 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -409,9 +409,9 @@ ColumnPtr ColumnFixedString::compress() const if (!compressed) return ColumnCompressed::wrap(this->getPtr()); - size_t column_size = size(); - - return ColumnCompressed::create(column_size, compressed->size(), + const size_t column_size = size(); + const size_t compressed_size = compressed->size(); + return ColumnCompressed::create(column_size, compressed_size, [compressed = std::move(compressed), column_size, n = n] { size_t chars_size = n * column_size; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 2beb9add318..cd8a3e698d8 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -474,8 +474,9 @@ void ColumnString::getExtremes(Field & min, Field & max) const ColumnPtr ColumnString::compress() const { - size_t source_chars_size = chars.size(); - size_t source_offsets_size = offsets.size() * sizeof(Offset); + const size_t source_chars_size = chars.size(); + const size_t source_offsets_elements = offsets.size(); + const size_t source_offsets_size = source_offsets_elements * sizeof(Offset); /// Don't compress small blocks. if (source_chars_size < 4096) /// A wild guess. @@ -489,12 +490,14 @@ ColumnPtr ColumnString::compress() const auto offsets_compressed = ColumnCompressed::compressBuffer(offsets.data(), source_offsets_size, true); - return ColumnCompressed::create(offsets.size(), chars_compressed->size() + offsets_compressed->size(), + const size_t chars_compressed_size = chars_compressed->size(); + const size_t offsets_compressed_size = offsets_compressed->size(); + return ColumnCompressed::create(source_offsets_elements, chars_compressed_size + offsets_compressed_size, [ chars_compressed = std::move(chars_compressed), offsets_compressed = std::move(offsets_compressed), source_chars_size, - source_offsets_elements = offsets.size() + source_offsets_elements ] { auto res = ColumnString::create(); diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index 13ba522b2ac..d6a7fad8c8c 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -481,7 +481,8 @@ void ColumnVector::getExtremes(Field & min, Field & max) const template ColumnPtr ColumnVector::compress() const { - size_t source_size = data.size() * sizeof(T); + const size_t data_size = data.size(); + const size_t source_size = data_size * sizeof(T); /// Don't compress small blocks. if (source_size < 4096) /// A wild guess. @@ -492,8 +493,9 @@ ColumnPtr ColumnVector::compress() const if (!compressed) return ColumnCompressed::wrap(this->getPtr()); - return ColumnCompressed::create(data.size(), compressed->size(), - [compressed = std::move(compressed), column_size = data.size()] + const size_t compressed_size = compressed->size(); + return ColumnCompressed::create(data_size, compressed_size, + [compressed = std::move(compressed), column_size = data_size] { auto res = ColumnVector::create(column_size); ColumnCompressed::decompressBuffer( From 7f5ac732efece89c9d5f2beb37055c55783b2f13 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 19 Dec 2021 08:03:14 +0300 Subject: [PATCH 310/358] Update tests.md --- docs/en/development/tests.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index b61bfef50bb..bd31fecd87b 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -106,20 +106,20 @@ Build ClickHouse. Run ClickHouse from the terminal: change directory to `program Note that all clickhouse tools (server, client, etc) are just symlinks to a single binary named `clickhouse`. You can find this binary at `programs/clickhouse`. All tools can also be invoked as `clickhouse tool` instead of `clickhouse-tool`. -Alternatively you can install ClickHouse package: either stable release from ClickHouse repository or you can build package for yourself with `./release` in ClickHouse sources root. Then start the server with `sudo service clickhouse-server start` (or stop to stop the server). Look for logs at `/etc/clickhouse-server/clickhouse-server.log`. +Alternatively you can install ClickHouse package: either stable release from ClickHouse repository or you can build package for yourself with `./release` in ClickHouse sources root. Then start the server with `sudo clickhouse start` (or stop to stop the server). Look for logs at `/etc/clickhouse-server/clickhouse-server.log`. When ClickHouse is already installed on your system, you can build a new `clickhouse` binary and replace the existing binary: ``` bash -$ sudo service clickhouse-server stop +$ sudo clickhouse stop $ sudo cp ./clickhouse /usr/bin/ -$ sudo service clickhouse-server start +$ sudo clickhouse start ``` Also you can stop system clickhouse-server and run your own with the same configuration but with logging to terminal: ``` bash -$ sudo service clickhouse-server stop +$ sudo clickhouse stop $ sudo -u clickhouse /usr/bin/clickhouse server --config-file /etc/clickhouse-server/config.xml ``` From 492cc94c382bb607b8f66c58d63fb1427dca974e Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 19 Dec 2021 08:04:06 +0300 Subject: [PATCH 311/358] Update tests.md --- docs/en/development/tests.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index bd31fecd87b..7d57e16b67a 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -257,6 +257,10 @@ There are five variants (Debug, ASan, TSan, MSan, UBSan). Thread Fuzzer (please don't mix up with Thread Sanitizer) is another kind of fuzzing that allows to randomize thread order of execution. It helps to find even more special cases. +## Security Audit + +People from Yandex Security Team did some basic overview of ClickHouse capabilities from the security standpoint. + ## Static Analyzers {#static-analyzers} We run `clang-tidy` and `PVS-Studio` on per-commit basis. `clang-static-analyzer` checks are also enabled. `clang-tidy` is also used for some style checks. From 851f5a11d567083017c3c6a46dddd5faa11f959d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 19 Dec 2021 08:04:55 +0300 Subject: [PATCH 312/358] Update tests.md --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index 7d57e16b67a..5f3245c4d60 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -326,7 +326,7 @@ There is automated check for flaky tests. It runs all new tests 100 times (for f ## Testflows -[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to implement a subset of ClickHouse. +[Testflows](https://testflows.com/) is an enterprise-grade open-source testing framework, which is used to test a subset of ClickHouse. ## Test Automation {#test-automation} From 7df4d3f4f37852cbbc16136b1b169f6ac7e11ddd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 09:36:03 +0300 Subject: [PATCH 313/358] Add new adopters --- docs/en/introduction/adopters.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 325cd1ff825..c60e822438d 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -178,5 +178,9 @@ toc_title: Adopters | Цифровой Рабочий | Industrial IoT, Analytics | — | — | — | [Blog post in Russian, March 2021](https://habr.com/en/company/croc/blog/548018/) | | ООО «МПЗ БогородÑкий» | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) | | ДомКлик | Real Estate | — | — | — | [Article in Russian, October 2021](https://habr.com/ru/company/domclick/blog/585936/) | +| Futurra Group | Analytics | — | — | — | [Article in Russian, December 2021](https://dou.ua/forums/topic/35587/) | +| UseTech | Software Development | — | — | — | [Job Posting, December 2021](https://vk.com/wall136266658_2418) | +| Lookforsale | E-Commerce | — | — | — | [Job Posting, December 2021](https://telegram.me/javascript_jobs/587318) | +| R-Vision | Information Security | — | — | — | [Article in Russian, December 2021](https://www.anti-malware.ru/reviews/R-Vision-SENSE-15) | [Original article](https://clickhouse.com/docs/en/introduction/adopters/) From cee850b3beb906c5de5403a9024c91f5997d917a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 19 Dec 2021 11:32:34 +0300 Subject: [PATCH 314/358] Revert "fix replaceRegexpAll bug" --- src/Functions/ReplaceRegexpImpl.h | 16 +--------------- .../02100_replaceRegexpAll_bug.reference | 11 ----------- .../0_stateless/02100_replaceRegexpAll_bug.sql | 14 -------------- 3 files changed, 1 insertion(+), 40 deletions(-) delete mode 100644 tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference delete mode 100644 tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 4320f47c424..1539ecf49a4 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -94,9 +94,6 @@ struct ReplaceRegexpImpl re2_st::StringPiece matches[max_captures]; size_t start_pos = 0; - bool is_first_match = true; - bool is_start_pos_added_one = false; - while (start_pos < static_cast(input.length())) { /// If no more replacements possible for current string @@ -104,9 +101,6 @@ struct ReplaceRegexpImpl if (searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) { - if (is_start_pos_added_one) - start_pos -= 1; - const auto & match = matches[0]; size_t bytes_to_copy = (match.data() - input.data()) - start_pos; @@ -116,13 +110,6 @@ struct ReplaceRegexpImpl res_offset += bytes_to_copy; start_pos += bytes_to_copy + match.length(); - /// To avoid infinite loop. - if (is_first_match && match.length() == 0 && !replace_one && input.length() > 1) - { - start_pos += 1; - is_start_pos_added_one = true; - } - /// Do substitution instructions for (const auto & it : instructions) { @@ -140,9 +127,8 @@ struct ReplaceRegexpImpl } } - if (replace_one || (!is_first_match && match.length() == 0)) + if (replace_one || match.length() == 0) /// Stop after match of zero length, to avoid infinite loop. can_finish_current_string = true; - is_first_match = false; } else can_finish_current_string = true; diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference deleted file mode 100644 index 993dd9b1cde..00000000000 --- a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference +++ /dev/null @@ -1,11 +0,0 @@ -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql deleted file mode 100644 index 32f7f63f6d0..00000000000 --- a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql +++ /dev/null @@ -1,14 +0,0 @@ -SELECT 'aaaabb ' == trim(leading 'b ' FROM 'b aaaabb ') x; -SELECT 'b aaaa' == trim(trailing 'b ' FROM 'b aaaabb ') x; -SELECT 'aaaa' == trim(both 'b ' FROM 'b aaaabb ') x; - -SELECT '1' == replaceRegexpAll(',,1,,', '^[,]*|[,]*$', '') x; -SELECT '1' == replaceRegexpAll(',,1', '^[,]*|[,]*$', '') x; -SELECT '1' == replaceRegexpAll('1,,', '^[,]*|[,]*$', '') x; - -SELECT '1,,' == replaceRegexpOne(',,1,,', '^[,]*|[,]*$', '') x; -SELECT '1' == replaceRegexpOne(',,1', '^[,]*|[,]*$', '') x; -SELECT '1,,' == replaceRegexpOne('1,,', '^[,]*|[,]*$', '') x; - -SELECT '5935,5998,6014' == trim(BOTH ', ' FROM '5935,5998,6014, ') x; -SELECT '5935,5998,6014' == replaceRegexpAll('5935,5998,6014, ', concat('^[', regexpQuoteMeta(', '), ']*|[', regexpQuoteMeta(', '), ']*$'), '') AS x; From 0db88e3700378ad2440d4a29aeb2cf828fe3740a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 12:11:23 +0300 Subject: [PATCH 315/358] Add a test --- .../0_stateless/02150_replace_regexp_all_empty_match.reference | 1 + .../queries/0_stateless/02150_replace_regexp_all_empty_match.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02150_replace_regexp_all_empty_match.reference create mode 100644 tests/queries/0_stateless/02150_replace_regexp_all_empty_match.sql diff --git a/tests/queries/0_stateless/02150_replace_regexp_all_empty_match.reference b/tests/queries/0_stateless/02150_replace_regexp_all_empty_match.reference new file mode 100644 index 00000000000..6e269c2a690 --- /dev/null +++ b/tests/queries/0_stateless/02150_replace_regexp_all_empty_match.reference @@ -0,0 +1 @@ +here: Hello, World! diff --git a/tests/queries/0_stateless/02150_replace_regexp_all_empty_match.sql b/tests/queries/0_stateless/02150_replace_regexp_all_empty_match.sql new file mode 100644 index 00000000000..a7b52a1c8b6 --- /dev/null +++ b/tests/queries/0_stateless/02150_replace_regexp_all_empty_match.sql @@ -0,0 +1 @@ +select replaceRegexpAll('Hello, World!', '^', 'here: '); From 0d6e8e90eb5ad58378920d4715047202fc0bd73c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 12:44:42 +0300 Subject: [PATCH 316/358] Proper fix for #30245 --- src/Functions/ReplaceRegexpImpl.h | 77 +++++++++++-------- ...gexp_all_empty_match_alternative.reference | 3 + ...ace_regexp_all_empty_match_alternative.sql | 3 + 3 files changed, 53 insertions(+), 30 deletions(-) create mode 100644 tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.reference create mode 100644 tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.sql diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 1539ecf49a4..56b7dd3fd7e 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -32,10 +32,18 @@ template struct ReplaceRegexpImpl { /// Sequence of instructions, describing how to get resulting string. - /// Each element is either: - /// - substitution (in that case first element of pair is their number and second element is empty) - /// - string that need to be inserted (in that case, first element of pair is that string and second element is -1) - using Instructions = std::vector>; + struct Instruction + { + /// If non zero - perform substitution of n-th subpattern from the regexp match. + int substitution_num = -1; + /// Otherwise - paste this string verbatim. + std::string literal; + + Instruction(int substitution_num_) : substitution_num(substitution_num_) {} + Instruction(std::string literal_) : literal(std::move(literal_)) {} + }; + + using Instructions = std::vector; static const size_t max_captures = 10; @@ -53,10 +61,10 @@ struct ReplaceRegexpImpl { if (!now.empty()) { - instructions.emplace_back(-1, now); + instructions.emplace_back(now); now = ""; } - instructions.emplace_back(s[i + 1] - '0', String()); + instructions.emplace_back(s[i + 1] - '0'); } else now += s[i + 1]; /// Escaping @@ -68,16 +76,15 @@ struct ReplaceRegexpImpl if (!now.empty()) { - instructions.emplace_back(-1, now); + instructions.emplace_back(now); now = ""; } for (const auto & it : instructions) - if (it.first >= num_captures) - throw Exception( - "Invalid replace instruction in replacement string. Id: " + toString(it.first) + ", but regexp has only " - + toString(num_captures - 1) + " subpatterns", - ErrorCodes::BAD_ARGUMENTS); + if (it.substitution_num >= num_captures) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Invalid replace instruction in replacement string. Id: {}, but regexp has only {} subpatterns", + it.substitution_num, num_captures - 1); return instructions; } @@ -93,42 +100,51 @@ struct ReplaceRegexpImpl { re2_st::StringPiece matches[max_captures]; - size_t start_pos = 0; - while (start_pos < static_cast(input.length())) + size_t copy_pos = 0; + size_t match_pos = 0; + + while (match_pos < static_cast(input.length())) { /// If no more replacements possible for current string bool can_finish_current_string = false; - if (searcher.Match(input, start_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) + if (searcher.Match(input, match_pos, input.length(), re2_st::RE2::Anchor::UNANCHORED, matches, num_captures)) { const auto & match = matches[0]; - size_t bytes_to_copy = (match.data() - input.data()) - start_pos; + size_t bytes_to_copy = (match.data() - input.data()) - copy_pos; /// Copy prefix before matched regexp without modification res_data.resize(res_data.size() + bytes_to_copy); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, bytes_to_copy); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + copy_pos, bytes_to_copy); res_offset += bytes_to_copy; - start_pos += bytes_to_copy + match.length(); + copy_pos += bytes_to_copy + match.length(); + match_pos = copy_pos; /// Do substitution instructions for (const auto & it : instructions) { - if (it.first >= 0) + if (it.substitution_num >= 0) { - res_data.resize(res_data.size() + matches[it.first].length()); - memcpy(&res_data[res_offset], matches[it.first].data(), matches[it.first].length()); - res_offset += matches[it.first].length(); + const auto & substitution = matches[it.substitution_num]; + + res_data.resize(res_data.size() + substitution.length()); + memcpy(&res_data[res_offset], substitution.data(), substitution.length()); + res_offset += substitution.length(); } else { - res_data.resize(res_data.size() + it.second.size()); - memcpy(&res_data[res_offset], it.second.data(), it.second.size()); - res_offset += it.second.size(); + const auto & literal = it.literal; + + res_data.resize(res_data.size() + literal.size()); + memcpy(&res_data[res_offset], literal.data(), literal.size()); + res_offset += literal.size(); } } - if (replace_one || match.length() == 0) /// Stop after match of zero length, to avoid infinite loop. + if (replace_one) can_finish_current_string = true; + else if (match.length() == 0) + ++match_pos; /// Step one character to avoid infinite loop. } else can_finish_current_string = true; @@ -136,10 +152,11 @@ struct ReplaceRegexpImpl /// If ready, append suffix after match to end of string. if (can_finish_current_string) { - res_data.resize(res_data.size() + input.length() - start_pos); - memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + start_pos, input.length() - start_pos); - res_offset += input.length() - start_pos; - start_pos = input.length(); + res_data.resize(res_data.size() + input.length() - copy_pos); + memcpySmallAllowReadWriteOverflow15(&res_data[res_offset], input.data() + copy_pos, input.length() - copy_pos); + res_offset += input.length() - copy_pos; + copy_pos = input.length(); + match_pos = copy_pos; } } diff --git a/tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.reference b/tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.sql b/tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.sql new file mode 100644 index 00000000000..6725fa04114 --- /dev/null +++ b/tests/queries/0_stateless/02151_replace_regexp_all_empty_match_alternative.sql @@ -0,0 +1,3 @@ +select replaceRegexpAll(',,1,,', '^[,]*|[,]*$', '') x; +select replaceRegexpAll(',,1', '^[,]*|[,]*$', '') x; +select replaceRegexpAll('1,,', '^[,]*|[,]*$', '') x; From c61885488d1a52348eb03b2aae7e08203d226ff1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 19 Dec 2021 12:46:19 +0300 Subject: [PATCH 317/358] Return more tests --- .../02100_replaceRegexpAll_bug.reference | 11 +++++++++++ .../0_stateless/02100_replaceRegexpAll_bug.sql | 14 ++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference create mode 100644 tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference new file mode 100644 index 00000000000..993dd9b1cde --- /dev/null +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.reference @@ -0,0 +1,11 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql new file mode 100644 index 00000000000..32f7f63f6d0 --- /dev/null +++ b/tests/queries/0_stateless/02100_replaceRegexpAll_bug.sql @@ -0,0 +1,14 @@ +SELECT 'aaaabb ' == trim(leading 'b ' FROM 'b aaaabb ') x; +SELECT 'b aaaa' == trim(trailing 'b ' FROM 'b aaaabb ') x; +SELECT 'aaaa' == trim(both 'b ' FROM 'b aaaabb ') x; + +SELECT '1' == replaceRegexpAll(',,1,,', '^[,]*|[,]*$', '') x; +SELECT '1' == replaceRegexpAll(',,1', '^[,]*|[,]*$', '') x; +SELECT '1' == replaceRegexpAll('1,,', '^[,]*|[,]*$', '') x; + +SELECT '1,,' == replaceRegexpOne(',,1,,', '^[,]*|[,]*$', '') x; +SELECT '1' == replaceRegexpOne(',,1', '^[,]*|[,]*$', '') x; +SELECT '1,,' == replaceRegexpOne('1,,', '^[,]*|[,]*$', '') x; + +SELECT '5935,5998,6014' == trim(BOTH ', ' FROM '5935,5998,6014, ') x; +SELECT '5935,5998,6014' == replaceRegexpAll('5935,5998,6014, ', concat('^[', regexpQuoteMeta(', '), ']*|[', regexpQuoteMeta(', '), ']*$'), '') AS x; From 13d1377ad76993e894542c24bdcfa3da32295a94 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 19 Dec 2021 12:59:28 +0300 Subject: [PATCH 318/358] Update src/Functions/ReplaceRegexpImpl.h Co-authored-by: Vladimir C --- src/Functions/ReplaceRegexpImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index 56b7dd3fd7e..e6305431d8f 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -34,7 +34,7 @@ struct ReplaceRegexpImpl /// Sequence of instructions, describing how to get resulting string. struct Instruction { - /// If non zero - perform substitution of n-th subpattern from the regexp match. + /// If not negative - perform substitution of n-th subpattern from the regexp match. int substitution_num = -1; /// Otherwise - paste this string verbatim. std::string literal; From 708439b0364b46e89762f408e09df3e6101d0faa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 16 Dec 2021 00:06:17 +0300 Subject: [PATCH 319/358] Support customized compression for input/output data in gRPC protocol. --- docker/test/integration/runner/Dockerfile | 1 + src/Server/GRPCServer.cpp | 109 +++++++++++++++---- src/Server/grpc_protos/clickhouse_grpc.proto | 23 ++++ tests/integration/test_grpc_protocol/test.py | 66 +++++++++++ 4 files changed, 179 insertions(+), 20 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index a953a8a904a..eee974f2cc2 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -72,6 +72,7 @@ RUN python3 -m pip install \ grpcio-tools \ kafka-python \ kazoo \ + lz4 \ minio \ protobuf \ psycopg2-binary==2.8.6 \ diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 096194455b1..589bdd63f41 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -618,7 +619,11 @@ namespace ASTInsertQuery * insert_query = nullptr; String input_format; String input_data_delimiter; + PODArray output; String output_format; + CompressionMethod compression_method = CompressionMethod::None; + int compression_level = 0; + uint64_t interactive_delay = 100000; bool send_exception_with_stacktrace = true; bool input_function_is_used = false; @@ -635,8 +640,10 @@ namespace bool responder_finished = false; bool cancelled = false; - std::optional read_buffer; - std::optional write_buffer; + std::unique_ptr read_buffer; + std::unique_ptr write_buffer; + WriteBufferFromVector> * nested_write_buffer = nullptr; + WriteBuffer * compressing_write_buffer = nullptr; std::unique_ptr pipeline; std::unique_ptr pipeline_executor; std::shared_ptr output_format_processor; @@ -818,6 +825,10 @@ namespace if (output_format.empty()) output_format = query_context->getDefaultFormat(); + /// Choose compression. + compression_method = chooseCompressionMethod("", query_info.compression_type()); + compression_level = query_info.compression_level(); + /// Set callback to create and fill external tables query_context->setExternalTablesInitializer([this] (ContextPtr context) { @@ -891,7 +902,7 @@ namespace void Call::initializeBlockInputStream(const Block & header) { assert(!read_buffer); - read_buffer.emplace([this]() -> std::pair + read_buffer = std::make_unique([this]() -> std::pair { if (need_input_data_from_insert_query) { @@ -947,6 +958,8 @@ namespace return {nullptr, 0}; /// no more input data }); + read_buffer = wrapReadBufferWithCompressionMethod(std::move(read_buffer), compression_method); + assert(!pipeline); auto source = query_context->getInputFormat( input_format, *read_buffer, header, query_context->getSettings().max_insert_block_size); @@ -1030,7 +1043,10 @@ namespace /// The data will be written directly to the table. auto metadata_snapshot = storage->getInMemoryMetadataPtr(); auto sink = storage->write(ASTPtr(), metadata_snapshot, query_context); - ReadBufferFromMemory data(external_table.data().data(), external_table.data().size()); + + std::unique_ptr buf = std::make_unique(external_table.data().data(), external_table.data().size()); + buf = wrapReadBufferWithCompressionMethod(std::move(buf), chooseCompressionMethod("", external_table.compression_type())); + String format = external_table.format(); if (format.empty()) format = "TabSeparated"; @@ -1047,7 +1063,7 @@ namespace external_table_context->applySettingsChanges(settings_changes); } auto in = external_table_context->getInputFormat( - format, data, metadata_snapshot->getSampleBlock(), + format, *buf, metadata_snapshot->getSampleBlock(), external_table_context->getSettings().max_insert_block_size); QueryPipelineBuilder cur_pipeline; @@ -1101,7 +1117,18 @@ namespace if (io.pipeline.pulling()) header = io.pipeline.getHeader(); - write_buffer.emplace(*result.mutable_output()); + if (compression_method != CompressionMethod::None) + output.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data. + write_buffer = std::make_unique>>(output); + nested_write_buffer = static_cast> *>(write_buffer.get()); + if (compression_method != CompressionMethod::None) + { + write_buffer = wrapWriteBufferWithCompressionMethod(std::move(write_buffer), compression_method, compression_level); + compressing_write_buffer = write_buffer.get(); + } + + auto has_output = [&] { return (nested_write_buffer->position() != output.data()) || (compressing_write_buffer && compressing_write_buffer->offset()); }; + output_format_processor = query_context->getOutputFormat(output_format, *write_buffer, header); Stopwatch after_send_progress; @@ -1143,8 +1170,7 @@ namespace addLogsToResult(); - bool has_output = write_buffer->offset(); - if (has_output || result.has_progress() || result.logs_size()) + if (has_output() || result.has_progress() || result.logs_size()) sendResult(); throwIfFailedToSendResult(); @@ -1164,13 +1190,11 @@ namespace auto executor = std::make_shared(io.pipeline); auto callback = [&]() -> bool { - throwIfFailedToSendResult(); addProgressToResult(); addLogsToResult(); - bool has_output = write_buffer->offset(); - if (has_output || result.has_progress() || result.logs_size()) + if (has_output() || result.has_progress() || result.logs_size()) sendResult(); throwIfFailedToSendResult(); @@ -1260,6 +1284,8 @@ namespace /// immediately after it receives our final result, and it's prohibited to have /// two queries executed at the same time with the same query ID or session ID. io.process_list_entry.reset(); + if (query_context) + query_context->setProcessListElement(nullptr); if (session) session->releaseSessionID(); } @@ -1272,6 +1298,8 @@ namespace output_format_processor.reset(); read_buffer.reset(); write_buffer.reset(); + nested_write_buffer = nullptr; + compressing_write_buffer = nullptr; io = {}; query_scope.reset(); query_context.reset(); @@ -1390,10 +1418,17 @@ namespace if (!totals) return; - WriteBufferFromString buf{*result.mutable_totals()}; - auto format = query_context->getOutputFormat(output_format, buf, totals); + PODArray memory; + if (compression_method != CompressionMethod::None) + memory.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data. + std::unique_ptr buf = std::make_unique>>(memory); + buf = wrapWriteBufferWithCompressionMethod(std::move(buf), compression_method, compression_level); + auto format = query_context->getOutputFormat(output_format, *buf, totals); format->write(materializeBlock(totals)); format->finalize(); + buf->finalize(); + + result.mutable_totals()->assign(memory.data(), memory.size()); } void Call::addExtremesToResult(const Block & extremes) @@ -1401,10 +1436,17 @@ namespace if (!extremes) return; - WriteBufferFromString buf{*result.mutable_extremes()}; - auto format = query_context->getOutputFormat(output_format, buf, extremes); + PODArray memory; + if (compression_method != CompressionMethod::None) + memory.resize(DBMS_DEFAULT_BUFFER_SIZE); /// Must have enough space for compressed data. + std::unique_ptr buf = std::make_unique>>(memory); + buf = wrapWriteBufferWithCompressionMethod(std::move(buf), compression_method, compression_level); + auto format = query_context->getOutputFormat(output_format, *buf, extremes); format->write(materializeBlock(extremes)); format->finalize(); + buf->finalize(); + + result.mutable_extremes()->assign(memory.data(), memory.size()); } void Call::addProfileInfoToResult(const ProfileInfo & info) @@ -1475,6 +1517,38 @@ namespace if (!send_final_message && !isOutputStreaming(call_type)) return; + /// Copy output to `result.output`, with optional compressing. + if (write_buffer) + { + size_t output_size; + if (send_final_message) + { + if (compressing_write_buffer) + LOG_DEBUG(log, "Compressing final {} bytes", compressing_write_buffer->offset()); + write_buffer->finalize(); + output_size = output.size(); + } + else + { + if (compressing_write_buffer && compressing_write_buffer->offset()) + { + LOG_DEBUG(log, "Compressing {} bytes", compressing_write_buffer->offset()); + compressing_write_buffer->sync(); + } + output_size = nested_write_buffer->position() - output.data(); + } + + if (output_size) + { + result.mutable_output()->assign(output.data(), output_size); + nested_write_buffer->restart(); /// We're going to reuse the same buffer again for next block of data. + } + } + + if (!send_final_message && result.output().empty() && result.totals().empty() && result.extremes().empty() && !result.logs_size() + && !result.has_progress() && !result.has_stats() && !result.has_exception() && !result.cancelled()) + return; /// Nothing to send. + /// Wait for previous write to finish. /// (gRPC doesn't allow to start sending another result while the previous is still being sending.) if (sending_result.get()) @@ -1488,9 +1562,6 @@ namespace /// Start sending the result. LOG_DEBUG(log, "Sending {} result to the client: {}", (send_final_message ? "final" : "intermediate"), getResultDescription(result)); - if (write_buffer) - write_buffer->finalize(); - sending_result.set(true); auto callback = [this](bool ok) { @@ -1511,8 +1582,6 @@ namespace /// gRPC has already retrieved all data from `result`, so we don't have to keep it. result.Clear(); - if (write_buffer) - write_buffer->restart(); if (send_final_message) { diff --git a/src/Server/grpc_protos/clickhouse_grpc.proto b/src/Server/grpc_protos/clickhouse_grpc.proto index c6cafaf6e40..c86c74535c5 100644 --- a/src/Server/grpc_protos/clickhouse_grpc.proto +++ b/src/Server/grpc_protos/clickhouse_grpc.proto @@ -37,6 +37,10 @@ message ExternalTable { // Format of the data to insert to the external table. string format = 4; + // Compression type used to compress `data`. + // Supported values: none, gzip(gz), deflate, brotli(br), lzma(xz), zstd(zst), lz4, bz2. + string compression_type = 6; + // Settings for executing that insertion, applied after QueryInfo.settings. map settings = 5; } @@ -101,6 +105,25 @@ message QueryInfo { /// Controls how a ClickHouse server will compress query execution results before sending back to the client. /// If not set the compression settings from the configuration file will be used. Compression result_compression = 17; + + // Compression type for `input_data`, `output_data`, `totals` and `extremes`. + // Supported compression types: none, gzip(gz), deflate, brotli(br), lzma(xz), zstd(zst), lz4, bz2. + // When used for `input_data` the client is responsible to compress data before putting it into `input_data`. + // When used for `output_data` or `totals` or `extremes` the client receives compressed data and should decompress it by itself. + // In the latter case consider to specify also `compression_level`. + string compression_type = 18; + + // Compression level. + // WARNING: If it's not specified the compression level is set to zero by default which might be not the best choice for some compression types (see below). + // The compression level should be in the following range (the higher the number, the better the compression): + // none: compression level isn't used + // gzip: 0..9; 0 means no compression, 6 is recommended by default (compression level -1 also means 6) + // brotli: 0..11 + // lzma: 0..9; 6 is recommended by default + // zstd: 1..22; 3 is recommended by default (compression level 0 also means 3) + // lz4: 0..16; values < 0 mean fast acceleration + // bz2: 1..9 + int32 compression_level = 19; } enum LogsLevel { diff --git a/tests/integration/test_grpc_protocol/test.py b/tests/integration/test_grpc_protocol/test.py index c892fc94712..e17ed0d9c8e 100644 --- a/tests/integration/test_grpc_protocol/test.py +++ b/tests/integration/test_grpc_protocol/test.py @@ -5,6 +5,8 @@ import time import grpc from helpers.cluster import ClickHouseCluster, run_and_check from threading import Thread +import gzip +import lz4.frame GRPC_PORT = 9100 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -365,3 +367,67 @@ def test_result_compression(): stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) result = stub.ExecuteQuery(query_info) assert result.output == (b'0\n')*1000000 + +def test_compressed_output(): + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", compression_type="lz4") + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) + result = stub.ExecuteQuery(query_info) + assert lz4.frame.decompress(result.output) == (b'0\n')*1000 + +def test_compressed_output_streaming(): + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(100000)", compression_type="lz4") + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) + d_context = lz4.frame.create_decompression_context() + data = b'' + for result in stub.ExecuteQueryWithStreamOutput(query_info): + d1, _, _ = lz4.frame.decompress_chunk(d_context, result.output) + data += d1 + assert data == (b'0\n')*100000 + +def test_compressed_output_gzip(): + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT 0 FROM numbers(1000)", compression_type="gzip", compression_level=6) + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) + result = stub.ExecuteQuery(query_info) + assert gzip.decompress(result.output) == (b'0\n')*1000 + +def test_compressed_totals_and_extremes(): + query("CREATE TABLE t (x UInt8, y UInt8) ENGINE = Memory") + query("INSERT INTO t VALUES (1, 2), (2, 4), (3, 2), (3, 3), (3, 4)") + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT sum(x), y FROM t GROUP BY y WITH TOTALS", compression_type="lz4") + result = stub.ExecuteQuery(query_info) + assert lz4.frame.decompress(result.totals) == b'12\t0\n' + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT x, y FROM t", settings={"extremes": "1"}, compression_type="lz4") + result = stub.ExecuteQuery(query_info) + assert lz4.frame.decompress(result.extremes) == b'1\t2\n3\t4\n' + +def test_compressed_insert_query_streaming(): + query("CREATE TABLE t (a UInt8) ENGINE = Memory") + data = lz4.frame.compress(b'(1),(2),(3),(5),(4),(6),(7),(8),(9)') + sz1 = len(data) // 3 + sz2 = len(data) // 3 + d1 = data[:sz1] + d2 = data[sz1:sz1+sz2] + d3 = data[sz1+sz2:] + def send_query_info(): + yield clickhouse_grpc_pb2.QueryInfo(query="INSERT INTO t VALUES", input_data=d1, compression_type="lz4", next_query_info=True) + yield clickhouse_grpc_pb2.QueryInfo(input_data=d2, next_query_info=True) + yield clickhouse_grpc_pb2.QueryInfo(input_data=d3) + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) + stub.ExecuteQueryWithStreamInput(send_query_info()) + assert query("SELECT a FROM t ORDER BY a") == "1\n2\n3\n4\n5\n6\n7\n8\n9\n" + +def test_compressed_external_table(): + columns = [clickhouse_grpc_pb2.NameAndType(name='UserID', type='UInt64'), clickhouse_grpc_pb2.NameAndType(name='UserName', type='String')] + d1 = lz4.frame.compress(b'1\tAlex\n2\tBen\n3\tCarl\n') + d2 = gzip.compress(b'4,Daniel\n5,Ethan\n') + ext1 = clickhouse_grpc_pb2.ExternalTable(name='ext1', columns=columns, data=d1, format='TabSeparated', compression_type="lz4") + ext2 = clickhouse_grpc_pb2.ExternalTable(name='ext2', columns=columns, data=d2, format='CSV', compression_type="gzip") + stub = clickhouse_grpc_pb2_grpc.ClickHouseStub(main_channel) + query_info = clickhouse_grpc_pb2.QueryInfo(query="SELECT * FROM (SELECT * FROM ext1 UNION ALL SELECT * FROM ext2) ORDER BY UserID", external_tables=[ext1, ext2]) + result = stub.ExecuteQuery(query_info) + assert result.output == b"1\tAlex\n"\ + b"2\tBen\n"\ + b"3\tCarl\n"\ + b"4\tDaniel\n"\ + b"5\tEthan\n" From 02d6cea857cad223a15cb8de19173b56e54d26d6 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 16 Dec 2021 00:06:41 +0300 Subject: [PATCH 320/358] Lz4DeflatingWriteBuffer now supports changes in the nested buffer between calls. --- src/IO/Lz4DeflatingWriteBuffer.cpp | 40 ++++++++++++++++++++---------- 1 file changed, 27 insertions(+), 13 deletions(-) diff --git a/src/IO/Lz4DeflatingWriteBuffer.cpp b/src/IO/Lz4DeflatingWriteBuffer.cpp index 5d9c5d40e6f..da954b13df9 100644 --- a/src/IO/Lz4DeflatingWriteBuffer.cpp +++ b/src/IO/Lz4DeflatingWriteBuffer.cpp @@ -54,14 +54,19 @@ void Lz4DeflatingWriteBuffer::nextImpl() in_data = reinterpret_cast(working_buffer.begin()); in_capacity = offset(); + out_capacity = out->buffer().end() - out->position(); + out_data = reinterpret_cast(out->position()); + try { if (first_time) { - out->nextIfAtEnd(); - - out_data = reinterpret_cast(out->position()); - out_capacity = out->buffer().end() - out->position(); + if (out_capacity < LZ4F_HEADER_SIZE_MAX) + { + out->next(); + out_capacity = out->buffer().end() - out->position(); + out_data = reinterpret_cast(out->position()); + } /// write frame header and check for errors size_t header_size = LZ4F_compressBegin(ctx, out_data, out_capacity, &kPrefs); @@ -74,24 +79,29 @@ void Lz4DeflatingWriteBuffer::nextImpl() out_capacity -= header_size; out->position() = out->buffer().end() - out_capacity; + out_data = reinterpret_cast(out->position()); + first_time = false; } do { /// Ensure that there is enough space for compressed block of minimal size - if (out_capacity < LZ4F_compressBound(0, &kPrefs)) + size_t min_compressed_block_size = LZ4F_compressBound(1, &kPrefs); + if (out_capacity < min_compressed_block_size) { out->next(); out_capacity = out->buffer().end() - out->position(); + out_data = reinterpret_cast(out->position()); } - out_data = reinterpret_cast(out->position()); - /// LZ4F_compressUpdate compresses whole input buffer at once so we need to shink it manually size_t cur_buffer_size = in_capacity; - while (out_capacity < LZ4F_compressBound(cur_buffer_size, &kPrefs)) - cur_buffer_size /= 2; + if (out_capacity >= min_compressed_block_size) /// We cannot shrink the input buffer if it's already too small. + { + while (out_capacity < LZ4F_compressBound(cur_buffer_size, &kPrefs)) + cur_buffer_size /= 2; + } size_t compressed_size = LZ4F_compressUpdate(ctx, out_data, out_capacity, in_data, cur_buffer_size, nullptr); @@ -101,11 +111,12 @@ void Lz4DeflatingWriteBuffer::nextImpl() "LZ4 failed to encode stream. LZ4F version: {}", LZ4F_VERSION); - out_capacity -= compressed_size; in_capacity -= cur_buffer_size; - in_data = reinterpret_cast(working_buffer.end() - in_capacity); + + out_capacity -= compressed_size; out->position() = out->buffer().end() - out_capacity; + out_data = reinterpret_cast(out->position()); } while (in_capacity > 0); } @@ -120,14 +131,16 @@ void Lz4DeflatingWriteBuffer::finalizeBefore() { next(); + out_capacity = out->buffer().end() - out->position(); + out_data = reinterpret_cast(out->position()); + if (out_capacity < LZ4F_compressBound(0, &kPrefs)) { out->next(); out_capacity = out->buffer().end() - out->position(); + out_data = reinterpret_cast(out->position()); } - out_data = reinterpret_cast(out->position()); - /// compression end size_t end_size = LZ4F_compressEnd(ctx, out_data, out_capacity, nullptr); @@ -139,6 +152,7 @@ void Lz4DeflatingWriteBuffer::finalizeBefore() out_capacity -= end_size; out->position() = out->buffer().end() - out_capacity; + out_data = reinterpret_cast(out->position()); } void Lz4DeflatingWriteBuffer::finalizeAfter() From ffda16b5a405220cafdf99de6a1d96993bb9cf19 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 19 Dec 2021 21:34:24 +0800 Subject: [PATCH 321/358] Better build of azure blob --- cmake/find/blob_storage.cmake | 6 ++++-- contrib/azure-cmake/CMakeLists.txt | 9 ++++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/cmake/find/blob_storage.cmake b/cmake/find/blob_storage.cmake index 4df25abb1ab..74a907da7db 100644 --- a/cmake/find/blob_storage.cmake +++ b/cmake/find/blob_storage.cmake @@ -1,8 +1,10 @@ +option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) + option(USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY "Set to FALSE to use system Azure SDK instead of bundled (OFF currently not implemented)" - ${ENABLE_LIBRARIES}) + ON) -if (USE_INTERNAL_AZURE_BLOB_STORAGE_LIBRARY) +if (ENABLE_AZURE_BLOB_STORAGE) set(USE_AZURE_BLOB_STORAGE 1) set(AZURE_BLOB_STORAGE_LIBRARY azure_sdk) endif() diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 7f9476e37b7..527503b85a2 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -46,14 +46,17 @@ include("${AZURE_DIR}/cmake-modules/AzureTransportAdapters.cmake") add_library(azure_sdk ${AZURE_SDK_UNIFIED_SRC}) if (COMPILER_CLANG) - target_compile_options(azure_sdk PUBLIC + target_compile_options(azure_sdk PRIVATE -Wno-deprecated-copy-dtor -Wno-extra-semi -Wno-suggest-destructor-override -Wno-inconsistent-missing-destructor-override -Wno-error=unknown-warning-option - -Wno-reserved-identifier ) + + if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 13) + target_compile_options(azure_sdk PRIVATE -Wno-reserved-identifier) + endif() endif() # Originally, on Windows azure-core is built with bcrypt and crypt32 by default @@ -68,4 +71,4 @@ endif() target_link_libraries(azure_sdk PRIVATE ${LIBXML2_LIBRARIES}) -target_include_directories(azure_sdk PUBLIC ${AZURE_SDK_INCLUDES}) +target_include_directories(azure_sdk SYSTEM PUBLIC ${AZURE_SDK_INCLUDES}) From f06c37d20645a630f9f9829584c484169fbaa353 Mon Sep 17 00:00:00 2001 From: kreuzerkrieg Date: Sat, 18 Dec 2021 11:25:25 +0200 Subject: [PATCH 322/358] Stop reading incomplete stripes and skip rows. --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/ORCBlockInputFormat.cpp | 45 ++++++------------- .../Formats/Impl/ORCBlockInputFormat.h | 6 --- 5 files changed, 16 insertions(+), 38 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 00ab0b73807..1332d844ff3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -593,6 +593,7 @@ class IColumn; M(Bool, input_format_null_as_default, true, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ M(Bool, input_format_arrow_import_nested, false, "Allow to insert array of structs into Nested table in Arrow input format.", 0) \ M(Bool, input_format_orc_import_nested, false, "Allow to insert array of structs into Nested table in ORC input format.", 0) \ + M(Int64, input_format_orc_row_batch_size, 100'000, "Batch size when reading ORC stripes.", 0) \ M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \ M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \ \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 75b096de425..09e0876bb4f 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -114,6 +114,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.arrow.low_cardinality_as_dictionary = settings.output_format_arrow_low_cardinality_as_dictionary; format_settings.arrow.import_nested = settings.input_format_arrow_import_nested; format_settings.orc.import_nested = settings.input_format_orc_import_nested; + format_settings.orc.row_batch_size = settings.input_format_orc_row_batch_size; format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields; format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode; format_settings.seekable_read = settings.input_format_allow_seeks; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index a18a20bac7b..909b173007a 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -200,6 +200,7 @@ struct FormatSettings struct { bool import_nested = false; + int64_t row_batch_size = 100'000; } orc; /// For capnProto format we should determine how to diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 8768e2f5f14..c645595919e 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include "ArrowBufferedStreams.h" #include "ArrowColumnToCHColumn.h" #include @@ -38,37 +37,22 @@ Chunk ORCBlockInputFormat::generate() if (!file_reader) prepareReader(); + std::shared_ptr batch_reader; + arrow::Status reader_status = file_reader->NextStripeReader(format_settings.orc.row_batch_size, include_indices, &batch_reader); + if (!reader_status.ok()) + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", reader_status.ToString()); if (!batch_reader) - { - arrow::Status reader_status = file_reader->NextStripeReader( - DBMS_DEFAULT_BUFFER_SIZE, include_indices, &batch_reader); - if (!reader_status.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, - "Failed to create batch reader: {}", - reader_status.ToString()); - if (!batch_reader) - return res; - } - - std::shared_ptr batch_result; - arrow::Status batch_status = batch_reader->ReadNext(&batch_result); - if (!batch_status.ok()) - throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, - "Error while reading batch of ORC data: {}", - batch_status.ToString()); - - if (!batch_result || !batch_result->num_rows()) return res; - ArrowColumnToCHColumn::NameToColumnPtr name_to_column_ptr; - for (const auto & column_name : column_names) - { - arrow::ArrayVector vec = {batch_result->GetColumnByName(column_name)}; - std::shared_ptr arrow_column = std::make_shared(vec); - name_to_column_ptr[column_name] = arrow_column; - } - arrow_column_to_ch_column->arrowColumnsToCHChunk(res, name_to_column_ptr); - batch_reader.reset(); + std::shared_ptr table; + arrow::Status table_status = batch_reader->ReadAll(&table); + if (!table_status.ok()) + throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_status.ToString()); + + if (!table || !table->num_rows()) + return res; + + arrow_column_to_ch_column->arrowTableToCHChunk(res, table); return res; } @@ -79,7 +63,6 @@ void ORCBlockInputFormat::resetParser() file_reader.reset(); include_indices.clear(); - stripe_current = 0; } static size_t countIndicesForType(std::shared_ptr type) @@ -108,8 +91,6 @@ static size_t countIndicesForType(std::shared_ptr type) void ORCBlockInputFormat::prepareReader() { THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(*in, format_settings), arrow::default_memory_pool(), &file_reader)); - stripe_total = file_reader->NumberOfStripes(); - stripe_current = 0; std::shared_ptr schema; THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 857ec7937b7..639aaee73bb 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -35,16 +35,10 @@ private: std::unique_ptr file_reader; - std::shared_ptr batch_reader; - std::unique_ptr arrow_column_to_ch_column; std::vector column_names; - int stripe_total = 0; - - int stripe_current = 0; - // indices of columns to read from ORC file std::vector include_indices; From 77c2180394efdefc6b7997bf5f28fcb9a7632249 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 19 Dec 2021 16:04:39 -0400 Subject: [PATCH 323/358] test for _base64encode_trailing_bytes #31797 --- ...13_base64encode_trailing_bytes_1.reference | 100 ++++++++++++++++++ .../02113_base64encode_trailing_bytes_1.sql | 4 + 2 files changed, 104 insertions(+) create mode 100644 tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.reference create mode 100644 tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql diff --git a/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.reference b/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.reference new file mode 100644 index 00000000000..8d9df2da010 --- /dev/null +++ b/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.reference @@ -0,0 +1,100 @@ +0 +1 61 +2 6161 +3 616161 +4 61616161 +5 6161616161 +6 616161616161 +7 61616161616161 +8 6161616161616161 +9 616161616161616161 +10 61616161616161616161 +11 6161616161616161616161 +12 616161616161616161616161 +13 61616161616161616161616161 +14 6161616161616161616161616161 +15 616161616161616161616161616161 +16 61616161616161616161616161616161 +17 6161616161616161616161616161616161 +18 616161616161616161616161616161616161 +19 61616161616161616161616161616161616161 +20 6161616161616161616161616161616161616161 +21 616161616161616161616161616161616161616161 +22 61616161616161616161616161616161616161616161 +23 6161616161616161616161616161616161616161616161 +24 616161616161616161616161616161616161616161616161 +25 61616161616161616161616161616161616161616161616161 +26 6161616161616161616161616161616161616161616161616161 +27 616161616161616161616161616161616161616161616161616161 +28 61616161616161616161616161616161616161616161616161616161 +29 6161616161616161616161616161616161616161616161616161616161 +30 616161616161616161616161616161616161616161616161616161616161 +31 61616161616161616161616161616161616161616161616161616161616161 +32 6161616161616161616161616161616161616161616161616161616161616161 +33 616161616161616161616161616161616161616161616161616161616161616161 +34 61616161616161616161616161616161616161616161616161616161616161616161 +35 6161616161616161616161616161616161616161616161616161616161616161616161 +36 616161616161616161616161616161616161616161616161616161616161616161616161 +37 61616161616161616161616161616161616161616161616161616161616161616161616161 +38 6161616161616161616161616161616161616161616161616161616161616161616161616161 +39 616161616161616161616161616161616161616161616161616161616161616161616161616161 +40 61616161616161616161616161616161616161616161616161616161616161616161616161616161 +41 6161616161616161616161616161616161616161616161616161616161616161616161616161616161 +42 616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +43 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +44 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +45 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +46 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +47 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +48 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +49 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +50 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +51 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +52 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +53 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +54 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +55 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +56 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +57 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +58 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +59 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +60 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +61 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +62 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +63 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +64 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +65 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +66 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +67 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +68 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +69 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +70 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +71 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +72 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +73 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +74 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +75 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +76 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +77 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +78 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +79 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +80 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +81 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +82 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +83 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +84 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +85 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +86 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +87 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +88 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +89 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +90 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +91 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +92 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +93 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +94 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +95 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +96 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +97 61616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +98 6161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 +99 616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161616161 diff --git a/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql b/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql new file mode 100644 index 00000000000..7166c8df2f3 --- /dev/null +++ b/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql @@ -0,0 +1,4 @@ +SELECT + number, + hex(base64Decode(base64Encode(repeat('a', number)))) r +FROM numbers(100); From a34c3511af06122e230eb9a77b716a33da81bc77 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Dec 2021 10:25:30 +0300 Subject: [PATCH 324/358] tests: fix 02050_client_profile_events flakiness CI: https://s3.amazonaws.com/clickhouse-test-reports/32303/24751e7d45d94541be854c86ce46d65c2e0f66da/stateless_tests__thread__actions_.html --- tests/queries/0_stateless/02050_client_profile_events.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index 5c3887cf5fb..fb54ebfb7ff 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -10,6 +9,8 @@ $CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'Selec # print only last $CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5) format Null' |& grep -o 'SelectedRows: .*$' # print everything -test "$($CLICKHOUSE_CLIENT --print-profile-events -q 'select * from numbers(1e9) format Null' |& grep -c 'SelectedRows')" -gt 1 && echo OK || echo FAIL +profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" +test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" # print each 100 ms -test "$($CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=100 -q 'select * from numbers(1e9) format Null' |& grep -c 'SelectedRows')" -gt 1 && echo OK || echo FAIL +profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events --profile-events-delay-ms=100 -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" +test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" From b7f18e23338b2645995bd81a6dd1c45f1382129d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Dec 2021 10:25:30 +0300 Subject: [PATCH 325/358] tests: add more information for 02050_client_profile_events in case of failure Sometimes [1] the test fails like this: 2021-12-10 03:41:19 --- /usr/share/clickhouse-test/queries/0_stateless/02050_client_profile_events.reference 2021-12-10 03:23:43.000000000 -0500 2021-12-10 03:41:19 +++ /tmp/clickhouse-test/0_stateless/02050_client_profile_events.617.stdout 2021-12-10 03:41:19.509611205 -0500 2021-12-10 03:41:19 @@ -1,4 +1,3 @@ 2021-12-10 03:41:19 0 2021-12-10 03:41:19 -SelectedRows: 131010 (increment) 2021-12-10 03:41:19 OK 2021-12-10 03:41:19 OK 2021-12-10 03:41:19 2021-12-10 03:41:19 2021-12-10 03:41:19 Database: test_73d5o0 [1]: https://s3.amazonaws.com/clickhouse-test-reports/32493/703213a6444f8014e3324df4b6e44d03fa351294/stateless_tests_flaky_check__address__actions_.html And I did not find anything strange in server logs (and there was 0 exceptions). --- .../queries/0_stateless/02050_client_profile_events.reference | 1 + tests/queries/0_stateless/02050_client_profile_events.sh | 4 ++-- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index 00fc3b5d06a..29d68082647 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -1,4 +1,5 @@ 0 +100000 SelectedRows: 131010 (increment) OK OK diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index fb54ebfb7ff..f91396d35cd 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -6,8 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # do not print any ProfileEvents packets $CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'SelectedRows' -# print only last -$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5) format Null' |& grep -o 'SelectedRows: .*$' +# print only last (and also number of rows to provide more info in case of failures) +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e 'SelectedRows: .*$' -e Exception) 1> >(wc -l) # print everything profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" From 2461cc5f9319831e7edfa098d53ccbba57eebde6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Dec 2021 10:25:30 +0300 Subject: [PATCH 326/358] tests: add thread_id into 02050_client_profile_events --- tests/queries/0_stateless/02050_client_profile_events.reference | 2 +- tests/queries/0_stateless/02050_client_profile_events.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02050_client_profile_events.reference b/tests/queries/0_stateless/02050_client_profile_events.reference index 29d68082647..2451417ddf0 100644 --- a/tests/queries/0_stateless/02050_client_profile_events.reference +++ b/tests/queries/0_stateless/02050_client_profile_events.reference @@ -1,5 +1,5 @@ 0 100000 -SelectedRows: 131010 (increment) +[ 0 ] SelectedRows: 131010 (increment) OK OK diff --git a/tests/queries/0_stateless/02050_client_profile_events.sh b/tests/queries/0_stateless/02050_client_profile_events.sh index f91396d35cd..459e8505e22 100755 --- a/tests/queries/0_stateless/02050_client_profile_events.sh +++ b/tests/queries/0_stateless/02050_client_profile_events.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # do not print any ProfileEvents packets $CLICKHOUSE_CLIENT -q 'select * from numbers(1e5) format Null' |& grep -c 'SelectedRows' # print only last (and also number of rows to provide more info in case of failures) -$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e 'SelectedRows: .*$' -e Exception) 1> >(wc -l) +$CLICKHOUSE_CLIENT --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers(1e5)' 2> >(grep -o -e '\[ 0 \] SelectedRows: .*$' -e Exception) 1> >(wc -l) # print everything profile_events="$($CLICKHOUSE_CLIENT --max_block_size 1 --print-profile-events -q 'select sleep(1) from numbers(2) format Null' |& grep -c 'SelectedRows')" test "$profile_events" -gt 1 && echo OK || echo "FAIL ($profile_events)" From 909ce68b5a195616c527d5d3e1ccdfe147e8f107 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Dec 2021 10:25:30 +0300 Subject: [PATCH 327/358] Do not suppress exception in ThreadStatus::~ThreadStatus() --- src/Common/ThreadStatus.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index c976e4ca16a..ff69163958d 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -139,6 +139,7 @@ ThreadStatus::~ThreadStatus() { /// It's a minor tracked memory leak here (not the memory itself but it's counter). /// We've already allocated a little bit more than the limit and cannot track it in the thread memory tracker or its parent. + tryLogCurrentException(log); } if (thread_group) From 6aebc3e94ca1159431a957f4b49b1f2235849843 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Dec 2021 10:25:30 +0300 Subject: [PATCH 328/358] Do not loose ProfileEvents in case of thread destroyed before v2: drop std::move and add copy ctor for ProfileEvents::Counter::Snapshot v2: remove std::move --- src/Common/ThreadStatus.cpp | 23 +++++++++++++++++++++++ src/Common/ThreadStatus.h | 11 +++++++++++ src/Server/TCPHandler.cpp | 19 +++++++++---------- 3 files changed, 43 insertions(+), 10 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index ff69163958d..411f725f2db 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -72,6 +72,24 @@ static thread_local bool has_alt_stack = false; #endif +std::vector ThreadGroupStatus::getProfileEventsCountersAndMemoryForThreads() +{ + std::lock_guard guard(mutex); + + /// It is OK to move it, since it is enough to report statistics for the thread at least once. + auto stats = std::move(finished_threads_counters_memory); + for (auto * thread : threads) + { + stats.emplace_back(ProfileEventsCountersAndMemory{ + thread->performance_counters.getPartiallyAtomicSnapshot(), + thread->memory_tracker.get(), + thread->thread_id, + }); + } + + return stats; +} + ThreadStatus::ThreadStatus() : thread_id{getThreadId()} { @@ -145,6 +163,11 @@ ThreadStatus::~ThreadStatus() if (thread_group) { std::lock_guard guard(thread_group->mutex); + thread_group->finished_threads_counters_memory.emplace_back(ThreadGroupStatus::ProfileEventsCountersAndMemory{ + performance_counters.getPartiallyAtomicSnapshot(), + memory_tracker.get(), + thread_id, + }); thread_group->threads.erase(this); } diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 97ddda1ea30..f3920474111 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -61,6 +61,13 @@ using ThreadStatusPtr = ThreadStatus *; class ThreadGroupStatus { public: + struct ProfileEventsCountersAndMemory + { + ProfileEvents::Counters::Snapshot counters; + Int64 memory_usage; + UInt64 thread_id; + }; + mutable std::mutex mutex; ProfileEvents::Counters performance_counters{VariableContext::Process}; @@ -83,6 +90,10 @@ public: String query; UInt64 normalized_query_hash = 0; + + std::vector finished_threads_counters_memory; + + std::vector getProfileEventsCountersAndMemoryForThreads(); }; using ThreadGroupStatusPtr = std::shared_ptr; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 36d126559a7..3b1ce4cc846 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -947,28 +947,27 @@ void TCPHandler::sendProfileEvents() ThreadIdToCountersSnapshot new_snapshots; ProfileEventsSnapshot group_snapshot; { - std::lock_guard guard(thread_group->mutex); - snapshots.reserve(thread_group->threads.size()); - for (auto * thread : thread_group->threads) + auto stats = thread_group->getProfileEventsCountersAndMemoryForThreads(); + snapshots.reserve(stats.size()); + + for (auto & stat : stats) { - auto const thread_id = thread->thread_id; + auto const thread_id = stat.thread_id; if (thread_id == current_thread_id) continue; auto current_time = time(nullptr); - auto counters = thread->performance_counters.getPartiallyAtomicSnapshot(); - auto memory_usage = thread->memory_tracker.get(); auto previous_snapshot = last_sent_snapshots.find(thread_id); auto increment = previous_snapshot != last_sent_snapshots.end() - ? CountersIncrement(counters, previous_snapshot->second) - : CountersIncrement(counters); + ? CountersIncrement(stat.counters, previous_snapshot->second) + : CountersIncrement(stat.counters); snapshots.push_back(ProfileEventsSnapshot{ thread_id, std::move(increment), - memory_usage, + stat.memory_usage, current_time }); - new_snapshots[thread_id] = std::move(counters); + new_snapshots[thread_id] = std::move(stat.counters); } group_snapshot.thread_id = 0; From 1d25ec3e82fd6385dd2bc85b864c5996cdc0c82f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 14 Dec 2021 10:25:30 +0300 Subject: [PATCH 329/358] Merge ProfileEvents in case they were not printed That way with --profile-events-delay-ms=-1 you will always get totals. Plus, this will fix periodic failures, that can be reproduced by limitting CPU (5% is enough in my setup), i.e.: $ systemd-run --collect --unit ch -p CPUQuota=5% --user clickhouse-server $ while clickhouse-client --print-profile-events --profile-events-delay-ms=-1 -q 'select * from numbers (1e5) format Null' |& tee /dev/stderr | fgrep 'SelectedRows: 131010 (increment)'; do :; done And as a bonus it will make 02050_client_profile_events deterministic. --- src/Client/ClientBase.cpp | 98 ++++++++++++++++++++++++++++++++++++++- 1 file changed, 97 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 58bc239f003..e662bad1086 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -4,6 +4,8 @@ #include #include #include +#include +#include #include #include @@ -52,6 +54,7 @@ #include #include #include +#include #include #include #include @@ -105,6 +108,99 @@ namespace ProfileEvents namespace DB { +static void incrementProfileEventsBlock(Block & dst, const Block & src) +{ + if (!dst) + { + dst = src; + return; + } + + assertBlocksHaveEqualStructure(src, dst, "ProfileEvents"); + + std::unordered_map name_pos; + for (size_t i = 0; i < dst.columns(); ++i) + name_pos[dst.getByPosition(i).name] = i; + + size_t dst_rows = dst.rows(); + MutableColumns mutable_columns = dst.mutateColumns(); + + auto & dst_column_host_name = typeid_cast(*mutable_columns[name_pos["host_name"]]); + auto & dst_array_current_time = typeid_cast(*mutable_columns[name_pos["current_time"]]).getData(); + auto & dst_array_thread_id = typeid_cast(*mutable_columns[name_pos["thread_id"]]).getData(); + auto & dst_array_type = typeid_cast(*mutable_columns[name_pos["type"]]).getData(); + auto & dst_column_name = typeid_cast(*mutable_columns[name_pos["name"]]); + auto & dst_array_value = typeid_cast(*mutable_columns[name_pos["value"]]).getData(); + + const auto & src_column_host_name = typeid_cast(*src.getByName("host_name").column); + const auto & src_array_current_time = typeid_cast(*src.getByName("current_time").column).getData(); + const auto & src_array_thread_id = typeid_cast(*src.getByName("thread_id").column).getData(); + const auto & src_column_name = typeid_cast(*src.getByName("name").column); + const auto & src_array_value = typeid_cast(*src.getByName("value").column).getData(); + + struct Id + { + StringRef name; + StringRef host_name; + UInt64 thread_id; + + bool operator<(const Id & rhs) const + { + return std::tie(name, host_name, thread_id) + < std::tie(rhs.name, rhs.host_name, rhs.thread_id); + } + }; + std::map rows_by_name; + for (size_t src_row = 0; src_row < src.rows(); ++src_row) + { + Id id{ + src_column_name.getDataAt(src_row), + src_column_host_name.getDataAt(src_row), + src_array_thread_id[src_row], + }; + rows_by_name[id] = src_row; + } + + /// Merge src into dst. + for (size_t dst_row = 0; dst_row < dst_rows; ++dst_row) + { + Id id{ + dst_column_name.getDataAt(dst_row), + dst_column_host_name.getDataAt(dst_row), + dst_array_thread_id[dst_row], + }; + + if (auto it = rows_by_name.find(id); it != rows_by_name.end()) + { + size_t src_row = it->second; + dst_array_current_time[dst_row] = src_array_current_time[src_row]; + + switch (dst_array_type[dst_row]) + { + case ProfileEvents::Type::INCREMENT: + dst_array_value[dst_row] += src_array_value[src_row]; + break; + case ProfileEvents::Type::GAUGE: + dst_array_value[dst_row] = src_array_value[src_row]; + break; + } + + rows_by_name.erase(it); + } + } + + /// Copy rows from src that dst does not contains. + for (const auto & [id, pos] : rows_by_name) + { + for (size_t col = 0; col < src.columns(); ++col) + { + mutable_columns[col]->insert((*src.getByPosition(col).column)[pos]); + } + } + + dst.setColumns(std::move(mutable_columns)); +} + std::atomic_flag exit_on_signal = ATOMIC_FLAG_INIT; @@ -753,7 +849,7 @@ void ClientBase::onProfileEvents(Block & block) } else { - profile_events.last_block = block; + incrementProfileEventsBlock(profile_events.last_block, block); } } profile_events.watch.restart(); From 393e99e8996a2181fd6def5f4a25d06a48b62374 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 19 Dec 2021 18:48:12 -0400 Subject: [PATCH 330/358] Update 02113_base64encode_trailing_bytes_1.sql --- .../queries/0_stateless/02113_base64encode_trailing_bytes_1.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql b/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql index 7166c8df2f3..56edf5dbf6f 100644 --- a/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql +++ b/tests/queries/0_stateless/02113_base64encode_trailing_bytes_1.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + SELECT number, hex(base64Decode(base64Encode(repeat('a', number)))) r From 15c555c161eb3f946a37fb6bdee4edfb96ba687e Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sun, 19 Dec 2021 19:49:30 -0800 Subject: [PATCH 331/358] simplify looping in S2 functions Previously replaced range based for loops in H3 functions with simple for loops based on review. Similarly doing the same for the S2 functions also. --- src/Functions/geoToS2.cpp | 2 +- src/Functions/s2CapContains.cpp | 2 +- src/Functions/s2CapUnion.cpp | 2 +- src/Functions/s2CellsIntersect.cpp | 2 +- src/Functions/s2GetNeighbors.cpp | 2 +- src/Functions/s2RectAdd.cpp | 2 +- src/Functions/s2RectContains.cpp | 2 +- src/Functions/s2RectIntersection.cpp | 2 +- src/Functions/s2RectUnion.cpp | 2 +- src/Functions/s2ToGeo.cpp | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Functions/geoToS2.cpp b/src/Functions/geoToS2.cpp index 644e4661412..5ef5b0763b6 100644 --- a/src/Functions/geoToS2.cpp +++ b/src/Functions/geoToS2.cpp @@ -73,7 +73,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const Float64 lon = col_lon->getFloat64(row); const Float64 lat = col_lat->getFloat64(row); diff --git a/src/Functions/s2CapContains.cpp b/src/Functions/s2CapContains.cpp index c3ebbf0d251..74fccbe00ff 100644 --- a/src/Functions/s2CapContains.cpp +++ b/src/Functions/s2CapContains.cpp @@ -91,7 +91,7 @@ public: auto & dst_data = dst->getData(); dst_data.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row=0 ; row < input_rows_count; row++) { const auto center = S2CellId(col_center->getUInt(row)); const Float64 degrees = col_degrees->getFloat64(row); diff --git a/src/Functions/s2CapUnion.cpp b/src/Functions/s2CapUnion.cpp index 2328db4cb52..57ec6eaee5e 100644 --- a/src/Functions/s2CapUnion.cpp +++ b/src/Functions/s2CapUnion.cpp @@ -95,7 +95,7 @@ public: auto & vec_res_radius = col_res_radius->getData(); vec_res_radius.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const UInt64 first_center = col_center1->getUInt(row); const Float64 first_radius = col_radius1->getFloat64(row); diff --git a/src/Functions/s2CellsIntersect.cpp b/src/Functions/s2CellsIntersect.cpp index d7801afe0d0..0365b9f8529 100644 --- a/src/Functions/s2CellsIntersect.cpp +++ b/src/Functions/s2CellsIntersect.cpp @@ -72,7 +72,7 @@ public: auto & dst_data = dst->getData(); dst_data.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const UInt64 id_first = col_id_first->getInt(row); const UInt64 id_second = col_id_second->getInt(row); diff --git a/src/Functions/s2GetNeighbors.cpp b/src/Functions/s2GetNeighbors.cpp index 99c1395f3cd..e321309b651 100644 --- a/src/Functions/s2GetNeighbors.cpp +++ b/src/Functions/s2GetNeighbors.cpp @@ -72,7 +72,7 @@ public: dst_offsets.resize(input_rows_count); size_t current_offset = 0; - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const UInt64 id = col_id->getUInt(row); diff --git a/src/Functions/s2RectAdd.cpp b/src/Functions/s2RectAdd.cpp index 9a6fcd25e5a..ca796228bb9 100644 --- a/src/Functions/s2RectAdd.cpp +++ b/src/Functions/s2RectAdd.cpp @@ -77,7 +77,7 @@ public: auto & vec_res_second = col_res_second->getData(); vec_res_second.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const auto lo = S2CellId(col_lo->getUInt(row)); const auto hi = S2CellId(col_hi->getUInt(row)); diff --git a/src/Functions/s2RectContains.cpp b/src/Functions/s2RectContains.cpp index 11db27e68ca..7460c2e9ae3 100644 --- a/src/Functions/s2RectContains.cpp +++ b/src/Functions/s2RectContains.cpp @@ -70,7 +70,7 @@ public: auto & dst_data = dst->getData(); dst_data.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const auto lo = S2CellId(col_lo->getUInt(row)); const auto hi = S2CellId(col_hi->getUInt(row)); diff --git a/src/Functions/s2RectIntersection.cpp b/src/Functions/s2RectIntersection.cpp index a8a4536c9e7..829098a1cab 100644 --- a/src/Functions/s2RectIntersection.cpp +++ b/src/Functions/s2RectIntersection.cpp @@ -81,7 +81,7 @@ public: auto & vec_res_second = col_res_second->getData(); vec_res_second.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const auto lo1 = S2CellId(col_lo1->getUInt(row)); const auto hi1 = S2CellId(col_hi1->getUInt(row)); diff --git a/src/Functions/s2RectUnion.cpp b/src/Functions/s2RectUnion.cpp index f187c068345..ef09dcd7e0a 100644 --- a/src/Functions/s2RectUnion.cpp +++ b/src/Functions/s2RectUnion.cpp @@ -79,7 +79,7 @@ public: auto & vec_res_second = col_res_second->getData(); vec_res_second.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const auto lo1 = S2CellId(col_lo1->getUInt(row)); const auto hi1 = S2CellId(col_hi1->getUInt(row)); diff --git a/src/Functions/s2ToGeo.cpp b/src/Functions/s2ToGeo.cpp index 032fdbfe323..f2b4a44ae32 100644 --- a/src/Functions/s2ToGeo.cpp +++ b/src/Functions/s2ToGeo.cpp @@ -78,7 +78,7 @@ public: auto & latitude = col_latitude->getData(); latitude.reserve(input_rows_count); - for (const auto row : collections::range(0, input_rows_count)) + for (size_t row = 0; row < input_rows_count; row++) { const auto id = S2CellId(col_id->getUInt(row)); From b9787f730edf6753f1a81ece95d6d131f51b7d22 Mon Sep 17 00:00:00 2001 From: bharatnc Date: Sun, 19 Dec 2021 20:32:52 -0800 Subject: [PATCH 332/358] use prefix increment for rows --- src/Functions/geoToS2.cpp | 2 +- src/Functions/s2CapContains.cpp | 2 +- src/Functions/s2CapUnion.cpp | 2 +- src/Functions/s2CellsIntersect.cpp | 2 +- src/Functions/s2GetNeighbors.cpp | 2 +- src/Functions/s2RectAdd.cpp | 2 +- src/Functions/s2RectContains.cpp | 2 +- src/Functions/s2RectIntersection.cpp | 2 +- src/Functions/s2RectUnion.cpp | 2 +- src/Functions/s2ToGeo.cpp | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Functions/geoToS2.cpp b/src/Functions/geoToS2.cpp index 5ef5b0763b6..32d2a1d7a10 100644 --- a/src/Functions/geoToS2.cpp +++ b/src/Functions/geoToS2.cpp @@ -73,7 +73,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const Float64 lon = col_lon->getFloat64(row); const Float64 lat = col_lat->getFloat64(row); diff --git a/src/Functions/s2CapContains.cpp b/src/Functions/s2CapContains.cpp index 74fccbe00ff..100b028646c 100644 --- a/src/Functions/s2CapContains.cpp +++ b/src/Functions/s2CapContains.cpp @@ -91,7 +91,7 @@ public: auto & dst_data = dst->getData(); dst_data.reserve(input_rows_count); - for (size_t row=0 ; row < input_rows_count; row++) + for (size_t row=0 ; row < input_rows_count; ++row) { const auto center = S2CellId(col_center->getUInt(row)); const Float64 degrees = col_degrees->getFloat64(row); diff --git a/src/Functions/s2CapUnion.cpp b/src/Functions/s2CapUnion.cpp index 57ec6eaee5e..263163963af 100644 --- a/src/Functions/s2CapUnion.cpp +++ b/src/Functions/s2CapUnion.cpp @@ -95,7 +95,7 @@ public: auto & vec_res_radius = col_res_radius->getData(); vec_res_radius.reserve(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 first_center = col_center1->getUInt(row); const Float64 first_radius = col_radius1->getFloat64(row); diff --git a/src/Functions/s2CellsIntersect.cpp b/src/Functions/s2CellsIntersect.cpp index 0365b9f8529..f8273a1fcca 100644 --- a/src/Functions/s2CellsIntersect.cpp +++ b/src/Functions/s2CellsIntersect.cpp @@ -72,7 +72,7 @@ public: auto & dst_data = dst->getData(); dst_data.reserve(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 id_first = col_id_first->getInt(row); const UInt64 id_second = col_id_second->getInt(row); diff --git a/src/Functions/s2GetNeighbors.cpp b/src/Functions/s2GetNeighbors.cpp index e321309b651..c0b2e634e6f 100644 --- a/src/Functions/s2GetNeighbors.cpp +++ b/src/Functions/s2GetNeighbors.cpp @@ -72,7 +72,7 @@ public: dst_offsets.resize(input_rows_count); size_t current_offset = 0; - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 id = col_id->getUInt(row); diff --git a/src/Functions/s2RectAdd.cpp b/src/Functions/s2RectAdd.cpp index ca796228bb9..f7c39b2a6b1 100644 --- a/src/Functions/s2RectAdd.cpp +++ b/src/Functions/s2RectAdd.cpp @@ -77,7 +77,7 @@ public: auto & vec_res_second = col_res_second->getData(); vec_res_second.reserve(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const auto lo = S2CellId(col_lo->getUInt(row)); const auto hi = S2CellId(col_hi->getUInt(row)); diff --git a/src/Functions/s2RectContains.cpp b/src/Functions/s2RectContains.cpp index 7460c2e9ae3..90ced5450bc 100644 --- a/src/Functions/s2RectContains.cpp +++ b/src/Functions/s2RectContains.cpp @@ -70,7 +70,7 @@ public: auto & dst_data = dst->getData(); dst_data.reserve(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const auto lo = S2CellId(col_lo->getUInt(row)); const auto hi = S2CellId(col_hi->getUInt(row)); diff --git a/src/Functions/s2RectIntersection.cpp b/src/Functions/s2RectIntersection.cpp index 829098a1cab..b108cc1b64f 100644 --- a/src/Functions/s2RectIntersection.cpp +++ b/src/Functions/s2RectIntersection.cpp @@ -81,7 +81,7 @@ public: auto & vec_res_second = col_res_second->getData(); vec_res_second.reserve(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const auto lo1 = S2CellId(col_lo1->getUInt(row)); const auto hi1 = S2CellId(col_hi1->getUInt(row)); diff --git a/src/Functions/s2RectUnion.cpp b/src/Functions/s2RectUnion.cpp index ef09dcd7e0a..bd40a747a09 100644 --- a/src/Functions/s2RectUnion.cpp +++ b/src/Functions/s2RectUnion.cpp @@ -79,7 +79,7 @@ public: auto & vec_res_second = col_res_second->getData(); vec_res_second.reserve(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const auto lo1 = S2CellId(col_lo1->getUInt(row)); const auto hi1 = S2CellId(col_hi1->getUInt(row)); diff --git a/src/Functions/s2ToGeo.cpp b/src/Functions/s2ToGeo.cpp index f2b4a44ae32..03a67d49e45 100644 --- a/src/Functions/s2ToGeo.cpp +++ b/src/Functions/s2ToGeo.cpp @@ -78,7 +78,7 @@ public: auto & latitude = col_latitude->getData(); latitude.reserve(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const auto id = S2CellId(col_id->getUInt(row)); From 0efc01be05df0a72c7a25f602443d9db632a9aaf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=87=8C=E6=B6=9B?= Date: Fri, 17 Dec 2021 11:17:18 +0800 Subject: [PATCH 333/358] add test --- src/Client/ClientBase.cpp | 15 +++++++++++---- .../02151_clickhouse_client_hints.reference | 1 + .../0_stateless/02151_clickhouse_client_hints.sh | 8 ++++++++ 3 files changed, 20 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02151_clickhouse_client_hints.reference create mode 100755 tests/queries/0_stateless/02151_clickhouse_client_hints.sh diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 3fa39b13551..bdd75886661 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1725,10 +1725,17 @@ void ClientBase::init(int argc, char ** argv) return "--" + String(op_long_name); }; - const auto & main_options = options_description.main_description->options(); - const auto & external_options = options_description.external_description->options(); - std::transform(main_options.begin(), main_options.end(), std::back_inserter(cmd_options), getter); - std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter); + if (options_description.main_description) + { + const auto & main_options = options_description.main_description->options(); + std::transform(main_options.begin(), main_options.end(), std::back_inserter(cmd_options), getter); + } + + if (options_description.external_description) + { + const auto & external_options = options_description.external_description->options(); + std::transform(external_options.begin(), external_options.end(), std::back_inserter(cmd_options), getter); + } parseAndCheckOptions(options_description, options, common_arguments); po::notify(options); diff --git a/tests/queries/0_stateless/02151_clickhouse_client_hints.reference b/tests/queries/0_stateless/02151_clickhouse_client_hints.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02151_clickhouse_client_hints.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02151_clickhouse_client_hints.sh b/tests/queries/0_stateless/02151_clickhouse_client_hints.sh new file mode 100755 index 00000000000..3e6c6cb16a5 --- /dev/null +++ b/tests/queries/0_stateless/02151_clickhouse_client_hints.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +$CLICKHOUSE_CLIENT --hardware_utilization 2>&1 | grep -q "Code: 552. DB::Exception: Unrecognized option '--hardware_utilization'. Maybe you meant \['--hardware-utilization'\]. (UNRECOGNIZED_ARGUMENTS)" && echo 'OK' || echo 'FAIL' ||: From 1abe9784552f429226a028600dc32f1aa7ee382d Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Mon, 20 Dec 2021 16:29:06 +0800 Subject: [PATCH 334/358] fix wrong modification_time after move part --- src/Storages/MergeTree/MergeTreePartsMover.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 15e7ed4c1d0..83ffcc41fd8 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -228,6 +228,7 @@ MergeTreeData::DataPartPtr MergeTreePartsMover::clonePart(const MergeTreeMoveEnt LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part->getFullPath()); cloned_part->loadColumnsChecksumsIndexes(true, true); + cloned_part->modification_time = disk->getLastModified(cloned_part->getFullRelativePath()).epochTime(); return cloned_part; } From 3edb2ca8065ecf98ada9c9e020bd4a1a9f86297a Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Mon, 20 Dec 2021 16:32:38 +0800 Subject: [PATCH 335/358] Update external-dicts-dict-polygon.md fix a typo `tne` -> `the` --- .../external-dictionaries/external-dicts-dict-polygon.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index 5fedd5cf8ad..b49f384367d 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -41,7 +41,7 @@ Example of a polygon dictionary configuration: ``` -Tne corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md#create-dictionary-query): +The corresponding [DDL-query](../../../sql-reference/statements/create/dictionary.md#create-dictionary-query): ``` sql CREATE DICTIONARY polygon_dict_name ( key Array(Array(Array(Array(Float64)))), From 0e6b1b0ec08222128997acca658afffbfbed5aaa Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 Dec 2021 11:58:52 +0300 Subject: [PATCH 336/358] Sync release branches in master --- .github/workflows/release_branches.yml | 536 ++++++++++++++++++++++++- 1 file changed, 519 insertions(+), 17 deletions(-) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index bad444d9961..c35b18f8b14 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -407,7 +407,7 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - FunctionalStatelessTestAsan: + FunctionalStatelessTestAsan0: needs: [BuilderDebAsan] runs-on: [self-hosted, func-tester] steps: @@ -419,6 +419,8 @@ jobs: CHECK_NAME=Stateless tests (address, actions) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 0 + RUN_BY_HASH_TOTAL: 2 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -442,7 +444,44 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - FunctionalStatelessTestTsan: + FunctionalStatelessTestAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 1 + RUN_BY_HASH_TOTAL: 2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestTsan0: needs: [BuilderDebTsan] runs-on: [self-hosted, func-tester] steps: @@ -454,6 +493,82 @@ jobs: CHECK_NAME=Stateless tests (thread, actions) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 0 + RUN_BY_HASH_TOTAL: 3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 1 + RUN_BY_HASH_TOTAL: 3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 2 + RUN_BY_HASH_TOTAL: 3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -512,7 +627,7 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - FunctionalStatelessTestMsan: + FunctionalStatelessTestMsan0: needs: [BuilderDebMsan] runs-on: [self-hosted, func-tester] steps: @@ -524,6 +639,8 @@ jobs: CHECK_NAME=Stateless tests (memory, actions) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 0 + RUN_BY_HASH_TOTAL: 3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -547,7 +664,81 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - FunctionalStatelessTestDebug: + FunctionalStatelessTestMsan1: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 1 + RUN_BY_HASH_TOTAL: 3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestMsan2: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 2 + RUN_BY_HASH_TOTAL: 3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestDebug0: needs: [BuilderDebDebug] runs-on: [self-hosted, func-tester] steps: @@ -559,6 +750,82 @@ jobs: CHECK_NAME=Stateless tests (debug, actions) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 0 + RUN_BY_HASH_TOTAL: 3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestDebug1: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 1 + RUN_BY_HASH_TOTAL: 3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestDebug2: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM: 2 + RUN_BY_HASH_TOTAL: 3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -975,8 +1242,8 @@ jobs: ############################################################################################# ############################# INTEGRATION TESTS ############################################# ############################################################################################# - IntegrationTestsAsan: - needs: [BuilderDebAsan, FunctionalStatelessTestAsan] + IntegrationTestsAsan0: + needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: - name: Set envs @@ -986,6 +1253,8 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=Integration tests (asan, actions) REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -1009,8 +1278,80 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - IntegrationTestsTsan: - needs: [BuilderDebTsan, FunctionalStatelessTestTsan] + IntegrationTestsAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsTsan0: + needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: - name: Set envs @@ -1020,6 +1361,8 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=Integration tests (thread, actions) REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -1043,8 +1386,116 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - IntegrationTestsRelease: - needs: [BuilderDebRelease, FunctionalStatelessTestRelease] + IntegrationTestsTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsTsan3: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsRelease0: + needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: - name: Set envs @@ -1054,6 +1505,44 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=Integration tests (release, actions) REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr $GITHUB_WORKSPACE && mkdir $GITHUB_WORKSPACE + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsRelease1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -1081,11 +1570,18 @@ jobs: needs: - DockerHubPush - BuilderReport - - FunctionalStatelessTestDebug + - FunctionalStatelessTestDebug0 + - FunctionalStatelessTestDebug1 + - FunctionalStatelessTestDebug2 - FunctionalStatelessTestRelease - - FunctionalStatelessTestAsan - - FunctionalStatelessTestTsan - - FunctionalStatelessTestMsan + - FunctionalStatelessTestAsan0 + - FunctionalStatelessTestAsan1 + - FunctionalStatelessTestTsan0 + - FunctionalStatelessTestTsan1 + - FunctionalStatelessTestTsan2 + - FunctionalStatelessTestMsan0 + - FunctionalStatelessTestMsan1 + - FunctionalStatelessTestMsan2 - FunctionalStatelessTestUBsan - FunctionalStatefulTestDebug - FunctionalStatefulTestRelease @@ -1098,9 +1594,15 @@ jobs: - StressTestTsan - StressTestMsan - StressTestUBsan - - IntegrationTestsAsan - - IntegrationTestsRelease - - IntegrationTestsTsan + - IntegrationTestsAsan0 + - IntegrationTestsAsan1 + - IntegrationTestsAsan2 + - IntegrationTestsRelease0 + - IntegrationTestsRelease1 + - IntegrationTestsTsan0 + - IntegrationTestsTsan1 + - IntegrationTestsTsan2 + - IntegrationTestsTsan3 - CompatibilityCheck runs-on: [self-hosted, style-checker] steps: From f56e3942e3ef5442a9da95aaecf890f9742d54fd Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 20 Dec 2021 16:59:20 +0800 Subject: [PATCH 337/358] Hermetic shared build --- PreLoad.cmake | 3 +-- cmake/linux/toolchain-x86_64.cmake | 9 ++++++--- contrib/sysroot | 2 +- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/PreLoad.cmake b/PreLoad.cmake index 9fba896d72e..46bf8efed31 100644 --- a/PreLoad.cmake +++ b/PreLoad.cmake @@ -27,8 +27,7 @@ execute_process(COMMAND uname -m OUTPUT_VARIABLE ARCH) if (OS MATCHES "Linux" AND NOT DEFINED CMAKE_TOOLCHAIN_FILE AND NOT DISABLE_HERMETIC_BUILD - AND ($ENV{CC} MATCHES ".*clang.*" OR CMAKE_C_COMPILER MATCHES ".*clang.*") - AND (USE_STATIC_LIBRARIES OR NOT DEFINED USE_STATIC_LIBRARIES)) + AND ($ENV{CC} MATCHES ".*clang.*" OR CMAKE_C_COMPILER MATCHES ".*clang.*")) if (ARCH MATCHES "amd64|x86_64") set (CMAKE_TOOLCHAIN_FILE "cmake/linux/toolchain-x86_64.cmake" CACHE INTERNAL "" FORCE) diff --git a/cmake/linux/toolchain-x86_64.cmake b/cmake/linux/toolchain-x86_64.cmake index 879f35feb83..965ea024ab7 100644 --- a/cmake/linux/toolchain-x86_64.cmake +++ b/cmake/linux/toolchain-x86_64.cmake @@ -14,9 +14,12 @@ set (TOOLCHAIN_PATH "${CMAKE_CURRENT_LIST_DIR}/../../contrib/sysroot/linux-x86_6 set (CMAKE_SYSROOT "${TOOLCHAIN_PATH}/x86_64-linux-gnu/libc") -set (CMAKE_C_FLAGS_INIT "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_CXX_FLAGS_INIT "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") -set (CMAKE_ASM_FLAGS_INIT "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") +set (CMAKE_MODULE_LINKER_FLAGS "${CMAKE_MODULE_LINKER_FLAGS} --gcc-toolchain=${TOOLCHAIN_PATH}") set (HAS_PRE_1970_EXITCODE "0" CACHE STRING "Result from TRY_RUN" FORCE) set (HAS_PRE_1970_EXITCODE__TRYRUN_OUTPUT "" CACHE STRING "Output from TRY_RUN" FORCE) diff --git a/contrib/sysroot b/contrib/sysroot index 410845187f5..bbcac834526 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit 410845187f582c5e6692b53dddbe43efbb728734 +Subproject commit bbcac834526d90d1e764164b861be426891d1743 From 7785eac384e6a8cf2cfdb8756f5bdb8d94e0e660 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Dec 2021 13:32:13 +0300 Subject: [PATCH 338/358] Loops remove postfix increment --- src/Functions/geoToH3.cpp | 2 +- src/Functions/h3EdgeAngle.cpp | 2 +- src/Functions/h3EdgeLengthM.cpp | 2 +- src/Functions/h3GetBaseCell.cpp | 2 +- src/Functions/h3GetFaces.cpp | 2 +- src/Functions/h3GetResolution.cpp | 2 +- src/Functions/h3HexAreaM2.cpp | 2 +- src/Functions/h3IndexesAreNeighbors.cpp | 2 +- src/Functions/h3IsPentagon.cpp | 2 +- src/Functions/h3IsResClassIII.cpp | 2 +- src/Functions/h3IsValid.cpp | 2 +- src/Functions/h3ToChildren.cpp | 2 +- src/Functions/h3ToParent.cpp | 2 +- src/Functions/h3kRing.cpp | 2 +- src/Functions/map.cpp | 4 ++-- 15 files changed, 16 insertions(+), 16 deletions(-) diff --git a/src/Functions/geoToH3.cpp b/src/Functions/geoToH3.cpp index 93865782c8e..18951d1a03f 100644 --- a/src/Functions/geoToH3.cpp +++ b/src/Functions/geoToH3.cpp @@ -76,7 +76,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const double lon = col_lon->getFloat64(row); const double lat = col_lat->getFloat64(row); diff --git a/src/Functions/h3EdgeAngle.cpp b/src/Functions/h3EdgeAngle.cpp index 68e44e38bb9..5d5ad6cd1d3 100644 --- a/src/Functions/h3EdgeAngle.cpp +++ b/src/Functions/h3EdgeAngle.cpp @@ -58,7 +58,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const int resolution = col_hindex->getUInt(row); if (resolution > MAX_H3_RES) diff --git a/src/Functions/h3EdgeLengthM.cpp b/src/Functions/h3EdgeLengthM.cpp index eb0aab029b7..3eef9be9345 100644 --- a/src/Functions/h3EdgeLengthM.cpp +++ b/src/Functions/h3EdgeLengthM.cpp @@ -63,7 +63,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 resolution = col_hindex->getUInt(row); if (resolution > MAX_H3_RES) diff --git a/src/Functions/h3GetBaseCell.cpp b/src/Functions/h3GetBaseCell.cpp index 1f635fda715..83978919f2c 100644 --- a/src/Functions/h3GetBaseCell.cpp +++ b/src/Functions/h3GetBaseCell.cpp @@ -55,7 +55,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 hindex = col_hindex->getUInt(row); diff --git a/src/Functions/h3GetFaces.cpp b/src/Functions/h3GetFaces.cpp index 5d82c16296c..e67ab15128f 100644 --- a/src/Functions/h3GetFaces.cpp +++ b/src/Functions/h3GetFaces.cpp @@ -64,7 +64,7 @@ public: auto current_offset = 0; std::vector faces; - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { int max_faces = maxFaceCount(data[row]); diff --git a/src/Functions/h3GetResolution.cpp b/src/Functions/h3GetResolution.cpp index cc4a3c7443d..02b634dac89 100644 --- a/src/Functions/h3GetResolution.cpp +++ b/src/Functions/h3GetResolution.cpp @@ -55,7 +55,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 hindex = col_hindex->getUInt(row); diff --git a/src/Functions/h3HexAreaM2.cpp b/src/Functions/h3HexAreaM2.cpp index 6aa8fb31aab..96b301806a5 100644 --- a/src/Functions/h3HexAreaM2.cpp +++ b/src/Functions/h3HexAreaM2.cpp @@ -58,7 +58,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 resolution = col_hindex->getUInt(row); if (resolution > MAX_H3_RES) diff --git a/src/Functions/h3IndexesAreNeighbors.cpp b/src/Functions/h3IndexesAreNeighbors.cpp index f938f7fe784..27eaacad4d6 100644 --- a/src/Functions/h3IndexesAreNeighbors.cpp +++ b/src/Functions/h3IndexesAreNeighbors.cpp @@ -63,7 +63,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 hindex_origin = col_hindex_origin->getUInt(row); const UInt64 hindex_dest = col_hindex_dest->getUInt(row); diff --git a/src/Functions/h3IsPentagon.cpp b/src/Functions/h3IsPentagon.cpp index 039fea39f2a..a6726fe1656 100644 --- a/src/Functions/h3IsPentagon.cpp +++ b/src/Functions/h3IsPentagon.cpp @@ -56,7 +56,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0 ; row < input_rows_count ; row++) + for (size_t row = 0 ; row < input_rows_count ; ++row) { UInt8 res = isPentagon(data[row]); dst_data[row] = res; diff --git a/src/Functions/h3IsResClassIII.cpp b/src/Functions/h3IsResClassIII.cpp index f2f7ae445f2..c6b79d404a4 100644 --- a/src/Functions/h3IsResClassIII.cpp +++ b/src/Functions/h3IsResClassIII.cpp @@ -56,7 +56,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0 ; row < input_rows_count ; row++) + for (size_t row = 0 ; row < input_rows_count ; ++row) { UInt8 res = isResClassIII(data[row]); dst_data[row] = res; diff --git a/src/Functions/h3IsValid.cpp b/src/Functions/h3IsValid.cpp index 891d534375e..aa109eee6b4 100644 --- a/src/Functions/h3IsValid.cpp +++ b/src/Functions/h3IsValid.cpp @@ -55,7 +55,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 hindex = col_hindex->getUInt(row); diff --git a/src/Functions/h3ToChildren.cpp b/src/Functions/h3ToChildren.cpp index 5745838e9cb..56b3dd9a88c 100644 --- a/src/Functions/h3ToChildren.cpp +++ b/src/Functions/h3ToChildren.cpp @@ -76,7 +76,7 @@ public: std::vector hindex_vec; - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 parent_hindex = col_hindex->getUInt(row); const UInt8 child_resolution = col_resolution->getUInt(row); diff --git a/src/Functions/h3ToParent.cpp b/src/Functions/h3ToParent.cpp index 76ebea6daf6..fef1b16696f 100644 --- a/src/Functions/h3ToParent.cpp +++ b/src/Functions/h3ToParent.cpp @@ -66,7 +66,7 @@ public: auto & dst_data = dst->getData(); dst_data.resize(input_rows_count); - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const UInt64 hindex = col_hindex->getUInt(row); const UInt8 resolution = col_resolution->getUInt(row); diff --git a/src/Functions/h3kRing.cpp b/src/Functions/h3kRing.cpp index 1bcb3e1ab6c..9fc6312daa4 100644 --- a/src/Functions/h3kRing.cpp +++ b/src/Functions/h3kRing.cpp @@ -73,7 +73,7 @@ public: std::vector hindex_vec; - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { const H3Index origin_hindex = col_hindex->getUInt(row); const int k = col_k->getInt(row); diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 03a9da404c2..0dda46e16d9 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -310,7 +310,7 @@ public: FunctionLike func_like; - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { size_t element_start_row = row != 0 ? column_array.getOffsets()[row-1] : 0; size_t elem_size = column_array.getOffsets()[row]- element_start_row; @@ -457,7 +457,7 @@ public: IColumn::Offset current_offset = 0; - for (size_t row = 0; row < input_rows_count; row++) + for (size_t row = 0; row < input_rows_count; ++row) { size_t element_start_row = row != 0 ? nested_column.getOffsets()[row-1] : 0; size_t element_size = nested_column.getOffsets()[row]- element_start_row; From 3feab5a975c639ca9a63166016b9a80044eb000a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Dec 2021 13:42:31 +0300 Subject: [PATCH 339/358] Containers iteration fix erase --- src/Core/Block.cpp | 2 +- src/Disks/DiskMemory.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 4 ++-- src/Interpreters/getTableExpressions.cpp | 2 +- src/Storages/MergeTree/ActiveDataPartSet.cpp | 4 ++-- src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 1bebe5f5efc..1d23325d473 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -209,7 +209,7 @@ void Block::eraseImpl(size_t position) for (auto it = index_by_name.begin(); it != index_by_name.end();) { if (it->second == position) - index_by_name.erase(it++); + it = index_by_name.erase(it); else { if (it->second > position) diff --git a/src/Disks/DiskMemory.cpp b/src/Disks/DiskMemory.cpp index ea8bf719de6..834ed3e0c65 100644 --- a/src/Disks/DiskMemory.cpp +++ b/src/Disks/DiskMemory.cpp @@ -253,7 +253,7 @@ void DiskMemory::clearDirectory(const String & path) throw Exception( "Failed to clear directory '" + path + "'. " + iter->first + " is a directory", ErrorCodes::CANNOT_DELETE_DIRECTORY); - files.erase(iter++); + iter = files.erase(iter); } } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 44b0c760d8d..9fc16bd9757 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -957,7 +957,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select unknown_required_source_columns.erase(column_name); if (!required.count(column_name)) - source_columns.erase(it++); + it = source_columns.erase(it); else ++it; } @@ -973,7 +973,7 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (column) { source_columns.push_back(*column); - unknown_required_source_columns.erase(it++); + it = unknown_required_source_columns.erase(it); } else ++it; diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index d82c7fc1332..830f0ea4411 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -16,7 +16,7 @@ NameSet removeDuplicateColumns(NamesAndTypesList & columns) if (names.emplace(it->name).second) ++it; else - columns.erase(it++); + it = columns.erase(it); } return names; } diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index 0f6cd8050ca..b21910158ad 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -49,7 +49,7 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) if (out_replaced_parts) out_replaced_parts->push_back(it->second); - part_info_to_name.erase(it++); + it = part_info_to_name.erase(it); } if (out_replaced_parts) @@ -61,7 +61,7 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) assert(part_info != it->first); if (out_replaced_parts) out_replaced_parts->push_back(it->second); - part_info_to_name.erase(it++); + it = part_info_to_name.erase(it); } if (it != part_info_to_name.end() && !part_info.isDisjoint(it->first)) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index cc9a142c65c..b3da3d47684 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1033,7 +1033,7 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( min_unprocessed_insert_time_changed, max_processed_insert_time_changed, lock); (*it)->removed_by_other_entry = true; - queue.erase(it++); + it = queue.erase(it); ++removed_entries; } else From 6e15ff7d31b20df0bd4d1937b877f3a867e3acfa Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 20 Dec 2021 13:48:15 +0300 Subject: [PATCH 340/358] Fix envs --- .github/workflows/release_branches.yml | 44 +++++++++++++------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index c35b18f8b14..7a898c79b1b 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -419,8 +419,8 @@ jobs: CHECK_NAME=Stateless tests (address, actions) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 2 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -456,8 +456,8 @@ jobs: CHECK_NAME=Stateless tests (address, actions) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 2 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -493,8 +493,8 @@ jobs: CHECK_NAME=Stateless tests (thread, actions) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -530,8 +530,8 @@ jobs: CHECK_NAME=Stateless tests (thread, actions) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -567,8 +567,8 @@ jobs: CHECK_NAME=Stateless tests (thread, actions) REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -639,8 +639,8 @@ jobs: CHECK_NAME=Stateless tests (memory, actions) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -676,8 +676,8 @@ jobs: CHECK_NAME=Stateless tests (memory, actions) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -713,8 +713,8 @@ jobs: CHECK_NAME=Stateless tests (memory, actions) REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -750,8 +750,8 @@ jobs: CHECK_NAME=Stateless tests (debug, actions) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 0 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -787,8 +787,8 @@ jobs: CHECK_NAME=Stateless tests (debug, actions) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 1 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 @@ -824,8 +824,8 @@ jobs: CHECK_NAME=Stateless tests (debug, actions) REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM: 2 - RUN_BY_HASH_TOTAL: 3 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports uses: actions/download-artifact@v2 From 20e4ce3314acf3d21bb10d7a86cc84b658ab744e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Dec 2021 14:49:05 +0300 Subject: [PATCH 341/358] Short circuit evaluation function throwIf support --- src/Columns/ColumnFunction.h | 7 ++++++- src/Columns/MaskOperations.cpp | 2 +- src/Columns/MaskOperations.h | 2 +- src/Functions/FunctionsLogical.cpp | 2 +- src/Functions/if.cpp | 2 +- src/Functions/multiIf.cpp | 2 +- src/Functions/throwIf.cpp | 15 +++++++++++---- .../02152_short_circuit_throw_if.reference | 2 ++ .../0_stateless/02152_short_circuit_throw_if.sql | 2 ++ 9 files changed, 26 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02152_short_circuit_throw_if.reference create mode 100644 tests/queries/0_stateless/02152_short_circuit_throw_if.sql diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index 8e39551676c..2592dc01f98 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -24,7 +24,12 @@ class ColumnFunction final : public COWHelper private: friend class COWHelper; - ColumnFunction(size_t size, FunctionBasePtr function_, const ColumnsWithTypeAndName & columns_to_capture, bool is_short_circuit_argument_ = false, bool is_function_compiled_ = false); + ColumnFunction( + size_t size, + FunctionBasePtr function_, + const ColumnsWithTypeAndName & columns_to_capture, + bool is_short_circuit_argument_ = false, + bool is_function_compiled_ = false); public: const char * getFamilyName() const override { return "Function"; } diff --git a/src/Columns/MaskOperations.cpp b/src/Columns/MaskOperations.cpp index 9499185da30..1641bdf5a4c 100644 --- a/src/Columns/MaskOperations.cpp +++ b/src/Columns/MaskOperations.cpp @@ -293,7 +293,7 @@ void executeColumnIfNeeded(ColumnWithTypeAndName & column, bool empty) column.column = column_function->getResultType()->createColumn(); } -int checkShirtCircuitArguments(const ColumnsWithTypeAndName & arguments) +int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments) { int last_short_circuit_argument_index = -1; for (size_t i = 0; i != arguments.size(); ++i) diff --git a/src/Columns/MaskOperations.h b/src/Columns/MaskOperations.h index bd6c5e8fe2c..e43b4588258 100644 --- a/src/Columns/MaskOperations.h +++ b/src/Columns/MaskOperations.h @@ -66,7 +66,7 @@ void executeColumnIfNeeded(ColumnWithTypeAndName & column, bool empty = false); /// Check if arguments contain lazy executed argument. If contain, return index of the last one, /// otherwise return -1. -int checkShirtCircuitArguments(const ColumnsWithTypeAndName & arguments); +int checkShortCircuitArguments(const ColumnsWithTypeAndName & arguments); void copyMask(const PaddedPODArray & from, PaddedPODArray & to); diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index f427deced3a..87a2ecd4c57 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -609,7 +609,7 @@ ColumnPtr FunctionAnyArityLogical::executeImpl( ColumnsWithTypeAndName arguments = std::move(args); /// Special implementation for short-circuit arguments. - if (checkShirtCircuitArguments(arguments) != -1) + if (checkShortCircuitArguments(arguments) != -1) return executeShortCircuit(arguments, result_type); ColumnRawPtrs args_in; diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 953aff3568e..6841098ebcf 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -969,7 +969,7 @@ private: static void executeShortCircuitArguments(ColumnsWithTypeAndName & arguments) { - int last_short_circuit_argument_index = checkShirtCircuitArguments(arguments); + int last_short_circuit_argument_index = checkShortCircuitArguments(arguments); if (last_short_circuit_argument_index == -1) return; diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index 3e5242d5f9b..070a7c2f05e 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -262,7 +262,7 @@ public: private: static void executeShortCircuitArguments(ColumnsWithTypeAndName & arguments) { - int last_short_circuit_argument_index = checkShirtCircuitArguments(arguments); + int last_short_circuit_argument_index = checkShortCircuitArguments(arguments); if (last_short_circuit_argument_index < 0) return; diff --git a/src/Functions/throwIf.cpp b/src/Functions/throwIf.cpp index d499f1f492f..2af6b7eede8 100644 --- a/src/Functions/throwIf.cpp +++ b/src/Functions/throwIf.cpp @@ -63,11 +63,15 @@ public: return std::make_shared(); } - bool useDefaultImplementationForConstants() const override { return true; } + bool useDefaultImplementationForConstants() const override { return false; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + bool isSuitableForConstantFolding() const override { return false; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { + if (input_rows_count == 0) + return result_type->createColumn(); + std::optional custom_message; if (arguments.size() == 2) { @@ -77,7 +81,10 @@ public: custom_message = msg_column->getValue(); } - const auto * in = arguments.front().column.get(); + auto first_argument_column = arguments.front().column; + auto first_argument_column_non_const = first_argument_column->convertToFullColumnIfConst(); + + const auto * in = first_argument_column_non_const.get(); ColumnPtr res; if (!((res = execute(in, custom_message)) @@ -106,7 +113,7 @@ public: ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO}; /// We return non constant to avoid constant folding. - return ColumnUInt8::create(in_data.size(), 0); + return ColumnUInt8::create(in_data.size(), 0); } return nullptr; diff --git a/tests/queries/0_stateless/02152_short_circuit_throw_if.reference b/tests/queries/0_stateless/02152_short_circuit_throw_if.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02152_short_circuit_throw_if.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02152_short_circuit_throw_if.sql b/tests/queries/0_stateless/02152_short_circuit_throw_if.sql new file mode 100644 index 00000000000..3fdc3cc48c8 --- /dev/null +++ b/tests/queries/0_stateless/02152_short_circuit_throw_if.sql @@ -0,0 +1,2 @@ +SELECT if(1, 0, throwIf(1, 'Executing FALSE branch')); +SELECT if(empty(''), 0, throwIf(1, 'Executing FALSE branch')); From 51477adf1bf85c92ebbadb6bc650f3407836ff8a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Dec 2021 15:55:07 +0300 Subject: [PATCH 342/358] Updated additional cases --- src/Access/Authentication.cpp | 2 +- src/Access/LDAPClient.cpp | 4 ++-- .../AggregateFunctionForEach.h | 2 +- .../AggregateFunctionHistogram.h | 2 +- src/AggregateFunctions/AggregateFunctionIf.cpp | 2 +- src/Common/Dwarf.cpp | 2 +- src/Common/TraceCollector.cpp | 2 +- src/Common/parseRemoteDescription.cpp | 2 +- src/Core/MySQL/Authentication.cpp | 4 ++-- src/Core/MySQL/MySQLGtid.cpp | 6 +++--- src/Core/MySQL/MySQLReplication.cpp | 16 ++++++++-------- src/Core/MySQL/PacketsProtocolText.cpp | 4 ++-- src/Dictionaries/PolygonDictionaryUtils.cpp | 2 +- src/Disks/S3/DiskS3.cpp | 2 +- src/Functions/CRC.cpp | 2 +- src/Functions/FunctionMathUnary.h | 2 +- src/Functions/FunctionsLogical.h | 4 ++-- src/Functions/array/mapOp.cpp | 4 ++-- src/Functions/formatString.h | 2 +- src/Functions/h3GetFaces.cpp | 2 +- src/Functions/isIPAddressContainedIn.cpp | 6 +++--- src/Functions/map.cpp | 2 +- src/Functions/pointInPolygon.cpp | 2 +- src/Functions/polygonArea.cpp | 2 +- src/Functions/polygonConvexHull.cpp | 2 +- src/Functions/polygonPerimeter.cpp | 2 +- src/Functions/polygonsDistance.cpp | 2 +- src/Functions/polygonsEquals.cpp | 2 +- src/Functions/polygonsSymDifference.cpp | 2 +- src/Functions/polygonsUnion.cpp | 2 +- src/Functions/polygonsWithin.cpp | 2 +- src/Functions/readWkt.cpp | 2 +- src/Functions/svg.cpp | 2 +- src/Functions/wkt.cpp | 2 +- src/IO/AIO.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/InterpreterInsertQuery.cpp | 2 +- .../Formats/Impl/AvroRowInputFormat.cpp | 10 +++++----- .../Formats/Impl/MySQLOutputFormat.cpp | 4 ++-- .../Formats/Impl/PostgreSQLOutputFormat.cpp | 2 +- .../Transforms/AggregatingTransform.cpp | 2 +- src/Processors/Transforms/WindowTransform.cpp | 4 ++-- .../tests/gtest_blocks_size_merging_streams.cpp | 2 +- .../tests/gtest_check_sorted_stream.cpp | 6 +++--- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- .../MergeTree/MergeTreeIndexFullText.cpp | 2 +- src/Storages/RocksDB/EmbeddedRocksDBSink.cpp | 2 +- src/Storages/StorageBuffer.cpp | 2 +- 48 files changed, 72 insertions(+), 72 deletions(-) diff --git a/src/Access/Authentication.cpp b/src/Access/Authentication.cpp index 794c0a0d5d5..6bc9aeec4c2 100644 --- a/src/Access/Authentication.cpp +++ b/src/Access/Authentication.cpp @@ -54,7 +54,7 @@ namespace const Poco::SHA1Engine::Digest & digest = engine.digest(); Poco::SHA1Engine::Digest calculated_password_sha1(sha1_size); - for (size_t i = 0; i < sha1_size; i++) + for (size_t i = 0; i < sha1_size; ++i) calculated_password_sha1[i] = scrambled_password[i] ^ digest[i]; auto calculated_password_double_sha1 = Util::encodeSHA1(calculated_password_sha1); diff --git a/src/Access/LDAPClient.cpp b/src/Access/LDAPClient.cpp index c666520c069..49d01074f6a 100644 --- a/src/Access/LDAPClient.cpp +++ b/src/Access/LDAPClient.cpp @@ -448,7 +448,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) vals = nullptr; }); - for (std::size_t i = 0; vals[i]; i++) + for (size_t i = 0; vals[i]; ++i) { if (vals[i]->bv_val && vals[i]->bv_len > 0) result.emplace(vals[i]->bv_val, vals[i]->bv_len); @@ -473,7 +473,7 @@ LDAPClient::SearchResults LDAPClient::search(const SearchParams & search_params) referrals = nullptr; }); - for (std::size_t i = 0; referrals[i]; i++) + for (size_t i = 0; referrals[i]; ++i) { LOG_WARNING(&Poco::Logger::get("LDAPClient"), "Received reference during LDAP search but not following it: {}", referrals[i]); } diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index 0de6272d23e..064b7b00c86 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -90,7 +90,7 @@ private: throw; } - for (i = 0; i < old_size; i++) + for (i = 0; i < old_size; ++i) { nested_func->merge(&new_state[i * nested_size_of_data], &old_state[i * nested_size_of_data], diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 665e505aa4e..b858c6b628c 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -271,7 +271,7 @@ public: { lower_bound = std::min(lower_bound, other.lower_bound); upper_bound = std::max(upper_bound, other.upper_bound); - for (size_t i = 0; i < other.size; i++) + for (size_t i = 0; i < other.size; ++i) add(other.points[i].mean, other.points[i].weight, max_bins); } diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 4ac6a2dce21..d752900c018 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -56,7 +56,7 @@ static bool ALWAYS_INLINE inline is_all_zeros(const UInt8 * flags, size_t size) i += 8; } - for (; i < size; i++) + for (; i < size; ++i) if (flags[i]) return false; diff --git a/src/Common/Dwarf.cpp b/src/Common/Dwarf.cpp index 18e9315d5c3..a85bbe818b5 100644 --- a/src/Common/Dwarf.cpp +++ b/src/Common/Dwarf.cpp @@ -838,7 +838,7 @@ bool Dwarf::findLocation( // The next inlined subroutine's call file and call line is the current // caller's location. - for (size_t i = 0; i < num_found - 1; i++) + for (size_t i = 0; i < num_found - 1; ++i) { call_locations[i].file = call_locations[i + 1].file; call_locations[i].line = call_locations[i + 1].line; diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index d84202449d1..523251fa2a2 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -153,7 +153,7 @@ void TraceCollector::run() Array trace; trace.reserve(trace_size); - for (size_t i = 0; i < trace_size; i++) + for (size_t i = 0; i < trace_size; ++i) { uintptr_t addr = 0; readPODBinary(addr, in); diff --git a/src/Common/parseRemoteDescription.cpp b/src/Common/parseRemoteDescription.cpp index 7c8053037ea..fa5d3a8fbd5 100644 --- a/src/Common/parseRemoteDescription.cpp +++ b/src/Common/parseRemoteDescription.cpp @@ -41,7 +41,7 @@ static void append(std::vector & to, const std::vector & what, s static bool parseNumber(const String & description, size_t l, size_t r, size_t & res) { res = 0; - for (size_t pos = l; pos < r; pos ++) + for (size_t pos = l; pos < r; ++pos) { if (!isNumericASCII(description[pos])) return false; diff --git a/src/Core/MySQL/Authentication.cpp b/src/Core/MySQL/Authentication.cpp index 4dd20ff585e..0492211c51f 100644 --- a/src/Core/MySQL/Authentication.cpp +++ b/src/Core/MySQL/Authentication.cpp @@ -71,7 +71,7 @@ Native41::Native41(const String & password_, const String & scramble_) const Poco::SHA1Engine::Digest & digest = engine3.digest(); scramble.resize(SCRAMBLE_LENGTH); - for (size_t i = 0; i < SCRAMBLE_LENGTH; i++) + for (size_t i = 0; i < SCRAMBLE_LENGTH; ++i) scramble[i] = static_cast(password_sha1[i] ^ digest[i]); } @@ -191,7 +191,7 @@ void Sha256Password::authenticate( } password.resize(plaintext_size); - for (int i = 0; i < plaintext_size; i++) + for (int i = 0; i < plaintext_size; ++i) { password[i] = plaintext[i] ^ static_cast(scramble[i % SCRAMBLE_LENGTH]); } diff --git a/src/Core/MySQL/MySQLGtid.cpp b/src/Core/MySQL/MySQLGtid.cpp index a441bccb076..bfd0bd02b45 100644 --- a/src/Core/MySQL/MySQLGtid.cpp +++ b/src/Core/MySQL/MySQLGtid.cpp @@ -41,7 +41,7 @@ void GTIDSets::parse(const String gtid_format) GTIDSet set; set.uuid = DB::parse(server_ids[0]); - for (size_t k = 1; k < server_ids.size(); k++) + for (size_t k = 1; k < server_ids.size(); ++k) { std::vector inters; boost::split(inters, server_ids[k], [](char c) { return c == '-'; }); @@ -74,7 +74,7 @@ void GTIDSets::update(const GTID & other) { if (set.uuid == other.uuid) { - for (auto i = 0U; i < set.intervals.size(); i++) + for (auto i = 0U; i < set.intervals.size(); ++i) { auto & current = set.intervals[i]; @@ -134,7 +134,7 @@ String GTIDSets::toString() const { WriteBufferFromOwnString buffer; - for (size_t i = 0; i < sets.size(); i++) + for (size_t i = 0; i < sets.size(); ++i) { GTIDSet set = sets[i]; writeUUIDText(set.uuid, buffer); diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index f734154f4ba..663d246b0d3 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -159,7 +159,7 @@ namespace MySQLReplication payload.ignore(1); column_count = readLengthEncodedNumber(payload); - for (auto i = 0U; i < column_count; i++) + for (auto i = 0U; i < column_count; ++i) { UInt8 v = 0x00; payload.readStrict(reinterpret_cast(&v), 1); @@ -188,7 +188,7 @@ namespace MySQLReplication { auto pos = 0; column_meta.reserve(column_count); - for (auto i = 0U; i < column_count; i++) + for (auto i = 0U; i < column_count; ++i) { UInt16 typ = column_type[i]; switch (typ) @@ -255,7 +255,7 @@ namespace MySQLReplication out << "Table Len: " << std::to_string(this->table_len) << '\n'; out << "Table: " << this->table << '\n'; out << "Column Count: " << this->column_count << '\n'; - for (auto i = 0U; i < column_count; i++) + for (UInt32 i = 0; i < column_count; ++i) { out << "Column Type [" << i << "]: " << std::to_string(column_type[i]) << ", Meta: " << column_meta[i] << '\n'; } @@ -312,7 +312,7 @@ namespace MySQLReplication UInt32 null_index = 0; UInt32 re_count = 0; - for (auto i = 0U; i < number_columns; i++) + for (UInt32 i = 0; i < number_columns; ++i) { if (bitmap[i]) re_count++; @@ -321,7 +321,7 @@ namespace MySQLReplication boost::dynamic_bitset<> columns_null_set; readBitmap(payload, columns_null_set, re_count); - for (auto i = 0U; i < number_columns; i++) + for (UInt32 i = 0; i < number_columns; ++i) { UInt32 field_len = 0; @@ -523,7 +523,7 @@ namespace MySQLReplication res += (val ^ (mask & compressed_integer_align_numbers[compressed_integers])); } - for (auto k = 0U; k < uncompressed_integers; k++) + for (size_t k = 0; k < uncompressed_integers; ++k) { UInt32 val = 0; readBigEndianStrict(payload, reinterpret_cast(&val), 4); @@ -536,7 +536,7 @@ namespace MySQLReplication size_t uncompressed_decimals = scale / digits_per_integer; size_t compressed_decimals = scale - (uncompressed_decimals * digits_per_integer); - for (auto k = 0U; k < uncompressed_decimals; k++) + for (size_t k = 0; k < uncompressed_decimals; ++k) { UInt32 val = 0; readBigEndianStrict(payload, reinterpret_cast(&val), 4); @@ -669,7 +669,7 @@ namespace MySQLReplication header.dump(out); out << "Schema: " << this->schema << '\n'; out << "Table: " << this->table << '\n'; - for (auto i = 0U; i < rows.size(); i++) + for (size_t i = 0; i < rows.size(); ++i) { out << "Row[" << i << "]: " << applyVisitor(to_string, rows[i]) << '\n'; } diff --git a/src/Core/MySQL/PacketsProtocolText.cpp b/src/Core/MySQL/PacketsProtocolText.cpp index 0494a146c47..728e8061e87 100644 --- a/src/Core/MySQL/PacketsProtocolText.cpp +++ b/src/Core/MySQL/PacketsProtocolText.cpp @@ -15,7 +15,7 @@ namespace ProtocolText ResultSetRow::ResultSetRow(const Serializations & serializations, const Columns & columns_, int row_num_) : columns(columns_), row_num(row_num_) { - for (size_t i = 0; i < columns.size(); i++) + for (size_t i = 0; i < columns.size(); ++i) { if (columns[i]->isNullAt(row_num)) { @@ -39,7 +39,7 @@ size_t ResultSetRow::getPayloadSize() const void ResultSetRow::writePayloadImpl(WriteBuffer & buffer) const { - for (size_t i = 0; i < columns.size(); i++) + for (size_t i = 0; i < columns.size(); ++i) { if (columns[i]->isNullAt(row_num)) buffer.write(serialized[i].data(), 1); diff --git a/src/Dictionaries/PolygonDictionaryUtils.cpp b/src/Dictionaries/PolygonDictionaryUtils.cpp index fced18a6f88..15267481c0b 100644 --- a/src/Dictionaries/PolygonDictionaryUtils.cpp +++ b/src/Dictionaries/PolygonDictionaryUtils.cpp @@ -151,7 +151,7 @@ void SlabsPolygonIndex::indexBuild(const std::vector & polygons) } } - for (size_t i = 0; i != all_edges.size(); i++) + for (size_t i = 0; i != all_edges.size(); ++i) { size_t l = edge_left[i]; size_t r = edge_right[i]; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 0e2f75505fa..29f40a52bd5 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -355,7 +355,7 @@ void DiskS3::findLastRevision() /// Construct revision number from high to low bits. String revision; revision.reserve(64); - for (int bit = 0; bit < 64; bit++) + for (int bit = 0; bit < 64; ++bit) { auto revision_prefix = revision + "1"; diff --git a/src/Functions/CRC.cpp b/src/Functions/CRC.cpp index 00aa631c85b..abcf137f2e7 100644 --- a/src/Functions/CRC.cpp +++ b/src/Functions/CRC.cpp @@ -33,7 +33,7 @@ struct CRCImpl static CRCBase base(polynomial); T crc = 0; - for (size_t i = 0; i < size; i++) + for (size_t i = 0; i < size; ++i) crc = base.tab[(crc ^ buf[i]) & 0xff] ^ (crc >> 8); return crc; } diff --git a/src/Functions/FunctionMathUnary.h b/src/Functions/FunctionMathUnary.h index 2d39daac366..d9ca162ba16 100644 --- a/src/Functions/FunctionMathUnary.h +++ b/src/Functions/FunctionMathUnary.h @@ -94,7 +94,7 @@ private: Impl::execute(src_remaining, dst_remaining); if constexpr (is_big_int_v || std::is_same_v) - for (size_t i = 0; i < rows_remaining; i++) + for (size_t i = 0; i < rows_remaining; ++i) dst_data[rows_size + i] = dst_remaining[i]; else memcpy(&dst_data[rows_size], dst_remaining, rows_remaining * sizeof(ReturnType)); diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 3ddf7ea84eb..7d4f5489e86 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -185,7 +185,7 @@ public: if constexpr (!Impl::isSaturable()) { auto * result = nativeBoolCast(b, types[0], values[0]); - for (size_t i = 1; i < types.size(); i++) + for (size_t i = 1; i < types.size(); ++i) result = Impl::apply(b, result, nativeBoolCast(b, types[i], values[i])); return b.CreateSelect(result, b.getInt8(1), b.getInt8(0)); } @@ -194,7 +194,7 @@ public: auto * stop = llvm::BasicBlock::Create(next->getContext(), "", next->getParent()); b.SetInsertPoint(stop); auto * phi = b.CreatePHI(b.getInt8Ty(), values.size()); - for (size_t i = 0; i < types.size(); i++) + for (size_t i = 0; i < types.size(); ++i) { b.SetInsertPoint(next); auto * value = values[i]; diff --git a/src/Functions/array/mapOp.cpp b/src/Functions/array/mapOp.cpp index a5913105146..b928254e454 100644 --- a/src/Functions/array/mapOp.cpp +++ b/src/Functions/array/mapOp.cpp @@ -204,7 +204,7 @@ private: std::map summing_map; - for (size_t i = 0; i < row_count; i++) + for (size_t i = 0; i < row_count; ++i) { [[maybe_unused]] bool first = true; for (auto & arg : args) @@ -222,7 +222,7 @@ private: } Field temp_val; - for (size_t j = 0; j < len; j++) + for (size_t j = 0; j < len; ++j) { KeyType key; if constexpr (std::is_same::value) diff --git a/src/Functions/formatString.h b/src/Functions/formatString.h index c72e7db9579..419ecf1c773 100644 --- a/src/Functions/formatString.h +++ b/src/Functions/formatString.h @@ -42,7 +42,7 @@ struct FormatImpl static void parseNumber(const String & description, UInt64 l, UInt64 r, UInt64 & res) { res = 0; - for (UInt64 pos = l; pos < r; pos++) + for (UInt64 pos = l; pos < r; ++pos) { if (!isNumericASCII(description[pos])) throw Exception("Not a number in curly braces at position " + std::to_string(pos), ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Functions/h3GetFaces.cpp b/src/Functions/h3GetFaces.cpp index e67ab15128f..c0300e7212b 100644 --- a/src/Functions/h3GetFaces.cpp +++ b/src/Functions/h3GetFaces.cpp @@ -73,7 +73,7 @@ public: // function name h3GetFaces (v3.x) changed to getIcosahedronFaces (v4.0.0). getIcosahedronFaces(data[row], faces.data()); - for (int i = 0; i < max_faces; i++) + for (int i = 0; i < max_faces; ++i) { // valid icosahedron faces are represented by integers 0-19 if (faces[i] >= 0 && faces[i] <= 19) diff --git a/src/Functions/isIPAddressContainedIn.cpp b/src/Functions/isIPAddressContainedIn.cpp index 048fa04adb1..3d2a38ef4c0 100644 --- a/src/Functions/isIPAddressContainedIn.cpp +++ b/src/Functions/isIPAddressContainedIn.cpp @@ -210,7 +210,7 @@ namespace DB ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; @@ -227,7 +227,7 @@ namespace DB ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { const auto addr = IPAddressVariant(col_addr.getDataAt(i)); vec_res[i] = isAddressInRange(addr, cidr) ? 1 : 0; @@ -241,7 +241,7 @@ namespace DB ColumnUInt8::MutablePtr col_res = ColumnUInt8::create(input_rows_count); ColumnUInt8::Container & vec_res = col_res->getData(); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { const auto addr = IPAddressVariant(col_addr.getDataAt(i)); const auto cidr = parseIPWithCIDR(col_cidr.getDataAt(i)); diff --git a/src/Functions/map.cpp b/src/Functions/map.cpp index 0dda46e16d9..4e242c4348b 100644 --- a/src/Functions/map.cpp +++ b/src/Functions/map.cpp @@ -492,7 +492,7 @@ public: auto res = func_like.executeImpl(new_arguments, result_type, input_rows_count); const auto & container = checkAndGetColumn(res.get())->getData(); - for (size_t row_num = 0; row_num < element_size; row_num++) + for (size_t row_num = 0; row_num < element_size; ++row_num) { if (container[row_num] == 1) { diff --git a/src/Functions/pointInPolygon.cpp b/src/Functions/pointInPolygon.cpp index 03e46541cdf..c3a9c411cbc 100644 --- a/src/Functions/pointInPolygon.cpp +++ b/src/Functions/pointInPolygon.cpp @@ -139,7 +139,7 @@ public: } else { - for (size_t i = 1; i < arguments.size(); i++) + for (size_t i = 1; i < arguments.size(); ++i) { const auto * array = checkAndGetDataType(arguments[i].get()); if (array == nullptr) diff --git a/src/Functions/polygonArea.cpp b/src/Functions/polygonArea.cpp index 2e38d6c74b9..c4c573490f6 100644 --- a/src/Functions/polygonArea.cpp +++ b/src/Functions/polygonArea.cpp @@ -78,7 +78,7 @@ public: { auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst()); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) res_data.emplace_back(boost::geometry::area(geometries[i])); } } diff --git a/src/Functions/polygonConvexHull.cpp b/src/Functions/polygonConvexHull.cpp index 887a12b8b6a..e8756f11bba 100644 --- a/src/Functions/polygonConvexHull.cpp +++ b/src/Functions/polygonConvexHull.cpp @@ -75,7 +75,7 @@ public: { auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst()); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { Polygon convex_hull{}; boost::geometry::convex_hull(geometries[i], convex_hull); diff --git a/src/Functions/polygonPerimeter.cpp b/src/Functions/polygonPerimeter.cpp index 8291020197a..eedb91a1622 100644 --- a/src/Functions/polygonPerimeter.cpp +++ b/src/Functions/polygonPerimeter.cpp @@ -77,7 +77,7 @@ public: { auto geometries = Converter::convert(arguments[0].column->convertToFullColumnIfConst()); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) res_data.emplace_back(boost::geometry::perimeter(geometries[i])); } } diff --git a/src/Functions/polygonsDistance.cpp b/src/Functions/polygonsDistance.cpp index 8dd88e1c3bd..51c0198b465 100644 --- a/src/Functions/polygonsDistance.cpp +++ b/src/Functions/polygonsDistance.cpp @@ -83,7 +83,7 @@ public: auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst()); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { boost::geometry::correct(first[i]); boost::geometry::correct(second[i]); diff --git a/src/Functions/polygonsEquals.cpp b/src/Functions/polygonsEquals.cpp index da1db43229b..5c572a16d0e 100644 --- a/src/Functions/polygonsEquals.cpp +++ b/src/Functions/polygonsEquals.cpp @@ -82,7 +82,7 @@ public: auto first = LeftConverter::convert(arguments[0].column->convertToFullColumnIfConst()); auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst()); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { boost::geometry::correct(first[i]); boost::geometry::correct(second[i]); diff --git a/src/Functions/polygonsSymDifference.cpp b/src/Functions/polygonsSymDifference.cpp index 8ef0142072a..4f718760124 100644 --- a/src/Functions/polygonsSymDifference.cpp +++ b/src/Functions/polygonsSymDifference.cpp @@ -81,7 +81,7 @@ public: auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst()); /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign) - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { boost::geometry::correct(first[i]); boost::geometry::correct(second[i]); diff --git a/src/Functions/polygonsUnion.cpp b/src/Functions/polygonsUnion.cpp index 770aa14ac52..e0c6f208c91 100644 --- a/src/Functions/polygonsUnion.cpp +++ b/src/Functions/polygonsUnion.cpp @@ -82,7 +82,7 @@ public: /// We are not interested in some pitfalls in third-party libraries /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign) - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { /// Orient the polygons correctly. boost::geometry::correct(first[i]); diff --git a/src/Functions/polygonsWithin.cpp b/src/Functions/polygonsWithin.cpp index 66e5b4e6e17..0412c9a656d 100644 --- a/src/Functions/polygonsWithin.cpp +++ b/src/Functions/polygonsWithin.cpp @@ -85,7 +85,7 @@ public: auto second = RightConverter::convert(arguments[1].column->convertToFullColumnIfConst()); /// NOLINTNEXTLINE(clang-analyzer-core.uninitialized.Assign) - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { boost::geometry::correct(first[i]); boost::geometry::correct(second[i]); diff --git a/src/Functions/readWkt.cpp b/src/Functions/readWkt.cpp index c3ae6516e0f..b8d0d20acb3 100644 --- a/src/Functions/readWkt.cpp +++ b/src/Functions/readWkt.cpp @@ -55,7 +55,7 @@ public: Serializer serializer; Geometry geometry; - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { const auto & str = column_string->getDataAt(i).toString(); boost::geometry::read_wkt(str, geometry); diff --git a/src/Functions/svg.cpp b/src/Functions/svg.cpp index b3a89c0393c..e1d48ffc061 100644 --- a/src/Functions/svg.cpp +++ b/src/Functions/svg.cpp @@ -79,7 +79,7 @@ public: auto figures = Converter::convert(arguments[0].column->convertToFullColumnIfConst()); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { std::stringstream str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM boost::geometry::correct(figures[i]); diff --git a/src/Functions/wkt.cpp b/src/Functions/wkt.cpp index 8fbb8f59d33..732441eeef2 100644 --- a/src/Functions/wkt.cpp +++ b/src/Functions/wkt.cpp @@ -49,7 +49,7 @@ public: auto figures = Converter::convert(arguments[0].column->convertToFullColumnIfConst()); - for (size_t i = 0; i < input_rows_count; i++) + for (size_t i = 0; i < input_rows_count; ++i) { std::stringstream str; // STYLE_CHECK_ALLOW_STD_STRING_STREAM str << boost::geometry::wkt(figures[i]); diff --git a/src/IO/AIO.cpp b/src/IO/AIO.cpp index 777d9bbbc7f..97e5a470463 100644 --- a/src/IO/AIO.cpp +++ b/src/IO/AIO.cpp @@ -95,7 +95,7 @@ int io_destroy(int ctx) int io_submit(int ctx, long nr, struct iocb * iocbpp[]) { - for (long i = 0; i < nr; i++) + for (long i = 0; i < nr; ++i) { struct aiocb * iocb = &iocbpp[i]->aio; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index f71675910d8..abf1ae5472b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1973,7 +1973,7 @@ std::string ExpressionAnalysisResult::dump() const if (!selected_columns.empty()) { ss << "selected_columns "; - for (size_t i = 0; i < selected_columns.size(); i++) + for (size_t i = 0; i < selected_columns.size(); ++i) { if (i > 0) { diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 26a20f73d24..8677cf59d79 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -375,7 +375,7 @@ BlockIO InterpreterInsertQuery::execute() pipeline = interpreter_watch.buildQueryPipeline(); } - for (size_t i = 0; i < out_streams_size; i++) + for (size_t i = 0; i < out_streams_size; ++i) { auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, nullptr, nullptr); out_chains.emplace_back(std::move(out)); diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 1f806d47c45..ba0d9490618 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -280,7 +280,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node for (size_t n = decoder.arrayStart(); n != 0; n = decoder.arrayNext()) { total += n; - for (size_t i = 0; i < n; i++) + for (size_t i = 0; i < n; ++i) { nested_deserialize(nested_column, decoder); } @@ -344,7 +344,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node if (target.isString()) { std::vector symbols; - for (size_t i = 0; i < root_node->names(); i++) + for (size_t i = 0; i < root_node->names(); ++i) { symbols.push_back(root_node->nameAt(i)); } @@ -359,7 +359,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node { const auto & enum_type = dynamic_cast(*target_type); Row symbol_mapping; - for (size_t i = 0; i < root_node->names(); i++) + for (size_t i = 0; i < root_node->names(); ++i) { symbol_mapping.push_back(enum_type.castToValue(root_node->nameAt(i))); } @@ -443,7 +443,7 @@ AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(avro::NodePtr root_node) case avro::AVRO_UNION: { std::vector union_skip_fns; - for (size_t i = 0; i < root_node->leaves(); i++) + for (size_t i = 0; i < root_node->leaves(); ++i) { union_skip_fns.push_back(createSkipFn(root_node->leafAt(i))); } @@ -476,7 +476,7 @@ AvroDeserializer::SkipFn AvroDeserializer::createSkipFn(avro::NodePtr root_node) case avro::AVRO_RECORD: { std::vector field_skip_fns; - for (size_t i = 0; i < root_node->leaves(); i++) + for (size_t i = 0; i < root_node->leaves(); ++i) { field_skip_fns.push_back(createSkipFn(root_node->leafAt(i))); } diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 5033176ca4b..74070252ebb 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -48,7 +48,7 @@ void MySQLOutputFormat::writePrefix() { packet_endpoint->sendPacket(LengthEncodedNumber(header.columns())); - for (size_t i = 0; i < header.columns(); i++) + for (size_t i = 0; i < header.columns(); ++i) { const auto & column_name = header.getColumnsWithTypeAndName()[i].name; packet_endpoint->sendPacket(getColumnDefinition(column_name, data_types[i]->getTypeId())); @@ -63,7 +63,7 @@ void MySQLOutputFormat::writePrefix() void MySQLOutputFormat::consume(Chunk chunk) { - for (size_t i = 0; i < chunk.getNumRows(); i++) + for (size_t i = 0; i < chunk.getNumRows(); ++i) { ProtocolText::ResultSetRow row_packet(serializations, chunk.getColumns(), i); packet_endpoint->sendPacket(row_packet); diff --git a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp index f46488fd0a8..0450051daf8 100644 --- a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp @@ -21,7 +21,7 @@ void PostgreSQLOutputFormat::writePrefix() std::vector columns; columns.reserve(header.columns()); - for (size_t i = 0; i < header.columns(); i++) + for (size_t i = 0; i < header.columns(); ++i) { const auto & column_name = header.getColumnsWithTypeAndName()[i].name; columns.emplace_back(column_name, data_types[i]->getTypeId()); diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 8357a997960..5b58530f3d5 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -329,7 +329,7 @@ private: if (num_threads > first->aggregates_pools.size()) { Arenas & first_pool = first->aggregates_pools; - for (size_t j = first_pool.size(); j < num_threads; j++) + for (size_t j = first_pool.size(); j < num_threads; ++j) first_pool.emplace_back(std::make_shared()); } diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 7a3bb25d2c6..0da7541556b 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -385,7 +385,7 @@ void WindowTransform::advancePartitionEnd() // prev_frame_start, partition_end); size_t i = 0; - for (; i < partition_by_columns; i++) + for (; i < partition_by_columns; ++i) { const auto * reference_column = inputAt(prev_frame_start)[partition_by_indices[i]].get(); @@ -667,7 +667,7 @@ bool WindowTransform::arePeers(const RowNumber & x, const RowNumber & y) const } size_t i = 0; - for (; i < n; i++) + for (; i < n; ++i) { const auto * column_x = inputAt(x)[order_by_indices[i]].get(); const auto * column_y = inputAt(y)[order_by_indices[i]].get(); diff --git a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp index fb10601216e..e19d2c7114b 100644 --- a/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp +++ b/src/QueryPipeline/tests/gtest_blocks_size_merging_streams.cpp @@ -16,7 +16,7 @@ static Block getBlockWithSize(const std::vector & columns, size_t r ColumnsWithTypeAndName cols; size_t size_of_row_in_bytes = columns.size() * sizeof(UInt64); - for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; i++) + for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; ++i) { auto column = ColumnUInt64::create(rows, 0); for (size_t j = 0; j < rows; ++j) diff --git a/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp index 751f7ef8635..7b30958f0c4 100644 --- a/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp +++ b/src/QueryPipeline/tests/gtest_check_sorted_stream.cpp @@ -29,7 +29,7 @@ static Block getSortedBlockWithSize( { ColumnsWithTypeAndName cols; size_t size_of_row_in_bytes = columns.size() * sizeof(UInt64); - for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; i++) + for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; ++i) { auto column = ColumnUInt64::create(rows, 0); for (size_t j = 0; j < rows; ++j) @@ -47,7 +47,7 @@ static Block getUnSortedBlockWithSize(const std::vector & columns, { ColumnsWithTypeAndName cols; size_t size_of_row_in_bytes = columns.size() * sizeof(UInt64); - for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; i++) + for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; ++i) { auto column = ColumnUInt64::create(rows, 0); for (size_t j = 0; j < rows; ++j) @@ -71,7 +71,7 @@ static Block getEqualValuesBlockWithSize( { ColumnsWithTypeAndName cols; size_t size_of_row_in_bytes = columns.size() * sizeof(UInt64); - for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; i++) + for (size_t i = 0; i * sizeof(UInt64) < size_of_row_in_bytes; ++i) { auto column = ColumnUInt64::create(rows, 0); for (size_t j = 0; j < rows; ++j) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index f49c31cba0c..83328594363 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1357,7 +1357,7 @@ String IMergeTreeDataPart::getRelativePathForPrefix(const String & prefix, bool else if (parent_part) full_relative_path /= parent_part->relative_path; - for (int try_no = 0; try_no < 10; try_no++) + for (int try_no = 0; try_no < 10; ++try_no) { res = (prefix.empty() ? "" : prefix + "_") + name + (try_no ? "_try" + DB::toString(try_no) : ""); diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index a8820b3f6d4..9332f4fd442 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -112,7 +112,7 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos, size_t element_start_row = column_offsets[current_position - 1]; size_t elements_size = column_offsets[current_position] - element_start_row; - for (size_t row_num = 0; row_num < elements_size; row_num++) + for (size_t row_num = 0; row_num < elements_size; ++row_num) { auto ref = column_key.getDataAt(element_start_row + row_num); token_extractor->stringPaddedToBloomFilter(ref.data, ref.size, granule->bloom_filters[col]); diff --git a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp index 1c918c15775..b42f2214d88 100644 --- a/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp +++ b/src/Storages/RocksDB/EmbeddedRocksDBSink.cpp @@ -38,7 +38,7 @@ void EmbeddedRocksDBSink::consume(Chunk chunk) rocksdb::WriteBatch batch; rocksdb::Status status; - for (size_t i = 0; i < rows; i++) + for (size_t i = 0; i < rows; ++i) { wb_key.restart(); wb_value.restart(); diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 6417aa9f72c..f5526781f41 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -1108,7 +1108,7 @@ void registerStorageBuffer(StorageFactory & factory) // After we evaluated all expressions, check that all arguments are // literals. - for (size_t i = 0; i < engine_args.size(); i++) + for (size_t i = 0; i < engine_args.size(); ++i) { if (!typeid_cast(engine_args[i].get())) { From 60d4295d8a1f599f338e325934dd89771f651e6b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Dec 2021 16:00:43 +0300 Subject: [PATCH 343/358] FunctionThrowIf added comment --- src/Functions/throwIf.cpp | 45 +++++++++++++++++++++++++++------------ 1 file changed, 31 insertions(+), 14 deletions(-) diff --git a/src/Functions/throwIf.cpp b/src/Functions/throwIf.cpp index 2af6b7eede8..d2af6781764 100644 --- a/src/Functions/throwIf.cpp +++ b/src/Functions/throwIf.cpp @@ -48,16 +48,21 @@ public: const size_t number_of_arguments = arguments.size(); if (number_of_arguments < 1 || number_of_arguments > 2) - throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed " - + toString(number_of_arguments) + ", should be 1 or 2", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1 or 2", + getName(), + toString(number_of_arguments)); if (!isNativeNumber(arguments[0])) - throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Argument for function {} must be number", + getName()); if (number_of_arguments > 1 && !isString(arguments[1])) - throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[1]->getName(), + getName()); return std::make_shared(); @@ -65,6 +70,10 @@ public: bool useDefaultImplementationForConstants() const override { return false; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + + /** Prevent constant folding for FunctionThrowIf because for short circuit evaluation + * it is unsafe to evaluate this function during DAG analysis. + */ bool isSuitableForConstantFolding() const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override @@ -75,16 +84,17 @@ public: std::optional custom_message; if (arguments.size() == 2) { - const auto * msg_column = checkAndGetColumnConst(arguments[1].column.get()); - if (!msg_column) - throw Exception{"Second argument for function " + getName() + " must be constant String", ErrorCodes::ILLEGAL_COLUMN}; - custom_message = msg_column->getValue(); + const auto * message_column = checkAndGetColumnConst(arguments[1].column.get()); + if (!message_column) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Second argument for function {} must be constant String", + getName()); + + custom_message = message_column->getValue(); } auto first_argument_column = arguments.front().column; - auto first_argument_column_non_const = first_argument_column->convertToFullColumnIfConst(); - - const auto * in = first_argument_column_non_const.get(); + const auto * in = first_argument_column.get(); ColumnPtr res; if (!((res = execute(in, custom_message)) @@ -97,7 +107,9 @@ public: || (res = execute(in, custom_message)) || (res = execute(in, custom_message)) || (res = execute(in, custom_message)))) + { throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } return res; } @@ -105,7 +117,12 @@ public: template ColumnPtr execute(const IColumn * in_untyped, const std::optional & message) const { - if (const auto in = checkAndGetColumn>(in_untyped)) + const auto * in = checkAndGetColumn>(in_untyped); + + if (!in) + in = checkAndGetColumnConstData>(in_untyped); + + if (in) { const auto & in_data = in->getData(); if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0]))) From 02b6ad52efc760c04e4ce0eb25e8b31d4a7163d4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 20 Dec 2021 16:55:15 +0300 Subject: [PATCH 344/358] FunctionThrowIf fixed exception --- src/Functions/throwIf.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/throwIf.cpp b/src/Functions/throwIf.cpp index d2af6781764..7533e30c9b9 100644 --- a/src/Functions/throwIf.cpp +++ b/src/Functions/throwIf.cpp @@ -126,8 +126,10 @@ public: { const auto & in_data = in->getData(); if (!memoryIsZero(in_data.data(), in_data.size() * sizeof(in_data[0]))) - throw Exception{message.value_or("Value passed to '" + getName() + "' function is non zero"), - ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO}; + { + throw Exception(ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO, + message.value_or("Value passed to '" + getName() + "' function is non zero")); + } /// We return non constant to avoid constant folding. return ColumnUInt8::create(in_data.size(), 0); From a5f28e51e5f1d90b415646784ad505361b79f7aa Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 20 Dec 2021 18:09:20 +0300 Subject: [PATCH 345/358] Try fix style check --- src/Client/ClientBase.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index bdd75886661..9328d3f369b 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1720,7 +1720,8 @@ void ClientBase::init(int argc, char ** argv) addOptions(options_description); - auto getter = [](const auto & op) { + auto getter = [](const auto & op) + { String op_long_name = op->long_name(); return "--" + String(op_long_name); }; From 7dea7b7f760ddfacd79c36bac5336b37ee45620e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 20 Dec 2021 18:18:54 +0300 Subject: [PATCH 346/358] Decrease log level for some s3 messages. --- .../AzureBlobStorage/DiskAzureBlobStorage.cpp | 2 +- src/Disks/DiskCacheWrapper.cpp | 10 +++++----- src/Disks/HDFS/DiskHDFS.cpp | 2 +- src/Disks/IDiskRemote.cpp | 6 +++--- src/Disks/S3/DiskS3.cpp | 2 +- src/IO/ReadBufferFromS3.cpp | 4 ++-- src/IO/S3/PocoHTTPClient.cpp | 14 +++++++------- src/IO/S3Common.cpp | 4 ++-- 8 files changed, 22 insertions(+), 22 deletions(-) diff --git a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp index 7883bc0b537..0b65f09338c 100644 --- a/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp +++ b/src/Disks/AzureBlobStorage/DiskAzureBlobStorage.cpp @@ -67,7 +67,7 @@ std::unique_ptr DiskAzureBlobStorage::readFile( auto settings = current_settings.get(); auto metadata = readMeta(path); - LOG_TRACE(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path)); + LOG_TEST(log, "Read from file by path: {}", backQuote(metadata_disk->getPath() + path)); bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; diff --git a/src/Disks/DiskCacheWrapper.cpp b/src/Disks/DiskCacheWrapper.cpp index e1e901f0d45..b09487c17bc 100644 --- a/src/Disks/DiskCacheWrapper.cpp +++ b/src/Disks/DiskCacheWrapper.cpp @@ -91,7 +91,7 @@ DiskCacheWrapper::readFile( if (!cache_file_predicate(path)) return DiskDecorator::readFile(path, settings, size); - LOG_DEBUG(log, "Read file {} from cache", backQuote(path)); + LOG_TEST(log, "Read file {} from cache", backQuote(path)); if (cache_disk->exists(path)) return cache_disk->readFile(path, settings, size); @@ -105,11 +105,11 @@ DiskCacheWrapper::readFile( { /// This thread will responsible for file downloading to cache. metadata->status = DOWNLOADING; - LOG_DEBUG(log, "File {} doesn't exist in cache. Will download it", backQuote(path)); + LOG_TEST(log, "File {} doesn't exist in cache. Will download it", backQuote(path)); } else if (metadata->status == DOWNLOADING) { - LOG_DEBUG(log, "Waiting for file {} download to cache", backQuote(path)); + LOG_TEST(log, "Waiting for file {} download to cache", backQuote(path)); metadata->condition.wait(lock, [metadata] { return metadata->status == DOWNLOADED || metadata->status == ERROR; }); } } @@ -134,7 +134,7 @@ DiskCacheWrapper::readFile( } cache_disk->moveFile(tmp_path, path); - LOG_DEBUG(log, "File {} downloaded to cache", backQuote(path)); + LOG_TEST(log, "File {} downloaded to cache", backQuote(path)); } catch (...) { @@ -163,7 +163,7 @@ DiskCacheWrapper::writeFile(const String & path, size_t buf_size, WriteMode mode if (!cache_file_predicate(path)) return DiskDecorator::writeFile(path, buf_size, mode); - LOG_DEBUG(log, "Write file {} to cache", backQuote(path)); + LOG_TRACE(log, "Write file {} to cache", backQuote(path)); auto dir_path = directoryPath(path); if (!cache_disk->exists(dir_path)) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 4d4a438f93b..41c407c10ee 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -75,7 +75,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, { auto metadata = readMeta(path); - LOG_TRACE(log, + LOG_TEST(log, "Read from file by path: {}. Existing HDFS objects: {}", backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size()); diff --git a/src/Disks/IDiskRemote.cpp b/src/Disks/IDiskRemote.cpp index e920e6fd5b9..848726f957d 100644 --- a/src/Disks/IDiskRemote.cpp +++ b/src/Disks/IDiskRemote.cpp @@ -177,7 +177,7 @@ IDiskRemote::Metadata IDiskRemote::createMeta(const String & path) const void IDiskRemote::removeMeta(const String & path, RemoteFSPathKeeperPtr fs_paths_keeper) { - LOG_DEBUG(log, "Remove file by path: {}", backQuote(metadata_disk->getPath() + path)); + LOG_TRACE(log, "Remove file by path: {}", backQuote(metadata_disk->getPath() + path)); if (!metadata_disk->isFile(path)) throw Exception(ErrorCodes::CANNOT_DELETE_DIRECTORY, "Path '{}' is a directory", path); @@ -464,7 +464,7 @@ bool IDiskRemote::tryReserve(UInt64 bytes) std::lock_guard lock(reservation_mutex); if (bytes == 0) { - LOG_DEBUG(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name)); + LOG_TRACE(log, "Reserving 0 bytes on remote_fs disk {}", backQuote(name)); ++reservation_count; return true; } @@ -473,7 +473,7 @@ bool IDiskRemote::tryReserve(UInt64 bytes) UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); if (unreserved_space >= bytes) { - LOG_DEBUG(log, "Reserving {} on disk {}, having unreserved {}.", + LOG_TRACE(log, "Reserving {} on disk {}, having unreserved {}.", ReadableSize(bytes), backQuote(name), ReadableSize(unreserved_space)); ++reservation_count; reserved_bytes += bytes; diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 0e2f75505fa..97d23d9e407 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -219,7 +219,7 @@ std::unique_ptr DiskS3::readFile(const String & path, co auto settings = current_settings.get(); auto metadata = readMeta(path); - LOG_TRACE(log, "Read from file by path: {}. Existing S3 objects: {}", + LOG_TEST(log, "Read from file by path: {}. Existing S3 objects: {}", backQuote(metadata_disk->getPath() + path), metadata.remote_fs_objects.size()); bool threadpool_read = read_settings.remote_fs_method == RemoteFSReadMethod::threadpool; diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 53d2067780e..30484b14021 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -235,12 +235,12 @@ std::unique_ptr ReadBufferFromS3::initialize() throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to read beyond right offset ({} > {})", offset, read_until_position - 1); req.SetRange(fmt::format("bytes={}-{}", offset, read_until_position - 1)); - LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, read_until_position - 1); + LOG_TEST(log, "Read S3 object. Bucket: {}, Key: {}, Range: {}-{}", bucket, key, offset, read_until_position - 1); } else { req.SetRange(fmt::format("bytes={}-", offset)); - LOG_DEBUG(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, offset); + LOG_TEST(log, "Read S3 object. Bucket: {}, Key: {}, Offset: {}", bucket, key, offset); } Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req); diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 68bdbc9cf86..25b03d66097 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -119,7 +119,7 @@ void PocoHTTPClient::makeRequestInternal( Poco::Logger * log = &Poco::Logger::get("AWSClient"); auto uri = request.GetUri().GetURIString(); - LOG_DEBUG(log, "Make request to: {}", uri); + LOG_TEST(log, "Make request to: {}", uri); enum class S3MetricType { @@ -251,7 +251,7 @@ void PocoHTTPClient::makeRequestInternal( if (request.GetContentBody()) { - LOG_TRACE(log, "Writing request body."); + LOG_TEST(log, "Writing request body."); if (attempt > 0) /// rewind content body buffer. { @@ -259,24 +259,24 @@ void PocoHTTPClient::makeRequestInternal( request.GetContentBody()->seekg(0); } auto size = Poco::StreamCopier::copyStream(*request.GetContentBody(), request_body_stream); - LOG_DEBUG(log, "Written {} bytes to request body", size); + LOG_TEST(log, "Written {} bytes to request body", size); } - LOG_TRACE(log, "Receiving response..."); + LOG_TEST(log, "Receiving response..."); auto & response_body_stream = session->receiveResponse(poco_response); watch.stop(); ProfileEvents::increment(select_metric(S3MetricType::Microseconds), watch.elapsedMicroseconds()); int status_code = static_cast(poco_response.getStatus()); - LOG_DEBUG(log, "Response status: {}, {}", status_code, poco_response.getReason()); + LOG_TEST(log, "Response status: {}, {}", status_code, poco_response.getReason()); if (poco_response.getStatus() == Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) { auto location = poco_response.get("location"); remote_host_filter.checkURL(Poco::URI(location)); uri = location; - LOG_DEBUG(log, "Redirecting request to new location: {}", location); + LOG_TEST(log, "Redirecting request to new location: {}", location); ProfileEvents::increment(select_metric(S3MetricType::Redirects)); @@ -292,7 +292,7 @@ void PocoHTTPClient::makeRequestInternal( response->AddHeader(header_name, header_value); headers_ss << header_name << ": " << header_value << "; "; } - LOG_DEBUG(log, "Received headers: {}", headers_ss.str()); + LOG_TEST(log, "Received headers: {}", headers_ss.str()); if (status_code == 429 || status_code == 503) { // API throttling diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 41b2b1f059a..432dc443300 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -51,8 +51,8 @@ const std::pair & convertLogLevel(Aws::U {Aws::Utils::Logging::LogLevel::Error, {DB::LogsLevel::error, Poco::Message::PRIO_ERROR}}, {Aws::Utils::Logging::LogLevel::Warn, {DB::LogsLevel::warning, Poco::Message::PRIO_WARNING}}, {Aws::Utils::Logging::LogLevel::Info, {DB::LogsLevel::information, Poco::Message::PRIO_INFORMATION}}, - {Aws::Utils::Logging::LogLevel::Debug, {DB::LogsLevel::debug, Poco::Message::PRIO_DEBUG}}, - {Aws::Utils::Logging::LogLevel::Trace, {DB::LogsLevel::trace, Poco::Message::PRIO_TRACE}}, + {Aws::Utils::Logging::LogLevel::Debug, {DB::LogsLevel::debug, Poco::Message::PRIO_TEST}}, + {Aws::Utils::Logging::LogLevel::Trace, {DB::LogsLevel::trace, Poco::Message::PRIO_TEST}}, }; return mapping.at(log_level); } From 25f88356e4358d3bd0055e317f51063219dc5c5b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 19 Dec 2021 11:10:36 +0100 Subject: [PATCH 347/358] Add hints for SET query and HTTP request param settings --- src/Access/SettingsConstraints.cpp | 19 ++++++++++++++++++- src/Core/Settings.cpp | 10 ++++++++++ src/Core/Settings.h | 5 ++++- 3 files changed, 32 insertions(+), 2 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index be59fc13136..9d21ba91f5d 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int READONLY; extern const int QUERY_IS_PROHIBITED; extern const int SETTING_CONSTRAINT_VIOLATION; + extern const int UNKNOWN_SETTING; } @@ -200,7 +201,23 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh }; if (reaction == THROW_ON_VIOLATION) - access_control->checkSettingNameIsAllowed(setting_name); + { + try + { + access_control->checkSettingNameIsAllowed(setting_name); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::UNKNOWN_SETTING) + { + if (const auto hints = current_settings.getHints(change.name); !hints.empty()) + { + e.addMessage(fmt::format("Maybe you meant {}", toString(hints))); + } + } + throw; + } + } else if (!access_control->isSettingNameAllowed(setting_name)) return false; diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 11c625007d9..8daf39d9928 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -117,6 +117,16 @@ void Settings::checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfigura } } +std::vector Settings::getAllRegisteredNames() const +{ + std::vector all_settings; + for (const auto & setting_field : all()) + { + all_settings.push_back(setting_field.getName()); + } + return all_settings; +} + IMPLEMENT_SETTINGS_TRAITS(FormatFactorySettingsTraits, FORMAT_FACTORY_SETTINGS) } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1332d844ff3..70fb5604997 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -675,7 +676,7 @@ DECLARE_SETTINGS_TRAITS_ALLOW_CUSTOM_SETTINGS(SettingsTraits, LIST_OF_SETTINGS) /** Settings of query execution. * These settings go to users.xml. */ -struct Settings : public BaseSettings +struct Settings : public BaseSettings, public IHints<2, Settings> { /// For initialization from empty initializer-list to be "value initialization", not "aggregate initialization" in C++14. /// http://en.cppreference.com/w/cpp/language/aggregate_initialization @@ -699,6 +700,8 @@ struct Settings : public BaseSettings /// Check that there is no user-level settings at the top level in config. /// This is a common source of mistake (user don't know where to write user-level setting). static void checkNoSettingNamesAtTopLevel(const Poco::Util::AbstractConfiguration & config, const String & config_path); + + std::vector getAllRegisteredNames() const override; }; /* From ef57b759e0b1588be3355ad3dd4f7dc3f5fd75c3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 19 Dec 2021 12:29:04 +0100 Subject: [PATCH 348/358] Add stateless tests for setting hints --- .../02151_invalid_setting_with_hints_in_query.reference | 1 + .../02151_invalid_setting_with_hints_in_query.sh | 7 +++++++ ...52_invalid_setting_with_hints_in_http_request.reference | 1 + .../02152_invalid_setting_with_hints_in_http_request.sh | 7 +++++++ 4 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.reference create mode 100755 tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.sh create mode 100644 tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.reference create mode 100755 tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh diff --git a/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.reference b/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.sh b/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.sh new file mode 100755 index 00000000000..a3a8eed60b3 --- /dev/null +++ b/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.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 + +$CLICKHOUSE_LOCAL --query="SET input_format_with_names_use_headers = 1" 2>&1 | grep -q "Code: 115. DB::Exception: Unknown setting input_format_with_names_use_headers: Maybe you meant \['input_format_with_names_use_header','input_format_with_types_use_header'\]. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.reference b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh new file mode 100755 index 00000000000..21498e4c614 --- /dev/null +++ b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.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 + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&?input_format_with_names_use_headers=1" -d 'SELECT 1' 2>&1 | grep -q "Code: 115. DB::Exception: Unknown setting ?input_format_with_names_use_headers: Maybe you meant \['input_format_with_names_use_header','input_format_with_types_use_header'\]. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: From bc52758e4d19f345865c439a33be6862006ea801 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 20 Dec 2021 14:50:06 +0100 Subject: [PATCH 349/358] Explicitly default move/copy constructors --- src/Common/NamePrompter.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 4a44a3adeaa..74725ede08c 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -102,6 +102,13 @@ public: return prompter.getHints(name, getAllRegisteredNames()); } + IHints() = default; + + IHints(const IHints &) = default; + IHints(IHints &&) = default; + IHints & operator=(const IHints &) = default; + IHints & operator=(IHints &&) = default; + virtual ~IHints() = default; private: From e65df85412f15e661a38eae79a744c3cfbf4799f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 20 Dec 2021 16:29:23 +0100 Subject: [PATCH 350/358] Use -F flag for tests --- .../0_stateless/02151_invalid_setting_with_hints_in_query.sh | 2 +- .../02152_invalid_setting_with_hints_in_http_request.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.sh b/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.sh index a3a8eed60b3..2faaa3bb1b6 100755 --- a/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.sh +++ b/tests/queries/0_stateless/02151_invalid_setting_with_hints_in_query.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --query="SET input_format_with_names_use_headers = 1" 2>&1 | grep -q "Code: 115. DB::Exception: Unknown setting input_format_with_names_use_headers: Maybe you meant \['input_format_with_names_use_header','input_format_with_types_use_header'\]. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: +$CLICKHOUSE_LOCAL --query="SET input_format_with_names_use_headers = 1" 2>&1 | grep -qF "Code: 115. DB::Exception: Unknown setting input_format_with_names_use_headers: Maybe you meant ['input_format_with_names_use_header','input_format_with_types_use_header']. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh index 21498e4c614..03b7b58728e 100755 --- a/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh +++ b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&?input_format_with_names_use_headers=1" -d 'SELECT 1' 2>&1 | grep -q "Code: 115. DB::Exception: Unknown setting ?input_format_with_names_use_headers: Maybe you meant \['input_format_with_names_use_header','input_format_with_types_use_header'\]. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&input_format_with_names_use_headers=1" -d 'SELECT 1' 2>&1 | grep -qF "Code: 115. DB::Exception: Unknown setting input_format_with_names_use_headers: Maybe you meant ['input_format_with_names_use_header','input_format_with_types_use_header']. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: From 4eae5ce00ecac1f8bed55438e5a8dbe85536f421 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 20 Dec 2021 20:39:31 +0100 Subject: [PATCH 351/358] Fix http request check --- .../02152_invalid_setting_with_hints_in_http_request.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh index 03b7b58728e..1fbf747da4f 100755 --- a/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh +++ b/tests/queries/0_stateless/02152_invalid_setting_with_hints_in_http_request.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&input_format_with_names_use_headers=1" -d 'SELECT 1' 2>&1 | grep -qF "Code: 115. DB::Exception: Unknown setting input_format_with_names_use_headers: Maybe you meant ['input_format_with_names_use_header','input_format_with_types_use_header']. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&input_format_with_names_use_headers=1" -d 'SELECT 1' 2>&1 | grep -q "Code: 115.*Maybe you meant \['input_format_with_names_use_header','input_format_with_types_use_header'\]. (UNKNOWN_SETTING)" && echo 'OK' || echo 'FAIL' ||: From 30996b24485cd9bb021b089b7e7a4ca60498b51d Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 21 Dec 2021 01:19:51 +0300 Subject: [PATCH 352/358] Fix --- src/Interpreters/getTableOverride.cpp | 2 +- src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/getTableOverride.cpp b/src/Interpreters/getTableOverride.cpp index fedcad85b3f..903d9e80836 100644 --- a/src/Interpreters/getTableOverride.cpp +++ b/src/Interpreters/getTableOverride.cpp @@ -13,7 +13,7 @@ ASTPtr tryGetTableOverride(const String & mapped_database, const String & table) if (auto database_ptr = DatabaseCatalog::instance().tryGetDatabase(mapped_database)) { auto create_query = database_ptr->getCreateDatabaseQuery(); - if (auto create_database_query = create_query->as()) + if (auto * create_database_query = create_query->as()) { if (create_database_query->table_overrides) { diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index ae9ade32ef0..63ba81289ac 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -468,7 +468,7 @@ ASTPtr StorageMaterializedPostgreSQL::getCreateNestedTableQuery( columns_declare_list->set(columns_declare_list->columns, getColumnsExpressionList(ordinary_columns_and_types)); } - auto columns = table_override->columns; + auto * columns = table_override->columns; if (columns && columns->constraints) constraints = ConstraintsDescription(columns->constraints->children); } From 7a195708533cfd70e649219a1ec6cf58d368e29b Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Tue, 21 Dec 2021 11:43:14 +0800 Subject: [PATCH 353/358] keeper handler should remove operation when response sent --- src/Server/KeeperTCPHandler.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 67abd6db13a..0c5d7d93689 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -557,6 +557,8 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response std::lock_guard lock(conn_stats_mutex); conn_stats.updateLatency(elapsed); } + + operations.erase(response->xid); keeper_dispatcher->updateKeeperStatLatency(elapsed); last_op.set(std::make_unique(LastOp{ From 11c85fe66b39c122865597e0f59c4e6163bc9fd0 Mon Sep 17 00:00:00 2001 From: dalei2019 Date: Tue, 21 Dec 2021 16:03:37 +0800 Subject: [PATCH 354/358] Update role.md --- docs/en/sql-reference/statements/create/role.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/role.md b/docs/en/sql-reference/statements/create/role.md index 4723613aeef..e0e58f7a0f6 100644 --- a/docs/en/sql-reference/statements/create/role.md +++ b/docs/en/sql-reference/statements/create/role.md @@ -31,7 +31,7 @@ CREATE ROLE accountant; GRANT SELECT ON db.* TO accountant; ``` -This sequence of queries creates the role `accountant` that has the privilege of reading data from the `accounting` database. +This sequence of queries creates the role `accountant` that has the privilege of reading data from the `db` database. Assigning the role to the user `mira`: From 754785fee5c8b8f972d11db0f1539fbfa8e71821 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Dec 2021 11:07:24 +0300 Subject: [PATCH 355/358] Better container --- src/Server/KeeperTCPHandler.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.h b/src/Server/KeeperTCPHandler.h index fb6541d1f53..f98b269b8be 100644 --- a/src/Server/KeeperTCPHandler.h +++ b/src/Server/KeeperTCPHandler.h @@ -93,7 +93,7 @@ private: Poco::Timestamp established; - using Operations = std::map; + using Operations = std::unordered_map; Operations operations; LastOpMultiVersion last_op; From de7a5dd1e66cd8cdeb6e55654e4219096952aceb Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 21 Dec 2021 12:39:31 +0300 Subject: [PATCH 356/358] Update ExpressionListParsers.cpp --- src/Parsers/ExpressionListParsers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index c06f262e605..157b67a6be3 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -402,7 +402,7 @@ bool ParserVariableArityOperatorList::parseImpl(Pos & pos, ASTPtr & node, Expect bool ParserBetweenExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { /// For the expression (subject [NOT] BETWEEN left AND right) - /// create an AST the same as for (subject> = left AND subject <= right). + /// create an AST the same as for (subject >= left AND subject <= right). ParserKeyword s_not("NOT"); ParserKeyword s_between("BETWEEN"); From 9a2b0680a14fe615e89ee5804f2bbc85a2e7cd5d Mon Sep 17 00:00:00 2001 From: Haavard Kvaalen Date: Tue, 21 Dec 2021 11:50:54 +0100 Subject: [PATCH 357/358] Fix weird permission on log directories Several log directories for integration tests were created with full permissions for "others" but no permission for the owner of the directory. This caused issues when trying to run integration tests again without manually wiping files from previous runs. The test would fail with "PermissionError: [Errno 13] Permission denied: 'logs'". The intention may have been to do the equivalent of 'chmod o+rwx logs', but it instead did the equivalent of 'chmod 007 logs'. --- tests/integration/helpers/cluster.py | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 96c44a9bbf6..d440f2de0ca 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1527,7 +1527,7 @@ class ClickHouseCluster: if os.path.exists(self.mysql_dir): shutil.rmtree(self.mysql_dir) os.makedirs(self.mysql_logs_dir) - os.chmod(self.mysql_logs_dir, stat.S_IRWXO) + os.chmod(self.mysql_logs_dir, stat.S_IRWXU | stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) self.up_called = True self.wait_mysql_to_start() @@ -1537,7 +1537,7 @@ class ClickHouseCluster: if os.path.exists(self.mysql8_dir): shutil.rmtree(self.mysql8_dir) os.makedirs(self.mysql8_logs_dir) - os.chmod(self.mysql8_logs_dir, stat.S_IRWXO) + os.chmod(self.mysql8_logs_dir, stat.S_IRWXU | stat.S_IRWXO) subprocess_check_call(self.base_mysql8_cmd + common_opts) self.wait_mysql8_to_start() @@ -1546,7 +1546,7 @@ class ClickHouseCluster: if os.path.exists(self.mysql_cluster_dir): shutil.rmtree(self.mysql_cluster_dir) os.makedirs(self.mysql_cluster_logs_dir) - os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXO) + os.chmod(self.mysql_cluster_logs_dir, stat.S_IRWXU | stat.S_IRWXO) subprocess_check_call(self.base_mysql_cluster_cmd + common_opts) self.up_called = True @@ -1557,7 +1557,7 @@ class ClickHouseCluster: if os.path.exists(self.postgres_dir): shutil.rmtree(self.postgres_dir) os.makedirs(self.postgres_logs_dir) - os.chmod(self.postgres_logs_dir, stat.S_IRWXO) + os.chmod(self.postgres_logs_dir, stat.S_IRWXU | stat.S_IRWXO) subprocess_check_call(self.base_postgres_cmd + common_opts) self.up_called = True @@ -1566,11 +1566,11 @@ class ClickHouseCluster: if self.with_postgres_cluster and self.base_postgres_cluster_cmd: print('Setup Postgres') os.makedirs(self.postgres2_logs_dir) - os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) + os.chmod(self.postgres2_logs_dir, stat.S_IRWXU | stat.S_IRWXO) os.makedirs(self.postgres3_logs_dir) - os.chmod(self.postgres3_logs_dir, stat.S_IRWXO) + os.chmod(self.postgres3_logs_dir, stat.S_IRWXU | stat.S_IRWXO) os.makedirs(self.postgres4_logs_dir) - os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) + os.chmod(self.postgres4_logs_dir, stat.S_IRWXU | stat.S_IRWXO) subprocess_check_call(self.base_postgres_cluster_cmd + common_opts) self.up_called = True self.wait_postgres_cluster_to_start() @@ -1591,7 +1591,7 @@ class ClickHouseCluster: if self.with_rabbitmq and self.base_rabbitmq_cmd: logging.debug('Setup RabbitMQ') os.makedirs(self.rabbitmq_logs_dir) - os.chmod(self.rabbitmq_logs_dir, stat.S_IRWXO) + os.chmod(self.rabbitmq_logs_dir, stat.S_IRWXU | stat.S_IRWXO) for i in range(5): subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) @@ -1604,7 +1604,7 @@ class ClickHouseCluster: if self.with_hdfs and self.base_hdfs_cmd: logging.debug('Setup HDFS') os.makedirs(self.hdfs_logs_dir) - os.chmod(self.hdfs_logs_dir, stat.S_IRWXO) + os.chmod(self.hdfs_logs_dir, stat.S_IRWXU | stat.S_IRWXO) subprocess_check_call(self.base_hdfs_cmd + common_opts) self.up_called = True self.make_hdfs_api() @@ -1613,7 +1613,7 @@ class ClickHouseCluster: if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') os.makedirs(self.hdfs_kerberized_logs_dir) - os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO) + os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXU | stat.S_IRWXO) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) self.up_called = True self.make_hdfs_api(kerberized=True) @@ -1669,7 +1669,7 @@ class ClickHouseCluster: if self.with_jdbc_bridge and self.base_jdbc_bridge_cmd: os.makedirs(self.jdbc_driver_logs_dir) - os.chmod(self.jdbc_driver_logs_dir, stat.S_IRWXO) + os.chmod(self.jdbc_driver_logs_dir, stat.S_IRWXU | stat.S_IRWXO) subprocess_check_call(self.base_jdbc_bridge_cmd + ['up', '-d']) self.up_called = True From 45aef649a120d47a20aef361b5ba066805c64f6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 21 Dec 2021 16:13:44 +0300 Subject: [PATCH 358/358] Add backport workflow rerun --- tests/ci/workflow_approve_rerun_lambda/app.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 0f601f7f52a..f2502f605af 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -23,7 +23,7 @@ SUSPICIOUS_PATTERNS = [ ] MAX_RETRY = 5 -MAX_WORKFLOW_RERUN = 5 +MAX_WORKFLOW_RERUN = 7 WorkflowDescription = namedtuple('WorkflowDescription', ['name', 'action', 'run_id', 'event', 'workflow_id', 'conclusion', 'status', 'api_url', @@ -44,6 +44,7 @@ NEED_RERUN_WORKFLOWS = { 15834118, # Docs 15522500, # MasterCI 15516108, # ReleaseCI + 15797242, # BackportPR } # Individual trusted contirbutors who are not in any trusted organization.