Merge pull request #24320 from ClickHouse/remove-auto-array

Remove AutoArray
This commit is contained in:
alexey-milovidov 2021-05-22 02:19:59 +03:00 committed by GitHub
commit a25f1910cc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 28 additions and 576 deletions

View File

@ -1,309 +0,0 @@
#pragma once
#include <cstddef>
#include <cstdlib>
#include <Common/Exception.h>
#include <Common/formatReadable.h>
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 <typename T>
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 <typename It>
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<T> & 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<T> & rhs) const
{
return !(*this == rhs);
}
bool operator< (const AutoArray<T> & 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<size_t *>(data_ptr)[-1];
}
size_t m_size() const
{
return reinterpret_cast<const size_t *>(data_ptr)[-1];
}
T * elemPtr(size_t i)
{
return reinterpret_cast<T *>(data_ptr) + i;
}
const T * elemPtr(size_t i) const
{
return reinterpret_cast<const T *>(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<char *>(reinterpret_cast<const char *>(&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<char *>(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);
}
}
};
}

View File

@ -7,9 +7,6 @@ endif()
add_executable (sip_hash_perf sip_hash_perf.cpp) add_executable (sip_hash_perf sip_hash_perf.cpp)
target_link_libraries (sip_hash_perf PRIVATE clickhouse_common_io) 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) add_executable (small_table small_table.cpp)
target_link_libraries (small_table PRIVATE clickhouse_common_io) target_link_libraries (small_table PRIVATE clickhouse_common_io)

View File

