diff --git a/src/Common/AutoArray.h b/src/Common/AutoArray.h deleted file mode 100644 index 1df8bde8c12..00000000000 --- a/src/Common/AutoArray.h +++ /dev/null @@ -1,309 +0,0 @@ -#pragma once - -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_ALLOCATE_MEMORY; -} - -/** An array of (almost) unchangeable size: - * the size is specified in the constructor; - * `resize` method removes old data, and necessary only for - * so that you can first create an empty object using the default constructor, - * and then decide on the size. - * - * There is a possibility to not initialize elements by default, but create them inplace. - * Member destructors are called automatically. - * - * `sizeof` is equal to the size of one pointer. - * - * Not exception-safe. - * - * Copying is supported via assign() method. Moving empties the original object. - * That is, it is inconvenient to use this array in many cases. - * - * Designed for situations in which many arrays of the same small size are created, - * but the size is not known at compile time. - * Also gives a significant advantage in cases where it is important that `sizeof` is minimal. - * For example, if arrays are put in an open-addressing hash table with inplace storage of values (like HashMap) - * - * In this case, compared to std::vector: - * - for arrays of 1 element size - an advantage of about 2 times; - * - for arrays of 5 elements - an advantage of about 1.5 times - * (DB::Field, containing UInt64 and String, used as T); - */ - -const size_t empty_auto_array_helper = 0; - -template -class AutoArray -{ -public: - /// For deferred creation. - AutoArray() - { - setEmpty(); - } - - explicit AutoArray(size_t size_) - { - init(size_, false); - } - - /** Initializes all elements with a copy constructor with the `value` parameter. - */ - AutoArray(size_t size_, const T & value) - { - init(size_, true); - - for (size_t i = 0; i < size_; ++i) - { - new (place(i)) T(value); - } - } - - /** `resize` removes all existing items. - */ - void resize(size_t size_, bool dont_init_elems = false) - { - uninit(); - init(size_, dont_init_elems); - } - - /** Move operations. - */ - AutoArray(AutoArray && src) - { - if (this == &src) - return; - setEmpty(); - data_ptr = src.data_ptr; - src.setEmpty(); - } - - AutoArray & operator= (AutoArray && src) - { - if (this == &src) - return *this; - uninit(); - data_ptr = src.data_ptr; - src.setEmpty(); - - return *this; - } - - ~AutoArray() - { - uninit(); - } - - size_t size() const - { - return m_size(); - } - - bool empty() const - { - return size() == 0; - } - - void clear() - { - uninit(); - setEmpty(); - } - - template - void assign(It from_begin, It from_end) - { - uninit(); - - size_t size = from_end - from_begin; - init(size, /* dont_init_elems = */ true); - - It it = from_begin; - for (size_t i = 0; i < size; ++i, ++it) - new (place(i)) T(*it); - } - - void assign(const AutoArray & from) - { - assign(from.begin(), from.end()); - } - - /** You can read and modify elements using the [] operator - * only if items were initialized - * (that is, into the constructor was not passed DontInitElemsTag, - * or you initialized them using `place` and `placement new`). - */ - T & operator[](size_t i) - { - return elem(i); - } - - const T & operator[](size_t i) const - { - return elem(i); - } - - T * data() - { - return elemPtr(0); - } - - const T * data() const - { - return elemPtr(0); - } - - /** Get the piece of memory in which the element should be located. - * The function is intended to initialize an element, - * which has not yet been initialized - * new (arr.place(i)) T(args); - */ - char * place(size_t i) - { - return data_ptr + sizeof(T) * i; - } - - using iterator = T *; - using const_iterator = const T *; - - iterator begin() { return elemPtr(0); } - iterator end() { return elemPtr(size()); } - - const_iterator begin() const { return elemPtr(0); } - const_iterator end() const { return elemPtr(size()); } - - bool operator== (const AutoArray & rhs) const - { - size_t s = size(); - - if (s != rhs.size()) - return false; - - for (size_t i = 0; i < s; ++i) - if (elem(i) != rhs.elem(i)) - return false; - - return true; - } - - bool operator!= (const AutoArray & rhs) const - { - return !(*this == rhs); - } - - bool operator< (const AutoArray & rhs) const - { - size_t s = size(); - size_t rhs_s = rhs.size(); - - if (s < rhs_s) - return true; - if (s > rhs_s) - return false; - - for (size_t i = 0; i < s; ++i) - { - if (elem(i) < rhs.elem(i)) - return true; - if (elem(i) > rhs.elem(i)) - return false; - } - - return false; - } - -private: - static constexpr size_t alignment = alignof(T); - /// Bytes allocated to store size of array before data. It is padded to have minimum size as alignment. - /// Padding is at left and the size is stored at right (just before the first data element). - static constexpr size_t prefix_size = std::max(sizeof(size_t), alignment); - - char * data_ptr; - - size_t & m_size() - { - return reinterpret_cast(data_ptr)[-1]; - } - - size_t m_size() const - { - return reinterpret_cast(data_ptr)[-1]; - } - - T * elemPtr(size_t i) - { - return reinterpret_cast(data_ptr) + i; - } - - const T * elemPtr(size_t i) const - { - return reinterpret_cast(data_ptr) + i; - } - - T & elem(size_t i) - { - return *elemPtr(i); - } - - const T & elem(size_t i) const - { - return *elemPtr(i); - } - - void setEmpty() - { - data_ptr = const_cast(reinterpret_cast(&empty_auto_array_helper)) + sizeof(size_t); - } - - void init(size_t new_size, bool dont_init_elems) - { - if (!new_size) - { - setEmpty(); - return; - } - - void * new_data = nullptr; - int res = posix_memalign(&new_data, alignment, prefix_size + new_size * sizeof(T)); - if (0 != res) - throwFromErrno(fmt::format("Cannot allocate memory (posix_memalign) {}.", ReadableSize(new_size)), - ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); - - data_ptr = static_cast(new_data); - data_ptr += prefix_size; - - m_size() = new_size; - - if (!dont_init_elems) - for (size_t i = 0; i < new_size; ++i) - new (place(i)) T(); - } - - void uninit() - { - size_t s = size(); - - if (s) - { - for (size_t i = 0; i < s; ++i) - elem(i).~T(); - - data_ptr -= prefix_size; - free(data_ptr); - } - } -}; - -} diff --git a/src/Common/examples/CMakeLists.txt b/src/Common/examples/CMakeLists.txt index c94ce580942..64d28fec5c2 100644 --- a/src/Common/examples/CMakeLists.txt +++ b/src/Common/examples/CMakeLists.txt @@ -7,9 +7,6 @@ endif() add_executable (sip_hash_perf sip_hash_perf.cpp) target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io) -add_executable (auto_array auto_array.cpp) -target_link_libraries (auto_array PRIVATE clickhouse_common_io) - add_executable (small_table small_table.cpp) target_link_libraries (small_table PRIVATE clickhouse_common_io) diff --git a/src/Common/examples/auto_array.cpp b/src/Common/examples/auto_array.cpp deleted file mode 100644 index 9e9a18ffec9..00000000000 --- a/src/Common/examples/auto_array.cpp +++ /dev/null @@ -1,197 +0,0 @@ -#include -#include -#include - -#include -#include -#include -#include -#include - -#include - - -int main(int argc, char ** argv) -{ - pcg64 rng; - - { - size_t n = 10; - using T = std::string; - DB::AutoArray arr(n); - - for (size_t i = 0; i < arr.size(); ++i) - arr[i] = "Hello, world! " + DB::toString(i); - - for (auto & elem : arr) - std::cerr << elem << std::endl; - } - - std::cerr << std::endl; - - { - size_t n = 10; - using T = std::string; - using Arr = DB::AutoArray; - Arr arr; - - arr.resize(n); - for (size_t i = 0; i < arr.size(); ++i) - arr[i] = "Hello, world! " + DB::toString(i); - - for (auto & elem : arr) - std::cerr << elem << std::endl; - - std::cerr << std::endl; - - Arr arr2 = std::move(arr); - - std::cerr << arr.size() << ", " << arr2.size() << std::endl; // NOLINT - - for (auto & elem : arr2) - std::cerr << elem << std::endl; - } - - std::cerr << std::endl; - - { - size_t n = 10; - size_t keys = 10; - using T = std::string; - using Arr = DB::AutoArray; - using Map = std::map; - Map map; - - for (size_t i = 0; i < keys; ++i) - { - Arr key(n); - for (size_t j = 0; j < n; ++j) - key[j] = DB::toString(rng()); - - map[std::move(key)] = "Hello, world! " + DB::toString(i); - } - - for (const auto & kv : map) - { - std::cerr << "["; - for (size_t j = 0; j < n; ++j) - std::cerr << (j == 0 ? "" : ", ") << kv.first[j]; - std::cerr << "]"; - - std::cerr << ":\t" << kv.second << std::endl; - } - - std::cerr << std::endl; - - Map map2 = std::move(map); - - for (const auto & kv : map2) - { - std::cerr << "["; - for (size_t j = 0; j < n; ++j) - std::cerr << (j == 0 ? "" : ", ") << kv.first[j]; - std::cerr << "]"; - - std::cerr << ":\t" << kv.second << std::endl; - } - } - - std::cerr << std::endl; - - { - size_t n = 10; - size_t keys = 10; - using T = std::string; - using Arr = DB::AutoArray; - using Vec = std::vector; - Vec vec; - - for (size_t i = 0; i < keys; ++i) - { - Arr key(n); - for (size_t j = 0; j < n; ++j) - key[j] = DB::toString(rng()); - - vec.push_back(std::move(key)); - } - - for (const auto & elem : vec) - { - std::cerr << "["; - for (size_t j = 0; j < n; ++j) - std::cerr << (j == 0 ? "" : ", ") << elem[j]; - std::cerr << "]" << std::endl; - } - - std::cerr << std::endl; - - Vec vec2 = std::move(vec); - - for (const auto & elem : vec2) - { - std::cerr << "["; - for (size_t j = 0; j < n; ++j) - std::cerr << (j == 0 ? "" : ", ") << elem[j]; - std::cerr << "]" << std::endl; - } - } - - if (argc == 2 && !strcmp(argv[1], "1")) - { - size_t n = 5; - size_t map_size = 1000000; - - using T = DB::Field; - T field = std::string("Hello, world"); - - using Arr = std::vector; - using Map = HashMap; - - Stopwatch watch; - - Map map; - for (size_t i = 0; i < map_size; ++i) - { - Map::LookupResult it; - bool inserted; - - map.emplace(rng(), it, inserted); - if (inserted) - { - new (&it->getMapped()) Arr(n); - - for (size_t j = 0; j < n; ++j) - (it->getMapped())[j] = field; - } - } - - std::cerr << std::fixed << std::setprecision(2) - << "Vector: Elapsed: " << watch.elapsedSeconds() - << " (" << map_size / watch.elapsedSeconds() << " rows/sec., " - << "sizeof(Map::value_type) = " << sizeof(Map::value_type) - << std::endl; - } - - { - size_t n = 10000; - using Arr = DB::AutoArray; - Arr arr1(n); - Arr arr2(n); - - for (size_t i = 0; i < n; ++i) - { - arr1[i] = "Hello, world! " + DB::toString(i); - arr2[i] = "Goodbye, world! " + DB::toString(i); - } - - arr2 = std::move(arr1); - arr1.resize(n); // NOLINT - - std::cerr - << "arr1.size(): " << arr1.size() << ", arr2.size(): " << arr2.size() << std::endl - << "arr1.data(): " << arr1.data() << ", arr2.data(): " << arr2.data() << std::endl - << "arr1[0]: " << arr1[0] << ", arr2[0]: " << arr2[0] << std::endl; - } - - return 0; -} diff --git a/src/DataStreams/MergingSortedBlockInputStream.h b/src/DataStreams/MergingSortedBlockInputStream.h index 7f8d594416d..582b41ff3af 100644 --- a/src/DataStreams/MergingSortedBlockInputStream.h +++ b/src/DataStreams/MergingSortedBlockInputStream.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include diff --git a/src/DataStreams/TTLAggregationAlgorithm.h b/src/DataStreams/TTLAggregationAlgorithm.h index c2f40bab6b9..7231506b7da 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.h +++ b/src/DataStreams/TTLAggregationAlgorithm.h @@ -32,7 +32,7 @@ private: const Block header; std::unique_ptr aggregator; - Row current_key_value; + std::vector current_key_value; AggregatedDataVariants aggregation_result; ColumnRawPtrs key_columns; Aggregator::AggregateColumns columns_for_aggregator; diff --git a/src/Functions/partitionId.cpp b/src/Functions/partitionId.cpp index cf679452da1..70ece96f983 100644 --- a/src/Functions/partitionId.cpp +++ b/src/Functions/partitionId.cpp @@ -52,7 +52,7 @@ public: auto result_column = ColumnString::create(); for (size_t j = 0; j < input_rows_count; ++j) { - Row row(size); + std::vector row(size); for (size_t i = 0; i < size; ++i) arguments[i].column->get(j, row[i]); MergeTreePartition partition(std::move(row)); diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index dace0beea11..2c66d2a9841 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -80,7 +79,7 @@ static Block createBlockFromCollection(const Collection & collection, const Data for (size_t i = 0; i < columns_num; ++i) columns[i] = types[i]->createColumn(); - Row tuple_values; + std::vector tuple_values; for (const auto & value : collection) { if (columns_num == 1) @@ -154,7 +153,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co MutableColumns columns = header.cloneEmptyColumns(); DataTypePtr tuple_type; - Row tuple_values; + std::vector tuple_values; const auto & list = node->as(); bool transform_null_in = context->getSettingsRef().transform_null_in; for (const auto & elem : list.children) diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index 713be3c1447..65896edfa69 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include diff --git a/src/Interpreters/examples/hash_map.cpp b/src/Interpreters/examples/hash_map.cpp index 620f2515825..66704ab432a 100644 --- a/src/Interpreters/examples/hash_map.cpp +++ b/src/Interpreters/examples/hash_map.cpp @@ -12,7 +12,6 @@ #define DBMS_HASH_MAP_COUNT_COLLISIONS */ #include -#include #include #include #include @@ -27,19 +26,6 @@ * This is important, because if you run all the tests one by one, the results will be incorrect. * (Due to the peculiarities of the work of the allocator, the first test takes advantage.) * - * Depending on USE_AUTO_ARRAY, one of the structures is selected as the value. - * USE_AUTO_ARRAY = 0 - uses std::vector (hard-copy structure, sizeof = 24 bytes). - * USE_AUTO_ARRAY = 1 - uses AutoArray (a structure specially designed for such cases, sizeof = 8 bytes). - * - * That is, the test also allows you to compare AutoArray and std::vector. - * - * If USE_AUTO_ARRAY = 0, then HashMap confidently overtakes all. - * If USE_AUTO_ARRAY = 1, then HashMap is slightly less serious (20%) ahead of google::dense_hash_map. - * - * When using HashMap, AutoArray has a rather serious (40%) advantage over std::vector. - * And when using other hash tables, AutoArray even more seriously overtakes std::vector - * (up to three and a half times in the case of std::unordered_map and google::sparse_hash_map). - * * HashMap, unlike google::dense_hash_map, much more depends on the quality of the hash function. * * PS. Measure everything yourself, otherwise I'm almost confused. @@ -49,9 +35,6 @@ * But in this test, there was something similar to the old scenario of using hash tables in the aggregation. */ -#define USE_AUTO_ARRAY 0 - - struct AlternativeHash { size_t operator() (UInt64 x) const @@ -85,12 +68,7 @@ int main(int argc, char ** argv) using namespace DB; using Key = UInt64; - -#if USE_AUTO_ARRAY - using Value = AutoArray; -#else using Value = std::vector; -#endif size_t n = argc < 2 ? 10000000 : std::stol(argv[1]); //size_t m = std::stol(argv[2]); @@ -119,13 +97,8 @@ int main(int argc, char ** argv) INIT -#ifndef USE_AUTO_ARRAY #undef INIT #define INIT -#endif - - Row row(1); - row[0] = UInt64(0); std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl; diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp index 749b460a934..19ead09ae4b 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.cpp @@ -12,6 +12,7 @@ #include #include + namespace DB { @@ -109,6 +110,9 @@ static bool isInPartitionKey(const std::string & column_name, const Names & part return is_in_partition_key != partition_key_columns.end(); } + +using Row = std::vector; + /// Returns true if merge result is not empty static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, Row & row, const ColumnRawPtrs & raw_columns, size_t row_number) diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index df3749b5e88..3e03f5e9ae3 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -2,7 +2,7 @@ #include #include -#include + namespace DB { @@ -83,7 +83,7 @@ public: bool is_group_started = false; - Row current_row; + std::vector current_row; bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. void addRowImpl(ColumnRawPtrs & raw_columns, size_t row); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 38aeb15ebeb..5f12498c08c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -2,7 +2,6 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 5def458a382..1280138d38b 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -211,9 +211,12 @@ namespace virtual void insertStringColumn(const ColumnPtr & column, const String & name) = 0; virtual void insertUInt64Column(const ColumnPtr & column, const String & name) = 0; virtual void insertUUIDColumn(const ColumnPtr & column, const String & name) = 0; - virtual void - insertPartitionValueColumn(size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name) - = 0; + + virtual void insertPartitionValueColumn( + size_t rows, + const std::vector & partition_value, + const DataTypePtr & partition_value_type, + const String & name) = 0; }; } @@ -319,7 +322,7 @@ namespace } void insertPartitionValueColumn( - size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name) final + size_t rows, const std::vector & partition_value, const DataTypePtr & partition_value_type, const String & name) final { ColumnPtr column; if (rows) @@ -358,8 +361,8 @@ namespace columns.push_back(column); } - void - insertPartitionValueColumn(size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String &) final + void insertPartitionValueColumn( + size_t rows, const std::vector & partition_value, const DataTypePtr & partition_value_type, const String &) final { ColumnPtr column; if (rows) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 51077996212..e884ff7f211 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3144,7 +3144,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, ContextPtr loc ErrorCodes::INVALID_PARTITION_VALUE); const FormatSettings format_settings; - Row partition_row(fields_count); + std::vector partition_row(fields_count); if (fields_count) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index e311cf8fa28..473f2e598a9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -79,7 +79,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri new_data_part->uuid = uuid; new_data_part->setColumns(columns); - new_data_part->partition.value.assign(partition.value); + new_data_part->partition.value = partition.value; new_data_part->minmax_idx = minmax_idx; if (disk->exists(destination_path)) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 2a85faa7083..33d0291946c 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -150,7 +150,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned. { - result.emplace_back(Block(block), Row()); + result.emplace_back(Block(block), std::vector{}); return result; } @@ -172,7 +172,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block auto get_partition = [&](size_t num) { - Row partition(partition_columns.size()); + std::vector partition(partition_columns.size()); for (size_t i = 0; i < partition_columns.size(); ++i) partition[i] = Field((*partition_columns[i])[partition_num_to_first_row[num]]); return partition; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 4c5b75657ee..f5eb0cd98e9 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -19,9 +18,9 @@ namespace DB struct BlockWithPartition { Block block; - Row partition; + std::vector partition; - BlockWithPartition(Block && block_, Row && partition_) + BlockWithPartition(Block && block_, std::vector && partition_) : block(block_), partition(std::move(partition_)) { } diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index f89b6f22d4f..c55eb8ce142 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -1,9 +1,10 @@ #pragma once -#include #include #include #include +#include + namespace DB { @@ -19,12 +20,12 @@ using StorageMetadataPtr = std::shared_ptr; /// This class represents a partition value of a single part and encapsulates its loading/storing logic. struct MergeTreePartition { - Row value; + std::vector value; public: MergeTreePartition() = default; - explicit MergeTreePartition(Row value_) : value(std::move(value_)) {} + explicit MergeTreePartition(std::vector value_) : value(std::move(value_)) {} /// For month-based partitioning. explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) {} @@ -38,7 +39,7 @@ public: void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; void store(const Block & partition_key_sample, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; - void assign(const MergeTreePartition & other) { value.assign(other.value); } + void assign(const MergeTreePartition & other) { value = other.value; } void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row); };