diff --git a/.clang-format b/.clang-format index 988c91c8c76..6af78340db1 100644 --- a/.clang-format +++ b/.clang-format @@ -50,7 +50,8 @@ IncludeCategories: - Regex: '.*' Priority: 40 ReflowComments: false -AlignEscapedNewlinesLeft: true +AlignEscapedNewlinesLeft: false +AlignEscapedNewlines: DontAlign # Not changed: AccessModifierOffset: -4 diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 276e0db293c..e623fca4204 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -1,5 +1,5 @@ # This strings autochanged from release_lib.sh: -set(VERSION_REVISION 54409 CACHE STRING "") +set(VERSION_REVISION 54410 CACHE STRING "") # changed manually for tests set(VERSION_MAJOR 18 CACHE STRING "") set(VERSION_MINOR 14 CACHE STRING "") set(VERSION_PATCH 17 CACHE STRING "") diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 1b7e48ac55f..e81baf891a3 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -43,6 +43,7 @@ #include #include #include +#include #include #include #include @@ -60,6 +61,7 @@ #include #include #include +#include #if USE_READLINE #include "Suggest.h" // Y_IGNORE @@ -69,7 +71,6 @@ #pragma GCC optimize("-fno-var-tracking-assignments") #endif - /// http://en.wikipedia.org/wiki/ANSI_escape_code /// Similar codes \e[s, \e[u don't work in VT100 and Mosh. @@ -875,11 +876,12 @@ private: /// Receive description of table structure. Block sample; - if (receiveSampleBlock(sample)) + ColumnsDescription columns_description; + if (receiveSampleBlock(sample, columns_description)) { /// If structure was received (thus, server has not thrown an exception), /// send our data with that structure. - sendData(sample); + sendData(sample, columns_description); receiveEndOfQuery(); } } @@ -917,7 +919,7 @@ private: } - void sendData(Block & sample) + void sendData(Block & sample, const ColumnsDescription & columns_description) { /// If INSERT data must be sent. const ASTInsertQuery * parsed_insert_query = typeid_cast(&*parsed_query); @@ -928,19 +930,19 @@ private: { /// Send data contained in the query. ReadBufferFromMemory data_in(parsed_insert_query->data, parsed_insert_query->end - parsed_insert_query->data); - sendDataFrom(data_in, sample); + sendDataFrom(data_in, sample, columns_description); } else if (!is_interactive) { /// Send data read from stdin. - sendDataFrom(std_in, sample); + sendDataFrom(std_in, sample, columns_description); } else throw Exception("No data to insert", ErrorCodes::NO_DATA_TO_INSERT); } - void sendDataFrom(ReadBuffer & buf, Block & sample) + void sendDataFrom(ReadBuffer & buf, Block & sample, const ColumnsDescription & columns_description) { String current_format = insert_format; @@ -952,6 +954,10 @@ private: BlockInputStreamPtr block_input = context.getInputFormat( current_format, buf, sample, insert_format_max_block_size); + const auto & column_defaults = columns_description.defaults; + if (!column_defaults.empty()) + block_input = std::make_shared(block_input, column_defaults, context); + BlockInputStreamPtr async_block_input = std::make_shared(block_input); async_block_input->readPrefix(); @@ -1089,7 +1095,7 @@ private: /// Receive the block that serves as an example of the structure of table where data will be inserted. - bool receiveSampleBlock(Block & out) + bool receiveSampleBlock(Block & out, ColumnsDescription & columns_description) { while (true) { @@ -1110,6 +1116,10 @@ private: onLogData(packet.block); break; + case Protocol::Server::TableColumns: + columns_description = ColumnsDescription::parse(packet.multistring_message[1]); + return receiveSampleBlock(out, columns_description); + default: throw NetException("Unexpected packet from server (expected Data, Exception or Log, got " + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 108a630a0dd..cfb0cd3cd58 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -30,6 +30,7 @@ #include #include #include +#include #include #include "TCPHandler.h" @@ -360,6 +361,14 @@ void TCPHandler::processInsertQuery(const Settings & global_settings) */ state.io.out->writePrefix(); + /// Send ColumnsDescription for insertion table + if (client_revision >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA) + { + const auto & db_and_table = query_context.getInsertionTable(); + if (auto * columns = ColumnsDescription::loadFromContext(query_context, db_and_table.first, db_and_table.second)) + sendTableColumns(*columns); + } + /// Send block to the client - table structure. Block block = state.io.out->getHeader(); @@ -389,6 +398,17 @@ void TCPHandler::processOrdinaryQuery() /// Send header-block, to allow client to prepare output format for data to send. { Block header = state.io.in->getHeader(); + + /// Send data to old clients without low cardinality type. + if (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) + { + for (auto & column : header) + { + column.column = recursiveRemoveLowCardinality(column.column); + column.type = recursiveRemoveLowCardinality(column.type); + } + } + if (header) sendData(header); } @@ -860,6 +880,16 @@ void TCPHandler::sendLogData(const Block & block) out->next(); } +void TCPHandler::sendTableColumns(const ColumnsDescription & columns) +{ + writeVarUInt(Protocol::Server::TableColumns, *out); + + /// Send external table name (empty name is the main table) + writeStringBinary("", *out); + writeStringBinary(columns.toString(), *out); + + out->next(); +} void TCPHandler::sendException(const Exception & e, bool with_stack_trace) { diff --git a/dbms/programs/server/TCPHandler.h b/dbms/programs/server/TCPHandler.h index 098019e86da..43c544cc401 100644 --- a/dbms/programs/server/TCPHandler.h +++ b/dbms/programs/server/TCPHandler.h @@ -144,6 +144,7 @@ private: void sendHello(); void sendData(const Block & block); /// Write a block to the network. void sendLogData(const Block & block); + void sendTableColumns(const ColumnsDescription & columns); void sendException(const Exception & e, bool with_stack_trace); void sendProgress(); void sendLogs(); diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 66bc04dcaf4..923c8179ca1 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -603,6 +603,10 @@ Connection::Packet Connection::receivePacket() res.block = receiveLogData(); return res; + case Protocol::Server::TableColumns: + res.multistring_message = receiveMultistringMessage(res.type); + return res; + case Protocol::Server::EndOfStream: return res; @@ -712,6 +716,16 @@ std::unique_ptr Connection::receiveException() } +std::vector Connection::receiveMultistringMessage(UInt64 msg_type) +{ + size_t num = Protocol::Server::stringsInMessage(msg_type); + std::vector out(num); + for (size_t i = 0; i < num; ++i) + readStringBinary(out[i], *in); + return out; +} + + Progress Connection::receiveProgress() { //LOG_TRACE(log_wrapper.get(), "Receiving progress"); diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index e3553e2a5cb..2c72e7e0a43 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include @@ -96,6 +98,7 @@ public: Block block; std::unique_ptr exception; + std::vector multistring_message; Progress progress; BlockStreamProfileInfo profile_info; @@ -254,6 +257,7 @@ private: Block receiveLogData(); Block receiveDataImpl(BlockInputStreamPtr & stream); + std::vector receiveMultistringMessage(UInt64 msg_type); std::unique_ptr receiveException(); Progress receiveProgress(); BlockStreamProfileInfo receiveProfileInfo(); diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index b78ce798ba2..2bbc0f6ad9e 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -142,7 +142,7 @@ struct HashTableCell /// Deserialization, in binary and text form. void read(DB::ReadBuffer & rb) { DB::readBinary(key, rb); } - void readText(DB::ReadBuffer & rb) { DB::writeDoubleQuoted(key, rb); } + void readText(DB::ReadBuffer & rb) { DB::readDoubleQuoted(key, rb); } }; diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 716de954974..de0e831f37c 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -99,6 +99,13 @@ void Block::insertUnique(ColumnWithTypeAndName && elem) } +void Block::erase(const std::set & positions) +{ + for (auto it = positions.rbegin(); it != positions.rend(); ++it) + erase(*it); +} + + void Block::erase(size_t position) { if (data.empty()) diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 690f9720af1..a3198a0fb74 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -51,6 +52,8 @@ public: void insertUnique(ColumnWithTypeAndName && elem); /// remove the column at the specified position void erase(size_t position); + /// remove the columns at the specified positions + void erase(const std::set & positions); /// remove the column with the specified name void erase(const String & name); diff --git a/dbms/src/Core/BlockInfo.cpp b/dbms/src/Core/BlockInfo.cpp index 77ef2e01007..aae9723d0ed 100644 --- a/dbms/src/Core/BlockInfo.cpp +++ b/dbms/src/Core/BlockInfo.cpp @@ -58,4 +58,20 @@ void BlockInfo::read(ReadBuffer & in) } } +void BlockMissingValues::setBit(size_t column_idx, size_t row_idx) +{ + RowsBitMask & mask = rows_mask_by_column_id[column_idx]; + mask.resize(row_idx + 1); + mask[row_idx] = true; +} + +const BlockMissingValues::RowsBitMask & BlockMissingValues::getDefaultsBitmask(size_t column_idx) const +{ + static RowsBitMask none; + auto it = rows_mask_by_column_id.find(column_idx); + if (it != rows_mask_by_column_id.end()) + return it->second; + return none; +} + } diff --git a/dbms/src/Core/BlockInfo.h b/dbms/src/Core/BlockInfo.h index ebfbd117de7..32a09d8cf70 100644 --- a/dbms/src/Core/BlockInfo.h +++ b/dbms/src/Core/BlockInfo.h @@ -1,5 +1,7 @@ #pragma once +#include + #include @@ -43,4 +45,24 @@ struct BlockInfo void read(ReadBuffer & in); }; +/// Block extention to support delayed defaults. AddingDefaultsBlockInputStream uses it to replace missing values with column defaults. +class BlockMissingValues +{ +public: + using RowsBitMask = std::vector; /// a bit per row for a column + + const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; + void setBit(size_t column_idx, size_t row_idx); + bool empty() const { return rows_mask_by_column_id.empty(); } + size_t size() const { return rows_mask_by_column_id.size(); } + void clear() { rows_mask_by_column_id.clear(); } + +private: + using RowsMaskByColumnId = std::unordered_map; + + /// If rows_mask_by_column_id[column_id][row_id] is true related value in Block should be replaced with column default. + /// It could contain less columns and rows then related block. + RowsMaskByColumnId rows_mask_by_column_id; +}; + } diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 4b99899213e..661dafeb0ee 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -51,6 +51,7 @@ /// Two-level (bucketed) aggregation is incompatible if servers are inconsistent in these rules /// (keys will be placed in different buckets and result will not be fully aggregated). #define DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD 54408 +#define DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA 54410 #define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405 diff --git a/dbms/src/Core/Protocol.h b/dbms/src/Core/Protocol.h index 5451e1550f6..b50d018f9ce 100644 --- a/dbms/src/Core/Protocol.h +++ b/dbms/src/Core/Protocol.h @@ -69,7 +69,8 @@ namespace Protocol Totals = 7, /// A block with totals (compressed or not). Extremes = 8, /// A block with minimums and maximums (compressed or not). TablesStatusResponse = 9, /// A response to TablesStatus request. - Log = 10 /// System logs of the query execution + Log = 10, /// System logs of the query execution + TableColumns = 11, /// Columns' description for default values calculation }; /// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10 @@ -78,11 +79,24 @@ namespace Protocol /// See https://www.securecoding.cert.org/confluence/display/cplusplus/INT36-CPP.+Do+not+use+out-of-range+enumeration+values inline const char * toString(UInt64 packet) { - static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes", "TablesStatusResponse", "Log" }; - return packet < 11 + static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", + "Extremes", "TablesStatusResponse", "Log", "TableColumns" }; + return packet < 12 ? data[packet] : "Unknown packet"; } + + inline size_t stringsInMessage(UInt64 msg_type) + { + switch (msg_type) + { + case TableColumns: + return 2; + default: + break; + } + return 0; + } } /// Packet types that client transmits. @@ -103,8 +117,8 @@ namespace Protocol inline const char * toString(UInt64 packet) { - static const char * data[] = { "Hello", "Query", "Data", "Cancel", "Ping", "TablesStatusRequest" }; - return packet < 6 + static const char * data[] = { "Hello", "Query", "Data", "Cancel", "Ping", "TablesStatusRequest", "KeepAlive" }; + return packet < 7 ? data[packet] : "Unknown packet"; } diff --git a/dbms/src/DataStreams/AddingDefaultsBlockInputStream.cpp b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.cpp new file mode 100644 index 00000000000..6b959cbc05b --- /dev/null +++ b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.cpp @@ -0,0 +1,205 @@ +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int TYPE_MISMATCH; +} + + +AddingDefaultsBlockInputStream::AddingDefaultsBlockInputStream(const BlockInputStreamPtr & input, + const ColumnDefaults & column_defaults_, + const Context & context_) + : column_defaults(column_defaults_), + context(context_) +{ + children.push_back(input); + header = input->getHeader(); +} + + +Block AddingDefaultsBlockInputStream::readImpl() +{ + Block res = children.back()->read(); + if (!res) + return res; + + if (column_defaults.empty()) + return res; + + const BlockMissingValues & block_missing_values = children.back()->getMissingValues(); + if (block_missing_values.empty()) + return res; + + Block evaluate_block{res}; + /// remove columns for recalculation + for (const auto & column : column_defaults) + if (evaluate_block.has(column.first)) + evaluate_block.erase(column.first); + + evaluateMissingDefaults(evaluate_block, header.getNamesAndTypesList(), column_defaults, context, false); + + std::unordered_map mixed_columns; + + for (const ColumnWithTypeAndName & column_def : evaluate_block) + { + const String & column_name = column_def.name; + + if (column_defaults.count(column_name) == 0) + continue; + + size_t block_column_position = res.getPositionByName(column_name); + ColumnWithTypeAndName & column_read = res.getByPosition(block_column_position); + const auto & defaults_mask = block_missing_values.getDefaultsBitmask(block_column_position); + + checkCalculated(column_read, column_def, defaults_mask.size()); + + if (!defaults_mask.empty()) + { + /// TODO: FixedString + if (isColumnedAsNumber(column_read.type) || isDecimal(column_read.type)) + { + MutableColumnPtr column_mixed = (*std::move(column_read.column)).mutate(); + mixNumberColumns(column_read.type->getTypeId(), column_mixed, column_def.column, defaults_mask); + column_read.column = std::move(column_mixed); + } + else + { + MutableColumnPtr column_mixed = mixColumns(column_read, column_def, defaults_mask); + mixed_columns.emplace(block_column_position, std::move(column_mixed)); + } + } + } + + if (!mixed_columns.empty()) + { + /// replace columns saving block structure + MutableColumns mutation = res.mutateColumns(); + for (size_t position = 0; position < mutation.size(); ++position) + { + auto it = mixed_columns.find(position); + if (it != mixed_columns.end()) + mutation[position] = std::move(it->second); + } + res.setColumns(std::move(mutation)); + } + + return res; +} + +void AddingDefaultsBlockInputStream::checkCalculated(const ColumnWithTypeAndName & col_read, + const ColumnWithTypeAndName & col_defaults, + size_t defaults_needed) const +{ + size_t column_size = col_read.column->size(); + + if (column_size != col_defaults.column->size()) + throw Exception("Mismatch column sizes while adding defaults", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + if (column_size < defaults_needed) + throw Exception("Unexpected defaults count", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + + if (!col_read.type->equals(*col_defaults.type)) + throw Exception("Mismach column types while adding defaults", ErrorCodes::TYPE_MISMATCH); +} + +void AddingDefaultsBlockInputStream::mixNumberColumns(TypeIndex type_idx, MutableColumnPtr & column_mixed, const ColumnPtr & column_defs, + const BlockMissingValues::RowsBitMask & defaults_mask) const +{ + auto call = [&](const auto & types) -> bool + { + using Types = std::decay_t; + using DataType = typename Types::LeftType; + + if constexpr (!std::is_same_v && !std::is_same_v) + { + using FieldType = typename DataType::FieldType; + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + + auto col_read = typeid_cast(column_mixed.get()); + if (!col_read) + return false; + + typename ColVecType::Container & dst = col_read->getData(); + + if (auto const_col_defs = checkAndGetColumnConst(column_defs.get())) + { + FieldType value = checkAndGetColumn(const_col_defs->getDataColumnPtr().get())->getData()[0]; + + for (size_t i = 0; i < defaults_mask.size(); ++i) + if (defaults_mask[i]) + dst[i] = value; + + return true; + } + else if (auto col_defs = checkAndGetColumn(column_defs.get())) + { + auto & src = col_defs->getData(); + for (size_t i = 0; i < defaults_mask.size(); ++i) + if (defaults_mask[i]) + dst[i] = src[i]; + + return true; + } + } + + return false; + }; + + if (!callOnIndexAndDataType(type_idx, call)) + throw Exception("Unexpected type on mixNumberColumns", ErrorCodes::LOGICAL_ERROR); +} + +MutableColumnPtr AddingDefaultsBlockInputStream::mixColumns(const ColumnWithTypeAndName & col_read, + const ColumnWithTypeAndName & col_defaults, + const BlockMissingValues::RowsBitMask & defaults_mask) const +{ + size_t column_size = col_read.column->size(); + size_t defaults_needed = defaults_mask.size(); + + MutableColumnPtr column_mixed = col_read.column->cloneEmpty(); + + for (size_t i = 0; i < defaults_needed; ++i) + { + if (defaults_mask[i]) + { + if (col_defaults.column->isColumnConst()) + column_mixed->insert((*col_defaults.column)[i]); + else + column_mixed->insertFrom(*col_defaults.column, i); + } + else + column_mixed->insertFrom(*col_read.column, i); + } + + for (size_t i = defaults_needed; i < column_size; ++i) + column_mixed->insertFrom(*col_read.column, i); + + return column_mixed; +} + +} diff --git a/dbms/src/DataStreams/AddingDefaultsBlockInputStream.h b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.h new file mode 100644 index 00000000000..6711a3daee9 --- /dev/null +++ b/dbms/src/DataStreams/AddingDefaultsBlockInputStream.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +/// Adds defaults to columns using BlockDelayedDefaults bitmask attached to Block by child InputStream. +class AddingDefaultsBlockInputStream : public IProfilingBlockInputStream +{ +public: + AddingDefaultsBlockInputStream( + const BlockInputStreamPtr & input, + const ColumnDefaults & column_defaults_, + const Context & context_); + + String getName() const override { return "AddingDefaults"; } + Block getHeader() const override { return header; } + +protected: + Block readImpl() override; + +private: + Block header; + const ColumnDefaults column_defaults; + const Context & context; + + void checkCalculated(const ColumnWithTypeAndName & col_read, const ColumnWithTypeAndName & col_defaults, size_t needed) const; + MutableColumnPtr mixColumns(const ColumnWithTypeAndName & col_read, const ColumnWithTypeAndName & col_defaults, + const BlockMissingValues::RowsBitMask & defaults_mask) const; + void mixNumberColumns(TypeIndex type_idx, MutableColumnPtr & col_mixed, const ColumnPtr & col_defaults, + const BlockMissingValues::RowsBitMask & defaults_mask) const; +}; + +} diff --git a/dbms/src/DataStreams/AddingDefaultBlockInputStream.cpp b/dbms/src/DataStreams/AddingMissedBlockInputStream.cpp similarity index 75% rename from dbms/src/DataStreams/AddingDefaultBlockInputStream.cpp rename to dbms/src/DataStreams/AddingMissedBlockInputStream.cpp index 749eebda1a5..e32a7024824 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockInputStream.cpp +++ b/dbms/src/DataStreams/AddingMissedBlockInputStream.cpp @@ -1,11 +1,11 @@ -#include +#include #include namespace DB { -AddingDefaultBlockInputStream::AddingDefaultBlockInputStream( +AddingMissedBlockInputStream::AddingMissedBlockInputStream( const BlockInputStreamPtr & input_, const Block & header_, const ColumnDefaults & column_defaults_, @@ -16,7 +16,7 @@ AddingDefaultBlockInputStream::AddingDefaultBlockInputStream( children.emplace_back(input); } -Block AddingDefaultBlockInputStream::readImpl() +Block AddingMissedBlockInputStream::readImpl() { Block src = children.back()->read(); if (!src) diff --git a/dbms/src/DataStreams/AddingDefaultBlockInputStream.h b/dbms/src/DataStreams/AddingMissedBlockInputStream.h similarity index 85% rename from dbms/src/DataStreams/AddingDefaultBlockInputStream.h rename to dbms/src/DataStreams/AddingMissedBlockInputStream.h index c0afffbfc17..b3b98509645 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockInputStream.h +++ b/dbms/src/DataStreams/AddingMissedBlockInputStream.h @@ -14,16 +14,16 @@ namespace DB * 3. Columns that materialized from other columns (materialized columns) * All three types of columns are materialized (not constants). */ -class AddingDefaultBlockInputStream : public IProfilingBlockInputStream +class AddingMissedBlockInputStream : public IProfilingBlockInputStream { public: - AddingDefaultBlockInputStream( + AddingMissedBlockInputStream( const BlockInputStreamPtr & input_, const Block & header_, const ColumnDefaults & column_defaults_, const Context & context_); - String getName() const override { return "AddingDefault"; } + String getName() const override { return "AddingMissed"; } Block getHeader() const override { return header; } private: diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index 3e7c59ff78f..c82761dc02b 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -63,6 +63,12 @@ public: */ virtual Block read() = 0; + virtual const BlockMissingValues & getMissingValues() const + { + static const BlockMissingValues none; + return none; + } + /** 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(). diff --git a/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp b/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp index 0e4f876925d..b78b7a59db6 100644 --- a/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp +++ b/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp @@ -4,7 +4,8 @@ #include #include #include - +#include +#include namespace DB { @@ -44,6 +45,10 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( input_buffer_contacenated = std::make_unique(buffers); res_stream = context.getInputFormat(format, *input_buffer_contacenated, streams.out->getHeader(), context.getSettings().max_insert_block_size); + + auto columns_description = ColumnsDescription::loadFromContext(context, ast_insert_query->database, ast_insert_query->table); + if (columns_description && !columns_description->defaults.empty()) + res_stream = std::make_shared(res_stream, columns_description->defaults, context); } } diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 33afbb0aa9e..7cd4a571a60 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -153,7 +153,8 @@ Block NativeBlockInputStream::readImpl() column.column = std::move(read_column); - if (server_revision && server_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) + /// Support insert from old clients without low cardinality type. + if (header && server_revision && server_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) { column.column = recursiveLowCardinalityConversion(column.column, column.type, header.getByPosition(i).type); column.type = header.getByPosition(i).type; diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index 8be3373fc44..c87d82b2506 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -9,6 +9,7 @@ #include #include +#include namespace DB { @@ -100,7 +101,14 @@ void NativeBlockOutputStream::write(const Block & block) mark.offset_in_decompressed_block = ostr_concrete->getRemainingBytes(); } - const ColumnWithTypeAndName & column = block.safeGetByPosition(i); + ColumnWithTypeAndName column = block.safeGetByPosition(i); + + /// Send data to old clients without low cardinality type. + if (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE) + { + column.column = recursiveRemoveLowCardinality(column.column); + column.type = recursiveRemoveLowCardinality(column.type); + } /// Name writeStringBinary(column.name, ostr); diff --git a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp index b212c8ebdaa..215b21f7994 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinalityHelpers.cpp @@ -69,6 +69,9 @@ ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column) ColumnPtr recursiveLowCardinalityConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type) { + if (!column) + return column; + if (from_type->equals(*to_type)) return column; diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index f88fdfc22a5..166320c75f2 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -514,6 +514,13 @@ inline bool isNumber(const T & data_type) return which.isInt() || which.isUInt() || which.isFloat(); } +template +inline bool isColumnedAsNumber(const T & data_type) +{ + WhichDataType which(data_type); + return which.isInt() || which.isUInt() || which.isFloat() || which.isDateOrDateTime() || which.isUUID(); +} + template inline bool isString(const T & data_type) { diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index 08a0752a23f..cd61d862c20 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -1,48 +1,47 @@ #include "CacheDictionary.h" #include -#include #include -#include +#include #include +#include #include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include #include -#include "DictionaryBlockInputStream.h" -#include -#include #include -#include "DictionaryFactory.h" +#include +#include #include "CacheDictionary.inc.h" +#include "DictionaryBlockInputStream.h" +#include "DictionaryFactory.h" namespace ProfileEvents { - extern const Event DictCacheKeysRequested; - extern const Event DictCacheKeysRequestedMiss; - extern const Event DictCacheKeysRequestedFound; - extern const Event DictCacheKeysExpired; - extern const Event DictCacheKeysNotFound; - extern const Event DictCacheKeysHit; - extern const Event DictCacheRequestTimeNs; - extern const Event DictCacheRequests; - extern const Event DictCacheLockWriteNs; - extern const Event DictCacheLockReadNs; +extern const Event DictCacheKeysRequested; +extern const Event DictCacheKeysRequestedMiss; +extern const Event DictCacheKeysRequestedFound; +extern const Event DictCacheKeysExpired; +extern const Event DictCacheKeysNotFound; +extern const Event DictCacheKeysHit; +extern const Event DictCacheRequestTimeNs; +extern const Event DictCacheRequests; +extern const Event DictCacheLockWriteNs; +extern const Event DictCacheLockReadNs; } namespace CurrentMetrics { - extern const Metric DictCacheRequests; +extern const Metric DictCacheRequests; } namespace DB { - namespace ErrorCodes { extern const int TYPE_MISMATCH; @@ -61,15 +60,20 @@ inline size_t CacheDictionary::getCellIdx(const Key id) const } -CacheDictionary::CacheDictionary(const std::string & name, const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, +CacheDictionary::CacheDictionary( + const std::string & name, + const DictionaryStructure & dict_struct, + DictionarySourcePtr source_ptr, + const DictionaryLifetime dict_lifetime, const size_t size) - : name{name}, dict_struct(dict_struct), - source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), - size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))}, - size_overlap_mask{this->size - 1}, - cells{this->size}, - rnd_engine(randomSeed()) + : name{name} + , dict_struct(dict_struct) + , source_ptr{std::move(source_ptr)} + , dict_lifetime(dict_lifetime) + , size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))} + , size_overlap_mask{this->size - 1} + , cells{this->size} + , rnd_engine(randomSeed()) { if (!this->source_ptr->supportsSelectiveLoad()) throw Exception{name + ": source cannot be used with CacheDictionary", ErrorCodes::UNSUPPORTED_METHOD}; @@ -79,32 +83,36 @@ CacheDictionary::CacheDictionary(const std::string & name, const DictionaryStruc CacheDictionary::CacheDictionary(const CacheDictionary & other) : CacheDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.size} -{} +{ +} void CacheDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray & out) const { const auto null_value = std::get(hierarchical_attribute->null_values); - getItemsNumber(*hierarchical_attribute, ids, out, [&] (const size_t) { return null_value; }); + getItemsNumber(*hierarchical_attribute, ids, out, [&](const size_t) { return null_value; }); } /// Allow to use single value in same way as array. -static inline CacheDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) { return arr[idx]; } -static inline CacheDictionary::Key getAt(const CacheDictionary::Key & value, const size_t) { return value; } +static inline CacheDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) +{ + return arr[idx]; +} +static inline CacheDictionary::Key getAt(const CacheDictionary::Key & value, const size_t) +{ + return value; +} template -void CacheDictionary::isInImpl( - const PaddedPODArray & child_ids, - const AncestorType & ancestor_ids, - PaddedPODArray & out) const +void CacheDictionary::isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const { /// Transform all children to parents until ancestor id or null_value will be reached. size_t out_size = out.size(); - memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated" + memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated" const auto null_value = std::get(hierarchical_attribute->null_values); @@ -164,25 +172,17 @@ void CacheDictionary::isInImpl( } void CacheDictionary::isInVectorVector( - const PaddedPODArray & child_ids, - const PaddedPODArray & ancestor_ids, - PaddedPODArray & out) const + const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const { isInImpl(child_ids, ancestor_ids, out); } -void CacheDictionary::isInVectorConstant( - const PaddedPODArray & child_ids, - const Key ancestor_id, - PaddedPODArray & out) const +void CacheDictionary::isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const { isInImpl(child_ids, ancestor_id, out); } -void CacheDictionary::isInConstantVector( - const Key child_id, - const PaddedPODArray & ancestor_ids, - PaddedPODArray & out) const +void CacheDictionary::isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const { /// Special case with single child value. @@ -213,33 +213,34 @@ void CacheDictionary::getString(const std::string & attribute_name, const Padded { auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; const auto null_value = StringRef{std::get(attribute.null_values)}; - getItemsString(attribute, ids, out, [&] (const size_t) { return null_value; }); + getItemsString(attribute, ids, out, [&](const size_t) { return null_value; }); } void CacheDictionary::getString( - const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, - ColumnString * const out) const + const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) const { auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, ids, out, [&] (const size_t row) { return def->getDataAt(row); }); + getItemsString(attribute, ids, out, [&](const size_t row) { return def->getDataAt(row); }); } void CacheDictionary::getString( - const std::string & attribute_name, const PaddedPODArray & ids, const String & def, - ColumnString * const out) const + const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const { auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, ids, out, [&] (const size_t) { return StringRef{def}; }); + getItemsString(attribute, ids, out, [&](const size_t) { return StringRef{def}; }); } @@ -329,21 +330,21 @@ void CacheDictionary::has(const PaddedPODArray & ids, PaddedPODArray return; std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); /// request new values - update(required_ids, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - out[row] = true; - }, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - out[row] = false; - }); + update( + required_ids, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + out[row] = true; + }, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + out[row] = false; + }); } @@ -362,7 +363,7 @@ void CacheDictionary::createAttributes() if (attribute.hierarchical) { - hierarchical_attribute = & attributes.back(); + hierarchical_attribute = &attributes.back(); if (hierarchical_attribute->type != AttributeUnderlyingType::UInt64) throw Exception{name + ": hierarchical attribute must be UInt64.", ErrorCodes::TYPE_MISMATCH}; @@ -377,11 +378,11 @@ CacheDictionary::Attribute CacheDictionary::createAttributeWithType(const Attrib switch (type) { #define DISPATCH(TYPE) \ - case AttributeUnderlyingType::TYPE: \ - attr.null_values = TYPE(null_value.get>()); \ - attr.arrays = std::make_unique>(size); \ - bytes_allocated += size * sizeof(TYPE); \ - break; + case AttributeUnderlyingType::TYPE: \ + attr.null_values = TYPE(null_value.get>()); \ + attr.arrays = std::make_unique>(size); \ + bytes_allocated += size * sizeof(TYPE); \ + break; DISPATCH(UInt8) DISPATCH(UInt16) DISPATCH(UInt32) @@ -413,17 +414,39 @@ void CacheDictionary::setDefaultAttributeValue(Attribute & attribute, const Key { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt128: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::UInt8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt128: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Float32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Float64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; case AttributeUnderlyingType::Decimal32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); @@ -457,21 +480,49 @@ void CacheDictionary::setAttributeValue(Attribute & attribute, const Key idx, co { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt128: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::UInt8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt128: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Float32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Float64: + std::get>(attribute.arrays)[idx] = value.get(); + break; - case AttributeUnderlyingType::Decimal32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Decimal64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Decimal128: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Decimal32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Decimal64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Decimal128: + std::get>(attribute.arrays)[idx] = value.get(); + break; case AttributeUnderlyingType::String: { @@ -509,8 +560,8 @@ CacheDictionary::Attribute & CacheDictionary::getAttribute(const std::string & a bool CacheDictionary::isEmptyCell(const UInt64 idx) const { - return (idx != zero_cell_idx && cells[idx].id == 0) || (cells[idx].data - == ext::safe_bit_cast(CellMetadata::time_point_t())); + return (idx != zero_cell_idx && cells[idx].id == 0) + || (cells[idx].data == ext::safe_bit_cast(CellMetadata::time_point_t())); } PaddedPODArray CacheDictionary::getCachedIds() const @@ -537,36 +588,32 @@ BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_na void registerDictionaryCache(DictionaryFactory & factory) { - auto create_layout = [=]( - const std::string & name, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr - ) -> DictionaryPtr { - + auto create_layout = [=](const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr) -> DictionaryPtr + { if (dict_struct.key) - throw Exception {"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD}; + throw Exception{"'key' is not supported for dictionary of layout 'cache'", ErrorCodes::UNSUPPORTED_METHOD}; if (dict_struct.range_min || dict_struct.range_max) - throw Exception {name - + ": elements .structure.range_min and .structure.range_max should be defined only " - "for a dictionary of layout 'range_hashed'", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception{name + + ": elements .structure.range_min and .structure.range_max should be defined only " + "for a dictionary of layout 'range_hashed'", + ErrorCodes::BAD_ARGUMENTS}; const auto & layout_prefix = config_prefix + ".layout"; const auto size = config.getInt(layout_prefix + ".cache.size_in_cells"); if (size == 0) - throw Exception {name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; + throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); if (require_nonempty) - throw Exception {name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", + ErrorCodes::BAD_ARGUMENTS}; - const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); - - }; factory.registerLayout("cache", create_layout); } diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index 8b72daaca23..9122b0d82bc 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -1,31 +1,33 @@ #pragma once -#include "IDictionary.h" -#include "IDictionarySource.h" -#include "DictionaryStructure.h" -#include -#include -#include -#include -#include -#include #include #include -#include +#include #include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "IDictionarySource.h" namespace DB { - class CacheDictionary final : public IDictionary { public: - CacheDictionary(const std::string & name, const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, + CacheDictionary( + const std::string & name, + const DictionaryStructure & dict_struct, + DictionarySourcePtr source_ptr, + const DictionaryLifetime dict_lifetime, const size_t size); CacheDictionary(const CacheDictionary & other); @@ -42,16 +44,12 @@ public: double getHitRate() const override { - return static_cast(hit_count.load(std::memory_order_acquire)) / - query_count.load(std::memory_order_relaxed); + return static_cast(hit_count.load(std::memory_order_acquire)) / query_count.load(std::memory_order_relaxed); } size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); } - double getLoadFactor() const override - { - return static_cast(element_count.load(std::memory_order_relaxed)) / size; - } + double getLoadFactor() const override { return static_cast(element_count.load(std::memory_order_relaxed)) / size; } bool isCached() const override { return true; } @@ -63,10 +61,7 @@ public: const DictionaryStructure & getStructure() const override { return dict_struct; } - std::chrono::time_point getCreationTime() const override - { - return creation_time; - } + std::chrono::time_point getCreationTime() const override { return creation_time; } bool isInjective(const std::string & attribute_name) const override { @@ -77,14 +72,15 @@ public: void toParent(const PaddedPODArray & ids, PaddedPODArray & out) const override; - void isInVectorVector(const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; + void isInVectorVector( + const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; void isInVectorConstant(const PaddedPODArray & child_ids, const Key ancestor_id, PaddedPODArray & out) const override; void isInConstantVector(const Key child_id, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const override; template using ResultArrayType = std::conditional_t, DecimalPaddedPODArray, PaddedPODArray>; -#define DECLARE(TYPE)\ +#define DECLARE(TYPE) \ void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) @@ -104,9 +100,11 @@ public: void getString(const std::string & attribute_name, const PaddedPODArray & ids, ColumnString * out) const; -#define DECLARE(TYPE)\ - void get##TYPE(\ - const std::string & attribute_name, const PaddedPODArray & ids, const PaddedPODArray & def,\ +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const PaddedPODArray & ids, \ + const PaddedPODArray & def, \ ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) @@ -124,11 +122,11 @@ public: DECLARE(Decimal128) #undef DECLARE - void getString( - const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, - ColumnString * const out) const; + void + getString(const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, ColumnString * const out) + const; -#define DECLARE(TYPE)\ +#define DECLARE(TYPE) \ void get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) @@ -146,17 +144,17 @@ public: DECLARE(Decimal128) #undef DECLARE - void getString( - const std::string & attribute_name, const PaddedPODArray & ids, const String & def, - ColumnString * const out) const; + void getString(const std::string & attribute_name, const PaddedPODArray & ids, const String & def, ColumnString * const out) const; void has(const PaddedPODArray & ids, PaddedPODArray & out) const override; BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: - template using ContainerType = Value[]; - template using ContainerPtrType = std::unique_ptr>; + template + using ContainerType = Value[]; + template + using ContainerPtrType = std::unique_ptr>; struct CellMetadata final { @@ -183,19 +181,39 @@ private: { AttributeUnderlyingType type; std::variant< - UInt8, UInt16, UInt32, UInt64, + UInt8, + UInt16, + UInt32, + UInt64, UInt128, - Int8, Int16, Int32, Int64, - Decimal32, Decimal64, Decimal128, - Float32, Float64, - String> null_values; + Int8, + Int16, + Int32, + Int64, + Decimal32, + Decimal64, + Decimal128, + Float32, + Float64, + String> + null_values; std::variant< - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, ContainerPtrType, - ContainerPtrType, ContainerPtrType, - ContainerPtrType> arrays; + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType> + arrays; }; void createAttributes(); @@ -205,29 +223,17 @@ private: template void getItemsNumber( - Attribute & attribute, - const PaddedPODArray & ids, - ResultArrayType & out, - DefaultGetter && get_default) const; + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const; template void getItemsNumberImpl( - Attribute & attribute, - const PaddedPODArray & ids, - ResultArrayType & out, - DefaultGetter && get_default) const; + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const; template - void getItemsString( - Attribute & attribute, - const PaddedPODArray & ids, - ColumnString * out, - DefaultGetter && get_default) const; + void getItemsString(Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const; template - void update( - const std::vector & requested_ids, PresentIdHandler && on_cell_updated, - AbsentIdHandler && on_id_not_found) const; + void update(const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const; PaddedPODArray getCachedIds() const; @@ -251,10 +257,7 @@ private: FindResult findCellIdx(const Key & id, const CellMetadata::time_point_t now) const; template - void isInImpl( - const PaddedPODArray & child_ids, - const AncestorType & ancestor_ids, - PaddedPODArray & out) const; + void isInImpl(const PaddedPODArray & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; const std::string name; const DictionaryStructure dict_struct; diff --git a/dbms/src/Dictionaries/CacheDictionary.inc.h b/dbms/src/Dictionaries/CacheDictionary.inc.h index 6fc082ab267..ac451d84990 100644 --- a/dbms/src/Dictionaries/CacheDictionary.inc.h +++ b/dbms/src/Dictionaries/CacheDictionary.inc.h @@ -1,34 +1,33 @@ #include "CacheDictionary.h" -#include -#include -#include +#include #include #include -#include +#include +#include +#include namespace ProfileEvents { - extern const Event DictCacheKeysRequested; - extern const Event DictCacheKeysRequestedMiss; - extern const Event DictCacheKeysRequestedFound; - extern const Event DictCacheKeysExpired; - extern const Event DictCacheKeysNotFound; - extern const Event DictCacheKeysHit; - extern const Event DictCacheRequestTimeNs; - extern const Event DictCacheRequests; - extern const Event DictCacheLockWriteNs; - extern const Event DictCacheLockReadNs; +extern const Event DictCacheKeysRequested; +extern const Event DictCacheKeysRequestedMiss; +extern const Event DictCacheKeysRequestedFound; +extern const Event DictCacheKeysExpired; +extern const Event DictCacheKeysNotFound; +extern const Event DictCacheKeysHit; +extern const Event DictCacheRequestTimeNs; +extern const Event DictCacheRequests; +extern const Event DictCacheLockWriteNs; +extern const Event DictCacheLockReadNs; } namespace CurrentMetrics { - extern const Metric DictCacheRequests; +extern const Metric DictCacheRequests; } namespace DB { - namespace ErrorCodes { extern const int TYPE_MISMATCH; @@ -36,12 +35,11 @@ namespace ErrorCodes template void CacheDictionary::getItemsNumber( - Attribute & attribute, - const PaddedPODArray & ids, - ResultArrayType & out, - DefaultGetter && get_default) const + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const { - if (false) {} + if (false) + { + } #define DISPATCH(TYPE) \ else if (attribute.type == AttributeUnderlyingType::TYPE) \ getItemsNumberImpl(attribute, ids, out, std::forward(get_default)); @@ -60,16 +58,12 @@ void CacheDictionary::getItemsNumber( DISPATCH(Decimal64) DISPATCH(Decimal128) #undef DISPATCH - else - throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); + else throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); } template void CacheDictionary::getItemsNumberImpl( - Attribute & attribute, - const PaddedPODArray & ids, - ResultArrayType & out, - DefaultGetter && get_default) const + Attribute & attribute, const PaddedPODArray & ids, ResultArrayType & out, DefaultGetter && get_default) const { /// Mapping: -> { all indices `i` of `ids` such that `ids[i]` = } std::unordered_map> outdated_ids; @@ -122,31 +116,28 @@ void CacheDictionary::getItemsNumberImpl( return; std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); /// request new values - update(required_ids, - [&] (const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; + update( + required_ids, + [&](const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; - for (const size_t row : outdated_ids[id]) - out[row] = static_cast(attribute_value); - }, - [&] (const auto id, const auto) - { - for (const size_t row : outdated_ids[id]) - out[row] = get_default(row); - }); + for (const size_t row : outdated_ids[id]) + out[row] = static_cast(attribute_value); + }, + [&](const auto id, const auto) + { + for (const size_t row : outdated_ids[id]) + out[row] = get_default(row); + }); } template void CacheDictionary::getItemsString( - Attribute & attribute, - const PaddedPODArray & ids, - ColumnString * out, - DefaultGetter && get_default) const + Attribute & attribute, const PaddedPODArray & ids, ColumnString * out, DefaultGetter && get_default) const { const auto rows = ext::size(ids); @@ -245,22 +236,22 @@ void CacheDictionary::getItemsString( if (!outdated_ids.empty()) { std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); - update(required_ids, - [&] (const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; + update( + required_ids, + [&](const auto id, const auto cell_idx) + { + const auto attribute_value = attribute_array[cell_idx]; - map[id] = String{attribute_value}; - total_length += (attribute_value.size + 1) * outdated_ids[id].size(); - }, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - total_length += get_default(row).size + 1; - }); + map[id] = String{attribute_value}; + total_length += (attribute_value.size + 1) * outdated_ids[id].size(); + }, + [&](const auto id, const auto) + { + for (const auto row : outdated_ids[id]) + total_length += get_default(row).size + 1; + }); } out->getChars().reserve(total_length); @@ -277,19 +268,13 @@ void CacheDictionary::getItemsString( template void CacheDictionary::update( - const std::vector & requested_ids, - PresentIdHandler && on_cell_updated, - AbsentIdHandler && on_id_not_found) const + const std::vector & requested_ids, PresentIdHandler && on_cell_updated, AbsentIdHandler && on_id_not_found) const { std::unordered_map remaining_ids{requested_ids.size()}; for (const auto id : requested_ids) - remaining_ids.insert({ id, 0 }); + remaining_ids.insert({id, 0}); - std::uniform_int_distribution distribution - { - dict_lifetime.min_sec, - dict_lifetime.max_sec - }; + std::uniform_int_distribution distribution{dict_lifetime.min_sec, dict_lifetime.max_sec}; const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; @@ -310,10 +295,8 @@ void CacheDictionary::update( const auto & ids = id_column->getData(); /// cache column pointers - const auto column_ptrs = ext::map(ext::range(0, attributes.size()), [&block] (size_t i) - { - return block.safeGetByPosition(i + 1).column.get(); - }); + const auto column_ptrs = ext::map( + ext::range(0, attributes.size()), [&block](size_t i) { return block.safeGetByPosition(i + 1).column.get(); }); for (const auto i : ext::range(0, ids.size())) { diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp index 161a157ffaa..faa93055073 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -1,21 +1,20 @@ #include "ClickHouseDictionarySource.h" -#include "ExternalQueryBuilder.h" -#include "writeParenthesisedString.h" +#include #include #include -#include "readInvalidateQuery.h" +#include #include #include -#include #include -#include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" +#include "ExternalQueryBuilder.h" +#include "readInvalidateQuery.h" +#include "writeParenthesisedString.h" namespace DB { - namespace ErrorCodes { extern const int UNSUPPORTED_METHOD; @@ -25,61 +24,81 @@ namespace ErrorCodes static const size_t MAX_CONNECTIONS = 16; static ConnectionPoolWithFailoverPtr createPool( - const std::string & host, UInt16 port, bool secure, const std::string & db, - const std::string & user, const std::string & password, const Context & context) + const std::string & host, + UInt16 port, + bool secure, + const std::string & db, + const std::string & user, + const std::string & password, + const Context & context) { auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(context.getSettingsRef()); ConnectionPoolPtrs pools; pools.emplace_back(std::make_shared( - MAX_CONNECTIONS, host, port, db, user, password, timeouts, "ClickHouseDictionarySource", - Protocol::Compression::Enable, - secure ? Protocol::Secure::Enable : Protocol::Secure::Disable)); + MAX_CONNECTIONS, + host, + port, + db, + user, + password, + timeouts, + "ClickHouseDictionarySource", + Protocol::Compression::Enable, + secure ? Protocol::Secure::Enable : Protocol::Secure::Disable)); return std::make_shared(pools, LoadBalancing::RANDOM); } ClickHouseDictionarySource::ClickHouseDictionarySource( - const DictionaryStructure & dict_struct_, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - const Block & sample_block, Context & context) - : update_time{std::chrono::system_clock::from_time_t(0)}, - dict_struct{dict_struct_}, - host{config.getString(config_prefix + ".host")}, - port(config.getInt(config_prefix + ".port")), - secure(config.getBool(config_prefix + ".secure", false)), - user{config.getString(config_prefix + ".user", "")}, - password{config.getString(config_prefix + ".password", "")}, - db{config.getString(config_prefix + ".db", "")}, - table{config.getString(config_prefix + ".table")}, - where{config.getString(config_prefix + ".where", "")}, - update_field{config.getString(config_prefix + ".update_field", "")}, - invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}, - query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}, - sample_block{sample_block}, context(context), - is_local{isLocalAddress({ host, port }, config.getInt("tcp_port", 0))}, - pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)}, - load_all_query{query_builder.composeLoadAllQuery()} -{} + const DictionaryStructure & dict_struct_, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + const Block & sample_block, + Context & context) + : update_time{std::chrono::system_clock::from_time_t(0)} + , dict_struct{dict_struct_} + , host{config.getString(config_prefix + ".host")} + , port(config.getInt(config_prefix + ".port")) + , secure(config.getBool(config_prefix + ".secure", false)) + , user{config.getString(config_prefix + ".user", "")} + , password{config.getString(config_prefix + ".password", "")} + , db{config.getString(config_prefix + ".db", "")} + , table{config.getString(config_prefix + ".table")} + , where{config.getString(config_prefix + ".where", "")} + , update_field{config.getString(config_prefix + ".update_field", "")} + , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} + , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks} + , sample_block{sample_block} + , context(context) + , is_local{isLocalAddress({host, port}, config.getInt("tcp_port", 0))} + , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)} + , load_all_query{query_builder.composeLoadAllQuery()} +{ +} ClickHouseDictionarySource::ClickHouseDictionarySource(const ClickHouseDictionarySource & other) - : update_time{other.update_time}, - dict_struct{other.dict_struct}, - host{other.host}, port{other.port}, - secure{other.secure}, - user{other.user}, password{other.password}, - db{other.db}, table{other.table}, - where{other.where}, - update_field{other.update_field}, - invalidate_query{other.invalidate_query}, - invalidate_query_response{other.invalidate_query_response}, - query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks}, - sample_block{other.sample_block}, context(other.context), - is_local{other.is_local}, - pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)}, - load_all_query{other.load_all_query} -{} + : update_time{other.update_time} + , dict_struct{other.dict_struct} + , host{other.host} + , port{other.port} + , secure{other.secure} + , user{other.user} + , password{other.password} + , db{other.db} + , table{other.table} + , where{other.where} + , update_field{other.update_field} + , invalidate_query{other.invalidate_query} + , invalidate_query_response{other.invalidate_query_response} + , query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::Backticks} + , sample_block{other.sample_block} + , context(other.context) + , is_local{other.is_local} + , pool{is_local ? nullptr : createPool(host, port, secure, db, user, password, context)} + , load_all_query{other.load_all_query} +{ +} std::string ClickHouseDictionarySource::getUpdateFieldAndDate() { @@ -119,17 +138,14 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadUpdatedAll() BlockInputStreamPtr ClickHouseDictionarySource::loadIds(const std::vector & ids) { - return createStreamForSelectiveLoad( - query_builder.composeLoadIdsQuery(ids)); + return createStreamForSelectiveLoad(query_builder.composeLoadIdsQuery(ids)); } -BlockInputStreamPtr ClickHouseDictionarySource::loadKeys( - const Columns & key_columns, const std::vector & requested_rows) +BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { return createStreamForSelectiveLoad( - query_builder.composeLoadKeysQuery( - key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES)); + query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES)); } bool ClickHouseDictionarySource::isModified() const @@ -167,7 +183,7 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (is_local) { auto input_block = executeQuery(request, context, true).in; - return readInvalidateQuery(dynamic_cast((*input_block))); + return readInvalidateQuery(dynamic_cast((*input_block))); } else { diff --git a/dbms/src/Dictionaries/ClickHouseDictionarySource.h b/dbms/src/Dictionaries/ClickHouseDictionarySource.h index 89db23737bc..bf8653932f7 100644 --- a/dbms/src/Dictionaries/ClickHouseDictionarySource.h +++ b/dbms/src/Dictionaries/ClickHouseDictionarySource.h @@ -1,15 +1,14 @@ #pragma once -#include "IDictionarySource.h" +#include +#include #include "DictionaryStructure.h" #include "ExternalQueryBuilder.h" -#include -#include +#include "IDictionarySource.h" namespace DB { - /** Allows loading dictionaries from local or remote ClickHouse instance * @todo use ConnectionPoolWithFailover * @todo invent a way to keep track of source modifications @@ -17,10 +16,12 @@ namespace DB class ClickHouseDictionarySource final : public IDictionarySource { public: - ClickHouseDictionarySource(const DictionaryStructure & dict_struct_, + ClickHouseDictionarySource( + const DictionaryStructure & dict_struct_, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, - const Block & sample_block, Context & context); + const Block & sample_block, + Context & context); /// copy-constructor is provided in order to support cloneability ClickHouseDictionarySource(const ClickHouseDictionarySource & other); @@ -31,8 +32,7 @@ public: BlockInputStreamPtr loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys( - const Columns & key_columns, const std::vector & requested_rows) override; + BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; bool supportsSelectiveLoad() const override { return true; } diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp index 61693a3538a..84ab8c569a4 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -1,40 +1,38 @@ #include "ComplexKeyCacheDictionary.h" -#include "DictionaryBlockInputStream.h" #include #include -#include -#include -#include -#include #include -#include +#include +#include +#include +#include #include +#include +#include "DictionaryBlockInputStream.h" #include "DictionaryFactory.h" namespace ProfileEvents { - - extern const Event DictCacheKeysRequested; - extern const Event DictCacheKeysRequestedMiss; - extern const Event DictCacheKeysRequestedFound; - extern const Event DictCacheKeysExpired; - extern const Event DictCacheKeysNotFound; - extern const Event DictCacheKeysHit; - extern const Event DictCacheRequestTimeNs; - extern const Event DictCacheLockWriteNs; - extern const Event DictCacheLockReadNs; +extern const Event DictCacheKeysRequested; +extern const Event DictCacheKeysRequestedMiss; +extern const Event DictCacheKeysRequestedFound; +extern const Event DictCacheKeysExpired; +extern const Event DictCacheKeysNotFound; +extern const Event DictCacheKeysHit; +extern const Event DictCacheRequestTimeNs; +extern const Event DictCacheLockWriteNs; +extern const Event DictCacheLockReadNs; } namespace CurrentMetrics { - extern const Metric DictCacheRequests; +extern const Metric DictCacheRequests; } namespace DB { - namespace ErrorCodes { extern const int TYPE_MISMATCH; @@ -52,13 +50,19 @@ inline UInt64 ComplexKeyCacheDictionary::getCellIdx(const StringRef key) const } -ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const std::string & name, const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, +ComplexKeyCacheDictionary::ComplexKeyCacheDictionary( + const std::string & name, + const DictionaryStructure & dict_struct, + DictionarySourcePtr source_ptr, + const DictionaryLifetime dict_lifetime, const size_t size) - : name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), - size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))}, - size_overlap_mask{this->size - 1}, - rnd_engine(randomSeed()) + : name{name} + , dict_struct(dict_struct) + , source_ptr{std::move(source_ptr)} + , dict_lifetime(dict_lifetime) + , size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))} + , size_overlap_mask{this->size - 1} + , rnd_engine(randomSeed()) { if (!this->source_ptr->supportsSelectiveLoad()) throw Exception{name + ": source cannot be used with ComplexKeyCacheDictionary", ErrorCodes::UNSUPPORTED_METHOD}; @@ -68,47 +72,56 @@ ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const std::string & name, c ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const ComplexKeyCacheDictionary & other) : ComplexKeyCacheDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.size} -{} +{ +} void ComplexKeyCacheDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - ColumnString * out) const + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const { dict_struct.validateKeyTypes(key_types); auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; const auto null_value = StringRef{std::get(attribute.null_values)}; - getItemsString(attribute, key_columns, out, [&] (const size_t) { return null_value; }); + getItemsString(attribute, key_columns, out, [&](const size_t) { return null_value; }); } void ComplexKeyCacheDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const ColumnString * const def, ColumnString * const out) const + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnString * const def, + ColumnString * const out) const { dict_struct.validateKeyTypes(key_types); auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, key_columns, out, [&] (const size_t row) { return def->getDataAt(row); }); + getItemsString(attribute, key_columns, out, [&](const size_t row) { return def->getDataAt(row); }); } void ComplexKeyCacheDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const String & def, ColumnString * const out) const + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const String & def, + ColumnString * const out) const { dict_struct.validateKeyTypes(key_types); auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, key_columns, out, [&] (const size_t) { return StringRef{def}; }); + getItemsString(attribute, key_columns, out, [&](const size_t) { return StringRef{def}; }); } /// returns cell_idx (always valid for replacing), 'cell is valid' flag, 'cell is outdated' flag, @@ -118,7 +131,8 @@ void ComplexKeyCacheDictionary::getString( /// true true impossible /// /// todo: split this func to two: find_for_get and find_for_set -ComplexKeyCacheDictionary::FindResult ComplexKeyCacheDictionary::findCellIdx(const StringRef & key, const CellMetadata::time_point_t now, const size_t hash) const +ComplexKeyCacheDictionary::FindResult +ComplexKeyCacheDictionary::findCellIdx(const StringRef & key, const CellMetadata::time_point_t now, const size_t hash) const { auto pos = hash; auto oldest_id = pos; @@ -211,17 +225,20 @@ void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes return; std::vector required_rows(outdated_keys.size()); - std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), - [] (auto & pair) { return pair.second.front(); }); + std::transform( + std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.second.front(); }); /// request new values - update(key_columns, keys_array, required_rows, - [&] (const StringRef key, const auto) + update( + key_columns, + keys_array, + required_rows, + [&](const StringRef key, const auto) { for (const auto out_idx : outdated_keys[key]) out[out_idx] = true; }, - [&] (const StringRef key, const auto) + [&](const StringRef key, const auto) { for (const auto out_idx : outdated_keys[key]) out[out_idx] = false; @@ -242,7 +259,8 @@ void ComplexKeyCacheDictionary::createAttributes() attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); if (attribute.hierarchical) - throw Exception{name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), + ErrorCodes::TYPE_MISMATCH}; } } @@ -273,8 +291,7 @@ void ComplexKeyCacheDictionary::freeKey(const StringRef key) const template StringRef ComplexKeyCacheDictionary::placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, - const std::vector & key_attributes, Pool & pool) + const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector & key_attributes, Pool & pool) { const auto keys_size = key_columns.size(); size_t sum_keys_size{}; @@ -313,22 +330,27 @@ StringRef ComplexKeyCacheDictionary::placeKeysInPool( } } - return { place, sum_keys_size }; + return {place, sum_keys_size}; } /// Explicit instantiations. template StringRef ComplexKeyCacheDictionary::placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, - const std::vector & key_attributes, Arena & pool); + const size_t row, + const Columns & key_columns, + StringRefs & keys, + const std::vector & key_attributes, + Arena & pool); template StringRef ComplexKeyCacheDictionary::placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, - const std::vector & key_attributes, ArenaWithFreeLists & pool); + const size_t row, + const Columns & key_columns, + StringRefs & keys, + const std::vector & key_attributes, + ArenaWithFreeLists & pool); -StringRef ComplexKeyCacheDictionary::placeKeysInFixedSizePool( - const size_t row, const Columns & key_columns) const +StringRef ComplexKeyCacheDictionary::placeKeysInFixedSizePool(const size_t row, const Columns & key_columns) const { const auto res = fixed_size_keys_pool->alloc(); auto place = res; @@ -340,14 +362,14 @@ StringRef ComplexKeyCacheDictionary::placeKeysInFixedSizePool( place += key.size; } - return { res, key_size }; + return {res, key_size}; } StringRef ComplexKeyCacheDictionary::copyIntoArena(StringRef src, Arena & arena) { char * allocated = arena.alloc(src.size); memcpy(allocated, src.data, src.size); - return { allocated, src.size }; + return {allocated, src.size}; } StringRef ComplexKeyCacheDictionary::copyKey(const StringRef key) const @@ -355,13 +377,14 @@ StringRef ComplexKeyCacheDictionary::copyKey(const StringRef key) const const auto res = key_size_is_fixed ? fixed_size_keys_pool->alloc() : keys_pool->alloc(key.size); memcpy(res, key.data, key.size); - return { res, key.size }; + return {res, key.size}; } bool ComplexKeyCacheDictionary::isEmptyCell(const UInt64 idx) const { - return (cells[idx].key == StringRef{} && (idx != zero_cell_idx - || cells[idx].data == ext::safe_bit_cast(CellMetadata::time_point_t()))); + return ( + cells[idx].key == StringRef{} + && (idx != zero_cell_idx || cells[idx].data == ext::safe_bit_cast(CellMetadata::time_point_t()))); } BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const @@ -371,8 +394,7 @@ BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names & const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; for (auto idx : ext::range(0, cells.size())) - if (!isEmptyCell(idx) - && !cells[idx].isDefault()) + if (!isEmptyCell(idx) && !cells[idx].isDefault()) keys.push_back(cells[idx].key); } @@ -382,26 +404,25 @@ BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names & void registerDictionaryComplexKeyCache(DictionaryFactory & factory) { - auto create_layout = [=]( - const std::string & name, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr - ) -> DictionaryPtr { + auto create_layout = [=](const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr) -> DictionaryPtr + { if (!dict_struct.key) - throw Exception {"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; + throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; const auto & layout_prefix = config_prefix + ".layout"; const auto size = config.getInt(layout_prefix + ".complex_key_cache.size_in_cells"); if (size == 0) - throw Exception {name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; + throw Exception{name + ": dictionary of layout 'cache' cannot have 0 cells", ErrorCodes::TOO_SMALL_BUFFER_SIZE}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); if (require_nonempty) - throw Exception {name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", - ErrorCodes::BAD_ARGUMENTS}; + throw Exception{name + ": dictionary of layout 'cache' cannot have 'require_nonempty' attribute set", + ErrorCodes::BAD_ARGUMENTS}; - const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, size); }; factory.registerLayout("complex_key_cache", create_layout); diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index f60e142db5e..017a638d776 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -3,23 +3,23 @@ #include #include #include +#include #include #include -#include #include #include +#include #include #include #include #include -#include "DictionaryStructure.h" -#include "IDictionary.h" -#include "IDictionarySource.h" #include #include #include #include -#include +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "IDictionarySource.h" namespace ProfileEvents @@ -40,7 +40,8 @@ namespace DB class ComplexKeyCacheDictionary final : public IDictionaryBase { public: - ComplexKeyCacheDictionary(const std::string & name, + ComplexKeyCacheDictionary( + const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, @@ -48,25 +49,13 @@ public: ComplexKeyCacheDictionary(const ComplexKeyCacheDictionary & other); - std::string getKeyDescription() const - { - return key_description; - } + std::string getKeyDescription() const { return key_description; } - std::exception_ptr getCreationException() const override - { - return {}; - } + std::exception_ptr getCreationException() const override { return {}; } - std::string getName() const override - { - return name; - } + std::string getName() const override { return name; } - std::string getTypeName() const override - { - return "ComplexKeyCache"; - } + std::string getTypeName() const override { return "ComplexKeyCache"; } size_t getBytesAllocated() const override { @@ -74,55 +63,28 @@ public: + (string_arena ? string_arena->size() : 0); } - size_t getQueryCount() const override - { - return query_count.load(std::memory_order_relaxed); - } + size_t getQueryCount() const override { return query_count.load(std::memory_order_relaxed); } double getHitRate() const override { return static_cast(hit_count.load(std::memory_order_acquire)) / query_count.load(std::memory_order_relaxed); } - size_t getElementCount() const override - { - return element_count.load(std::memory_order_relaxed); - } + size_t getElementCount() const override { return element_count.load(std::memory_order_relaxed); } - double getLoadFactor() const override - { - return static_cast(element_count.load(std::memory_order_relaxed)) / size; - } + double getLoadFactor() const override { return static_cast(element_count.load(std::memory_order_relaxed)) / size; } - bool isCached() const override - { - return true; - } + bool isCached() const override { return true; } - std::unique_ptr clone() const override - { - return std::make_unique(*this); - } + std::unique_ptr clone() const override { return std::make_unique(*this); } - const IDictionarySource * getSource() const override - { - return source_ptr.get(); - } + const IDictionarySource * getSource() const override { return source_ptr.get(); } - const DictionaryLifetime & getLifetime() const override - { - return dict_lifetime; - } + const DictionaryLifetime & getLifetime() const override { return dict_lifetime; } - const DictionaryStructure & getStructure() const override - { - return dict_struct; - } + const DictionaryStructure & getStructure() const override { return dict_struct; } - std::chrono::time_point getCreationTime() const override - { - return creation_time; - } + std::chrono::time_point getCreationTime() const override { return creation_time; } bool isInjective(const std::string & attribute_name) const override { @@ -135,7 +97,7 @@ public: /// In all functions below, key_columns must be full (non-constant) columns. /// See the requirement in IDataType.h for text-serialization functions. #define DECLARE(TYPE) \ - void get##TYPE( \ + void get##TYPE( \ const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) @@ -155,11 +117,12 @@ public: void getString(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const; -#define DECLARE(TYPE) \ - void get##TYPE(const std::string & attribute_name, \ - const Columns & key_columns, \ - const DataTypes & key_types, \ - const PaddedPODArray & def, \ +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) @@ -177,17 +140,19 @@ public: DECLARE(Decimal128) #undef DECLARE - void getString(const std::string & attribute_name, + void getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const ColumnString * const def, ColumnString * const out) const; -#define DECLARE(TYPE) \ - void get##TYPE(const std::string & attribute_name, \ - const Columns & key_columns, \ - const DataTypes & key_types, \ - const TYPE def, \ +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const TYPE def, \ ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) @@ -205,7 +170,8 @@ public: DECLARE(Decimal128) #undef DECLARE - void getString(const std::string & attribute_name, + void getString( + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, const String & def, @@ -216,9 +182,12 @@ public: BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: - template using MapType = HashMapWithSavedHash; - template using ContainerType = Value[]; - template using ContainerPtrType = std::unique_ptr>; + template + using MapType = HashMapWithSavedHash; + template + using ContainerType = Value[]; + template + using ContainerPtrType = std::unique_ptr>; struct CellMetadata final { @@ -235,32 +204,35 @@ private: time_point_urep_t data; /// Sets expiration time, resets `is_default` flag to false - time_point_t expiresAt() const - { - return ext::safe_bit_cast(data & EXPIRES_AT_MASK); - } - void setExpiresAt(const time_point_t & t) - { - data = ext::safe_bit_cast(t); - } + time_point_t expiresAt() const { return ext::safe_bit_cast(data & EXPIRES_AT_MASK); } + void setExpiresAt(const time_point_t & t) { data = ext::safe_bit_cast(t); } - bool isDefault() const - { - return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; - } - void setDefault() - { - data |= IS_DEFAULT_MASK; - } + bool isDefault() const { return (data & IS_DEFAULT_MASK) == IS_DEFAULT_MASK; } + void setDefault() { data |= IS_DEFAULT_MASK; } }; struct Attribute final { AttributeUnderlyingType type; - std::variant null_values; - std::variant, + std::variant< + UInt8, + UInt16, + UInt32, + UInt64, + UInt128, + Int8, + Int16, + Int32, + Int64, + Decimal32, + Decimal64, + Decimal128, + Float32, + Float64, + String> + null_values; + std::variant< + ContainerPtrType, ContainerPtrType, ContainerPtrType, ContainerPtrType, @@ -283,13 +255,13 @@ private: Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value); template - void getItemsNumber( - Attribute & attribute, const Columns & key_columns, PaddedPODArray & out, DefaultGetter && get_default) const + void + getItemsNumber(Attribute & attribute, const Columns & key_columns, PaddedPODArray & out, DefaultGetter && get_default) const { if (false) { } -#define DISPATCH(TYPE) \ +#define DISPATCH(TYPE) \ else if (attribute.type == AttributeUnderlyingType::TYPE) \ getItemsNumberImpl(attribute, key_columns, out, std::forward(get_default)); DISPATCH(UInt8) @@ -372,7 +344,8 @@ private: std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.second.front(); }); /// request new values - update(key_columns, + update( + key_columns, keys_array, required_rows, [&](const StringRef key, const size_t cell_idx) @@ -497,7 +470,8 @@ private: return pair.second.front(); }); - update(key_columns, + update( + key_columns, keys_array, required_rows, [&](const StringRef key, const size_t cell_idx) @@ -531,7 +505,8 @@ private: } template - void update(const Columns & in_key_columns, + void update( + const Columns & in_key_columns, const PODArray & in_keys, const std::vector & in_requested_rows, PresentKeyHandler && on_cell_updated, @@ -561,8 +536,10 @@ private: const auto key_columns = ext::map( ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); - const auto attribute_columns = ext::map(ext::range(0, attributes_size), - [&](const size_t attribute_idx) { return block.safeGetByPosition(keys_size + attribute_idx).column; }); + const auto attribute_columns = ext::map(ext::range(0, attributes_size), [&](const size_t attribute_idx) + { + return block.safeGetByPosition(keys_size + attribute_idx).column; + }); const auto rows_num = block.rows(); @@ -693,7 +670,8 @@ private: void freeKey(const StringRef key) const; template - static StringRef placeKeysInPool(const size_t row, + static StringRef placeKeysInPool( + const size_t row, const Columns & key_columns, StringRefs & keys, const std::vector & key_attributes, diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp index 843c389dcb0..aeb85881f86 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_createAttributeWithType.cpp @@ -2,19 +2,19 @@ namespace DB { - -ComplexKeyCacheDictionary::Attribute ComplexKeyCacheDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) +ComplexKeyCacheDictionary::Attribute +ComplexKeyCacheDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) { Attribute attr{type, {}, {}}; switch (type) { #define DISPATCH(TYPE) \ - case AttributeUnderlyingType::TYPE: \ - attr.null_values = TYPE(null_value.get>()); \ - attr.arrays = std::make_unique>(size); \ - bytes_allocated += size * sizeof(TYPE); \ - break; + case AttributeUnderlyingType::TYPE: \ + attr.null_values = TYPE(null_value.get>()); \ + attr.arrays = std::make_unique>(size); \ + bytes_allocated += size * sizeof(TYPE); \ + break; DISPATCH(UInt8) DISPATCH(UInt16) DISPATCH(UInt32) diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp index 9a3d34eb2c7..7b3a44214c5 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp @@ -2,26 +2,53 @@ namespace DB { - void ComplexKeyCacheDictionary::setAttributeValue(Attribute & attribute, const size_t idx, const Field & value) const { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt128: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::UInt8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt128: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Float32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Float64: + std::get>(attribute.arrays)[idx] = value.get(); + break; - case AttributeUnderlyingType::Decimal32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Decimal64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Decimal128: std::get>(attribute.arrays)[idx] = value.get(); break; + case AttributeUnderlyingType::Decimal32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Decimal64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Decimal128: + std::get>(attribute.arrays)[idx] = value.get(); + break; case AttributeUnderlyingType::String: { diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp index 7477e01da9c..89cf1506f90 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setDefaultAttributeValue.cpp @@ -2,22 +2,43 @@ namespace DB { - void ComplexKeyCacheDictionary::setDefaultAttributeValue(Attribute & attribute, const size_t idx) const { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt128: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; + case AttributeUnderlyingType::UInt8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt128: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Float32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Float64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; case AttributeUnderlyingType::Decimal32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp index cdf01668bd2..8422fb2a68f 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.cpp @@ -1,12 +1,11 @@ +#include "ComplexKeyHashedDictionary.h" #include #include -#include "ComplexKeyHashedDictionary.h" #include "DictionaryBlockInputStream.h" #include "DictionaryFactory.h" namespace DB { - namespace ErrorCodes { extern const int TYPE_MISMATCH; @@ -16,12 +15,19 @@ namespace ErrorCodes } ComplexKeyHashedDictionary::ComplexKeyHashedDictionary( - const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr, - const DictionaryLifetime dict_lifetime, bool require_nonempty, BlockPtr saved_block) - : name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), - require_nonempty(require_nonempty), saved_block{std::move(saved_block)} + const std::string & name, + const DictionaryStructure & dict_struct, + DictionarySourcePtr source_ptr, + const DictionaryLifetime dict_lifetime, + bool require_nonempty, + BlockPtr saved_block) + : name{name} + , dict_struct(dict_struct) + , source_ptr{std::move(source_ptr)} + , dict_lifetime(dict_lifetime) + , require_nonempty(require_nonempty) + , saved_block{std::move(saved_block)} { - createAttributes(); try @@ -38,27 +44,30 @@ ComplexKeyHashedDictionary::ComplexKeyHashedDictionary( } ComplexKeyHashedDictionary::ComplexKeyHashedDictionary(const ComplexKeyHashedDictionary & other) - : ComplexKeyHashedDictionary{other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block} + : ComplexKeyHashedDictionary{ + other.name, other.dict_struct, other.source_ptr->clone(), other.dict_lifetime, other.require_nonempty, other.saved_block} { } -#define DECLARE(TYPE)\ -void ComplexKeyHashedDictionary::get##TYPE(\ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\ - ResultArrayType & out) const\ -{\ - dict_struct.validateKeyTypes(key_types);\ - \ - const auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\ - \ - const auto null_value = std::get(attribute.null_values);\ - \ - getItemsNumber(attribute, key_columns,\ - [&] (const size_t row, const auto value) { out[row] = value; },\ - [&] (const size_t) { return null_value; });\ -} +#define DECLARE(TYPE) \ + void ComplexKeyHashedDictionary::get##TYPE( \ + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ +\ + const auto & attribute = getAttribute(attribute_name); \ + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ + ErrorCodes::TYPE_MISMATCH}; \ +\ + const auto null_value = std::get(attribute.null_values); \ +\ + getItemsNumber( \ + attribute, \ + key_columns, \ + [&](const size_t row, const auto value) { out[row] = value; }, \ + [&](const size_t) { return null_value; }); \ + } DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -76,37 +85,45 @@ DECLARE(Decimal128) #undef DECLARE void ComplexKeyHashedDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - ColumnString * out) const + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const { dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; const auto & null_value = StringRef{std::get(attribute.null_values)}; - getItemsImpl(attribute, key_columns, - [&] (const size_t, const StringRef value) { out->insertData(value.data, value.size); }, - [&] (const size_t) { return null_value; }); + getItemsImpl( + attribute, + key_columns, + [&](const size_t, const StringRef value) { out->insertData(value.data, value.size); }, + [&](const size_t) { return null_value; }); } -#define DECLARE(TYPE)\ -void ComplexKeyHashedDictionary::get##TYPE(\ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\ - const PaddedPODArray & def, ResultArrayType & out) const\ -{\ - dict_struct.validateKeyTypes(key_types);\ - \ - const auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\ - \ - getItemsNumber(attribute, key_columns,\ - [&] (const size_t row, const auto value) { out[row] = value; },\ - [&] (const size_t row) { return def[row]; });\ -} +#define DECLARE(TYPE) \ + void ComplexKeyHashedDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ + ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ +\ + const auto & attribute = getAttribute(attribute_name); \ + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ + ErrorCodes::TYPE_MISMATCH}; \ +\ + getItemsNumber( \ + attribute, \ + key_columns, \ + [&](const size_t row, const auto value) { out[row] = value; }, \ + [&](const size_t row) { return def[row]; }); \ + } DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -124,35 +141,44 @@ DECLARE(Decimal128) #undef DECLARE void ComplexKeyHashedDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const ColumnString * const def, ColumnString * const out) const + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnString * const def, + ColumnString * const out) const { dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; - getItemsImpl(attribute, key_columns, - [&] (const size_t, const StringRef value) { out->insertData(value.data, value.size); }, - [&] (const size_t row) { return def->getDataAt(row); }); + getItemsImpl( + attribute, + key_columns, + [&](const size_t, const StringRef value) { out->insertData(value.data, value.size); }, + [&](const size_t row) { return def->getDataAt(row); }); } -#define DECLARE(TYPE)\ -void ComplexKeyHashedDictionary::get##TYPE(\ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\ - const TYPE def, ResultArrayType & out) const\ -{\ - dict_struct.validateKeyTypes(key_types);\ - \ - const auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH};\ - \ - getItemsNumber(attribute, key_columns,\ - [&] (const size_t row, const auto value) { out[row] = value; },\ - [&] (const size_t) { return def; });\ -} +#define DECLARE(TYPE) \ + void ComplexKeyHashedDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const TYPE def, \ + ResultArrayType & out) const \ + { \ + dict_struct.validateKeyTypes(key_types); \ +\ + const auto & attribute = getAttribute(attribute_name); \ + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ + ErrorCodes::TYPE_MISMATCH}; \ +\ + getItemsNumber( \ + attribute, key_columns, [&](const size_t row, const auto value) { out[row] = value; }, [&](const size_t) { return def; }); \ + } DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -170,18 +196,24 @@ DECLARE(Decimal128) #undef DECLARE void ComplexKeyHashedDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const String & def, ColumnString * const out) const + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const String & def, + ColumnString * const out) const { dict_struct.validateKeyTypes(key_types); const auto & attribute = getAttribute(attribute_name); if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::String)) - throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), + ErrorCodes::TYPE_MISMATCH}; - getItemsImpl(attribute, key_columns, - [&] (const size_t, const StringRef value) { out->insertData(value.data, value.size); }, - [&] (const size_t) { return StringRef{def}; }); + getItemsImpl( + attribute, + key_columns, + [&](const size_t, const StringRef value) { out->insertData(value.data, value.size); }, + [&](const size_t) { return StringRef{def}; }); } void ComplexKeyHashedDictionary::has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray & out) const @@ -192,22 +224,52 @@ void ComplexKeyHashedDictionary::has(const Columns & key_columns, const DataType switch (attribute.type) { - case AttributeUnderlyingType::UInt8: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::UInt16: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::UInt32: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::UInt64: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::UInt128: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Int8: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Int16: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Int32: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Int64: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Float32: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Float64: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::String: has(attribute, key_columns, out); break; + case AttributeUnderlyingType::UInt8: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::UInt16: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::UInt32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::UInt64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::UInt128: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Int8: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Int16: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Int32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Int64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Float32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Float64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::String: + has(attribute, key_columns, out); + break; - case AttributeUnderlyingType::Decimal32: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Decimal64: has(attribute, key_columns, out); break; - case AttributeUnderlyingType::Decimal128: has(attribute, key_columns, out); break; + case AttributeUnderlyingType::Decimal32: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Decimal64: + has(attribute, key_columns, out); + break; + case AttributeUnderlyingType::Decimal128: + has(attribute, key_columns, out); + break; } } @@ -222,7 +284,8 @@ void ComplexKeyHashedDictionary::createAttributes() attributes.push_back(createAttributeWithType(attribute.underlying_type, attribute.null_value)); if (attribute.hierarchical) - throw Exception{name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), ErrorCodes::TYPE_MISMATCH}; + throw Exception{name + ": hierarchical attributes not supported for dictionary of type " + getTypeName(), + ErrorCodes::TYPE_MISMATCH}; } } @@ -236,17 +299,13 @@ void ComplexKeyHashedDictionary::blockToAttributes(const Block & block) const auto rows = block.rows(); element_count += rows; - const auto key_column_ptrs = ext::map(ext::range(0, keys_size), - [&](const size_t attribute_idx) - { - return block.safeGetByPosition(attribute_idx).column; - }); + const auto key_column_ptrs = ext::map( + ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); - const auto attribute_column_ptrs = ext::map(ext::range(0, attributes_size), - [&](const size_t attribute_idx) - { - return block.safeGetByPosition(keys_size + attribute_idx).column; - }); + const auto attribute_column_ptrs = ext::map(ext::range(0, attributes_size), [&](const size_t attribute_idx) + { + return block.safeGetByPosition(keys_size + attribute_idx).column; + }); for (const auto row_idx : ext::range(0, rows)) { @@ -304,18 +363,14 @@ void ComplexKeyHashedDictionary::updateData() stream->readPrefix(); while (Block block = stream->read()) { - const auto saved_key_column_ptrs = ext::map(ext::range(0, keys_size), [&](const size_t key_idx) - { - return saved_block->safeGetByPosition(key_idx).column; - }); + const auto saved_key_column_ptrs = ext::map( + ext::range(0, keys_size), [&](const size_t key_idx) { return saved_block->safeGetByPosition(key_idx).column; }); - const auto update_key_column_ptrs = ext::map(ext::range(0, keys_size), [&](const size_t key_idx) - { - return block.safeGetByPosition(key_idx).column; - }); + const auto update_key_column_ptrs = ext::map( + ext::range(0, keys_size), [&](const size_t key_idx) { return block.safeGetByPosition(key_idx).column; }); Arena temp_key_pool; - ContainerType > update_key_hash; + ContainerType> update_key_hash; for (size_t i = 0; i < block.rows(); ++i) { @@ -389,21 +444,49 @@ void ComplexKeyHashedDictionary::calculateBytesAllocated() { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: addAttributeSize(attribute); break; - case AttributeUnderlyingType::UInt16: addAttributeSize(attribute); break; - case AttributeUnderlyingType::UInt32: addAttributeSize(attribute); break; - case AttributeUnderlyingType::UInt64: addAttributeSize(attribute); break; - case AttributeUnderlyingType::UInt128: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Int8: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Int16: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Int32: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Int64: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Float32: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Float64: addAttributeSize(attribute); break; + case AttributeUnderlyingType::UInt8: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::UInt16: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::UInt32: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::UInt64: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::UInt128: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Int8: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Int16: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Int32: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Int64: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Float32: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Float64: + addAttributeSize(attribute); + break; - case AttributeUnderlyingType::Decimal32: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Decimal64: addAttributeSize(attribute); break; - case AttributeUnderlyingType::Decimal128: addAttributeSize(attribute); break; + case AttributeUnderlyingType::Decimal32: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Decimal64: + addAttributeSize(attribute); + break; + case AttributeUnderlyingType::Decimal128: + addAttributeSize(attribute); + break; case AttributeUnderlyingType::String: { @@ -425,27 +508,56 @@ void ComplexKeyHashedDictionary::createAttributeImpl(Attribute & attribute, cons attribute.maps.emplace>(); } -ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) +ComplexKeyHashedDictionary::Attribute +ComplexKeyHashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) { Attribute attr{type, {}, {}, {}}; switch (type) { - case AttributeUnderlyingType::UInt8: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::UInt16: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::UInt32: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::UInt64: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::UInt128: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Int8: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Int16: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Int32: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Int64: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Float32: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Float64: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::UInt8: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::UInt16: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::UInt32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::UInt64: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::UInt128: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Int8: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Int16: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Int32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Int64: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Float32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Float64: + createAttributeImpl(attr, null_value); + break; - case AttributeUnderlyingType::Decimal32: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Decimal64: createAttributeImpl(attr, null_value); break; - case AttributeUnderlyingType::Decimal128: createAttributeImpl(attr, null_value); break; + case AttributeUnderlyingType::Decimal32: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Decimal64: + createAttributeImpl(attr, null_value); + break; + case AttributeUnderlyingType::Decimal128: + createAttributeImpl(attr, null_value); + break; case AttributeUnderlyingType::String: { @@ -462,15 +574,14 @@ ComplexKeyHashedDictionary::Attribute ComplexKeyHashedDictionary::createAttribut template void ComplexKeyHashedDictionary::getItemsNumber( - const Attribute & attribute, - const Columns & key_columns, - ValueSetter && set_value, - DefaultGetter && get_default) const + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const { - if (false) {} + if (false) + { + } #define DISPATCH(TYPE) \ - else if (attribute.type == AttributeUnderlyingType::TYPE) \ - getItemsImpl(attribute, key_columns, std::forward(set_value), std::forward(get_default)); + else if (attribute.type == AttributeUnderlyingType::TYPE) getItemsImpl( \ + attribute, key_columns, std::forward(set_value), std::forward(get_default)); DISPATCH(UInt8) DISPATCH(UInt16) DISPATCH(UInt32) @@ -486,16 +597,12 @@ void ComplexKeyHashedDictionary::getItemsNumber( DISPATCH(Decimal64) DISPATCH(Decimal128) #undef DISPATCH - else - throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); + else throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); } template void ComplexKeyHashedDictionary::getItemsImpl( - const Attribute & attribute, - const Columns & key_columns, - ValueSetter && set_value, - DefaultGetter && get_default) const + const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const { const auto & attr = std::get>(attribute.maps); @@ -524,7 +631,7 @@ template bool ComplexKeyHashedDictionary::setAttributeValueImpl(Attribute & attribute, const StringRef key, const T value) { auto & map = std::get>(attribute.maps); - const auto pair = map.insert({ key, value }); + const auto pair = map.insert({key, value}); return pair.second; } @@ -532,28 +639,42 @@ bool ComplexKeyHashedDictionary::setAttributeValue(Attribute & attribute, const { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::UInt16: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::UInt32: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::UInt64: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::UInt128: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Int8: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Int16: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Int32: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Int64: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Float32: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Float64: return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::UInt8: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::UInt16: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::UInt32: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::UInt64: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::UInt128: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Int8: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Int16: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Int32: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Int64: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Float32: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Float64: + return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Decimal32: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Decimal64: return setAttributeValueImpl(attribute, key, value.get()); - case AttributeUnderlyingType::Decimal128: return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Decimal32: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Decimal64: + return setAttributeValueImpl(attribute, key, value.get()); + case AttributeUnderlyingType::Decimal128: + return setAttributeValueImpl(attribute, key, value.get()); case AttributeUnderlyingType::String: { auto & map = std::get>(attribute.maps); const auto & string = value.get(); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - const auto pair = map.insert({ key, StringRef{string_in_arena, string.size()} }); + const auto pair = map.insert({key, StringRef{string_in_arena, string.size()}}); return pair.second; } } @@ -570,8 +691,7 @@ const ComplexKeyHashedDictionary::Attribute & ComplexKeyHashedDictionary::getAtt return attributes[it->second]; } -StringRef ComplexKeyHashedDictionary::placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool) +StringRef ComplexKeyHashedDictionary::placeKeysInPool(const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool) { const auto keys_size = key_columns.size(); size_t sum_keys_size{}; @@ -590,7 +710,7 @@ StringRef ComplexKeyHashedDictionary::placeKeysInPool( key_start += keys[j].size; } - return { block_start, sum_keys_size }; + return {block_start, sum_keys_size}; } template @@ -623,22 +743,37 @@ std::vector ComplexKeyHashedDictionary::getKeys() const switch (attribute.type) { - case AttributeUnderlyingType::UInt8: return getKeys(attribute); - case AttributeUnderlyingType::UInt16: return getKeys(attribute); - case AttributeUnderlyingType::UInt32: return getKeys(attribute); - case AttributeUnderlyingType::UInt64: return getKeys(attribute); - case AttributeUnderlyingType::UInt128: return getKeys(attribute); - case AttributeUnderlyingType::Int8: return getKeys(attribute); - case AttributeUnderlyingType::Int16: return getKeys(attribute); - case AttributeUnderlyingType::Int32: return getKeys(attribute); - case AttributeUnderlyingType::Int64: return getKeys(attribute); - case AttributeUnderlyingType::Float32: return getKeys(attribute); - case AttributeUnderlyingType::Float64: return getKeys(attribute); - case AttributeUnderlyingType::String: return getKeys(attribute); + case AttributeUnderlyingType::UInt8: + return getKeys(attribute); + case AttributeUnderlyingType::UInt16: + return getKeys(attribute); + case AttributeUnderlyingType::UInt32: + return getKeys(attribute); + case AttributeUnderlyingType::UInt64: + return getKeys(attribute); + case AttributeUnderlyingType::UInt128: + return getKeys(attribute); + case AttributeUnderlyingType::Int8: + return getKeys(attribute); + case AttributeUnderlyingType::Int16: + return getKeys(attribute); + case AttributeUnderlyingType::Int32: + return getKeys(attribute); + case AttributeUnderlyingType::Int64: + return getKeys(attribute); + case AttributeUnderlyingType::Float32: + return getKeys(attribute); + case AttributeUnderlyingType::Float64: + return getKeys(attribute); + case AttributeUnderlyingType::String: + return getKeys(attribute); - case AttributeUnderlyingType::Decimal32: return getKeys(attribute); - case AttributeUnderlyingType::Decimal64: return getKeys(attribute); - case AttributeUnderlyingType::Decimal128: return getKeys(attribute); + case AttributeUnderlyingType::Decimal32: + return getKeys(attribute); + case AttributeUnderlyingType::Decimal64: + return getKeys(attribute); + case AttributeUnderlyingType::Decimal128: + return getKeys(attribute); } return {}; } @@ -663,17 +798,16 @@ BlockInputStreamPtr ComplexKeyHashedDictionary::getBlockInputStream(const Names void registerDictionaryComplexKeyHashed(DictionaryFactory & factory) { - auto create_layout = [=]( - const std::string & name, - const DictionaryStructure & dict_struct, - const Poco::Util::AbstractConfiguration & config, - const std::string & config_prefix, - DictionarySourcePtr source_ptr - ) -> DictionaryPtr { + auto create_layout = [=](const std::string & name, + const DictionaryStructure & dict_struct, + const Poco::Util::AbstractConfiguration & config, + const std::string & config_prefix, + DictionarySourcePtr source_ptr) -> DictionaryPtr + { if (!dict_struct.key) - throw Exception {"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; + throw Exception{"'key' is required for dictionary of layout 'complex_key_hashed'", ErrorCodes::BAD_ARGUMENTS}; - const DictionaryLifetime dict_lifetime {config, config_prefix + ".lifetime"}; + const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); }; diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h index 859266fb5d1..8362d1f6f4a 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -1,30 +1,33 @@ #pragma once -#include "IDictionary.h" -#include "IDictionarySource.h" -#include "DictionaryStructure.h" -#include -#include -#include -#include -#include -#include #include #include #include +#include +#include +#include +#include +#include +#include +#include "DictionaryStructure.h" +#include "IDictionary.h" +#include "IDictionarySource.h" namespace DB { - using BlockPtr = std::shared_ptr; class ComplexKeyHashedDictionary final : public IDictionaryBase { public: ComplexKeyHashedDictionary( - const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr, - const DictionaryLifetime dict_lifetime, bool require_nonempty, BlockPtr saved_block = nullptr); + const std::string & name, + const DictionaryStructure & dict_struct, + DictionarySourcePtr source_ptr, + const DictionaryLifetime dict_lifetime, + bool require_nonempty, + BlockPtr saved_block = nullptr); ComplexKeyHashedDictionary(const ComplexKeyHashedDictionary & other); @@ -56,10 +59,7 @@ public: const DictionaryStructure & getStructure() const override { return dict_struct; } - std::chrono::time_point getCreationTime() const override - { - return creation_time; - } + std::chrono::time_point getCreationTime() const override { return creation_time; } bool isInjective(const std::string & attribute_name) const override { @@ -69,9 +69,33 @@ public: template using ResultArrayType = std::conditional_t, DecimalPaddedPODArray, PaddedPODArray>; -#define DECLARE(TYPE)\ - void get##TYPE(\ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\ +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ResultArrayType & out) const; + DECLARE(UInt8) + DECLARE(UInt16) + DECLARE(UInt32) + DECLARE(UInt64) + DECLARE(UInt128) + DECLARE(Int8) + DECLARE(Int16) + DECLARE(Int32) + DECLARE(Int64) + DECLARE(Float32) + DECLARE(Float64) + DECLARE(Decimal32) + DECLARE(Decimal64) + DECLARE(Decimal128) +#undef DECLARE + + void getString(const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, ColumnString * out) const; + +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) @@ -90,13 +114,19 @@ public: #undef DECLARE void getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - ColumnString * out) const; + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnString * const def, + ColumnString * const out) const; -#define DECLARE(TYPE)\ - void get##TYPE(\ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\ - const PaddedPODArray & def, ResultArrayType & out) const; +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + const TYPE def, \ + ResultArrayType & out) const; DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -114,57 +144,57 @@ public: #undef DECLARE void getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const ColumnString * const def, ColumnString * const out) const; - -#define DECLARE(TYPE)\ - void get##TYPE(\ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types,\ - const TYPE def, ResultArrayType & out) const; - DECLARE(UInt8) - DECLARE(UInt16) - DECLARE(UInt32) - DECLARE(UInt64) - DECLARE(UInt128) - DECLARE(Int8) - DECLARE(Int16) - DECLARE(Int32) - DECLARE(Int64) - DECLARE(Float32) - DECLARE(Float64) - DECLARE(Decimal32) - DECLARE(Decimal64) - DECLARE(Decimal128) -#undef DECLARE - - void getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const String & def, ColumnString * const out) const; + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const String & def, + ColumnString * const out) const; void has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray & out) const; BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; private: - template using ContainerType = HashMapWithSavedHash; + template + using ContainerType = HashMapWithSavedHash; struct Attribute final { AttributeUnderlyingType type; std::variant< - UInt8, UInt16, UInt32, UInt64, + UInt8, + UInt16, + UInt32, + UInt64, UInt128, - Int8, Int16, Int32, Int64, - Decimal32, Decimal64, Decimal128, - Float32, Float64, - String> null_values; + Int8, + Int16, + Int32, + Int64, + Decimal32, + Decimal64, + Decimal128, + Float32, + Float64, + String> + null_values; std::variant< - ContainerType, ContainerType, ContainerType, ContainerType, + ContainerType, + ContainerType, + ContainerType, + ContainerType, ContainerType, - ContainerType, ContainerType, ContainerType, ContainerType, - ContainerType, ContainerType, ContainerType, - ContainerType, ContainerType, - ContainerType> maps; + ContainerType, + ContainerType, + ContainerType, + ContainerType, + ContainerType, + ContainerType, + ContainerType, + ContainerType, + ContainerType, + ContainerType> + maps; std::unique_ptr string_arena; }; @@ -188,18 +218,12 @@ private: template - void getItemsNumber( - const Attribute & attribute, - const Columns & key_columns, - ValueSetter && set_value, - DefaultGetter && get_default) const; + void + getItemsNumber(const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const; template - void getItemsImpl( - const Attribute & attribute, - const Columns & key_columns, - ValueSetter && set_value, - DefaultGetter && get_default) const; + void + getItemsImpl(const Attribute & attribute, const Columns & key_columns, ValueSetter && set_value, DefaultGetter && get_default) const; template @@ -209,8 +233,7 @@ private: const Attribute & getAttribute(const std::string & attribute_name) const; - static StringRef placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool); + static StringRef placeKeysInPool(const size_t row, const Columns & key_columns, StringRefs & keys, Arena & pool); template void has(const Attribute & attribute, const Columns & key_columns, PaddedPODArray & out) const; diff --git a/dbms/src/Dictionaries/DictionaryBlockInputStream.h b/dbms/src/Dictionaries/DictionaryBlockInputStream.h index f1778a9fa6d..afdc26cdba3 100644 --- a/dbms/src/Dictionaries/DictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/DictionaryBlockInputStream.h @@ -1,22 +1,21 @@ #pragma once -#include +#include #include #include +#include #include +#include #include #include +#include +#include #include "DictionaryBlockInputStreamBase.h" #include "DictionaryStructure.h" #include "IDictionary.h" -#include -#include -#include -#include namespace DB { - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -32,28 +31,30 @@ class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase public: using DictionaryPtr = std::shared_ptr; - DictionaryBlockInputStream(std::shared_ptr dictionary, size_t max_block_size, - PaddedPODArray && ids, const Names & column_names); + DictionaryBlockInputStream( + std::shared_ptr dictionary, size_t max_block_size, PaddedPODArray && ids, const Names & column_names); - DictionaryBlockInputStream(std::shared_ptr dictionary, size_t max_block_size, - const std::vector & keys, const Names & column_names); + DictionaryBlockInputStream( + std::shared_ptr dictionary, + size_t max_block_size, + const std::vector & keys, + const Names & column_names); - using GetColumnsFunction = - std::function & attributes)>; + using GetColumnsFunction = std::function & attributes)>; // Used to separate key columns format for storage and view. // Calls get_key_columns_function to get key column for dictionary get fuction call // and get_view_columns_function to get key representation. // Now used in trie dictionary, where columns are stored as ip and mask, and are showed as string - DictionaryBlockInputStream(std::shared_ptr dictionary, size_t max_block_size, - const Columns & data_columns, const Names & column_names, - GetColumnsFunction && get_key_columns_function, - GetColumnsFunction && get_view_columns_function); + DictionaryBlockInputStream( + std::shared_ptr dictionary, + size_t max_block_size, + const Columns & data_columns, + const Names & column_names, + GetColumnsFunction && get_key_columns_function, + GetColumnsFunction && get_view_columns_function); - String getName() const override - { - return "Dictionary"; - } + String getName() const override { return "Dictionary"; } protected: Block getBlock(size_t start, size_t size) const override; @@ -65,8 +66,8 @@ private: using DictionaryGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray &, PaddedPODArray &) const; template - using DictionaryDecimalGetter = - void (DictionaryType::*)(const std::string &, const PaddedPODArray &, DecimalPaddedPODArray &) const; + using DictionaryDecimalGetter + = void (DictionaryType::*)(const std::string &, const PaddedPODArray &, DecimalPaddedPODArray &) const; using DictionaryStringGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray &, ColumnString *) const; @@ -75,61 +76,103 @@ private: using GetterByKey = void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, PaddedPODArray & out) const; template - using DecimalGetterByKey = - void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, DecimalPaddedPODArray & out) const; + using DecimalGetterByKey + = void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, DecimalPaddedPODArray & out) const; using StringGetterByKey = void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, ColumnString * out) const; // call getXXX // for single key dictionaries template - void callGetter(DictionaryGetter getter, const PaddedPODArray & ids_to_fill, - const Columns & keys, const DataTypes & data_types, - Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const; + void callGetter( + DictionaryGetter getter, + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & data_types, + Container & container, + const DictionaryAttribute & attribute, + const DictionaryType & dictionary) const; template - void callGetter(DictionaryDecimalGetter getter, const PaddedPODArray & ids_to_fill, - const Columns & keys, const DataTypes & data_types, - Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const; + void callGetter( + DictionaryDecimalGetter getter, + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & data_types, + Container & container, + const DictionaryAttribute & attribute, + const DictionaryType & dictionary) const; template - void callGetter(DictionaryStringGetter getter, const PaddedPODArray & ids_to_fill, - const Columns & keys, const DataTypes & data_types, - Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const; + void callGetter( + DictionaryStringGetter getter, + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & data_types, + Container & container, + const DictionaryAttribute & attribute, + const DictionaryType & dictionary) const; // for complex complex key dictionaries template - void callGetter(GetterByKey getter, const PaddedPODArray & ids_to_fill, - const Columns & keys, const DataTypes & data_types, - Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const; + void callGetter( + GetterByKey getter, + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & data_types, + Container & container, + const DictionaryAttribute & attribute, + const DictionaryType & dictionary) const; template - void callGetter(DecimalGetterByKey getter, const PaddedPODArray & ids_to_fill, - const Columns & keys, const DataTypes & data_types, - Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const; + void callGetter( + DecimalGetterByKey getter, + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & data_types, + Container & container, + const DictionaryAttribute & attribute, + const DictionaryType & dictionary) const; template - void callGetter(StringGetterByKey getter, const PaddedPODArray & ids_to_fill, - const Columns & keys, const DataTypes & data_types, - Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const; + void callGetter( + StringGetterByKey getter, + const PaddedPODArray & ids_to_fill, + const Columns & keys, + const DataTypes & data_types, + Container & container, + const DictionaryAttribute & attribute, + const DictionaryType & dictionary) const; template