@ -1,197 +0,0 @@
#include <iostream>
#include <iomanip>
#include <map>
#include <pcg_random.hpp>
#include <Core/Field.h>
#include <Common/HashTable/HashMap.h>
#include <Common/AutoArray.h>
#include <IO/WriteHelpers.h>
#include <Common/Stopwatch.h>
int main(int argc, char ** argv)
{
pcg64 rng;
{
size_t n = 10;
using T = std::string;
DB::AutoArray<T> 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<T>;
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<T>;
using Map = std::map<Arr, T>;
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<T>;
using Vec = std::vector<Arr>;
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<T>;
using Map = HashMap<UInt64, Arr>;
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<std::string>;
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;
}

View File

@ -721,6 +721,9 @@ private:
#undef DBMS_MIN_FIELD_SIZE #undef DBMS_MIN_FIELD_SIZE
using Row = std::vector<Field>;
template <> struct Field::TypeToEnum<Null> { static const Types::Which value = Types::Null; }; template <> struct Field::TypeToEnum<Null> { static const Types::Which value = Types::Null; };
template <> struct Field::TypeToEnum<UInt64> { static const Types::Which value = Types::UInt64; }; template <> struct Field::TypeToEnum<UInt64> { static const Types::Which value = Types::UInt64; };
template <> struct Field::TypeToEnum<UInt128> { static const Types::Which value = Types::UInt128; }; template <> struct Field::TypeToEnum<UInt128> { static const Types::Which value = Types::UInt128; };

View File

@ -452,7 +452,7 @@ namespace MySQLReplication
UInt32 number_columns; UInt32 number_columns;
String schema; String schema;
String table; String table;
std::vector<Field> rows; Row rows;
RowsEvent(std::shared_ptr<TableMapEvent> table_map_, EventHeader && header_, const RowsEventHeader & rows_header) RowsEvent(std::shared_ptr<TableMapEvent> table_map_, EventHeader && header_, const RowsEventHeader & rows_header)
: EventBase(std::move(header_)), number_columns(0), table_map(table_map_) : EventBase(std::move(header_)), number_columns(0), table_map(table_map_)

View File

@ -1,18 +0,0 @@
#pragma once
#include <vector>
#include <Common/AutoArray.h>
#include <Core/Field.h>
namespace DB
{
/** The data type for representing one row of the table in the RAM.
* Warning! It is preferable to store column blocks instead of single rows. See Block.h
*/
using Row = AutoArray<Field>;
}

View File

@ -1,6 +1,5 @@
#pragma once #pragma once
#include <Core/Row.h>
#include <Core/SortDescription.h> #include <Core/SortDescription.h>
#include <Core/SortCursor.h> #include <Core/SortCursor.h>

View File

@ -193,7 +193,7 @@ void PostgreSQLBlockInputStream::insertValue(IColumn & column, std::string_view
size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions; size_t dimension = 0, max_dimension = 0, expected_dimensions = array_info[idx].num_dimensions;
const auto parse_value = array_info[idx].pqxx_parser; const auto parse_value = array_info[idx].pqxx_parser;
std::vector<std::vector<Field>> dimensions(expected_dimensions + 1); std::vector<Row> dimensions(expected_dimensions + 1);
while (parsed.first != pqxx::array_parser::juncture::done) while (parsed.first != pqxx::array_parser::juncture::done)
{ {

View File

@ -477,7 +477,7 @@ static inline void fillSignAndVersionColumnsData(Block & data, Int8 sign_value,
template <bool assert_nullable = false> template <bool assert_nullable = false>
static void writeFieldsToColumn( static void writeFieldsToColumn(
IColumn & column_to, const std::vector<Field> & rows_data, size_t column_index, const std::vector<bool> & mask, ColumnUInt8 * null_map_column = nullptr) IColumn & column_to, const Row & rows_data, size_t column_index, const std::vector<bool> & mask, ColumnUInt8 * null_map_column = nullptr)
{ {
if (ColumnNullable * column_nullable = typeid_cast<ColumnNullable *>(&column_to)) if (ColumnNullable * column_nullable = typeid_cast<ColumnNullable *>(&column_to))
writeFieldsToColumn<true>(column_nullable->getNestedColumn(), rows_data, column_index, mask, &column_nullable->getNullMapColumn()); writeFieldsToColumn<true>(column_nullable->getNestedColumn(), rows_data, column_index, mask, &column_nullable->getNullMapColumn());
@ -599,7 +599,7 @@ static void writeFieldsToColumn(
} }
template <Int8 sign> template <Int8 sign>
static size_t onWriteOrDeleteData(const std::vector<Field> & rows_data, Block & buffer, size_t version) static size_t onWriteOrDeleteData(const Row & rows_data, Block & buffer, size_t version)
{ {
size_t prev_bytes = buffer.bytes(); size_t prev_bytes = buffer.bytes();
for (size_t column = 0; column < buffer.columns() - 2; ++column) for (size_t column = 0; column < buffer.columns() - 2; ++column)
@ -623,7 +623,7 @@ static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple
return false; return false;
} }
static inline size_t onUpdateData(const std::vector<Field> & rows_data, Block & buffer, size_t version, const std::vector<size_t> & sorting_columns_index) static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t version, const std::vector<size_t> & sorting_columns_index)
{ {
if (rows_data.size() % 2 != 0) if (rows_data.size() % 2 != 0)
throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR);

View File

@ -1,6 +1,5 @@
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Core/Row.h>
#include <Functions/FunctionFactory.h> #include <Functions/FunctionFactory.h>
#include <Functions/FunctionsMiscellaneous.h> #include <Functions/FunctionsMiscellaneous.h>

View File

@ -34,7 +34,7 @@ public:
FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; }
private: private:
std::vector<Field> row; Row row;
SortDescription description; SortDescription description;
}; };

View File

@ -1,7 +1,6 @@
#include <optional> #include <optional>
#include <Core/Field.h> #include <Core/Field.h>
#include <Core/Row.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Columns/ColumnTuple.h> #include <Columns/ColumnTuple.h>

View File

@ -12,7 +12,6 @@
#define DBMS_HASH_MAP_COUNT_COLLISIONS #define DBMS_HASH_MAP_COUNT_COLLISIONS
*/ */
#include <common/types.h> #include <common/types.h>
#include <Core/Row.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <Compression/CompressedReadBuffer.h> #include <Compression/CompressedReadBuffer.h>
#include <Common/HashTable/HashMap.h> #include <Common/HashTable/HashMap.h>
@ -27,19 +26,6 @@
* This is important, because if you run all the tests one by one, the results will be incorrect. * 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.) * (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. * 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. * 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. * 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 struct AlternativeHash
{ {
size_t operator() (UInt64 x) const size_t operator() (UInt64 x) const
@ -85,12 +68,7 @@ int main(int argc, char ** argv)
using namespace DB; using namespace DB;
using Key = UInt64; using Key = UInt64;
#if USE_AUTO_ARRAY
using Value = AutoArray<IAggregateFunction*>;
#else
using Value = std::vector<IAggregateFunction*>; using Value = std::vector<IAggregateFunction*>;
#endif
size_t n = argc < 2 ? 10000000 : std::stol(argv[1]); size_t n = argc < 2 ? 10000000 : std::stol(argv[1]);
//size_t m = std::stol(argv[2]); //size_t m = std::stol(argv[2]);
@ -119,13 +97,8 @@ int main(int argc, char ** argv)
INIT INIT
#ifndef USE_AUTO_ARRAY
#undef INIT #undef INIT
#define INIT #define INIT
#endif
Row row(1);
row[0] = UInt64(0);
std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl; std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl;

View File

@ -344,7 +344,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node
if (target.isEnum()) if (target.isEnum())
{ {
const auto & enum_type = dynamic_cast<const IDataTypeEnum &>(*target_type); const auto & enum_type = dynamic_cast<const IDataTypeEnum &>(*target_type);
std::vector<Field> symbol_mapping; 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))); symbol_mapping.push_back(enum_type.castToValue(root_node->nameAt(i)));

View File

@ -12,6 +12,7 @@
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
namespace DB 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(); return is_in_partition_key != partition_key_columns.end();
} }
using Row = std::vector<Field>;
/// Returns true if merge result is not empty /// Returns true if merge result is not empty
static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc, static bool mergeMap(const SummingSortedAlgorithm::MapDescription & desc,
Row & row, const ColumnRawPtrs & raw_columns, size_t row_number) Row & row, const ColumnRawPtrs & raw_columns, size_t row_number)

View File

@ -2,7 +2,7 @@
#include <Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h> #include <Processors/Merges/Algorithms/IMergingAlgorithmWithDelayedChunk.h>
#include <Processors/Merges/Algorithms/MergedData.h> #include <Processors/Merges/Algorithms/MergedData.h>
#include <Core/Row.h>
namespace DB namespace DB
{ {

View File

@ -2,7 +2,6 @@
#include <DataStreams/IBlockInputStream.h> #include <DataStreams/IBlockInputStream.h>
#include <Core/Row.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <common/types.h> #include <common/types.h>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>

View File

@ -211,9 +211,12 @@ namespace
virtual void insertStringColumn(const ColumnPtr & column, const String & name) = 0; virtual void insertStringColumn(const ColumnPtr & column, const String & name) = 0;
virtual void insertUInt64Column(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 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) virtual void insertPartitionValueColumn(
= 0; size_t rows,
const Row & partition_value,
const DataTypePtr & partition_value_type,
const String & name) = 0;
}; };
} }
@ -358,8 +361,8 @@ namespace
columns.push_back(column); columns.push_back(column);
} }
void void insertPartitionValueColumn(
insertPartitionValueColumn(size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String &) final size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String &) final
{ {
ColumnPtr column; ColumnPtr column;
if (rows) if (rows)

View File

@ -79,7 +79,7 @@ void MergeTreeDataPartInMemory::flushToDisk(const String & base_path, const Stri
new_data_part->uuid = uuid; new_data_part->uuid = uuid;
new_data_part->setColumns(columns); 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; new_data_part->minmax_idx = minmax_idx;
if (disk->exists(destination_path)) if (disk->exists(destination_path))

View File

@ -150,7 +150,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block
if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned. if (!metadata_snapshot->hasPartitionKey()) /// Table is not partitioned.
{ {
result.emplace_back(Block(block), Row()); result.emplace_back(Block(block), Row{});
return result; return result;
} }

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include <Core/Block.h> #include <Core/Block.h>
#include <Core/Row.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
#include <Compression/CompressedWriteBuffer.h> #include <Compression/CompressedWriteBuffer.h>

View File

@ -1,9 +1,10 @@
#pragma once #pragma once
#include <Core/Row.h>
#include <common/types.h> #include <common/types.h>
#include <Disks/IDisk.h> #include <Disks/IDisk.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <Core/Field.h>
namespace DB namespace DB
{ {
@ -38,7 +39,7 @@ public:
void store(const MergeTreeData & storage, const DiskPtr & disk, const String & part_path, MergeTreeDataPartChecksums & checksums) const; 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 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); void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row);
}; };