diff --git a/src/Core/Field.h b/src/Core/Field.h index fb9f811ddf3..9c970fbbb31 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -721,6 +721,9 @@ private: #undef DBMS_MIN_FIELD_SIZE +using Row = std::vector; + + template <> struct Field::TypeToEnum { static const Types::Which value = Types::Null; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::UInt64; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::UInt128; }; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index d415bdda70d..cd24979caaa 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -452,7 +452,7 @@ namespace MySQLReplication UInt32 number_columns; String schema; String table; - std::vector rows; + Row rows; RowsEvent(std::shared_ptr table_map_, EventHeader && header_, const RowsEventHeader & rows_header) : EventBase(std::move(header_)), number_columns(0), table_map(table_map_) diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 63a8e9edb79..9ef0a45acc0 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -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; const auto parse_value = array_info[idx].pqxx_parser; - std::vector> dimensions(expected_dimensions + 1); + std::vector dimensions(expected_dimensions + 1); while (parsed.first != pqxx::array_parser::juncture::done) { diff --git a/src/DataStreams/TTLAggregationAlgorithm.h b/src/DataStreams/TTLAggregationAlgorithm.h index 7231506b7da..c2f40bab6b9 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.h +++ b/src/DataStreams/TTLAggregationAlgorithm.h @@ -32,7 +32,7 @@ private: const Block header; std::unique_ptr aggregator; - std::vector current_key_value; + Row current_key_value; AggregatedDataVariants aggregation_result; ColumnRawPtrs key_columns; Aggregator::AggregateColumns columns_for_aggregator; diff --git a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp index fff4739eff9..a7ebcb0d3f2 100644 --- a/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializeMySQLSyncThread.cpp @@ -477,7 +477,7 @@ static inline void fillSignAndVersionColumnsData(Block & data, Int8 sign_value, template static void writeFieldsToColumn( - IColumn & column_to, const std::vector & rows_data, size_t column_index, const std::vector & mask, ColumnUInt8 * null_map_column = nullptr) + IColumn & column_to, const Row & rows_data, size_t column_index, const std::vector & mask, ColumnUInt8 * null_map_column = nullptr) { if (ColumnNullable * column_nullable = typeid_cast(&column_to)) writeFieldsToColumn(column_nullable->getNestedColumn(), rows_data, column_index, mask, &column_nullable->getNullMapColumn()); @@ -599,7 +599,7 @@ static void writeFieldsToColumn( } template -static size_t onWriteOrDeleteData(const std::vector & 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(); 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; } -static inline size_t onUpdateData(const std::vector & rows_data, Block & buffer, size_t version, const std::vector & sorting_columns_index) +static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t version, const std::vector & sorting_columns_index) { if (rows_data.size() % 2 != 0) throw Exception("LOGICAL ERROR: It is a bug.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Dictionaries/CacheDictionaryStorage.h b/src/Dictionaries/CacheDictionaryStorage.h index a79d83eea05..07217369dc1 100644 --- a/src/Dictionaries/CacheDictionaryStorage.h +++ b/src/Dictionaries/CacheDictionaryStorage.h @@ -232,7 +232,7 @@ private: if (unlikely(attribute.is_complex_type)) { - auto & container = std::get>(attribute.attribute_container); + auto & container = std::get(attribute.attribute_container); for (size_t fetched_key_index = 0; fetched_key_index < fetched_columns_index; ++fetched_key_index) { @@ -506,7 +506,7 @@ private: if (unlikely(attribute.is_complex_type)) { - auto & container = std::get>(attribute.attribute_container); + auto & container = std::get(attribute.attribute_container); std::forward(func)(container); } else @@ -564,7 +564,7 @@ private: last_attribute.is_complex_type = dictionary_attribute.is_nullable || dictionary_attribute.is_array; if (dictionary_attribute.is_nullable) - last_attribute.attribute_container = std::vector(); + last_attribute.attribute_container = Row(); else last_attribute.attribute_container = PaddedPODArray(); }; @@ -609,7 +609,7 @@ private: PaddedPODArray, PaddedPODArray, PaddedPODArray, - std::vector> attribute_container; + Row> attribute_container; }; CacheDictionaryStorageConfiguration configuration; diff --git a/src/Functions/partitionId.cpp b/src/Functions/partitionId.cpp index 70ece96f983..cf679452da1 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) { - std::vector row(size); + Row 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 2c66d2a9841..a65e9337ecd 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -79,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(); - std::vector tuple_values; + Row tuple_values; for (const auto & value : collection) { if (columns_num == 1) @@ -153,7 +153,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co MutableColumns columns = header.cloneEmptyColumns(); DataTypePtr tuple_type; - std::vector tuple_values; + Row 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/FillingRow.h b/src/Interpreters/FillingRow.h index 0e1d60d0d7a..434a9270718 100644 --- a/src/Interpreters/FillingRow.h +++ b/src/Interpreters/FillingRow.h @@ -34,7 +34,7 @@ public: FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } private: - std::vector row; + Row row; SortDescription description; }; diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp index 95ee42b4d09..cd576916364 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.cpp @@ -344,7 +344,7 @@ AvroDeserializer::DeserializeFn AvroDeserializer::createDeserializeFn(avro::Node if (target.isEnum()) { const auto & enum_type = dynamic_cast(*target_type); - std::vector symbol_mapping; + Row symbol_mapping; for (size_t i = 0; i < root_node->names(); i++) { symbol_mapping.push_back(enum_type.castToValue(root_node->nameAt(i))); diff --git a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h index 3e03f5e9ae3..c77bf7c0ba5 100644 --- a/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/SummingSortedAlgorithm.h @@ -83,7 +83,7 @@ public: bool is_group_started = false; - std::vector current_row; + Row 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/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 1280138d38b..d9cb949042c 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -214,7 +214,7 @@ namespace virtual void insertPartitionValueColumn( size_t rows, - const std::vector & partition_value, + const Row & partition_value, const DataTypePtr & partition_value_type, const String & name) = 0; }; @@ -322,7 +322,7 @@ namespace } void insertPartitionValueColumn( - size_t rows, const std::vector & partition_value, const DataTypePtr & partition_value_type, const String & name) final + size_t rows, const Row & partition_value, const DataTypePtr & partition_value_type, const String & name) final { ColumnPtr column; if (rows) @@ -362,7 +362,7 @@ namespace } void insertPartitionValueColumn( - size_t rows, const std::vector & 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; if (rows) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e884ff7f211..51077996212 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; - std::vector partition_row(fields_count); + Row partition_row(fields_count); if (fields_count) { diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 33d0291946c..b875b3b2785 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), std::vector{}); + result.emplace_back(Block(block), Row{}); return result; } @@ -172,7 +172,7 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block auto get_partition = [&](size_t num) { - std::vector partition(partition_columns.size()); + Row 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 f5eb0cd98e9..8b7bb9d9165 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -18,9 +18,9 @@ namespace DB struct BlockWithPartition { Block block; - std::vector partition; + Row partition; - BlockWithPartition(Block && block_, std::vector && partition_) + BlockWithPartition(Block && block_, Row && partition_) : block(block_), partition(std::move(partition_)) { } diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index c55eb8ce142..f8ce4b60a34 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -20,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 { - std::vector value; + Row value; public: MergeTreePartition() = default; - explicit MergeTreePartition(std::vector value_) : value(std::move(value_)) {} + explicit MergeTreePartition(Row value_) : value(std::move(value_)) {} /// For month-based partitioning. explicit MergeTreePartition(UInt32 yyyymm) : value(1, yyyymm) {}